Nodes Stats API, closes #165,

This commit is contained in:
kimchy 2010-05-09 09:59:43 +03:00
parent 39cb08fc1c
commit 3e405c3ec7
29 changed files with 934 additions and 56 deletions

View File

@ -26,6 +26,7 @@
<w>desc</w>
<w>deserialize</w>
<w>elasticsearch</w>
<w>estab</w>
<w>failover</w>
<w>flushable</w>
<w>formatter</w>
@ -70,8 +71,12 @@
<w>queryparser</w>
<w>rackspace</w>
<w>rebalance</w>
<w>retrans</w>
<w>retval</w>
<w>rsts</w>
<w>sbuf</w>
<w>searchable</w>
<w>segs</w>
<w>serializers</w>
<w>sigar</w>
<w>snapshotting</w>

View File

@ -23,6 +23,7 @@ import org.elasticsearch.action.admin.cluster.health.TransportClusterHealthActio
import org.elasticsearch.action.admin.cluster.node.info.TransportNodesInfo;
import org.elasticsearch.action.admin.cluster.node.restart.TransportNodesRestartAction;
import org.elasticsearch.action.admin.cluster.node.shutdown.TransportNodesShutdownAction;
import org.elasticsearch.action.admin.cluster.node.stats.TransportNodesStats;
import org.elasticsearch.action.admin.cluster.ping.broadcast.TransportBroadcastPingAction;
import org.elasticsearch.action.admin.cluster.ping.replication.TransportIndexReplicationPingAction;
import org.elasticsearch.action.admin.cluster.ping.replication.TransportReplicationPingAction;
@ -63,6 +64,7 @@ public class TransportActionModule extends AbstractModule {
@Override protected void configure() {
bind(TransportNodesInfo.class).asEagerSingleton();
bind(TransportNodesStats.class).asEagerSingleton();
bind(TransportNodesShutdownAction.class).asEagerSingleton();
bind(TransportNodesRestartAction.class).asEagerSingleton();
bind(TransportClusterStateAction.class).asEagerSingleton();

View File

@ -73,6 +73,7 @@ public class TransportActions {
public static class Node {
public static final String INFO = "/cluster/nodes/info";
public static final String STATS = "/cluster/nodes/stats";
public static final String SHUTDOWN = "/cluster/nodes/shutdown";
public static final String RESTART = "/cluster/nodes/restart";
}

View File

@ -35,6 +35,8 @@ import java.io.IOException;
import java.util.Map;
/**
* Node information (static, does not change over time).
*
* @author kimchy (shay.banon)
*/
public class NodeInfo extends NodeOperationResponse {
@ -65,50 +67,86 @@ public class NodeInfo extends NodeOperationResponse {
this.network = network;
}
/**
* The attributes of the node.
*/
public ImmutableMap<String, String> attributes() {
return this.attributes;
}
/**
* The attributes of the node.
*/
public ImmutableMap<String, String> getAttributes() {
return attributes();
}
/**
* The settings of the node.
*/
public Settings settings() {
return this.settings;
}
/**
* The settings of the node.
*/
public Settings getSettings() {
return settings();
}
/**
* Operating System level information.
*/
public OsInfo os() {
return this.os;
}
/**
* Operating System level information.
*/
public OsInfo getOs() {
return os();
}
/**
* Process level information.
*/
public ProcessInfo process() {
return process;
}
/**
* Process level information.
*/
public ProcessInfo getProcess() {
return process();
}
/**
* JVM level information.
*/
public JvmInfo jvm() {
return jvm;
}
/**
* JVM level information.
*/
public JvmInfo getJvm() {
return jvm();
}
/**
* Network level information.
*/
public NetworkInfo network() {
return network;
}
/**
* Network level information.
*/
public NetworkInfo getNetwork() {
return network();
}

View File

@ -0,0 +1,165 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.action.admin.cluster.node.stats;
import org.elasticsearch.action.support.nodes.NodeOperationResponse;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.monitor.jvm.JvmStats;
import org.elasticsearch.monitor.network.NetworkStats;
import org.elasticsearch.monitor.os.OsStats;
import org.elasticsearch.monitor.process.ProcessStats;
import org.elasticsearch.util.io.stream.StreamInput;
import org.elasticsearch.util.io.stream.StreamOutput;
import java.io.IOException;
/**
* Node statistics (static, does not change over time).
*
* @author kimchy (shay.banon)
*/
public class NodeStats extends NodeOperationResponse {
private OsStats os;
private ProcessStats process;
private JvmStats jvm;
private NetworkStats network;
NodeStats() {
}
public NodeStats(DiscoveryNode node,
OsStats os, ProcessStats process, JvmStats jvm, NetworkStats network) {
super(node);
this.os = os;
this.process = process;
this.jvm = jvm;
this.network = network;
}
/**
* Operating System level statistics.
*/
public OsStats os() {
return this.os;
}
/**
* Operating System level statistics.
*/
public OsStats getOs() {
return os();
}
/**
* Process level statistics.
*/
public ProcessStats process() {
return process;
}
/**
* Process level statistics.
*/
public ProcessStats getProcess() {
return process();
}
/**
* JVM level statistics.
*/
public JvmStats jvm() {
return jvm;
}
/**
* JVM level statistics.
*/
public JvmStats getJvm() {
return jvm();
}
/**
* Network level statistics.
*/
public NetworkStats network() {
return network;
}
/**
* Network level statistics.
*/
public NetworkStats getNetwork() {
return network();
}
public static NodeStats readNodeStats(StreamInput in) throws IOException {
NodeStats nodeInfo = new NodeStats();
nodeInfo.readFrom(in);
return nodeInfo;
}
@Override public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
if (in.readBoolean()) {
os = OsStats.readOsStats(in);
}
if (in.readBoolean()) {
process = ProcessStats.readProcessStats(in);
}
if (in.readBoolean()) {
jvm = JvmStats.readJvmStats(in);
}
if (in.readBoolean()) {
network = NetworkStats.readNetworkStats(in);
}
}
@Override public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
if (os == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
os.writeTo(out);
}
if (process == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
process.writeTo(out);
}
if (jvm == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
jvm.writeTo(out);
}
if (network == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
network.writeTo(out);
}
}
}

View File

@ -0,0 +1,41 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.action.admin.cluster.node.stats;
import org.elasticsearch.action.support.nodes.NodesOperationRequest;
/**
* A request to get node (cluster) level stats.
*
* @author kimchy (shay.banon)
*/
public class NodesStatsRequest extends NodesOperationRequest {
protected NodesStatsRequest() {
}
/**
* Get stats from nodes based on the nodes ids specified. If none are passed, stats
* for all nodes will be returned.
*/
public NodesStatsRequest(String... nodesIds) {
super(nodesIds);
}
}

View File

@ -0,0 +1,56 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.action.admin.cluster.node.stats;
import org.elasticsearch.action.support.nodes.NodesOperationResponse;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.util.io.stream.StreamInput;
import org.elasticsearch.util.io.stream.StreamOutput;
import java.io.IOException;
/**
* @author kimchy (shay.banon)
*/
public class NodesStatsResponse extends NodesOperationResponse<NodeStats> {
public NodesStatsResponse() {
}
public NodesStatsResponse(ClusterName clusterName, NodeStats[] nodes) {
super(clusterName, nodes);
}
@Override public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
nodes = new NodeStats[in.readVInt()];
for (int i = 0; i < nodes.length; i++) {
nodes[i] = NodeStats.readNodeStats(in);
}
}
@Override public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(nodes.length);
for (NodeStats node : nodes) {
node.writeTo(out);
}
}
}

View File

@ -0,0 +1,106 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.action.admin.cluster.node.stats;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.action.TransportActions;
import org.elasticsearch.action.support.nodes.NodeOperationRequest;
import org.elasticsearch.action.support.nodes.TransportNodesOperationAction;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.monitor.MonitorService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.util.collect.Lists;
import org.elasticsearch.util.guice.inject.Inject;
import org.elasticsearch.util.settings.Settings;
import java.util.List;
import java.util.concurrent.atomic.AtomicReferenceArray;
/**
* @author kimchy (shay.banon)
*/
public class TransportNodesStats extends TransportNodesOperationAction<NodesStatsRequest, NodesStatsResponse, TransportNodesStats.NodeStatsRequest, NodeStats> {
private final MonitorService monitorService;
@Inject public TransportNodesStats(Settings settings, ClusterName clusterName, ThreadPool threadPool,
ClusterService clusterService, TransportService transportService,
MonitorService monitorService) {
super(settings, clusterName, threadPool, clusterService, transportService);
this.monitorService = monitorService;
}
@Override protected String transportAction() {
return TransportActions.Admin.Cluster.Node.STATS;
}
@Override protected String transportNodeAction() {
return "/cluster/nodes/stats/node";
}
@Override protected NodesStatsResponse newResponse(NodesStatsRequest nodesInfoRequest, AtomicReferenceArray responses) {
final List<NodeStats> nodeStats = Lists.newArrayList();
for (int i = 0; i < responses.length(); i++) {
Object resp = responses.get(i);
if (resp instanceof NodeStats) {
nodeStats.add((NodeStats) resp);
}
}
return new NodesStatsResponse(clusterName, nodeStats.toArray(new NodeStats[nodeStats.size()]));
}
@Override protected NodesStatsRequest newRequest() {
return new NodesStatsRequest();
}
@Override protected NodeStatsRequest newNodeRequest() {
return new NodeStatsRequest();
}
@Override protected NodeStatsRequest newNodeRequest(String nodeId, NodesStatsRequest request) {
return new NodeStatsRequest(nodeId);
}
@Override protected NodeStats newNodeResponse() {
return new NodeStats();
}
@Override protected NodeStats nodeOperation(NodeStatsRequest request) throws ElasticSearchException {
return new NodeStats(clusterService.state().nodes().localNode(),
monitorService.osService().stats(), monitorService.processService().stats(),
monitorService.jvmService().stats(), monitorService.networkService().stats());
}
@Override protected boolean accumulateExceptions() {
return false;
}
protected static class NodeStatsRequest extends NodeOperationRequest {
private NodeStatsRequest() {
}
private NodeStatsRequest(String nodeId) {
super(nodeId);
}
}
}

View File

@ -27,6 +27,8 @@ import org.elasticsearch.util.io.stream.Streamable;
import java.io.IOException;
/**
* A base class for node level operations.
*
* @author kimchy (shay.banon)
*/
public abstract class NodeOperationResponse implements Streamable {
@ -40,10 +42,16 @@ public abstract class NodeOperationResponse implements Streamable {
this.node = node;
}
/**
* The node this information relates to.
*/
public DiscoveryNode node() {
return node;
}
/**
* The node this information relates to.
*/
public DiscoveryNode getNode() {
return node();
}

View File

@ -29,6 +29,8 @@ import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartRequest;
import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartResponse;
import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownRequest;
import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownResponse;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.cluster.ping.broadcast.BroadcastPingRequest;
import org.elasticsearch.action.admin.cluster.ping.broadcast.BroadcastPingResponse;
import org.elasticsearch.action.admin.cluster.ping.replication.ReplicationPingRequest;
@ -96,10 +98,28 @@ public interface ClusterAdminClient {
*
* @param request The nodes info request
* @param listener A listener to be notified with a result
* @see org.elasticsearch.client.Requests#nodesShutdown(String...)
* @see org.elasticsearch.client.Requests#nodesInfo(String...)
*/
void nodesInfo(NodesInfoRequest request, ActionListener<NodesInfoResponse> listener);
/**
* Nodes stats of the cluster.
*
* @param request The nodes info request
* @return The result future
* @see org.elasticsearch.client.Requests#nodesStats(String...)
*/
ActionFuture<NodesStatsResponse> nodesStats(NodesStatsRequest request);
/**
* Nodes stats of the cluster.
*
* @param request The nodes info request
* @param listener A listener to be notified with a result
* @see org.elasticsearch.client.Requests#nodesStats(String...)
*/
void nodesStats(NodesStatsRequest request, ActionListener<NodesStatsResponse> listener);
/**
* Shutdown nodes in the cluster.
*

View File

@ -23,6 +23,7 @@ import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest;
import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartRequest;
import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownRequest;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest;
import org.elasticsearch.action.admin.cluster.ping.broadcast.BroadcastPingRequest;
import org.elasticsearch.action.admin.cluster.ping.replication.ReplicationPingRequest;
import org.elasticsearch.action.admin.cluster.ping.single.SinglePingRequest;
@ -314,12 +315,23 @@ public class Requests {
*
* @param nodesIds The nodes ids to get the status for
* @return The nodes info request
* @see org.elasticsearch.client.ClusterAdminClient#nodesInfo(org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest)
* @see org.elasticsearch.client.ClusterAdminClient#nodesStats(org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest)
*/
public static NodesInfoRequest nodesInfo(String... nodesIds) {
return new NodesInfoRequest(nodesIds);
}
/**
* Creates a nodes stats request against one or more nodes. Pass <tt>null</tt> or an empty array for all nodes.
*
* @param nodesIds The nodes ids to get the stats for
* @return The nodes info request
* @see org.elasticsearch.client.ClusterAdminClient#nodesStats(org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest)
*/
public static NodesStatsRequest nodesStats(String... nodesIds) {
return new NodesStatsRequest(nodesIds);
}
/**
* Shuts down all nodes in the cluster.
*/

View File

@ -33,6 +33,9 @@ import org.elasticsearch.action.admin.cluster.node.restart.TransportNodesRestart
import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownRequest;
import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownResponse;
import org.elasticsearch.action.admin.cluster.node.shutdown.TransportNodesShutdownAction;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.cluster.node.stats.TransportNodesStats;
import org.elasticsearch.action.admin.cluster.ping.broadcast.BroadcastPingRequest;
import org.elasticsearch.action.admin.cluster.ping.broadcast.BroadcastPingResponse;
import org.elasticsearch.action.admin.cluster.ping.broadcast.TransportBroadcastPingAction;
@ -67,6 +70,8 @@ public class NodeClusterAdminClient extends AbstractComponent implements Cluster
private final TransportNodesInfo nodesInfo;
private final TransportNodesStats nodesStats;
private final TransportNodesShutdownAction nodesShutdown;
private final TransportNodesRestartAction nodesRestart;
@ -74,7 +79,7 @@ public class NodeClusterAdminClient extends AbstractComponent implements Cluster
@Inject public NodeClusterAdminClient(Settings settings,
TransportClusterHealthAction clusterHealthAction, TransportClusterStateAction clusterStateAction,
TransportSinglePingAction singlePingAction, TransportBroadcastPingAction broadcastPingAction, TransportReplicationPingAction replicationPingAction,
TransportNodesInfo nodesInfo, TransportNodesShutdownAction nodesShutdown, TransportNodesRestartAction nodesRestart) {
TransportNodesInfo nodesInfo, TransportNodesShutdownAction nodesShutdown, TransportNodesRestartAction nodesRestart, TransportNodesStats nodesStats) {
super(settings);
this.clusterHealthAction = clusterHealthAction;
this.clusterStateAction = clusterStateAction;
@ -84,6 +89,7 @@ public class NodeClusterAdminClient extends AbstractComponent implements Cluster
this.singlePingAction = singlePingAction;
this.broadcastPingAction = broadcastPingAction;
this.replicationPingAction = replicationPingAction;
this.nodesStats = nodesStats;
}
@Override public ActionFuture<ClusterHealthResponse> health(ClusterHealthRequest request) {
@ -134,6 +140,14 @@ public class NodeClusterAdminClient extends AbstractComponent implements Cluster
nodesInfo.execute(request, listener);
}
@Override public ActionFuture<NodesStatsResponse> nodesStats(NodesStatsRequest request) {
return nodesStats.execute(request);
}
@Override public void nodesStats(NodesStatsRequest request, ActionListener<NodesStatsResponse> listener) {
nodesStats.execute(request, listener);
}
@Override public ActionFuture<NodesShutdownResponse> nodesShutdown(NodesShutdownRequest request) {
return nodesShutdown.execute(request);
}

View File

@ -23,6 +23,7 @@ import org.elasticsearch.client.transport.action.admin.cluster.health.ClientTran
import org.elasticsearch.client.transport.action.admin.cluster.node.info.ClientTransportNodesInfoAction;
import org.elasticsearch.client.transport.action.admin.cluster.node.restart.ClientTransportNodesRestartAction;
import org.elasticsearch.client.transport.action.admin.cluster.node.shutdown.ClientTransportNodesShutdownAction;
import org.elasticsearch.client.transport.action.admin.cluster.node.stats.ClientTransportNodesStatsAction;
import org.elasticsearch.client.transport.action.admin.cluster.ping.broadcast.ClientTransportBroadcastPingAction;
import org.elasticsearch.client.transport.action.admin.cluster.ping.replication.ClientTransportReplicationPingAction;
import org.elasticsearch.client.transport.action.admin.cluster.ping.single.ClientTransportSinglePingAction;
@ -74,6 +75,7 @@ public class ClientTransportActionModule extends AbstractModule {
bind(ClientTransportClearIndicesCacheAction.class).asEagerSingleton();
bind(ClientTransportNodesInfoAction.class).asEagerSingleton();
bind(ClientTransportNodesStatsAction.class).asEagerSingleton();
bind(ClientTransportNodesShutdownAction.class).asEagerSingleton();
bind(ClientTransportNodesRestartAction.class).asEagerSingleton();
bind(ClientTransportSinglePingAction.class).asEagerSingleton();

View File

@ -0,0 +1,42 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.client.transport.action.admin.cluster.node.stats;
import org.elasticsearch.action.TransportActions;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.client.transport.action.support.BaseClientTransportAction;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.util.guice.inject.Inject;
import org.elasticsearch.util.settings.Settings;
/**
* @author kimchy (shay.banon)
*/
public class ClientTransportNodesStatsAction extends BaseClientTransportAction<NodesStatsRequest, NodesStatsResponse> {
@Inject public ClientTransportNodesStatsAction(Settings settings, TransportService transportService) {
super(settings, transportService, NodesStatsResponse.class);
}
@Override protected String action() {
return TransportActions.Admin.Cluster.Node.STATS;
}
}

View File

@ -30,6 +30,8 @@ import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartRequest;
import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartResponse;
import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownRequest;
import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownResponse;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.cluster.ping.broadcast.BroadcastPingRequest;
import org.elasticsearch.action.admin.cluster.ping.broadcast.BroadcastPingResponse;
import org.elasticsearch.action.admin.cluster.ping.replication.ReplicationPingRequest;
@ -44,6 +46,7 @@ import org.elasticsearch.client.transport.action.admin.cluster.health.ClientTran
import org.elasticsearch.client.transport.action.admin.cluster.node.info.ClientTransportNodesInfoAction;
import org.elasticsearch.client.transport.action.admin.cluster.node.restart.ClientTransportNodesRestartAction;
import org.elasticsearch.client.transport.action.admin.cluster.node.shutdown.ClientTransportNodesShutdownAction;
import org.elasticsearch.client.transport.action.admin.cluster.node.stats.ClientTransportNodesStatsAction;
import org.elasticsearch.client.transport.action.admin.cluster.ping.broadcast.ClientTransportBroadcastPingAction;
import org.elasticsearch.client.transport.action.admin.cluster.ping.replication.ClientTransportReplicationPingAction;
import org.elasticsearch.client.transport.action.admin.cluster.ping.single.ClientTransportSinglePingAction;
@ -72,6 +75,8 @@ public class InternalTransportClusterAdminClient extends AbstractComponent imple
private final ClientTransportNodesInfoAction nodesInfoAction;
private final ClientTransportNodesStatsAction nodesStatsAction;
private final ClientTransportNodesShutdownAction nodesShutdownAction;
private final ClientTransportNodesRestartAction nodesRestartAction;
@ -79,7 +84,7 @@ public class InternalTransportClusterAdminClient extends AbstractComponent imple
@Inject public InternalTransportClusterAdminClient(Settings settings, TransportClientNodesService nodesService,
ClientTransportClusterHealthAction clusterHealthAction, ClientTransportClusterStateAction clusterStateAction,
ClientTransportSinglePingAction singlePingAction, ClientTransportReplicationPingAction replicationPingAction, ClientTransportBroadcastPingAction broadcastPingAction,
ClientTransportNodesInfoAction nodesInfoAction, ClientTransportNodesShutdownAction nodesShutdownAction, ClientTransportNodesRestartAction nodesRestartAction) {
ClientTransportNodesInfoAction nodesInfoAction, ClientTransportNodesShutdownAction nodesShutdownAction, ClientTransportNodesRestartAction nodesRestartAction, ClientTransportNodesStatsAction nodesStatsAction) {
super(settings);
this.nodesService = nodesService;
this.clusterHealthAction = clusterHealthAction;
@ -90,6 +95,7 @@ public class InternalTransportClusterAdminClient extends AbstractComponent imple
this.singlePingAction = singlePingAction;
this.replicationPingAction = replicationPingAction;
this.broadcastPingAction = broadcastPingAction;
this.nodesStatsAction = nodesStatsAction;
}
@Override public ActionFuture<ClusterHealthResponse> health(final ClusterHealthRequest request) {
@ -194,6 +200,23 @@ public class InternalTransportClusterAdminClient extends AbstractComponent imple
});
}
@Override public ActionFuture<NodesStatsResponse> nodesStats(final NodesStatsRequest request) {
return nodesService.execute(new TransportClientNodesService.NodeCallback<ActionFuture<NodesStatsResponse>>() {
@Override public ActionFuture<NodesStatsResponse> doWithNode(DiscoveryNode node) throws ElasticSearchException {
return nodesStatsAction.execute(node, request);
}
});
}
@Override public void nodesStats(final NodesStatsRequest request, final ActionListener<NodesStatsResponse> listener) {
nodesService.execute(new TransportClientNodesService.NodeCallback<Void>() {
@Override public Void doWithNode(DiscoveryNode node) throws ElasticSearchException {
nodesStatsAction.execute(node, request, listener);
return null;
}
});
}
@Override public ActionFuture<NodesShutdownResponse> nodesShutdown(final NodesShutdownRequest request) {
return nodesService.execute(new TransportClientNodesService.NodeCallback<ActionFuture<NodesShutdownResponse>>() {
@Override public ActionFuture<NodesShutdownResponse> doWithNode(DiscoveryNode node) throws ElasticSearchException {

View File

@ -37,6 +37,8 @@ import java.util.Map;
import static org.elasticsearch.util.transport.TransportAddressSerializers.*;
/**
* A discovery node represents a node that is part of the cluster.
*
* @author kimchy (shay.banon)
*/
public class DiscoveryNode implements Streamable, Serializable {

View File

@ -39,7 +39,7 @@ public class JvmService extends AbstractComponent {
return this.jvmInfo;
}
public JvmStats state() {
public JvmStats stats() {
return JvmStats.jvmStats();
}
}

View File

@ -25,6 +25,8 @@ import org.elasticsearch.util.collect.Iterators;
import org.elasticsearch.util.io.stream.StreamInput;
import org.elasticsearch.util.io.stream.StreamOutput;
import org.elasticsearch.util.io.stream.Streamable;
import org.elasticsearch.util.xcontent.ToXContent;
import org.elasticsearch.util.xcontent.builder.XContentBuilder;
import java.io.IOException;
import java.io.Serializable;
@ -36,7 +38,7 @@ import java.util.concurrent.TimeUnit;
/**
* @author kimchy (shay.banon)
*/
public class JvmStats implements Streamable, Serializable {
public class JvmStats implements Streamable, Serializable, ToXContent {
private static RuntimeMXBean runtimeMXBean;
private static MemoryMXBean memoryMXBean;
@ -134,6 +136,51 @@ public class JvmStats implements Streamable, Serializable {
return gc();
}
@Override public void toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject("jvm");
builder.field("timestamp", timestamp);
builder.field("uptime", uptime().format());
builder.field("uptime_in_millis", uptime().millis());
if (mem != null) {
builder.startObject("mem");
builder.field("heap_used", mem.heapUsed().toString());
builder.field("heap_used_in_bytes", mem.heapUsed().bytes());
builder.field("heap_committed", mem.heapCommitted().toString());
builder.field("heap_committed_in_bytes", mem.heapCommitted().bytes());
builder.field("non_heap_used", mem.nonHeapUsed().toString());
builder.field("non_heap_used_in_bytes", mem.nonHeapUsed().bytes());
builder.field("non_heap_committed", mem.nonHeapCommitted().toString());
builder.field("non_heap_committed_in_bytes", mem.nonHeapCommitted().bytes());
builder.endObject();
}
if (threads != null) {
builder.startObject("threads");
builder.field("count", threads.count());
builder.field("peak_count", threads.peakCount());
builder.endObject();
}
if (gc != null) {
builder.startObject("gc");
builder.field("collection_count", gc.collectionCount());
builder.field("collection_time", gc.collectionTime().format());
builder.field("collection_time_in_millis", gc.collectionTime().millis());
builder.startObject("collectors");
for (GarbageCollector collector : gc) {
builder.startObject(collector.name());
builder.field("collection_count", collector.collectionCount());
builder.field("collection_time", collector.collectionTime().format());
builder.field("collection_time_in_millis", collector.collectionTime().millis());
builder.endObject();
}
builder.endObject();
builder.endObject();
}
builder.endObject();
}
public static JvmStats readJvmStats(StreamInput in) throws IOException {
JvmStats jvmStats = new JvmStats();
jvmStats.readFrom(in);
@ -158,7 +205,7 @@ public class JvmStats implements Streamable, Serializable {
gc.writeTo(out);
}
public static class GarbageCollectors implements Streamable, Serializable, Iterable {
public static class GarbageCollectors implements Streamable, Serializable, Iterable<GarbageCollector> {
private GarbageCollector[] collectors;
@ -266,6 +313,22 @@ public class JvmStats implements Streamable, Serializable {
Threads() {
}
public int count() {
return count;
}
public int getCount() {
return count();
}
public int peakCount() {
return peakCount;
}
public int getPeakCount() {
return peakCount();
}
public static Threads readThreads(StreamInput in) throws IOException {
Threads threads = new Threads();
threads.readFrom(in);

View File

@ -22,6 +22,8 @@ package org.elasticsearch.monitor.network;
import org.elasticsearch.util.io.stream.StreamInput;
import org.elasticsearch.util.io.stream.StreamOutput;
import org.elasticsearch.util.io.stream.Streamable;
import org.elasticsearch.util.xcontent.ToXContent;
import org.elasticsearch.util.xcontent.builder.XContentBuilder;
import java.io.IOException;
import java.io.Serializable;
@ -29,12 +31,41 @@ import java.io.Serializable;
/**
* @author kimchy (shay.banon)
*/
public class NetworkStats implements Streamable, Serializable {
public class NetworkStats implements Streamable, Serializable, ToXContent {
long timestamp;
Tcp tcp = null;
NetworkStats() {
}
@Override public void toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject("network");
if (tcp != null) {
builder.startObject("tcp");
builder.field("active_opens", tcp.getActiveOpens());
builder.field("passive_opens", tcp.getPassiveOpens());
builder.field("curr_estab", tcp.getCurrEstab());
builder.field("in_segs", tcp.getInSegs());
builder.field("out_segs", tcp.getOutSegs());
builder.field("retrans_segs", tcp.getRetransSegs());
builder.field("estab_resets", tcp.getEstabResets());
builder.field("attempt_fails", tcp.getAttemptFails());
builder.field("in_errs", tcp.getInErrs());
builder.field("out_rsts", tcp.getOutRsts());
builder.endObject();
}
builder.endObject();
}
public static NetworkStats readNetworkStats(StreamInput in) throws IOException {
NetworkStats stats = new NetworkStats();
stats.readFrom(in);
return stats;
}
@Override public void readFrom(StreamInput in) throws IOException {
timestamp = in.readVLong();
if (in.readBoolean()) {

View File

@ -19,12 +19,13 @@
package org.elasticsearch.monitor.os;
import org.elasticsearch.util.Percent;
import org.elasticsearch.util.SizeValue;
import org.elasticsearch.util.TimeValue;
import org.elasticsearch.util.io.stream.StreamInput;
import org.elasticsearch.util.io.stream.StreamOutput;
import org.elasticsearch.util.io.stream.Streamable;
import org.elasticsearch.util.xcontent.ToXContent;
import org.elasticsearch.util.xcontent.builder.XContentBuilder;
import java.io.IOException;
import java.io.Serializable;
@ -33,7 +34,7 @@ import java.util.concurrent.TimeUnit;
/**
* @author kimchy (shay.banon)
*/
public class OsStats implements Streamable, Serializable {
public class OsStats implements Streamable, Serializable, ToXContent {
public static final double[] EMPTY_LOAD = new double[0];
@ -101,6 +102,57 @@ public class OsStats implements Streamable, Serializable {
return swap();
}
@Override public void toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject("os");
builder.field("timestamp", timestamp);
builder.field("uptime", uptime().format());
builder.field("uptime_in_millis", uptime().millis());
builder.startArray("load_average");
for (double value : loadAverage) {
builder.value(value);
}
builder.endArray();
if (cpu != null) {
builder.startObject("cpu");
builder.field("sys", cpu.sys());
builder.field("user", cpu.user());
builder.field("idle", cpu.idle());
builder.endObject();
}
if (mem != null) {
builder.startObject("mem");
builder.field("free", mem.free().toString());
builder.field("free_in_bytes", mem.free().bytes());
builder.field("used", mem.used().toString());
builder.field("used_in_bytes", mem.used().bytes());
builder.field("free_percent", mem.freePercent());
builder.field("used_percent", mem.usedPercent());
builder.field("actual_free", mem.actualFree().toString());
builder.field("actual_free_in_bytes", mem.actualFree().bytes());
builder.field("actual_used", mem.actualUsed().toString());
builder.field("actual_used_in_bytes", mem.actualUsed().bytes());
builder.endObject();
}
if (swap != null) {
builder.startObject("swap");
builder.field("used", swap.used().toString());
builder.field("used_in_bytes", swap.used().bytes());
builder.field("free", swap.free().toString());
builder.field("free_in_bytes", swap.free().bytes());
builder.endObject();
}
builder.endObject();
}
public static OsStats readOsStats(StreamInput in) throws IOException {
OsStats stats = new OsStats();
stats.readFrom(in);
@ -157,6 +209,22 @@ public class OsStats implements Streamable, Serializable {
long free = -1;
long used = -1;
public SizeValue free() {
return new SizeValue(free);
}
public SizeValue getFree() {
return free();
}
public SizeValue used() {
return new SizeValue(used);
}
public SizeValue getUsed() {
return used();
}
public static Swap readSwap(StreamInput in) throws IOException {
Swap swap = new Swap();
swap.readFrom(in);
@ -177,9 +245,9 @@ public class OsStats implements Streamable, Serializable {
public static class Mem implements Streamable, Serializable {
long free = -1;
double freePercent = -1;
short freePercent = -1;
long used = -1;
double usedPercent = -1;
short usedPercent = -1;
long actualFree = -1;
long actualUsed = -1;
@ -191,18 +259,18 @@ public class OsStats implements Streamable, Serializable {
@Override public void readFrom(StreamInput in) throws IOException {
free = in.readLong();
freePercent = in.readDouble();
freePercent = in.readShort();
used = in.readLong();
usedPercent = in.readDouble();
usedPercent = in.readShort();
actualFree = in.readLong();
actualUsed = in.readLong();
}
@Override public void writeTo(StreamOutput out) throws IOException {
out.writeLong(free);
out.writeDouble(freePercent);
out.writeShort(freePercent);
out.writeLong(used);
out.writeDouble(usedPercent);
out.writeShort(usedPercent);
out.writeLong(actualFree);
out.writeLong(actualUsed);
}
@ -215,11 +283,11 @@ public class OsStats implements Streamable, Serializable {
return used();
}
public Percent usedPercent() {
return new Percent(usedPercent);
public short usedPercent() {
return usedPercent;
}
public Percent getUsedPercent() {
public short getUsedPercent() {
return usedPercent();
}
@ -231,11 +299,11 @@ public class OsStats implements Streamable, Serializable {
return free();
}
public Percent freePercent() {
return new Percent(freePercent);
public short freePercent() {
return freePercent;
}
public Percent getFreePercent() {
public short getFreePercent() {
return freePercent();
}
@ -258,9 +326,9 @@ public class OsStats implements Streamable, Serializable {
public static class Cpu implements Streamable, Serializable {
double sys = -1;
double user = -1;
double idle = -1;
short sys = -1;
short user = -1;
short idle = -1;
Cpu() {
@ -273,38 +341,38 @@ public class OsStats implements Streamable, Serializable {
}
@Override public void readFrom(StreamInput in) throws IOException {
sys = in.readDouble();
user = in.readDouble();
idle = in.readDouble();
sys = in.readShort();
user = in.readShort();
idle = in.readShort();
}
@Override public void writeTo(StreamOutput out) throws IOException {
out.writeDouble(sys);
out.writeDouble(user);
out.writeDouble(idle);
out.writeShort(sys);
out.writeShort(user);
out.writeShort(idle);
}
public Percent sys() {
return new Percent(sys);
public short sys() {
return sys;
}
public Percent getSys() {
public short getSys() {
return sys();
}
public Percent user() {
return new Percent(user);
public short user() {
return user;
}
public Percent getUser() {
public short getUser() {
return user();
}
public Percent idle() {
return new Percent(idle);
public short idle() {
return idle;
}
public Percent getIdle() {
public short getIdle() {
return idle();
}
}

View File

@ -95,9 +95,9 @@ public class SigarOsProbe extends AbstractComponent implements OsProbe {
try {
CpuPerc cpuPerc = sigar.getCpuPerc();
stats.cpu = new OsStats.Cpu();
stats.cpu.sys = cpuPerc.getSys();
stats.cpu.user = cpuPerc.getUser();
stats.cpu.idle = cpuPerc.getIdle();
stats.cpu.sys = (short) (cpuPerc.getSys() * 100);
stats.cpu.user = (short) (cpuPerc.getUser() * 100);
stats.cpu.idle = (short) (cpuPerc.getIdle() * 100);
} catch (SigarException e) {
// ignore
}
@ -106,9 +106,9 @@ public class SigarOsProbe extends AbstractComponent implements OsProbe {
Mem mem = sigar.getMem();
stats.mem = new OsStats.Mem();
stats.mem.free = mem.getFree();
stats.mem.freePercent = mem.getFreePercent() / 100;
stats.mem.freePercent = (short) mem.getFreePercent();
stats.mem.used = mem.getUsed();
stats.mem.usedPercent = mem.getUsedPercent() / 100;
stats.mem.usedPercent = (short) mem.getUsedPercent();
stats.mem.actualFree = mem.getActualFree();
stats.mem.actualUsed = mem.getActualUsed();
} catch (SigarException e) {

View File

@ -19,12 +19,13 @@
package org.elasticsearch.monitor.process;
import org.elasticsearch.util.Percent;
import org.elasticsearch.util.SizeValue;
import org.elasticsearch.util.TimeValue;
import org.elasticsearch.util.io.stream.StreamInput;
import org.elasticsearch.util.io.stream.StreamOutput;
import org.elasticsearch.util.io.stream.Streamable;
import org.elasticsearch.util.xcontent.ToXContent;
import org.elasticsearch.util.xcontent.builder.XContentBuilder;
import java.io.IOException;
import java.io.Serializable;
@ -32,7 +33,7 @@ import java.io.Serializable;
/**
* @author kimchy (shay.banon)
*/
public class ProcessStats implements Streamable, Serializable {
public class ProcessStats implements Streamable, Serializable, ToXContent {
long timestamp = -1;
@ -77,6 +78,38 @@ public class ProcessStats implements Streamable, Serializable {
return fd();
}
@Override public void toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject("process");
builder.field("timestamp", timestamp);
if (cpu != null) {
builder.startObject("cpu");
builder.field("percent", cpu.percent());
builder.field("sys", cpu.sys().format());
builder.field("sys_in_millis", cpu.sys().millis());
builder.field("user", cpu.user().format());
builder.field("user_in_millis", cpu.user().millis());
builder.field("total", cpu.total().format());
builder.field("total_in_millis", cpu.total().millis());
builder.endObject();
}
if (mem != null) {
builder.startObject("mem");
builder.field("resident", mem.resident().toString());
builder.field("resident_in_bytes", mem.resident().bytes());
builder.field("share", mem.share().toString());
builder.field("share_in_bytes", mem.share().bytes());
builder.field("total_virtual", mem.totalVirtual().toString());
builder.field("total_virtual_in_bytes", mem.totalVirtual().bytes());
builder.endObject();
}
if (fd != null) {
builder.startObject("fd");
builder.field("total", fd.total());
builder.endObject();
}
builder.endObject();
}
public static ProcessStats readProcessStats(StreamInput in) throws IOException {
ProcessStats stats = new ProcessStats();
stats.readFrom(in);
@ -207,7 +240,7 @@ public class ProcessStats implements Streamable, Serializable {
public static class Cpu implements Streamable, Serializable {
double percent = -1;
short percent = -1;
long sys = -1;
long user = -1;
long total = -1;
@ -223,14 +256,14 @@ public class ProcessStats implements Streamable, Serializable {
}
@Override public void readFrom(StreamInput in) throws IOException {
percent = in.readDouble();
percent = in.readShort();
sys = in.readLong();
user = in.readLong();
total = in.readLong();
}
@Override public void writeTo(StreamOutput out) throws IOException {
out.writeDouble(percent);
out.writeShort(percent);
out.writeLong(sys);
out.writeLong(user);
out.writeLong(total);
@ -241,8 +274,8 @@ public class ProcessStats implements Streamable, Serializable {
*
* <p>Supported Platforms: All.
*/
public Percent percent() {
return new Percent(percent);
public short percent() {
return percent;
}
/**
@ -250,7 +283,7 @@ public class ProcessStats implements Streamable, Serializable {
*
* <p>Supported Platforms: All.
*/
public Percent getPercent() {
public short getPercent() {
return percent();
}

View File

@ -49,7 +49,7 @@ public class SigarProcessProbe extends AbstractComponent implements ProcessProbe
try {
ProcCpu cpu = sigar.getProcCpu(sigar.getPid());
stats.cpu = new ProcessStats.Cpu();
stats.cpu.percent = cpu.getPercent();
stats.cpu.percent = (short) (cpu.getPercent() * 100);
stats.cpu.sys = cpu.getSys();
stats.cpu.user = cpu.getUser();
} catch (SigarException e) {

View File

@ -23,6 +23,7 @@ import org.elasticsearch.rest.action.admin.cluster.health.RestClusterHealthActio
import org.elasticsearch.rest.action.admin.cluster.node.info.RestNodesInfoAction;
import org.elasticsearch.rest.action.admin.cluster.node.restart.RestNodesRestartAction;
import org.elasticsearch.rest.action.admin.cluster.node.shutdown.RestNodesShutdownAction;
import org.elasticsearch.rest.action.admin.cluster.node.stats.RestNodesStatsAction;
import org.elasticsearch.rest.action.admin.cluster.ping.broadcast.RestBroadcastPingAction;
import org.elasticsearch.rest.action.admin.cluster.ping.replication.RestReplicationPingAction;
import org.elasticsearch.rest.action.admin.cluster.ping.single.RestSinglePingAction;
@ -58,6 +59,7 @@ public class RestActionModule extends AbstractModule {
bind(RestMainAction.class).asEagerSingleton();
bind(RestNodesInfoAction.class).asEagerSingleton();
bind(RestNodesStatsAction.class).asEagerSingleton();
bind(RestNodesShutdownAction.class).asEagerSingleton();
bind(RestNodesRestartAction.class).asEagerSingleton();
bind(RestClusterStateAction.class).asEagerSingleton();

View File

@ -0,0 +1,97 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.admin.cluster.node.stats;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.rest.*;
import org.elasticsearch.rest.action.support.RestActions;
import org.elasticsearch.rest.action.support.RestXContentBuilder;
import org.elasticsearch.util.guice.inject.Inject;
import org.elasticsearch.util.settings.Settings;
import org.elasticsearch.util.xcontent.builder.XContentBuilder;
import java.io.IOException;
/**
* @author kimchy (shay.banon)
*/
public class RestNodesStatsAction extends BaseRestHandler {
@Inject public RestNodesStatsAction(Settings settings, Client client, RestController controller) {
super(settings, client);
controller.registerHandler(RestRequest.Method.GET, "/_cluster/nodes/stats", this);
controller.registerHandler(RestRequest.Method.GET, "/_cluster/nodes/{nodeId}/stats", this);
}
@Override public void handleRequest(final RestRequest request, final RestChannel channel) {
String[] nodesIds = RestActions.splitNodes(request.param("nodeId"));
NodesStatsRequest nodesStatsRequest = new NodesStatsRequest(nodesIds);
nodesStatsRequest.listenerThreaded(false);
client.admin().cluster().nodesStats(nodesStatsRequest, new ActionListener<NodesStatsResponse>() {
@Override public void onResponse(NodesStatsResponse result) {
try {
XContentBuilder builder = RestXContentBuilder.restContentBuilder(request);
builder.startObject();
builder.field("cluster_name", result.clusterName().value());
builder.startObject("nodes");
for (NodeStats nodeStats : result) {
builder.startObject(nodeStats.node().id());
builder.field("name", nodeStats.node().name());
if (nodeStats.os() != null) {
nodeStats.os().toXContent(builder, request);
}
if (nodeStats.process() != null) {
nodeStats.process().toXContent(builder, request);
}
if (nodeStats.jvm() != null) {
nodeStats.jvm().toXContent(builder, request);
}
if (nodeStats.network() != null) {
nodeStats.network().toXContent(builder, request);
}
builder.endObject();
}
builder.endObject();
builder.endObject();
channel.sendResponse(new XContentRestResponse(request, RestResponse.Status.OK, builder));
} catch (Exception 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);
}
}
});
}
}

View File

@ -108,7 +108,7 @@ public class SizeValue implements Serializable, Streamable {
@Override public String toString() {
long bytes = bytes();
double value = bytes;
String suffix = "b";
String suffix = "";
if (bytes >= SizeUnit.C3) {
value = gbFrac();
suffix = "g";

View File

@ -23,6 +23,10 @@ import org.elasticsearch.ElasticSearchParseException;
import org.elasticsearch.util.io.stream.StreamInput;
import org.elasticsearch.util.io.stream.StreamOutput;
import org.elasticsearch.util.io.stream.Streamable;
import org.joda.time.Period;
import org.joda.time.PeriodType;
import org.joda.time.format.PeriodFormat;
import org.joda.time.format.PeriodFormatter;
import java.io.IOException;
import java.io.Serializable;
@ -170,6 +174,19 @@ public class TimeValue implements Serializable, Streamable {
return daysFrac();
}
private final PeriodFormatter defaultFormatter = PeriodFormat.getDefault()
.withParseType(PeriodType.standard());
public String format() {
Period period = new Period(millis());
return defaultFormatter.print(period);
}
public String format(PeriodType type) {
Period period = new Period(millis());
return PeriodFormat.getDefault().withParseType(type).print(period);
}
@Override public String toString() {
long nanos = nanos();
if (nanos == 0) {

View File

@ -19,6 +19,7 @@
package org.elasticsearch.util;
import org.joda.time.PeriodType;
import org.testng.annotations.Test;
import java.util.concurrent.TimeUnit;
@ -27,7 +28,7 @@ import static org.hamcrest.MatcherAssert.*;
import static org.hamcrest.Matchers.*;
/**
* @author kimchy (Shay Banon)
* @author kimchy (shay.banon)
*/
public class TimeValueTests {
@ -48,4 +49,11 @@ public class TimeValueTests {
assertThat("1.5d", equalTo(new TimeValue(36, TimeUnit.HOURS).toString()));
assertThat("1000d", equalTo(new TimeValue(1000, TimeUnit.DAYS).toString()));
}
@Test public void testFormat() {
assertThat(new TimeValue(1025, TimeUnit.MILLISECONDS).format(PeriodType.dayTime()), equalTo("1 second and 25 milliseconds"));
assertThat(new TimeValue(1, TimeUnit.MINUTES).format(PeriodType.dayTime()), equalTo("1 minute"));
assertThat(new TimeValue(65, TimeUnit.MINUTES).format(PeriodType.dayTime()), equalTo("1 hour and 5 minutes"));
assertThat(new TimeValue(24 * 600 + 85, TimeUnit.MINUTES).format(PeriodType.dayTime()), equalTo("241 hours and 25 minutes"));
}
}

View File

@ -28,6 +28,8 @@ import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartRequest
import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartResponse
import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownRequest
import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownResponse
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse
import org.elasticsearch.client.ClusterAdminClient
@ -111,6 +113,26 @@ class GClusterAdminClient {
clusterAdminClient.nodesInfo(request, listener)
}
// NODES STATS
GActionFuture<NodesStatsResponse> nodesStats(Closure c) {
NodesStatsRequest request = new NodesStatsRequest()
c.setDelegate request
c.resolveStrategy = gClient.resolveStrategy
c.call()
nodesStats(request)
}
GActionFuture<NodesStatsResponse> nodesStats(NodesStatsRequest request) {
GActionFuture<NodesStatsResponse> future = new GActionFuture<NodesStatsResponse>(internalClient.threadPool(), request);
clusterAdminClient.nodesStats(request, future)
return future
}
void nodesStats(NodesStatsRequest request, ActionListener<NodesStatsResponse> listener) {
clusterAdminClient.nodesStats(request, listener)
}
// NODES SHUTDOWN
GActionFuture<NodesShutdownResponse> nodesShutdown(Closure c) {