Răsfoiți Sursa

[CORE] Remove Restart API and remove Node#stop()

The restart API seems pretty esoteric and is entirely untested.
This commit removes it without a replacement.

Closes #9841
Simon Willnauer 10 ani în urmă
părinte
comite
ce53e20351

+ 0 - 3
src/main/java/org/elasticsearch/action/ActionModule.java

@@ -28,8 +28,6 @@ import org.elasticsearch.action.admin.cluster.node.hotthreads.TransportNodesHotT
 import org.elasticsearch.action.admin.cluster.node.info.NodesInfoAction;
 import org.elasticsearch.action.admin.cluster.node.info.TransportNodesInfoAction;
 import org.elasticsearch.action.admin.cluster.node.liveness.TransportLivenessAction;
-import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartAction;
-import org.elasticsearch.action.admin.cluster.node.restart.TransportNodesRestartAction;
 import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownAction;
 import org.elasticsearch.action.admin.cluster.node.shutdown.TransportNodesShutdownAction;
 import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsAction;
@@ -228,7 +226,6 @@ public class ActionModule extends AbstractModule {
         registerAction(NodesInfoAction.INSTANCE, TransportNodesInfoAction.class);
         registerAction(NodesStatsAction.INSTANCE, TransportNodesStatsAction.class);
         registerAction(NodesShutdownAction.INSTANCE, TransportNodesShutdownAction.class);
-        registerAction(NodesRestartAction.INSTANCE, TransportNodesRestartAction.class);
         registerAction(NodesHotThreadsAction.INSTANCE, TransportNodesHotThreadsAction.class);
 
         registerAction(ClusterStatsAction.INSTANCE, TransportClusterStatsAction.class);

+ 0 - 45
src/main/java/org/elasticsearch/action/admin/cluster/node/restart/NodesRestartAction.java

@@ -1,45 +0,0 @@
-/*
- * Licensed to Elasticsearch 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.action.admin.cluster.node.restart;
-
-import org.elasticsearch.action.admin.cluster.ClusterAction;
-import org.elasticsearch.client.ClusterAdminClient;
-
-/**
- */
-public class NodesRestartAction extends ClusterAction<NodesRestartRequest, NodesRestartResponse, NodesRestartRequestBuilder> {
-
-    public static final NodesRestartAction INSTANCE = new NodesRestartAction();
-    public static final String NAME = "cluster:admin/nodes/restart";
-
-    private NodesRestartAction() {
-        super(NAME);
-    }
-
-    @Override
-    public NodesRestartResponse newResponse() {
-        return new NodesRestartResponse();
-    }
-
-    @Override
-    public NodesRestartRequestBuilder newRequestBuilder(ClusterAdminClient client) {
-        return new NodesRestartRequestBuilder(client);
-    }
-}

+ 0 - 79
src/main/java/org/elasticsearch/action/admin/cluster/node/restart/NodesRestartRequest.java

@@ -1,79 +0,0 @@
-/*
- * Licensed to Elasticsearch 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.action.admin.cluster.node.restart;
-
-import org.elasticsearch.action.support.nodes.NodesOperationRequest;
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.io.stream.StreamOutput;
-import org.elasticsearch.common.unit.TimeValue;
-
-import java.io.IOException;
-
-import static org.elasticsearch.common.unit.TimeValue.readTimeValue;
-
-/**
- * A request to restart one ore more nodes (or the whole cluster).
- */
-public class NodesRestartRequest extends NodesOperationRequest<NodesRestartRequest> {
-
-    TimeValue delay = TimeValue.timeValueSeconds(1);
-
-    protected NodesRestartRequest() {
-    }
-
-    /**
-     * Restarts down nodes based on the nodes ids specified. If none are passed, <b>all</b>
-     * nodes will be shutdown.
-     */
-    public NodesRestartRequest(String... nodesIds) {
-        super(nodesIds);
-    }
-
-    /**
-     * The delay for the restart to occur. Defaults to <tt>1s</tt>.
-     */
-    public NodesRestartRequest delay(TimeValue delay) {
-        this.delay = delay;
-        return this;
-    }
-
-    /**
-     * The delay for the restart to occur. Defaults to <tt>1s</tt>.
-     */
-    public NodesRestartRequest delay(String delay) {
-        return delay(TimeValue.parseTimeValue(delay, null));
-    }
-
-    public TimeValue delay() {
-        return this.delay;
-    }
-
-    @Override
-    public void readFrom(StreamInput in) throws IOException {
-        super.readFrom(in);
-        delay = readTimeValue(in);
-    }
-
-    @Override
-    public void writeTo(StreamOutput out) throws IOException {
-        super.writeTo(out);
-        delay.writeTo(out);
-    }
-}

+ 0 - 56
src/main/java/org/elasticsearch/action/admin/cluster/node/restart/NodesRestartRequestBuilder.java

@@ -1,56 +0,0 @@
-/*
- * Licensed to Elasticsearch 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.action.admin.cluster.node.restart;
-
-import org.elasticsearch.action.ActionListener;
-import org.elasticsearch.action.support.nodes.NodesOperationRequestBuilder;
-import org.elasticsearch.client.ClusterAdminClient;
-import org.elasticsearch.common.unit.TimeValue;
-
-/**
- *
- */
-public class NodesRestartRequestBuilder extends NodesOperationRequestBuilder<NodesRestartRequest, NodesRestartResponse, NodesRestartRequestBuilder> {
-
-    public NodesRestartRequestBuilder(ClusterAdminClient clusterClient) {
-        super(clusterClient, new NodesRestartRequest());
-    }
-
-    /**
-     * The delay for the restart to occur. Defaults to <tt>1s</tt>.
-     */
-    public NodesRestartRequestBuilder setDelay(TimeValue delay) {
-        request.delay(delay);
-        return this;
-    }
-
-    /**
-     * The delay for the restart to occur. Defaults to <tt>1s</tt>.
-     */
-    public NodesRestartRequestBuilder setDelay(String delay) {
-        request.delay(delay);
-        return this;
-    }
-
-    @Override
-    protected void doExecute(ActionListener<NodesRestartResponse> listener) {
-        client.nodesRestart(request, listener);
-    }
-}

+ 0 - 76
src/main/java/org/elasticsearch/action/admin/cluster/node/restart/NodesRestartResponse.java

@@ -1,76 +0,0 @@
-/*
- * Licensed to Elasticsearch 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.action.admin.cluster.node.restart;
-
-import org.elasticsearch.action.support.nodes.NodeOperationResponse;
-import org.elasticsearch.action.support.nodes.NodesOperationResponse;
-import org.elasticsearch.cluster.ClusterName;
-import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.io.stream.StreamOutput;
-
-import java.io.IOException;
-
-/**
- *
- */
-public class NodesRestartResponse extends NodesOperationResponse<NodesRestartResponse.NodeRestartResponse> {
-
-    NodesRestartResponse() {
-    }
-
-    public NodesRestartResponse(ClusterName clusterName, NodeRestartResponse[] nodes) {
-        super(clusterName, nodes);
-    }
-
-    @Override
-    public void readFrom(StreamInput in) throws IOException {
-        super.readFrom(in);
-        nodes = new NodeRestartResponse[in.readVInt()];
-        for (int i = 0; i < nodes.length; i++) {
-            nodes[i] = NodeRestartResponse.readNodeRestartResponse(in);
-        }
-    }
-
-    @Override
-    public void writeTo(StreamOutput out) throws IOException {
-        super.writeTo(out);
-        out.writeVInt(nodes.length);
-        for (NodeRestartResponse node : nodes) {
-            node.writeTo(out);
-        }
-    }
-
-    public static class NodeRestartResponse extends NodeOperationResponse {
-
-        NodeRestartResponse() {
-        }
-
-        public NodeRestartResponse(DiscoveryNode node) {
-            super(node);
-        }
-
-        public static NodeRestartResponse readNodeRestartResponse(StreamInput in) throws IOException {
-            NodeRestartResponse res = new NodeRestartResponse();
-            res.readFrom(in);
-            return res;
-        }
-    }
-}

+ 0 - 177
src/main/java/org/elasticsearch/action/admin/cluster/node/restart/TransportNodesRestartAction.java

@@ -1,177 +0,0 @@
-/*
- * Licensed to Elasticsearch 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.action.admin.cluster.node.restart;
-
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.ElasticsearchIllegalStateException;
-import org.elasticsearch.action.ActionListener;
-import org.elasticsearch.action.support.ActionFilters;
-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.common.inject.Inject;
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.io.stream.StreamOutput;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.unit.TimeValue;
-import org.elasticsearch.node.Node;
-import org.elasticsearch.threadpool.ThreadPool;
-import org.elasticsearch.transport.TransportService;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReferenceArray;
-
-import static com.google.common.collect.Lists.newArrayList;
-import static org.elasticsearch.common.unit.TimeValue.readTimeValue;
-
-/**
- *
- */
-public class TransportNodesRestartAction extends TransportNodesOperationAction<NodesRestartRequest, NodesRestartResponse, TransportNodesRestartAction.NodeRestartRequest, NodesRestartResponse.NodeRestartResponse> {
-
-    private final Node node;
-
-    private final boolean disabled;
-
-    private AtomicBoolean restartRequested = new AtomicBoolean();
-
-    @Inject
-    public TransportNodesRestartAction(Settings settings, ClusterName clusterName, ThreadPool threadPool,
-                                       ClusterService clusterService, TransportService transportService,
-                                       Node node, ActionFilters actionFilters) {
-        super(settings, NodesRestartAction.NAME, clusterName, threadPool, clusterService, transportService, actionFilters);
-        this.node = node;
-        disabled = this.settings.getAsBoolean("admin.cluster.node.restart.disabled", false);
-    }
-
-    @Override
-    protected void doExecute(NodesRestartRequest nodesRestartRequest, ActionListener<NodesRestartResponse> listener) {
-        listener.onFailure(new ElasticsearchIllegalStateException("restart is disabled (for now) ...."));
-    }
-
-    @Override
-    protected String executor() {
-        return ThreadPool.Names.GENERIC;
-    }
-
-    @Override
-    protected NodesRestartResponse newResponse(NodesRestartRequest nodesShutdownRequest, AtomicReferenceArray responses) {
-        final List<NodesRestartResponse.NodeRestartResponse> nodeRestartResponses = newArrayList();
-        for (int i = 0; i < responses.length(); i++) {
-            Object resp = responses.get(i);
-            if (resp instanceof NodesRestartResponse.NodeRestartResponse) {
-                nodeRestartResponses.add((NodesRestartResponse.NodeRestartResponse) resp);
-            }
-        }
-        return new NodesRestartResponse(clusterName, nodeRestartResponses.toArray(new NodesRestartResponse.NodeRestartResponse[nodeRestartResponses.size()]));
-    }
-
-    @Override
-    protected NodesRestartRequest newRequest() {
-        return new NodesRestartRequest();
-    }
-
-    @Override
-    protected NodeRestartRequest newNodeRequest() {
-        return new NodeRestartRequest();
-    }
-
-    @Override
-    protected NodeRestartRequest newNodeRequest(String nodeId, NodesRestartRequest request) {
-        return new NodeRestartRequest(nodeId, request);
-    }
-
-    @Override
-    protected NodesRestartResponse.NodeRestartResponse newNodeResponse() {
-        return new NodesRestartResponse.NodeRestartResponse();
-    }
-
-    @Override
-    protected NodesRestartResponse.NodeRestartResponse nodeOperation(NodeRestartRequest request) throws ElasticsearchException {
-        if (disabled) {
-            throw new ElasticsearchIllegalStateException("Restart is disabled");
-        }
-        if (!restartRequested.compareAndSet(false, true)) {
-            return new NodesRestartResponse.NodeRestartResponse(clusterService.localNode());
-        }
-        logger.info("Restarting in [{}]", request.delay);
-        threadPool.schedule(request.delay, ThreadPool.Names.GENERIC, new Runnable() {
-            @Override
-            public void run() {
-                boolean restartWithWrapper = false;
-                if (System.getProperty("elasticsearch-service") != null) {
-                    try {
-                        Class wrapperManager = settings.getClassLoader().loadClass("org.tanukisoftware.wrapper.WrapperManager");
-                        logger.info("Initiating requested restart (using service)");
-                        wrapperManager.getMethod("restartAndReturn").invoke(null);
-                        restartWithWrapper = true;
-                    } catch (Throwable e) {
-                        logger.error("failed to initial restart on service wrapper", e);
-                    }
-                }
-                if (!restartWithWrapper) {
-                    logger.info("Initiating requested restart");
-                    try {
-                        node.stop();
-                        node.start();
-                    } catch (Exception e) {
-                        logger.warn("Failed to restart", e);
-                    } finally {
-                        restartRequested.set(false);
-                    }
-                }
-            }
-        });
-        return new NodesRestartResponse.NodeRestartResponse(clusterService.localNode());
-    }
-
-    @Override
-    protected boolean accumulateExceptions() {
-        return false;
-    }
-
-    protected static class NodeRestartRequest extends NodeOperationRequest {
-
-        TimeValue delay;
-
-        private NodeRestartRequest() {
-        }
-
-        private NodeRestartRequest(String nodeId, NodesRestartRequest request) {
-            super(request, nodeId);
-            this.delay = request.delay;
-        }
-
-        @Override
-        public void readFrom(StreamInput in) throws IOException {
-            super.readFrom(in);
-            delay = readTimeValue(in);
-        }
-
-        @Override
-        public void writeTo(StreamOutput out) throws IOException {
-            super.writeTo(out);
-            delay.writeTo(out);
-        }
-    }
-}

+ 1 - 1
src/main/java/org/elasticsearch/bootstrap/Bootstrap.java

@@ -128,7 +128,7 @@ public class Bootstrap {
      * hook for JSVC
      */
     public void stop() {
-        node.stop();
+       destroy();
     }
 
 

+ 0 - 26
src/main/java/org/elasticsearch/client/ClusterAdminClient.java

@@ -29,9 +29,6 @@ import org.elasticsearch.action.admin.cluster.node.hotthreads.NodesHotThreadsRes
 import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest;
 import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequestBuilder;
 import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse;
-import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartRequest;
-import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartRequestBuilder;
-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.NodesShutdownRequestBuilder;
 import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownResponse;
@@ -274,29 +271,6 @@ public interface ClusterAdminClient extends ElasticsearchClient<ClusterAdminClie
      */
     NodesShutdownRequestBuilder prepareNodesShutdown(String... nodesIds);
 
-    /**
-     * Restarts nodes in the cluster.
-     *
-     * @param request The nodes restart request
-     * @return The result future
-     * @see org.elasticsearch.client.Requests#nodesRestartRequest(String...)
-     */
-    ActionFuture<NodesRestartResponse> nodesRestart(NodesRestartRequest request);
-
-    /**
-     * Restarts nodes in the cluster.
-     *
-     * @param request  The nodes restart request
-     * @param listener A listener to be notified with a result
-     * @see org.elasticsearch.client.Requests#nodesRestartRequest(String...)
-     */
-    void nodesRestart(NodesRestartRequest request, ActionListener<NodesRestartResponse> listener);
-
-    /**
-     * Restarts nodes in the cluster.
-     */
-    NodesRestartRequestBuilder prepareNodesRestart(String... nodesIds);
-
     /**
      * Returns list of shards the given search would be executed on.
      */

+ 0 - 19
src/main/java/org/elasticsearch/client/Requests.java

@@ -21,7 +21,6 @@ package org.elasticsearch.client;
 
 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.repositories.delete.DeleteRepositoryRequest;
@@ -454,24 +453,6 @@ public class Requests {
         return new NodesShutdownRequest(nodesIds);
     }
 
-    /**
-     * Restarts all nodes in the cluster.
-     */
-    public static NodesRestartRequest nodesRestartRequest() {
-        return new NodesRestartRequest();
-    }
-
-    /**
-     * Restarts specific nodes in the cluster.
-     *
-     * @param nodesIds The nodes ids to restart
-     * @return The nodes info request
-     * @see org.elasticsearch.client.ClusterAdminClient#nodesRestart(org.elasticsearch.action.admin.cluster.node.restart.NodesRestartRequest)
-     */
-    public static NodesRestartRequest nodesRestartRequest(String... nodesIds) {
-        return new NodesRestartRequest(nodesIds);
-    }
-
     /**
      * Registers snapshot repository
      *

+ 0 - 19
src/main/java/org/elasticsearch/client/support/AbstractClusterAdminClient.java

@@ -33,10 +33,6 @@ import org.elasticsearch.action.admin.cluster.node.info.NodesInfoAction;
 import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest;
 import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequestBuilder;
 import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse;
-import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartAction;
-import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartRequest;
-import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartRequestBuilder;
-import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartResponse;
 import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownAction;
 import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownRequest;
 import org.elasticsearch.action.admin.cluster.node.shutdown.NodesShutdownRequestBuilder;
@@ -237,21 +233,6 @@ public abstract class AbstractClusterAdminClient implements ClusterAdminClient {
         return new NodesHotThreadsRequestBuilder(this).setNodesIds(nodesIds);
     }
 
-    @Override
-    public ActionFuture<NodesRestartResponse> nodesRestart(final NodesRestartRequest request) {
-        return execute(NodesRestartAction.INSTANCE, request);
-    }
-
-    @Override
-    public void nodesRestart(final NodesRestartRequest request, final ActionListener<NodesRestartResponse> listener) {
-        execute(NodesRestartAction.INSTANCE, request, listener);
-    }
-
-    @Override
-    public NodesRestartRequestBuilder prepareNodesRestart(String... nodesIds) {
-        return new NodesRestartRequestBuilder(this).setNodesIds(nodesIds);
-    }
-
     @Override
     public ActionFuture<NodesShutdownResponse> nodesShutdown(final NodesShutdownRequest request) {
         return execute(NodesShutdownAction.INSTANCE, request);

+ 1 - 1
src/main/java/org/elasticsearch/common/settings/ImmutableSettings.java

@@ -194,7 +194,7 @@ public class ImmutableSettings implements Settings {
         for (Map.Entry<String, String> entry : getAsMap().entrySet()) {
             if (entry.getKey().startsWith(prefix)) {
                 if (entry.getKey().length() < prefix.length()) {
-                    // ignore this one
+                    // ignore this. one
                     continue;
                 }
                 builder.put(entry.getKey().substring(prefix.length()), entry.getValue());

+ 1 - 1
src/main/java/org/elasticsearch/node/Node.java

@@ -274,7 +274,7 @@ public class Node implements Releasable {
         return this;
     }
 
-    public Node stop() {
+    private Node stop() {
         if (!lifecycle.moveToStopped()) {
             return this;
         }

+ 0 - 2
src/main/java/org/elasticsearch/rest/action/RestActionModule.java

@@ -28,7 +28,6 @@ import org.elasticsearch.rest.action.admin.cluster.repositories.verify.RestVerif
 import org.elasticsearch.rest.action.admin.cluster.health.RestClusterHealthAction;
 import org.elasticsearch.rest.action.admin.cluster.node.hotthreads.RestNodesHotThreadsAction;
 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.repositories.delete.RestDeleteRepositoryAction;
@@ -135,7 +134,6 @@ public class RestActionModule extends AbstractModule {
         bind(RestNodesStatsAction.class).asEagerSingleton();
         bind(RestNodesHotThreadsAction.class).asEagerSingleton();
         bind(RestNodesShutdownAction.class).asEagerSingleton();
-        bind(RestNodesRestartAction.class).asEagerSingleton();
         bind(RestClusterStatsAction.class).asEagerSingleton();
         bind(RestClusterStateAction.class).asEagerSingleton();
         bind(RestClusterHealthAction.class).asEagerSingleton();

+ 0 - 70
src/main/java/org/elasticsearch/rest/action/admin/cluster/node/restart/RestNodesRestartAction.java

@@ -1,70 +0,0 @@
-/*
- * Licensed to Elasticsearch 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.admin.cluster.node.restart;
-
-import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartRequest;
-import org.elasticsearch.action.admin.cluster.node.restart.NodesRestartResponse;
-import org.elasticsearch.client.Client;
-import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.rest.*;
-import org.elasticsearch.rest.action.support.RestBuilderListener;
-
-/**
- *
- */
-public class RestNodesRestartAction extends BaseRestHandler {
-
-    @Inject
-    public RestNodesRestartAction(Settings settings, RestController controller, Client client) {
-        super(settings, controller, client);
-
-        controller.registerHandler(RestRequest.Method.POST, "/_cluster/nodes/_restart", this);
-        controller.registerHandler(RestRequest.Method.POST, "/_cluster/nodes/{nodeId}/_restart", this);
-    }
-
-    @Override
-    public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) {
-        String[] nodesIds = Strings.splitStringByCommaToArray(request.param("nodeId"));
-        NodesRestartRequest nodesRestartRequest = new NodesRestartRequest(nodesIds);
-        nodesRestartRequest.listenerThreaded(false);
-        nodesRestartRequest.delay(request.paramAsTime("delay", nodesRestartRequest.delay()));
-        client.admin().cluster().nodesRestart(nodesRestartRequest, new RestBuilderListener<NodesRestartResponse>(channel) {
-            @Override
-            public RestResponse buildResponse(NodesRestartResponse result, XContentBuilder builder) throws Exception {
-                builder.startObject();
-                builder.field("cluster_name", result.getClusterName().value());
-
-                builder.startObject("nodes");
-                for (NodesRestartResponse.NodeRestartResponse nodeInfo : result) {
-                    builder.startObject(nodeInfo.getNode().id());
-                    builder.field("name", nodeInfo.getNode().name());
-                    builder.endObject();
-                }
-                builder.endObject();
-
-                builder.endObject();
-                return new BytesRestResponse(RestStatus.OK, builder);
-            }
-        });
-    }
-}

+ 1 - 9
src/main/java/org/elasticsearch/tribe/TribeService.java

@@ -186,13 +186,7 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
 
     @Override
     protected void doStop() throws ElasticsearchException {
-        for (Node node : nodes) {
-            try {
-                node.stop();
-            } catch (Throwable t) {
-                logger.warn("failed to stop node {}", t, node);
-            }
-        }
+        doClose();
     }
 
     @Override
@@ -208,11 +202,9 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
 
     class TribeClusterStateListener implements ClusterStateListener {
 
-        private final Node tribeNode;
         private final String tribeName;
 
         TribeClusterStateListener(Node tribeNode) {
-            this.tribeNode = tribeNode;
             this.tribeName = tribeNode.settings().get(TRIBE_NAME);
         }
 

+ 1 - 1
src/test/java/org/elasticsearch/benchmark/breaker/CircuitBreakerBenchmark.java

@@ -185,7 +185,7 @@ public class CircuitBreakerBenchmark {
             runMultiThreadedQueries(client);
         } finally {
             client.close();
-            node.stop();
+            node.close();
         }
     }
 }