1
0
Эх сурвалжийг харах

Remove Gateway abstraction

We only have a single gatweway since es 1.3. There is no need to keep all
these abstractsion and nested packages. We can fold most of it into simpler
structures.
Simon Willnauer 11 жил өмнө
parent
commit
1247774ff1
40 өөрчлөгдсөн 614 нэмэгдсэн , 1072 устгасан
  1. 2 2
      docs/reference/modules/gateway/local.asciidoc
  2. 0 40
      src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/GatewayAllocator.java
  3. 2 10
      src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/ShardsAllocatorModule.java
  4. 1 0
      src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/ShardsAllocators.java
  5. 1 1
      src/main/java/org/elasticsearch/gateway/CorruptStateException.java
  6. 154 11
      src/main/java/org/elasticsearch/gateway/Gateway.java
  7. 8 13
      src/main/java/org/elasticsearch/gateway/GatewayAllocator.java
  8. 13 9
      src/main/java/org/elasticsearch/gateway/GatewayMetaState.java
  9. 8 22
      src/main/java/org/elasticsearch/gateway/GatewayModule.java
  10. 0 3
      src/main/java/org/elasticsearch/gateway/GatewayService.java
  11. 3 5
      src/main/java/org/elasticsearch/gateway/GatewayShardsState.java
  12. 1 1
      src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java
  13. 2 3
      src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java
  14. 1 1
      src/main/java/org/elasticsearch/gateway/ShardStateInfo.java
  15. 23 23
      src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java
  16. 24 28
      src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayStartedShards.java
  17. 0 215
      src/main/java/org/elasticsearch/gateway/local/LocalGateway.java
  18. 0 54
      src/main/java/org/elasticsearch/gateway/local/LocalGatewayModule.java
  19. 2 10
      src/main/java/org/elasticsearch/index/IndexService.java
  20. 0 35
      src/main/java/org/elasticsearch/index/gateway/IndexGateway.java
  21. 0 52
      src/main/java/org/elasticsearch/index/gateway/IndexGatewayModule.java
  22. 326 11
      src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java
  23. 1 9
      src/main/java/org/elasticsearch/index/gateway/IndexShardGatewayModule.java
  24. 0 58
      src/main/java/org/elasticsearch/index/gateway/local/LocalIndexGateway.java
  25. 0 34
      src/main/java/org/elasticsearch/index/gateway/local/LocalIndexGatewayModule.java
  26. 0 377
      src/main/java/org/elasticsearch/index/gateway/local/LocalIndexShardGateway.java
  27. 2 2
      src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java
  28. 0 5
      src/main/java/org/elasticsearch/indices/IndicesService.java
  29. 1 1
      src/main/java/org/elasticsearch/node/internal/InternalNode.java
  30. 5 7
      src/test/java/org/elasticsearch/gateway/GatewayIndexStateTests.java
  31. 4 4
      src/test/java/org/elasticsearch/gateway/GatewayShardStateTests.java
  32. 13 14
      src/test/java/org/elasticsearch/gateway/MetaDataStateFormatTest.java
  33. 2 2
      src/test/java/org/elasticsearch/gateway/QuorumGatewayTests.java
  34. 1 1
      src/test/java/org/elasticsearch/gateway/RecoverAfterNodesTests.java
  35. 1 1
      src/test/java/org/elasticsearch/gateway/RecoveryBackwardsCompatibilityTests.java
  36. 2 2
      src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java
  37. 2 2
      src/test/java/org/elasticsearch/indices/state/RareClusterStateTests.java
  38. 2 2
      src/test/java/org/elasticsearch/indices/warmer/GatewayIndicesWarmerTests.java
  39. 7 2
      src/test/java/org/elasticsearch/test/gateway/NoopGatewayAllocator.java
  40. 0 0
      src/test/resources/org/elasticsearch/gateway/global-3.st

+ 2 - 2
docs/reference/modules/gateway/local.asciidoc

@@ -39,7 +39,7 @@ cluster by default.  This functionality has two purposes:
    cluster, the new node can be shut down and removed.
 
 The import of dangling indices can be controlled with the
-`gateway.local.auto_import_dangled` which accepts:
+`gateway.auto_import_dangled` which accepts:
 
 [horizontal]
 `yes`::
@@ -52,5 +52,5 @@ The import of dangling indices can be controlled with the
 
 `no`::
 
-    Delete dangling indices after `gateway.local.dangling_timeout`, which
+    Delete dangling indices after `gateway.dangling_timeout`, which
     defaults to 2 hours.

+ 0 - 40
src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/GatewayAllocator.java

@@ -1,40 +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.cluster.routing.allocation.allocator;
-
-import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation;
-import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
-import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation;
-
-/**
- * The gateway allocator allows for a pluggable control of the gateway to allocate unassigned shards.
- */
-public interface GatewayAllocator {
-
-    /**
-     * Apply all shards  
-     * @param allocation
-     */
-    void applyStartedShards(StartedRerouteAllocation allocation);
-
-    void applyFailedShards(FailedRerouteAllocation allocation);
-
-    boolean allocateUnassigned(RoutingAllocation allocation);
-}

+ 2 - 10
src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/ShardsAllocatorModule.java

@@ -21,7 +21,7 @@ package org.elasticsearch.cluster.routing.allocation.allocator;
 
 import org.elasticsearch.common.inject.AbstractModule;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.gateway.local.LocalGatewayAllocator;
+import org.elasticsearch.gateway.GatewayAllocator;
 
 /**
  */
@@ -37,27 +37,19 @@ public class ShardsAllocatorModule extends AbstractModule {
 
     private Class<? extends ShardsAllocator> shardsAllocator;
 
-    private Class<? extends GatewayAllocator> gatewayAllocator = LocalGatewayAllocator.class;
 
     public ShardsAllocatorModule(Settings settings) {
         this.settings = settings;
         shardsAllocator = loadShardsAllocator(settings);
     }
 
-    public void setGatewayAllocator(Class<? extends GatewayAllocator> gatewayAllocator) {
-        this.gatewayAllocator = gatewayAllocator;
-    }
-
-    public void setShardsAllocator(Class<? extends ShardsAllocator> shardsAllocator) {
-        this.shardsAllocator = shardsAllocator;
-    }
 
     @Override
     protected void configure() {
         if (shardsAllocator == null) {
             shardsAllocator = loadShardsAllocator(settings);
         }
-        bind(GatewayAllocator.class).to(gatewayAllocator).asEagerSingleton();
+        bind(GatewayAllocator.class).asEagerSingleton();
         bind(ShardsAllocator.class).to(shardsAllocator).asEagerSingleton();
     }
 

+ 1 - 0
src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/ShardsAllocators.java

@@ -28,6 +28,7 @@ import org.elasticsearch.common.component.AbstractComponent;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.gateway.GatewayAllocator;
 
 /**
  * The {@link ShardsAllocator} class offers methods for allocating shard within a cluster.

+ 1 - 1
src/main/java/org/elasticsearch/gateway/local/state/meta/CorruptStateException.java → src/main/java/org/elasticsearch/gateway/CorruptStateException.java

@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.elasticsearch.gateway.local.state.meta;
+package org.elasticsearch.gateway;
 
 import org.elasticsearch.ElasticsearchCorruptionException;
 

+ 154 - 11
src/main/java/org/elasticsearch/gateway/Gateway.java

@@ -19,26 +19,169 @@
 
 package org.elasticsearch.gateway;
 
-import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.common.component.LifecycleComponent;
-import org.elasticsearch.common.inject.Module;
+import com.carrotsearch.hppc.ObjectFloatOpenHashMap;
+import com.carrotsearch.hppc.ObjectOpenHashSet;
+import com.carrotsearch.hppc.cursors.ObjectCursor;
+import org.apache.lucene.util.IOUtils;
+import org.elasticsearch.action.FailedNodeException;
+import org.elasticsearch.cluster.*;
+import org.elasticsearch.cluster.metadata.IndexMetaData;
+import org.elasticsearch.cluster.metadata.MetaData;
+import org.elasticsearch.common.component.AbstractComponent;
+import org.elasticsearch.common.inject.Inject;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.env.NodeEnvironment;
 
 /**
  *
  */
-public interface Gateway extends LifecycleComponent<Gateway> {
+public class Gateway extends AbstractComponent implements ClusterStateListener {
 
-    String type();
+    private final ClusterService clusterService;
 
-    void performStateRecovery(GatewayStateRecoveredListener listener) throws GatewayException;
+    private final NodeEnvironment nodeEnv;
 
-    Class<? extends Module> suggestIndexGateway();
+    private final GatewayShardsState shardsState;
+    private final GatewayMetaState metaState;
 
-    void reset() throws Exception;
+    private final TransportNodesListGatewayMetaState listGatewayMetaState;
 
-    interface GatewayStateRecoveredListener {
-        void onSuccess(ClusterState recoveredState);
+    private final String initialMeta;
+    private final ClusterName clusterName;
 
-        void onFailure(String message);
+    @Inject
+    public Gateway(Settings settings, ClusterService clusterService, NodeEnvironment nodeEnv,
+                   GatewayShardsState shardsState, GatewayMetaState metaState,
+                   TransportNodesListGatewayMetaState listGatewayMetaState, ClusterName clusterName) {
+        super(settings);
+        this.clusterService = clusterService;
+        this.nodeEnv = nodeEnv;
+        this.metaState = metaState;
+        this.listGatewayMetaState = listGatewayMetaState;
+        this.clusterName = clusterName;
+
+        this.shardsState = shardsState;
+
+        clusterService.addLast(this);
+
+        // we define what is our minimum "master" nodes, use that to allow for recovery
+        this.initialMeta = settings.get("gateway.initial_meta", settings.get("gateway.local.initial_meta", settings.get("discovery.zen.minimum_master_nodes", "1")));
+    }
+
+    public void performStateRecovery(final GatewayStateRecoveredListener listener) throws GatewayException {
+        ObjectOpenHashSet<String> nodesIds = ObjectOpenHashSet.from(clusterService.state().nodes().masterNodes().keys());
+        logger.trace("performing state recovery from {}", nodesIds);
+        TransportNodesListGatewayMetaState.NodesGatewayMetaState nodesState = listGatewayMetaState.list(nodesIds.toArray(String.class), null).actionGet();
+
+
+        int requiredAllocation = 1;
+        try {
+            if ("quorum".equals(initialMeta)) {
+                if (nodesIds.size() > 2) {
+                    requiredAllocation = (nodesIds.size() / 2) + 1;
+                }
+            } else if ("quorum-1".equals(initialMeta) || "half".equals(initialMeta)) {
+                if (nodesIds.size() > 2) {
+                    requiredAllocation = ((1 + nodesIds.size()) / 2);
+                }
+            } else if ("one".equals(initialMeta)) {
+                requiredAllocation = 1;
+            } else if ("full".equals(initialMeta) || "all".equals(initialMeta)) {
+                requiredAllocation = nodesIds.size();
+            } else if ("full-1".equals(initialMeta) || "all-1".equals(initialMeta)) {
+                if (nodesIds.size() > 1) {
+                    requiredAllocation = nodesIds.size() - 1;
+                }
+            } else {
+                requiredAllocation = Integer.parseInt(initialMeta);
+            }
+        } catch (Exception e) {
+            logger.warn("failed to derived initial_meta from value {}", initialMeta);
+        }
+
+        if (nodesState.failures().length > 0) {
+            for (FailedNodeException failedNodeException : nodesState.failures()) {
+                logger.warn("failed to fetch state from node", failedNodeException);
+            }
+        }
+
+        ObjectFloatOpenHashMap<String> indices = new ObjectFloatOpenHashMap<>();
+        MetaData electedGlobalState = null;
+        int found = 0;
+        for (TransportNodesListGatewayMetaState.NodeGatewayMetaState nodeState : nodesState) {
+            if (nodeState.metaData() == null) {
+                continue;
+            }
+            found++;
+            if (electedGlobalState == null) {
+                electedGlobalState = nodeState.metaData();
+            } else if (nodeState.metaData().version() > electedGlobalState.version()) {
+                electedGlobalState = nodeState.metaData();
+            }
+            for (ObjectCursor<IndexMetaData> cursor : nodeState.metaData().indices().values()) {
+                indices.addTo(cursor.value.index(), 1);
+            }
+        }
+        if (found < requiredAllocation) {
+            listener.onFailure("found [" + found + "] metadata states, required [" + requiredAllocation + "]");
+            return;
+        }
+        // update the global state, and clean the indices, we elect them in the next phase
+        MetaData.Builder metaDataBuilder = MetaData.builder(electedGlobalState).removeAllIndices();
+        final boolean[] states = indices.allocated;
+        final Object[] keys = indices.keys;
+        for (int i = 0; i < states.length; i++) {
+            if (states[i]) {
+                String index = (String) keys[i];
+                IndexMetaData electedIndexMetaData = null;
+                int indexMetaDataCount = 0;
+                for (TransportNodesListGatewayMetaState.NodeGatewayMetaState nodeState : nodesState) {
+                    if (nodeState.metaData() == null) {
+                        continue;
+                    }
+                    IndexMetaData indexMetaData = nodeState.metaData().index(index);
+                    if (indexMetaData == null) {
+                        continue;
+                    }
+                    if (electedIndexMetaData == null) {
+                        electedIndexMetaData = indexMetaData;
+                    } else if (indexMetaData.version() > electedIndexMetaData.version()) {
+                        electedIndexMetaData = indexMetaData;
+                    }
+                    indexMetaDataCount++;
+                }
+                if (electedIndexMetaData != null) {
+                    if (indexMetaDataCount < requiredAllocation) {
+                        logger.debug("[{}] found [{}], required [{}], not adding", index, indexMetaDataCount, requiredAllocation);
+                    }
+                    metaDataBuilder.put(electedIndexMetaData, false);
+                }
+            }
+        }
+        ClusterState.Builder builder = ClusterState.builder(clusterName);
+        builder.metaData(metaDataBuilder);
+        listener.onSuccess(builder.build());
+    }
+
+    public void reset() throws Exception {
+        try {
+            IOUtils.rm(nodeEnv.nodeDataPaths());
+        } catch (Exception ex) {
+            logger.debug("failed to delete shard locations", ex);
+        }
+    }
+
+    @Override
+    public void clusterChanged(final ClusterChangedEvent event) {
+        // order is important, first metaState, and then shardsState
+        // so dangling indices will be recorded
+        metaState.clusterChanged(event);
+        shardsState.clusterChanged(event);
+    }
+
+    public interface GatewayStateRecoveredListener {
+        void onSuccess(ClusterState build);
+
+        void onFailure(String s);
     }
 }

+ 8 - 13
src/main/java/org/elasticsearch/gateway/local/LocalGatewayAllocator.java → src/main/java/org/elasticsearch/gateway/GatewayAllocator.java

@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.elasticsearch.gateway.local;
+package org.elasticsearch.gateway;
 
 import com.carrotsearch.hppc.ObjectLongOpenHashMap;
 import com.carrotsearch.hppc.ObjectOpenHashSet;
@@ -36,7 +36,6 @@ import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation;
 import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
 import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation;
-import org.elasticsearch.cluster.routing.allocation.allocator.GatewayAllocator;
 import org.elasticsearch.cluster.routing.allocation.decider.Decision;
 import org.elasticsearch.common.component.AbstractComponent;
 import org.elasticsearch.common.inject.Inject;
@@ -44,7 +43,6 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
-import org.elasticsearch.gateway.local.state.shards.TransportNodesListGatewayStartedShards;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.store.StoreFileMetaData;
 import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData;
@@ -56,7 +54,7 @@ import java.util.concurrent.ConcurrentMap;
 /**
  *
  */
-public class LocalGatewayAllocator extends AbstractComponent implements GatewayAllocator {
+public class GatewayAllocator extends AbstractComponent {
 
     public static final String INDEX_RECOVERY_INITIAL_SHARDS = "index.recovery.initial_shards";
 
@@ -73,19 +71,18 @@ public class LocalGatewayAllocator extends AbstractComponent implements GatewayA
     private final String initialShards;
 
     @Inject
-    public LocalGatewayAllocator(Settings settings,
-                                 TransportNodesListGatewayStartedShards listGatewayStartedShards, TransportNodesListShardStoreMetaData listShardStoreMetaData) {
+    public GatewayAllocator(Settings settings,
+                            TransportNodesListGatewayStartedShards listGatewayStartedShards, TransportNodesListShardStoreMetaData listShardStoreMetaData) {
         super(settings);
         this.listGatewayStartedShards = listGatewayStartedShards;
         this.listShardStoreMetaData = listShardStoreMetaData;
 
-        this.listTimeout = componentSettings.getAsTime("list_timeout", TimeValue.timeValueSeconds(30));
-        this.initialShards = componentSettings.get("initial_shards", "quorum");
+        this.listTimeout = componentSettings.getAsTime("list_timeout", settings.getAsTime("gateway.local.list_timeout", TimeValue.timeValueSeconds(30)));
+        this.initialShards = componentSettings.get("initial_shards", settings.get("gateway.local.initial_shards", "quorum"));
 
         logger.debug("using initial_shards [{}], list_timeout [{}]", initialShards, listTimeout);
     }
 
-    @Override
     public void applyStartedShards(StartedRerouteAllocation allocation) {
         for (ShardRouting shardRouting : allocation.startedShards()) {
             cachedStores.remove(shardRouting.shardId());
@@ -93,7 +90,6 @@ public class LocalGatewayAllocator extends AbstractComponent implements GatewayA
         }
     }
 
-    @Override
     public void applyFailedShards(FailedRerouteAllocation allocation) {
         for (ShardRouting failedShard : allocation.failedShards()) {
             cachedStores.remove(failedShard.shardId());
@@ -101,7 +97,6 @@ public class LocalGatewayAllocator extends AbstractComponent implements GatewayA
         }
     }
 
-    @Override
     public boolean allocateUnassigned(RoutingAllocation allocation) {
         boolean changed = false;
         DiscoveryNodes nodes = allocation.nodes();
@@ -399,7 +394,7 @@ public class LocalGatewayAllocator extends AbstractComponent implements GatewayA
         }
 
         String[] nodesIdsArray = nodeIds.toArray(String.class);
-        TransportNodesListGatewayStartedShards.NodesLocalGatewayStartedShards response = listGatewayStartedShards.list(shard.shardId(), nodesIdsArray, listTimeout).actionGet();
+        TransportNodesListGatewayStartedShards.NodesGatewayStartedShards response = listGatewayStartedShards.list(shard.shardId(), nodesIdsArray, listTimeout).actionGet();
         if (logger.isDebugEnabled()) {
             if (response.failures().length > 0) {
                 StringBuilder sb = new StringBuilder(shard + ": failures when trying to list shards on nodes:");
@@ -414,7 +409,7 @@ public class LocalGatewayAllocator extends AbstractComponent implements GatewayA
             }
         }
 
-        for (TransportNodesListGatewayStartedShards.NodeLocalGatewayStartedShards nodeShardState : response) {
+        for (TransportNodesListGatewayStartedShards.NodeGatewayStartedShards nodeShardState : response) {
             // -1 version means it does not exists, which is what the API returns, and what we expect to
             shardStates.put(nodeShardState.getNode(), nodeShardState.version());
         }

+ 13 - 9
src/main/java/org/elasticsearch/gateway/local/state/meta/LocalGatewayMetaState.java → src/main/java/org/elasticsearch/gateway/GatewayMetaState.java

@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.elasticsearch.gateway.local.state.meta;
+package org.elasticsearch.gateway;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -62,7 +62,7 @@ import java.util.regex.Pattern;
 /**
  *
  */
-public class LocalGatewayMetaState extends AbstractComponent implements ClusterStateListener {
+public class GatewayMetaState extends AbstractComponent implements ClusterStateListener {
 
     static final String GLOBAL_STATE_FILE_PREFIX = "global-";
     private static final String INDEX_STATE_FILE_PREFIX = "state-";
@@ -71,7 +71,11 @@ public class LocalGatewayMetaState extends AbstractComponent implements ClusterS
     private static final String GLOBAL_STATE_LOG_TYPE = "[_global]";
     private static final String DEPRECATED_SETTING_ROUTING_HASH_FUNCTION = "cluster.routing.operation.hash.type";
     private static final String DEPRECATED_SETTING_ROUTING_USE_TYPE = "cluster.routing.operation.use_type";
-
+    public static final String GATEWAY_DANGLING_TIMEOUT = "gateway.dangling_timeout";
+    public static final String GATEWAY_AUTO_IMPORT_DANGLED = "gateway.auto_import_dangled";
+    // legacy - this used to be in a different package
+    private static final String GATEWAY_LOCAL_DANGLING_TIMEOUT = "gateway.local.dangling_timeout";
+    private static final String GATEWAY_LOCAL_AUTO_IMPORT_DANGLED = "gateway.local.auto_import_dangled";
     static enum AutoImportDangledState {
         NO() {
             @Override
@@ -127,9 +131,9 @@ public class LocalGatewayMetaState extends AbstractComponent implements ClusterS
     private final Object danglingMutex = new Object();
 
     @Inject
-    public LocalGatewayMetaState(Settings settings, ThreadPool threadPool, NodeEnvironment nodeEnv,
-                                 TransportNodesListGatewayMetaState nodesListGatewayMetaState, LocalAllocateDangledIndices allocateDangledIndices,
-                                 NodeIndexDeletedAction nodeIndexDeletedAction) throws Exception {
+    public GatewayMetaState(Settings settings, ThreadPool threadPool, NodeEnvironment nodeEnv,
+                            TransportNodesListGatewayMetaState nodesListGatewayMetaState, LocalAllocateDangledIndices allocateDangledIndices,
+                            NodeIndexDeletedAction nodeIndexDeletedAction) throws Exception {
         super(settings);
         this.nodeEnv = nodeEnv;
         this.threadPool = threadPool;
@@ -153,10 +157,10 @@ public class LocalGatewayMetaState extends AbstractComponent implements ClusterS
             gatewayModeFormatParams = new ToXContent.MapParams(gatewayModeParams);
         }
 
-        this.autoImportDangled = AutoImportDangledState.fromString(settings.get("gateway.local.auto_import_dangled", AutoImportDangledState.YES.toString()));
-        this.danglingTimeout = settings.getAsTime("gateway.local.dangling_timeout", TimeValue.timeValueHours(2));
+        this.autoImportDangled = AutoImportDangledState.fromString(settings.get(GATEWAY_AUTO_IMPORT_DANGLED, settings.get(GATEWAY_LOCAL_AUTO_IMPORT_DANGLED, AutoImportDangledState.YES.toString())));
+        this.danglingTimeout = settings.getAsTime(GATEWAY_DANGLING_TIMEOUT, settings.getAsTime(GATEWAY_LOCAL_DANGLING_TIMEOUT, TimeValue.timeValueHours(2)));
 
-        logger.debug("using gateway.local.auto_import_dangled [{}], with gateway.local.dangling_timeout [{}]", this.autoImportDangled, this.danglingTimeout);
+        logger.debug("using gateway.local.auto_import_dangled [{}], with gateway.dangling_timeout [{}]", this.autoImportDangled, this.danglingTimeout);
         if (DiscoveryNode.masterNode(settings) || DiscoveryNode.dataNode(settings)) {
             nodeEnv.ensureAtomicMoveSupported();
         }

+ 8 - 22
src/main/java/org/elasticsearch/gateway/GatewayModule.java

@@ -19,35 +19,21 @@
 
 package org.elasticsearch.gateway;
 
-import com.google.common.collect.ImmutableList;
-import org.elasticsearch.common.inject.AbstractModule;
-import org.elasticsearch.common.inject.Module;
-import org.elasticsearch.common.inject.Modules;
-import org.elasticsearch.common.inject.SpawnModules;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.gateway.local.LocalGatewayModule;
+import org.elasticsearch.common.inject.*;
 
 /**
  *
  */
-public class GatewayModule extends AbstractModule implements SpawnModules {
-
-    public static String GATEWAY_TYPE_SETTING = "gateway.type";
-
-    private final Settings settings;
-
-    public GatewayModule(Settings settings) {
-        this.settings = settings;
-    }
-
-    @Override
-    public Iterable<? extends Module> spawnModules() {
-        Class gateway = settings.getAsClass(GATEWAY_TYPE_SETTING, LocalGatewayModule.class, "org.elasticsearch.gateway.", "GatewayModule");
-        return ImmutableList.of(Modules.createModule(gateway, settings));
-    }
+public class GatewayModule extends AbstractModule {
 
     @Override
     protected void configure() {
         bind(GatewayService.class).asEagerSingleton();
+        bind(Gateway.class).asEagerSingleton();
+        bind(GatewayShardsState.class).asEagerSingleton();
+        bind(TransportNodesListGatewayMetaState.class).asEagerSingleton();
+        bind(GatewayMetaState.class).asEagerSingleton();
+        bind(TransportNodesListGatewayStartedShards.class).asEagerSingleton();
+        bind(LocalAllocateDangledIndices.class).asEagerSingleton();
     }
 }

+ 0 - 3
src/main/java/org/elasticsearch/gateway/GatewayService.java

@@ -103,7 +103,6 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
 
     @Override
     protected void doStart() throws ElasticsearchException {
-        gateway.start();
         clusterService.addLast(this);
         // if we received initial state, see if we can recover within the start phase, so we hold the
         // node from starting until we recovered properly
@@ -120,12 +119,10 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
     @Override
     protected void doStop() throws ElasticsearchException {
         clusterService.remove(this);
-        gateway.stop();
     }
 
     @Override
     protected void doClose() throws ElasticsearchException {
-        gateway.close();
     }
 
     @Override

+ 3 - 5
src/main/java/org/elasticsearch/gateway/local/state/shards/LocalGatewayShardsState.java → src/main/java/org/elasticsearch/gateway/GatewayShardsState.java

@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.elasticsearch.gateway.local.state.shards;
+package org.elasticsearch.gateway;
 
 import com.google.common.collect.Maps;
 import org.elasticsearch.ElasticsearchIllegalStateException;
@@ -34,8 +34,6 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.*;
 import org.elasticsearch.env.NodeEnvironment;
-import org.elasticsearch.gateway.local.state.meta.CorruptStateException;
-import org.elasticsearch.gateway.local.state.meta.MetaDataStateFormat;
 import org.elasticsearch.index.shard.ShardId;
 
 import java.io.*;
@@ -46,7 +44,7 @@ import java.util.regex.Pattern;
 
 /**
  */
-public class LocalGatewayShardsState extends AbstractComponent implements ClusterStateListener {
+public class GatewayShardsState extends AbstractComponent implements ClusterStateListener {
 
     private static final String SHARD_STATE_FILE_PREFIX = "state-";
     private static final Pattern SHARD_STATE_FILE_PATTERN = Pattern.compile(SHARD_STATE_FILE_PREFIX + "(\\d+)(" + MetaDataStateFormat.STATE_FILE_EXTENSION + ")?");
@@ -58,7 +56,7 @@ public class LocalGatewayShardsState extends AbstractComponent implements Cluste
     private volatile Map<ShardId, ShardStateInfo> currentState = Maps.newHashMap();
 
     @Inject
-    public LocalGatewayShardsState(Settings settings, NodeEnvironment nodeEnv, TransportNodesListGatewayStartedShards listGatewayStartedShards) throws Exception {
+    public GatewayShardsState(Settings settings, NodeEnvironment nodeEnv, TransportNodesListGatewayStartedShards listGatewayStartedShards) throws Exception {
         super(settings);
         this.nodeEnv = nodeEnv;
         if (listGatewayStartedShards != null) { // for testing

+ 1 - 1
src/main/java/org/elasticsearch/gateway/local/state/meta/LocalAllocateDangledIndices.java → src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java

@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.elasticsearch.gateway.local.state.meta;
+package org.elasticsearch.gateway;
 
 import org.elasticsearch.cluster.ClusterService;
 import org.elasticsearch.cluster.ClusterState;

+ 2 - 3
src/main/java/org/elasticsearch/gateway/local/state/meta/MetaDataStateFormat.java → src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java

@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.elasticsearch.gateway.local.state.meta;
+package org.elasticsearch.gateway;
 
 import com.google.common.base.Predicate;
 import com.google.common.collect.Collections2;
@@ -29,7 +29,6 @@ import org.apache.lucene.util.IOUtils;
 import org.elasticsearch.ElasticsearchIllegalStateException;
 import org.elasticsearch.ExceptionsHelper;
 import org.elasticsearch.common.Preconditions;
-import org.elasticsearch.common.io.Streams;
 import org.elasticsearch.common.logging.ESLogger;
 import org.elasticsearch.common.lucene.store.InputStreamIndexInput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
@@ -293,7 +292,7 @@ public abstract class MetaDataStateFormat<T> {
     }
 
     /**
-     * Filters out all {@link org.elasticsearch.gateway.local.state.meta.MetaDataStateFormat.PathAndVersion} instances with a different version than
+     * Filters out all {@link MetaDataStateFormat.PathAndVersion} instances with a different version than
      * the given one.
      */
     private static final class VersionAndLegacyPredicate implements Predicate<PathAndVersion> {

+ 1 - 1
src/main/java/org/elasticsearch/gateway/local/state/shards/ShardStateInfo.java → src/main/java/org/elasticsearch/gateway/ShardStateInfo.java

@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.elasticsearch.gateway.local.state.shards;
+package org.elasticsearch.gateway;
 
 import org.elasticsearch.common.Nullable;
 

+ 23 - 23
src/main/java/org/elasticsearch/gateway/local/state/meta/TransportNodesListGatewayMetaState.java → src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java

@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.elasticsearch.gateway.local.state.meta;
+package org.elasticsearch.gateway;
 
 import com.google.common.collect.Lists;
 import org.elasticsearch.ElasticsearchException;
@@ -45,23 +45,23 @@ import java.util.concurrent.atomic.AtomicReferenceArray;
 /**
  *
  */
-public class TransportNodesListGatewayMetaState extends TransportNodesOperationAction<TransportNodesListGatewayMetaState.Request, TransportNodesListGatewayMetaState.NodesLocalGatewayMetaState, TransportNodesListGatewayMetaState.NodeRequest, TransportNodesListGatewayMetaState.NodeLocalGatewayMetaState> {
+public class TransportNodesListGatewayMetaState extends TransportNodesOperationAction<TransportNodesListGatewayMetaState.Request, TransportNodesListGatewayMetaState.NodesGatewayMetaState, TransportNodesListGatewayMetaState.NodeRequest, TransportNodesListGatewayMetaState.NodeGatewayMetaState> {
 
     public static final String ACTION_NAME = "internal:gateway/local/meta_state";
 
-    private LocalGatewayMetaState metaState;
+    private GatewayMetaState metaState;
 
     @Inject
     public TransportNodesListGatewayMetaState(Settings settings, ClusterName clusterName, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, ActionFilters actionFilters) {
         super(settings, ACTION_NAME, clusterName, threadPool, clusterService, transportService, actionFilters);
     }
 
-    TransportNodesListGatewayMetaState init(LocalGatewayMetaState metaState) {
+    TransportNodesListGatewayMetaState init(GatewayMetaState metaState) {
         this.metaState = metaState;
         return this;
     }
 
-    public ActionFuture<NodesLocalGatewayMetaState> list(String[] nodesIds, @Nullable TimeValue timeout) {
+    public ActionFuture<NodesGatewayMetaState> list(String[] nodesIds, @Nullable TimeValue timeout) {
         return execute(new Request(nodesIds).timeout(timeout));
     }
 
@@ -91,32 +91,32 @@ public class TransportNodesListGatewayMetaState extends TransportNodesOperationA
     }
 
     @Override
-    protected NodeLocalGatewayMetaState newNodeResponse() {
-        return new NodeLocalGatewayMetaState();
+    protected NodeGatewayMetaState newNodeResponse() {
+        return new NodeGatewayMetaState();
     }
 
     @Override
-    protected NodesLocalGatewayMetaState newResponse(Request request, AtomicReferenceArray responses) {
-        final List<NodeLocalGatewayMetaState> nodesList = Lists.newArrayList();
+    protected NodesGatewayMetaState newResponse(Request request, AtomicReferenceArray responses) {
+        final List<NodeGatewayMetaState> nodesList = Lists.newArrayList();
         final List<FailedNodeException> failures = Lists.newArrayList();
         for (int i = 0; i < responses.length(); i++) {
             Object resp = responses.get(i);
-            if (resp instanceof NodeLocalGatewayMetaState) { // will also filter out null response for unallocated ones
-                nodesList.add((NodeLocalGatewayMetaState) resp);
+            if (resp instanceof NodeGatewayMetaState) { // will also filter out null response for unallocated ones
+                nodesList.add((NodeGatewayMetaState) resp);
             } else if (resp instanceof FailedNodeException) {
                 failures.add((FailedNodeException) resp);
             } else {
                 logger.warn("unknown response type [{}], expected NodeLocalGatewayMetaState or FailedNodeException", resp);
             }
         }
-        return new NodesLocalGatewayMetaState(clusterName, nodesList.toArray(new NodeLocalGatewayMetaState[nodesList.size()]),
+        return new NodesGatewayMetaState(clusterName, nodesList.toArray(new NodeGatewayMetaState[nodesList.size()]),
                 failures.toArray(new FailedNodeException[failures.size()]));
     }
 
     @Override
-    protected NodeLocalGatewayMetaState nodeOperation(NodeRequest request) throws ElasticsearchException {
+    protected NodeGatewayMetaState nodeOperation(NodeRequest request) throws ElasticsearchException {
         try {
-            return new NodeLocalGatewayMetaState(clusterService.localNode(), metaState.loadMetaState());
+            return new NodeGatewayMetaState(clusterService.localNode(), metaState.loadMetaState());
         } catch (Exception e) {
             throw new ElasticsearchException("failed to load metadata", e);
         }
@@ -147,14 +147,14 @@ public class TransportNodesListGatewayMetaState extends TransportNodesOperationA
         }
     }
 
-    public static class NodesLocalGatewayMetaState extends NodesOperationResponse<NodeLocalGatewayMetaState> {
+    public static class NodesGatewayMetaState extends NodesOperationResponse<NodeGatewayMetaState> {
 
         private FailedNodeException[] failures;
 
-        NodesLocalGatewayMetaState() {
+        NodesGatewayMetaState() {
         }
 
-        public NodesLocalGatewayMetaState(ClusterName clusterName, NodeLocalGatewayMetaState[] nodes, FailedNodeException[] failures) {
+        public NodesGatewayMetaState(ClusterName clusterName, NodeGatewayMetaState[] nodes, FailedNodeException[] failures) {
             super(clusterName, nodes);
             this.failures = failures;
         }
@@ -166,9 +166,9 @@ public class TransportNodesListGatewayMetaState extends TransportNodesOperationA
         @Override
         public void readFrom(StreamInput in) throws IOException {
             super.readFrom(in);
-            nodes = new NodeLocalGatewayMetaState[in.readVInt()];
+            nodes = new NodeGatewayMetaState[in.readVInt()];
             for (int i = 0; i < nodes.length; i++) {
-                nodes[i] = new NodeLocalGatewayMetaState();
+                nodes[i] = new NodeGatewayMetaState();
                 nodes[i].readFrom(in);
             }
         }
@@ -177,7 +177,7 @@ public class TransportNodesListGatewayMetaState extends TransportNodesOperationA
         public void writeTo(StreamOutput out) throws IOException {
             super.writeTo(out);
             out.writeVInt(nodes.length);
-            for (NodeLocalGatewayMetaState response : nodes) {
+            for (NodeGatewayMetaState response : nodes) {
                 response.writeTo(out);
             }
         }
@@ -204,14 +204,14 @@ public class TransportNodesListGatewayMetaState extends TransportNodesOperationA
         }
     }
 
-    public static class NodeLocalGatewayMetaState extends NodeOperationResponse {
+    public static class NodeGatewayMetaState extends NodeOperationResponse {
 
         private MetaData metaData;
 
-        NodeLocalGatewayMetaState() {
+        NodeGatewayMetaState() {
         }
 
-        public NodeLocalGatewayMetaState(DiscoveryNode node, MetaData metaData) {
+        public NodeGatewayMetaState(DiscoveryNode node, MetaData metaData) {
             super(node);
             this.metaData = metaData;
         }

+ 24 - 28
src/main/java/org/elasticsearch/gateway/local/state/shards/TransportNodesListGatewayStartedShards.java → src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayStartedShards.java

@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.elasticsearch.gateway.local.state.shards;
+package org.elasticsearch.gateway;
 
 import com.google.common.collect.Lists;
 import org.elasticsearch.ElasticsearchException;
@@ -46,23 +46,23 @@ import java.util.concurrent.atomic.AtomicReferenceArray;
 /**
  *
  */
-public class TransportNodesListGatewayStartedShards extends TransportNodesOperationAction<TransportNodesListGatewayStartedShards.Request, TransportNodesListGatewayStartedShards.NodesLocalGatewayStartedShards, TransportNodesListGatewayStartedShards.NodeRequest, TransportNodesListGatewayStartedShards.NodeLocalGatewayStartedShards> {
+public class TransportNodesListGatewayStartedShards extends TransportNodesOperationAction<TransportNodesListGatewayStartedShards.Request, TransportNodesListGatewayStartedShards.NodesGatewayStartedShards, TransportNodesListGatewayStartedShards.NodeRequest, TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> {
 
     public static final String ACTION_NAME = "internal:gateway/local/started_shards";
 
-    private LocalGatewayShardsState shardsState;
+    private GatewayShardsState shardsState;
 
     @Inject
     public TransportNodesListGatewayStartedShards(Settings settings, ClusterName clusterName, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, ActionFilters actionFilters) {
         super(settings, ACTION_NAME, clusterName, threadPool, clusterService, transportService, actionFilters);
     }
 
-    TransportNodesListGatewayStartedShards initGateway(LocalGatewayShardsState shardsState) {
+    TransportNodesListGatewayStartedShards initGateway(GatewayShardsState shardsState) {
         this.shardsState = shardsState;
         return this;
     }
 
-    public ActionFuture<NodesLocalGatewayStartedShards> list(ShardId shardId, String[] nodesIds, @Nullable TimeValue timeout) {
+    public ActionFuture<NodesGatewayStartedShards> list(ShardId shardId, String[] nodesIds, @Nullable TimeValue timeout) {
         return execute(new Request(shardId, nodesIds).timeout(timeout));
     }
 
@@ -92,36 +92,36 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesOperat
     }
 
     @Override
-    protected NodeLocalGatewayStartedShards newNodeResponse() {
-        return new NodeLocalGatewayStartedShards();
+    protected NodeGatewayStartedShards newNodeResponse() {
+        return new NodeGatewayStartedShards();
     }
 
     @Override
-    protected NodesLocalGatewayStartedShards newResponse(Request request, AtomicReferenceArray responses) {
-        final List<NodeLocalGatewayStartedShards> nodesList = Lists.newArrayList();
+    protected NodesGatewayStartedShards newResponse(Request request, AtomicReferenceArray responses) {
+        final List<NodeGatewayStartedShards> nodesList = Lists.newArrayList();
         final List<FailedNodeException> failures = Lists.newArrayList();
         for (int i = 0; i < responses.length(); i++) {
             Object resp = responses.get(i);
-            if (resp instanceof NodeLocalGatewayStartedShards) { // will also filter out null response for unallocated ones
-                nodesList.add((NodeLocalGatewayStartedShards) resp);
+            if (resp instanceof NodeGatewayStartedShards) { // will also filter out null response for unallocated ones
+                nodesList.add((NodeGatewayStartedShards) resp);
             } else if (resp instanceof FailedNodeException) {
                 failures.add((FailedNodeException) resp);
             } else {
                 logger.warn("unknown response type [{}], expected NodeLocalGatewayStartedShards or FailedNodeException", resp);
             }
         }
-        return new NodesLocalGatewayStartedShards(clusterName, nodesList.toArray(new NodeLocalGatewayStartedShards[nodesList.size()]),
+        return new NodesGatewayStartedShards(clusterName, nodesList.toArray(new NodeGatewayStartedShards[nodesList.size()]),
                 failures.toArray(new FailedNodeException[failures.size()]));
     }
 
     @Override
-    protected NodeLocalGatewayStartedShards nodeOperation(NodeRequest request) throws ElasticsearchException {
+    protected NodeGatewayStartedShards nodeOperation(NodeRequest request) throws ElasticsearchException {
         try {
             ShardStateInfo shardStateInfo = shardsState.loadShardInfo(request.shardId);
             if (shardStateInfo != null) {
-                return new NodeLocalGatewayStartedShards(clusterService.localNode(), shardStateInfo.version);
+                return new NodeGatewayStartedShards(clusterService.localNode(), shardStateInfo.version);
             }
-            return new NodeLocalGatewayStartedShards(clusterService.localNode(), -1);
+            return new NodeGatewayStartedShards(clusterService.localNode(), -1);
         } catch (Exception e) {
             throw new ElasticsearchException("failed to load started shards", e);
         }
@@ -166,14 +166,14 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesOperat
         }
     }
 
-    public static class NodesLocalGatewayStartedShards extends NodesOperationResponse<NodeLocalGatewayStartedShards> {
+    public static class NodesGatewayStartedShards extends NodesOperationResponse<NodeGatewayStartedShards> {
 
         private FailedNodeException[] failures;
 
-        NodesLocalGatewayStartedShards() {
+        NodesGatewayStartedShards() {
         }
 
-        public NodesLocalGatewayStartedShards(ClusterName clusterName, NodeLocalGatewayStartedShards[] nodes, FailedNodeException[] failures) {
+        public NodesGatewayStartedShards(ClusterName clusterName, NodeGatewayStartedShards[] nodes, FailedNodeException[] failures) {
             super(clusterName, nodes);
             this.failures = failures;
         }
@@ -185,9 +185,9 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesOperat
         @Override
         public void readFrom(StreamInput in) throws IOException {
             super.readFrom(in);
-            nodes = new NodeLocalGatewayStartedShards[in.readVInt()];
+            nodes = new NodeGatewayStartedShards[in.readVInt()];
             for (int i = 0; i < nodes.length; i++) {
-                nodes[i] = new NodeLocalGatewayStartedShards();
+                nodes[i] = new NodeGatewayStartedShards();
                 nodes[i].readFrom(in);
             }
         }
@@ -196,7 +196,7 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesOperat
         public void writeTo(StreamOutput out) throws IOException {
             super.writeTo(out);
             out.writeVInt(nodes.length);
-            for (NodeLocalGatewayStartedShards response : nodes) {
+            for (NodeGatewayStartedShards response : nodes) {
                 response.writeTo(out);
             }
         }
@@ -228,22 +228,18 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesOperat
         }
     }
 
-    public static class NodeLocalGatewayStartedShards extends NodeOperationResponse {
+    public static class NodeGatewayStartedShards extends NodeOperationResponse {
 
         private long version = -1;
 
-        NodeLocalGatewayStartedShards() {
+        NodeGatewayStartedShards() {
         }
 
-        public NodeLocalGatewayStartedShards(DiscoveryNode node, long version) {
+        public NodeGatewayStartedShards(DiscoveryNode node, long version) {
             super(node);
             this.version = version;
         }
 
-        public boolean hasVersion() {
-            return version != -1;
-        }
-
         public long version() {
             return this.version;
         }

+ 0 - 215
src/main/java/org/elasticsearch/gateway/local/LocalGateway.java

@@ -1,215 +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.gateway.local;
-
-import com.carrotsearch.hppc.ObjectFloatOpenHashMap;
-import com.carrotsearch.hppc.ObjectOpenHashSet;
-import com.carrotsearch.hppc.cursors.ObjectCursor;
-import org.apache.lucene.util.IOUtils;
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.action.FailedNodeException;
-import org.elasticsearch.cluster.*;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
-import org.elasticsearch.cluster.metadata.MetaData;
-import org.elasticsearch.common.component.AbstractLifecycleComponent;
-import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.inject.Module;
-import org.elasticsearch.common.io.FileSystemUtils;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.env.NodeEnvironment;
-import org.elasticsearch.gateway.Gateway;
-import org.elasticsearch.gateway.GatewayException;
-import org.elasticsearch.gateway.local.state.meta.LocalGatewayMetaState;
-import org.elasticsearch.gateway.local.state.meta.TransportNodesListGatewayMetaState;
-import org.elasticsearch.gateway.local.state.shards.LocalGatewayShardsState;
-import org.elasticsearch.index.gateway.local.LocalIndexGatewayModule;
-
-/**
- *
- */
-public class LocalGateway extends AbstractLifecycleComponent<Gateway> implements Gateway, ClusterStateListener {
-
-    private final ClusterService clusterService;
-
-    private final NodeEnvironment nodeEnv;
-
-    private final LocalGatewayShardsState shardsState;
-    private final LocalGatewayMetaState metaState;
-
-    private final TransportNodesListGatewayMetaState listGatewayMetaState;
-
-    private final String initialMeta;
-    private final ClusterName clusterName;
-
-    @Inject
-    public LocalGateway(Settings settings, ClusterService clusterService, NodeEnvironment nodeEnv,
-                        LocalGatewayShardsState shardsState, LocalGatewayMetaState metaState,
-                        TransportNodesListGatewayMetaState listGatewayMetaState, ClusterName clusterName) {
-        super(settings);
-        this.clusterService = clusterService;
-        this.nodeEnv = nodeEnv;
-        this.metaState = metaState;
-        this.listGatewayMetaState = listGatewayMetaState;
-        this.clusterName = clusterName;
-
-        this.shardsState = shardsState;
-
-        clusterService.addLast(this);
-
-        // we define what is our minimum "master" nodes, use that to allow for recovery
-        this.initialMeta = componentSettings.get("initial_meta", settings.get("discovery.zen.minimum_master_nodes", "1"));
-    }
-
-    @Override
-    public String type() {
-        return "local";
-    }
-
-    @Override
-    protected void doStart() throws ElasticsearchException {
-    }
-
-    @Override
-    protected void doStop() throws ElasticsearchException {
-    }
-
-    @Override
-    protected void doClose() throws ElasticsearchException {
-        clusterService.remove(this);
-    }
-
-    @Override
-    public void performStateRecovery(final GatewayStateRecoveredListener listener) throws GatewayException {
-        ObjectOpenHashSet<String> nodesIds = ObjectOpenHashSet.from(clusterService.state().nodes().masterNodes().keys());
-        logger.trace("performing state recovery from {}", nodesIds);
-        TransportNodesListGatewayMetaState.NodesLocalGatewayMetaState nodesState = listGatewayMetaState.list(nodesIds.toArray(String.class), null).actionGet();
-
-
-        int requiredAllocation = 1;
-        try {
-            if ("quorum".equals(initialMeta)) {
-                if (nodesIds.size() > 2) {
-                    requiredAllocation = (nodesIds.size() / 2) + 1;
-                }
-            } else if ("quorum-1".equals(initialMeta) || "half".equals(initialMeta)) {
-                if (nodesIds.size() > 2) {
-                    requiredAllocation = ((1 + nodesIds.size()) / 2);
-                }
-            } else if ("one".equals(initialMeta)) {
-                requiredAllocation = 1;
-            } else if ("full".equals(initialMeta) || "all".equals(initialMeta)) {
-                requiredAllocation = nodesIds.size();
-            } else if ("full-1".equals(initialMeta) || "all-1".equals(initialMeta)) {
-                if (nodesIds.size() > 1) {
-                    requiredAllocation = nodesIds.size() - 1;
-                }
-            } else {
-                requiredAllocation = Integer.parseInt(initialMeta);
-            }
-        } catch (Exception e) {
-            logger.warn("failed to derived initial_meta from value {}", initialMeta);
-        }
-
-        if (nodesState.failures().length > 0) {
-            for (FailedNodeException failedNodeException : nodesState.failures()) {
-                logger.warn("failed to fetch state from node", failedNodeException);
-            }
-        }
-
-        ObjectFloatOpenHashMap<String> indices = new ObjectFloatOpenHashMap<>();
-        MetaData electedGlobalState = null;
-        int found = 0;
-        for (TransportNodesListGatewayMetaState.NodeLocalGatewayMetaState nodeState : nodesState) {
-            if (nodeState.metaData() == null) {
-                continue;
-            }
-            found++;
-            if (electedGlobalState == null) {
-                electedGlobalState = nodeState.metaData();
-            } else if (nodeState.metaData().version() > electedGlobalState.version()) {
-                electedGlobalState = nodeState.metaData();
-            }
-            for (ObjectCursor<IndexMetaData> cursor : nodeState.metaData().indices().values()) {
-                indices.addTo(cursor.value.index(), 1);
-            }
-        }
-        if (found < requiredAllocation) {
-            listener.onFailure("found [" + found + "] metadata states, required [" + requiredAllocation + "]");
-            return;
-        }
-        // update the global state, and clean the indices, we elect them in the next phase
-        MetaData.Builder metaDataBuilder = MetaData.builder(electedGlobalState).removeAllIndices();
-        final boolean[] states = indices.allocated;
-        final Object[] keys = indices.keys;
-        for (int i = 0; i < states.length; i++) {
-            if (states[i]) {
-                String index = (String) keys[i];
-                IndexMetaData electedIndexMetaData = null;
-                int indexMetaDataCount = 0;
-                for (TransportNodesListGatewayMetaState.NodeLocalGatewayMetaState nodeState : nodesState) {
-                    if (nodeState.metaData() == null) {
-                        continue;
-                    }
-                    IndexMetaData indexMetaData = nodeState.metaData().index(index);
-                    if (indexMetaData == null) {
-                        continue;
-                    }
-                    if (electedIndexMetaData == null) {
-                        electedIndexMetaData = indexMetaData;
-                    } else if (indexMetaData.version() > electedIndexMetaData.version()) {
-                        electedIndexMetaData = indexMetaData;
-                    }
-                    indexMetaDataCount++;
-                }
-                if (electedIndexMetaData != null) {
-                    if (indexMetaDataCount < requiredAllocation) {
-                        logger.debug("[{}] found [{}], required [{}], not adding", index, indexMetaDataCount, requiredAllocation);
-                    }
-                    metaDataBuilder.put(electedIndexMetaData, false);
-                }
-            }
-        }
-        ClusterState.Builder builder = ClusterState.builder(clusterName);
-        builder.metaData(metaDataBuilder);
-        listener.onSuccess(builder.build());
-    }
-
-    @Override
-    public Class<? extends Module> suggestIndexGateway() {
-        return LocalIndexGatewayModule.class;
-    }
-
-    @Override
-    public void reset() throws Exception {
-        try {
-            IOUtils.rm(nodeEnv.nodeDataPaths());
-        } catch (Exception ex) {
-            logger.debug("failed to delete shard locations", ex);
-        }
-    }
-
-    @Override
-    public void clusterChanged(final ClusterChangedEvent event) {
-        // order is important, first metaState, and then shardsState
-        // so dangling indices will be recorded
-        metaState.clusterChanged(event);
-        shardsState.clusterChanged(event);
-    }
-}

+ 0 - 54
src/main/java/org/elasticsearch/gateway/local/LocalGatewayModule.java

@@ -1,54 +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.gateway.local;
-
-import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocatorModule;
-import org.elasticsearch.common.inject.AbstractModule;
-import org.elasticsearch.common.inject.Module;
-import org.elasticsearch.common.inject.PreProcessModule;
-import org.elasticsearch.gateway.Gateway;
-import org.elasticsearch.gateway.local.state.meta.LocalAllocateDangledIndices;
-import org.elasticsearch.gateway.local.state.meta.LocalGatewayMetaState;
-import org.elasticsearch.gateway.local.state.meta.TransportNodesListGatewayMetaState;
-import org.elasticsearch.gateway.local.state.shards.LocalGatewayShardsState;
-import org.elasticsearch.gateway.local.state.shards.TransportNodesListGatewayStartedShards;
-
-/**
- *
- */
-public class LocalGatewayModule extends AbstractModule implements PreProcessModule {
-
-    @Override
-    protected void configure() {
-        bind(Gateway.class).to(LocalGateway.class).asEagerSingleton();
-        bind(LocalGatewayShardsState.class).asEagerSingleton();
-        bind(TransportNodesListGatewayMetaState.class).asEagerSingleton();
-        bind(LocalGatewayMetaState.class).asEagerSingleton();
-        bind(TransportNodesListGatewayStartedShards.class).asEagerSingleton();
-        bind(LocalAllocateDangledIndices.class).asEagerSingleton();
-    }
-
-    @Override
-    public void processModule(Module module) {
-        if (module instanceof ShardsAllocatorModule) {
-            ((ShardsAllocatorModule) module).setGatewayAllocator(LocalGatewayAllocator.class);
-        }
-    }
-}

+ 2 - 10
src/main/java/org/elasticsearch/index/IndexService.java

@@ -44,7 +44,6 @@ import org.elasticsearch.index.engine.Engine;
 import org.elasticsearch.index.engine.EngineModule;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
 import org.elasticsearch.index.fielddata.ShardFieldDataModule;
-import org.elasticsearch.index.gateway.IndexGateway;
 import org.elasticsearch.index.gateway.IndexShardGatewayModule;
 import org.elasticsearch.index.gateway.IndexShardGatewayService;
 import org.elasticsearch.index.get.ShardGetModule;
@@ -121,8 +120,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
 
     private final BitsetFilterCache bitsetFilterCache;
 
-    private final IndexGateway indexGateway;
-
     private final IndexStore indexStore;
 
     private final IndexSettingsService settingsService;
@@ -139,7 +136,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
     public IndexService(Injector injector, Index index, @IndexSettings Settings indexSettings, NodeEnvironment nodeEnv,
                         AnalysisService analysisService, MapperService mapperService, IndexQueryParserService queryParserService,
                         SimilarityService similarityService, IndexAliasesService aliasesService, IndexCache indexCache,
-                        IndexGateway indexGateway, IndexStore indexStore, IndexSettingsService settingsService,
+                        IndexStore indexStore, IndexSettingsService settingsService,
                         IndexFieldDataService indexFieldData, BitsetFilterCache bitSetFilterCache) {
         super(index, indexSettings);
         this.injector = injector;
@@ -151,7 +148,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
         this.aliasesService = aliasesService;
         this.indexCache = indexCache;
         this.indexFieldData = indexFieldData;
-        this.indexGateway = indexGateway;
         this.indexStore = indexStore;
         this.settingsService = settingsService;
         this.bitsetFilterCache = bitSetFilterCache;
@@ -205,10 +201,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
         return injector;
     }
 
-    public IndexGateway gateway() {
-        return indexGateway;
-    }
-
     public IndexSettingsService settingsService() {
         return this.settingsService;
     }
@@ -325,7 +317,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
             modules.add(new ShardFieldDataModule());
             modules.add(new TranslogModule(indexSettings));
             modules.add(new EngineModule(indexSettings));
-            modules.add(new IndexShardGatewayModule(injector.getInstance(IndexGateway.class)));
+            modules.add(new IndexShardGatewayModule());
             modules.add(new PercolatorShardModule());
             modules.add(new ShardTermVectorsModule());
             modules.add(new IndexShardSnapshotModule());

+ 0 - 35
src/main/java/org/elasticsearch/index/gateway/IndexGateway.java

@@ -1,35 +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.index.gateway;
-
-import org.elasticsearch.index.IndexComponent;
-
-import java.io.Closeable;
-
-/**
- *
- */
-public interface IndexGateway extends IndexComponent, Closeable {
-
-    String type();
-
-    Class<? extends IndexShardGateway> shardGatewayClass();
-
-}

+ 0 - 52
src/main/java/org/elasticsearch/index/gateway/IndexGatewayModule.java

@@ -1,52 +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.index.gateway;
-
-import com.google.common.collect.ImmutableList;
-import org.elasticsearch.common.inject.AbstractModule;
-import org.elasticsearch.common.inject.Module;
-import org.elasticsearch.common.inject.Modules;
-import org.elasticsearch.common.inject.SpawnModules;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.gateway.Gateway;
-
-/**
- *
- */
-public class IndexGatewayModule extends AbstractModule implements SpawnModules {
-
-    private final Settings settings;
-
-    private final Gateway gateway;
-
-    public IndexGatewayModule(Settings settings, Gateway gateway) {
-        this.settings = settings;
-        this.gateway = gateway;
-    }
-
-    @Override
-    public Iterable<? extends Module> spawnModules() {
-        return ImmutableList.of(Modules.createModule(settings.getAsClass("index.gateway.type", gateway.suggestIndexGateway(), "org.elasticsearch.index.gateway.", "IndexGatewayModule"), settings));
-    }
-
-    @Override
-    protected void configure() {
-    }
-}

+ 326 - 11
src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java

@@ -19,26 +19,341 @@
 
 package org.elasticsearch.index.gateway;
 
-import org.elasticsearch.index.shard.IndexShardComponent;
+import com.google.common.collect.Sets;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.util.IOUtils;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.ExceptionsHelper;
+import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
+import org.elasticsearch.common.inject.Inject;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.lucene.Lucene;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.util.concurrent.FutureUtils;
+import org.elasticsearch.index.engine.Engine;
+import org.elasticsearch.index.gateway.IndexShardGateway;
+import org.elasticsearch.index.gateway.IndexShardGatewayRecoveryException;
+import org.elasticsearch.index.IndexService;
+import org.elasticsearch.index.settings.IndexSettings;
+import org.elasticsearch.index.shard.AbstractIndexShardComponent;
+import org.elasticsearch.index.shard.IndexShardState;
+import org.elasticsearch.index.shard.ShardId;
+import org.elasticsearch.index.shard.IndexShard;
+import org.elasticsearch.index.translog.*;
 import org.elasticsearch.indices.recovery.RecoveryState;
+import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.threadpool.ThreadPool;
 
 import java.io.Closeable;
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
 
 /**
  *
  */
-public interface IndexShardGateway extends IndexShardComponent, Closeable {
+public class IndexShardGateway extends AbstractIndexShardComponent implements Closeable {
 
-    String type();
+    private static final int RECOVERY_TRANSLOG_RENAME_RETRIES = 3;
 
-    /**
-     * The last / on going recovery status.
-     */
-    RecoveryState recoveryState();
+    private final ThreadPool threadPool;
+    private final MappingUpdatedAction mappingUpdatedAction;
+    private final IndexService indexService;
+    private final IndexShard indexShard;
+    private final TimeValue waitForMappingUpdatePostRecovery;
+    private final TimeValue syncInterval;
 
-    /**
-     * Recovers the state of the shard from the gateway.
-     */
-    void recover(boolean indexShouldExists, RecoveryState recoveryState) throws IndexShardGatewayRecoveryException;
+    private volatile ScheduledFuture flushScheduler;
 
+
+    @Inject
+    public IndexShardGateway(ShardId shardId, @IndexSettings Settings indexSettings, ThreadPool threadPool, MappingUpdatedAction mappingUpdatedAction,
+                             IndexService indexService, IndexShard indexShard) {
+        super(shardId, indexSettings);
+        this.threadPool = threadPool;
+        this.mappingUpdatedAction = mappingUpdatedAction;
+        this.indexService = indexService;
+        this.indexShard = indexShard;
+
+        this.waitForMappingUpdatePostRecovery = componentSettings.getAsTime("wait_for_mapping_update_post_recovery", TimeValue.timeValueSeconds(30));
+        syncInterval = componentSettings.getAsTime("sync", TimeValue.timeValueSeconds(5));
+        if (syncInterval.millis() > 0) {
+            this.indexShard.translog().syncOnEachOperation(false);
+            flushScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, new Sync());
+        } else if (syncInterval.millis() == 0) {
+            flushScheduler = null;
+            this.indexShard.translog().syncOnEachOperation(true);
+        } else {
+            flushScheduler = null;
+        }
+    }
+
+    public void recover(boolean indexShouldExists, RecoveryState recoveryState) throws IndexShardGatewayRecoveryException {
+        recoveryState.getIndex().startTime(System.currentTimeMillis());
+        recoveryState.setStage(RecoveryState.Stage.INDEX);
+        long version = -1;
+        long translogId = -1;
+        indexShard.store().incRef();
+        try {
+            try {
+                indexShard.store().failIfCorrupted();
+                SegmentInfos si = null;
+                try {
+                    si = Lucene.readSegmentInfos(indexShard.store().directory());
+                } catch (Throwable e) {
+                    String files = "_unknown_";
+                    try {
+                        files = Arrays.toString(indexShard.store().directory().listAll());
+                    } catch (Throwable e1) {
+                        files += " (failure=" + ExceptionsHelper.detailedMessage(e1) + ")";
+                    }
+                    if (indexShouldExists) {
+                        throw new IndexShardGatewayRecoveryException(shardId(), "shard allocated for local recovery (post api), should exist, but doesn't, current files: " + files, e);
+                    }
+                }
+                if (si != null) {
+                    if (indexShouldExists) {
+                        version = si.getVersion();
+                        /**
+                         * We generate the translog ID before each lucene commit to ensure that
+                         * we can read the current translog ID safely when we recover. The commits metadata
+                         * therefor contains always the current / active translog ID.
+                         */
+                        if (si.getUserData().containsKey(Translog.TRANSLOG_ID_KEY)) {
+                            translogId = Long.parseLong(si.getUserData().get(Translog.TRANSLOG_ID_KEY));
+                        } else {
+                            translogId = version;
+                        }
+                        logger.trace("using existing shard data, translog id [{}]", translogId);
+                    } else {
+                        // it exists on the directory, but shouldn't exist on the FS, its a leftover (possibly dangling)
+                        // its a "new index create" API, we have to do something, so better to clean it than use same data
+                        logger.trace("cleaning existing shard, shouldn't exists");
+                        IndexWriter writer = new IndexWriter(indexShard.store().directory(), new IndexWriterConfig(Lucene.STANDARD_ANALYZER).setOpenMode(IndexWriterConfig.OpenMode.CREATE));
+                        writer.close();
+                    }
+                }
+            } catch (Throwable e) {
+                throw new IndexShardGatewayRecoveryException(shardId(), "failed to fetch index version after copying it over", e);
+            }
+            recoveryState.getIndex().updateVersion(version);
+            recoveryState.getIndex().time(System.currentTimeMillis() - recoveryState.getIndex().startTime());
+
+            // since we recover from local, just fill the files and size
+            try {
+                int numberOfFiles = 0;
+                long totalSizeInBytes = 0;
+                for (String name : indexShard.store().directory().listAll()) {
+                    numberOfFiles++;
+                    long length =  indexShard.store().directory().fileLength(name);
+                    totalSizeInBytes += length;
+                    recoveryState.getIndex().addFileDetail(name, length, length);
+                }
+                RecoveryState.Index index = recoveryState.getIndex();
+                index.totalFileCount(numberOfFiles);
+                index.totalByteCount(totalSizeInBytes);
+                index.reusedFileCount(numberOfFiles);
+                index.reusedByteCount(totalSizeInBytes);
+                index.recoveredFileCount(numberOfFiles);
+                index.recoveredByteCount(totalSizeInBytes);
+            } catch (Exception e) {
+                // ignore
+            }
+
+            recoveryState.getStart().startTime(System.currentTimeMillis());
+            recoveryState.setStage(RecoveryState.Stage.START);
+            if (translogId == -1) {
+                // no translog files, bail
+                indexShard.postRecovery("post recovery from gateway, no translog for id [" + translogId + "]");
+                // no index, just start the shard and bail
+                recoveryState.getStart().time(System.currentTimeMillis() - recoveryState.getStart().startTime());
+                recoveryState.getStart().checkIndexTime(indexShard.checkIndexTook());
+                return;
+            }
+
+            // move an existing translog, if exists, to "recovering" state, and start reading from it
+            Translog translog = indexShard.translog();
+            final Path translogName = translog.getPath(translogId);
+            final Path recoverTranslogName = translogName.resolveSibling(translogName.getFileName() + ".recovering");
+
+            logger.trace("try recover from translog file {} locations: {}", translogName, Arrays.toString(translog.locations()));
+            Path recoveringTranslogFile = null;
+            for (Path translogLocation : translog.locations()) {
+                final Path tmpRecoveringFile = translogLocation.resolve(recoverTranslogName);
+                if (Files.exists(tmpRecoveringFile) == false) {
+                    Path tmpTranslogFile = translogLocation.resolve(translogName);
+                    if (Files.exists(tmpTranslogFile)) {
+                        logger.trace("Translog file found in {} - renaming", translogLocation);
+                        for (int i = 0; i < RECOVERY_TRANSLOG_RENAME_RETRIES; i++) {
+                            try {
+                                Files.move(tmpTranslogFile, tmpRecoveringFile, StandardCopyOption.ATOMIC_MOVE);
+                                recoveringTranslogFile = tmpRecoveringFile;
+                                logger.trace("Renamed translog from {} to {}", tmpTranslogFile.getFileName(), recoveringTranslogFile.getFileName());
+                                break;
+                            } catch (Exception ex) {
+                                logger.debug("Failed to rename tmp recovery file", ex);
+                            }
+                        }
+                    } else {
+                        logger.trace("Translog file NOT found in {} - continue", translogLocation);
+                    }
+                } else {
+                    recoveringTranslogFile = tmpRecoveringFile;
+                    break;
+                }
+            }
+
+            if (recoveringTranslogFile == null || Files.exists(recoveringTranslogFile) == false) {
+                // no translog to recovery from, start and bail
+                // no translog files, bail
+                indexShard.postRecovery("post recovery from gateway, no translog");
+                // no index, just start the shard and bail
+                recoveryState.getStart().time(System.currentTimeMillis() - recoveryState.getStart().startTime());
+                recoveryState.getStart().checkIndexTime(indexShard.checkIndexTook());
+                return;
+            }
+
+            // recover from the translog file
+            indexShard.performRecoveryPrepareForTranslog();
+            recoveryState.getStart().time(System.currentTimeMillis() - recoveryState.getStart().startTime());
+            recoveryState.getStart().checkIndexTime(indexShard.checkIndexTook());
+
+            recoveryState.getTranslog().startTime(System.currentTimeMillis());
+            recoveryState.setStage(RecoveryState.Stage.TRANSLOG);
+            StreamInput in = null;
+
+            final Set<String> typesToUpdate = Sets.newHashSet();
+            try {
+                logger.trace("recovering translog file: {} length: {}", recoveringTranslogFile, Files.size(recoveringTranslogFile));
+                TranslogStream stream = TranslogStreams.translogStreamFor(recoveringTranslogFile);
+                try {
+                    in = stream.openInput(recoveringTranslogFile);
+                } catch (TruncatedTranslogException e) {
+                    // file is empty or header has been half-written and should be ignored
+                    logger.trace("ignoring truncation exception, the translog is either empty or half-written", e);
+                }
+                while (true) {
+                    if (in == null) {
+                        break;
+                    }
+                    Translog.Operation operation;
+                    try {
+                        if (stream instanceof LegacyTranslogStream) {
+                            in.readInt(); // ignored opSize
+                        }
+                        operation = stream.read(in);
+                    } catch (EOFException e) {
+                        // ignore, not properly written the last op
+                        logger.trace("ignoring translog EOF exception, the last operation was not properly written", e);
+                        break;
+                    } catch (IOException e) {
+                        // ignore, not properly written last op
+                        logger.trace("ignoring translog IO exception, the last operation was not properly written", e);
+                        break;
+                    }
+                    try {
+                        Engine.IndexingOperation potentialIndexOperation = indexShard.performRecoveryOperation(operation);
+                        if (potentialIndexOperation != null && potentialIndexOperation.parsedDoc().mappingsModified()) {
+                            if (!typesToUpdate.contains(potentialIndexOperation.docMapper().type())) {
+                                typesToUpdate.add(potentialIndexOperation.docMapper().type());
+                            }
+                        }
+                        recoveryState.getTranslog().addTranslogOperations(1);
+                    } catch (ElasticsearchException e) {
+                        if (e.status() == RestStatus.BAD_REQUEST) {
+                            // mainly for MapperParsingException and Failure to detect xcontent
+                            logger.info("ignoring recovery of a corrupt translog entry", e);
+                        } else {
+                            throw e;
+                        }
+                    }
+                }
+            } catch (Throwable e) {
+                IOUtils.closeWhileHandlingException(indexShard.translog());
+                throw new IndexShardGatewayRecoveryException(shardId, "failed to recover shard", e);
+            } finally {
+                IOUtils.closeWhileHandlingException(in);
+            }
+            indexShard.performRecoveryFinalization(true);
+
+            try {
+                Files.deleteIfExists(recoveringTranslogFile);
+            } catch (Exception ex) {
+                logger.debug("Failed to delete recovering translog file {}", ex, recoveringTranslogFile);
+            }
+
+            for (final String type : typesToUpdate) {
+                final CountDownLatch latch = new CountDownLatch(1);
+                mappingUpdatedAction.updateMappingOnMaster(indexService.index().name(), indexService.mapperService().documentMapper(type), indexService.indexUUID(), new MappingUpdatedAction.MappingUpdateListener() {
+                    @Override
+                    public void onMappingUpdate() {
+                        latch.countDown();
+                    }
+
+                    @Override
+                    public void onFailure(Throwable t) {
+                        latch.countDown();
+                        logger.debug("failed to send mapping update post recovery to master for [{}]", t, type);
+                    }
+                });
+
+                try {
+                    boolean waited = latch.await(waitForMappingUpdatePostRecovery.millis(), TimeUnit.MILLISECONDS);
+                    if (!waited) {
+                        logger.debug("waited for mapping update on master for [{}], yet timed out");
+                    }
+                } catch (InterruptedException e) {
+                    logger.debug("interrupted while waiting for mapping update");
+                }
+            }
+        } finally {
+            indexShard.store().decRef();
+        }
+
+        recoveryState.getTranslog().time(System.currentTimeMillis() - recoveryState.getTranslog().startTime());
+    }
+
+    @Override
+    public void close() {
+        FutureUtils.cancel(flushScheduler);
+    }
+
+    class Sync implements Runnable {
+        @Override
+        public void run() {
+            // don't re-schedule  if its closed..., we are done
+            if (indexShard.state() == IndexShardState.CLOSED) {
+                return;
+            }
+            if (indexShard.state() == IndexShardState.STARTED && indexShard.translog().syncNeeded()) {
+                threadPool.executor(ThreadPool.Names.FLUSH).execute(new Runnable() {
+                    @Override
+                    public void run() {
+                        try {
+                            indexShard.translog().sync();
+                        } catch (Exception e) {
+                            if (indexShard.state() == IndexShardState.STARTED) {
+                                logger.warn("failed to sync translog", e);
+                            }
+                        }
+                        if (indexShard.state() != IndexShardState.CLOSED) {
+                            flushScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, Sync.this);
+                        }
+                    }
+                });
+            } else {
+                flushScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, Sync.this);
+            }
+        }
+    }
 }

+ 1 - 9
src/main/java/org/elasticsearch/index/gateway/IndexShardGatewayModule.java

@@ -26,18 +26,10 @@ import org.elasticsearch.common.inject.AbstractModule;
  */
 public class IndexShardGatewayModule extends AbstractModule {
 
-    private final IndexGateway indexGateway;
-
-    public IndexShardGatewayModule(IndexGateway indexGateway) {
-        this.indexGateway = indexGateway;
-    }
 
     @Override
     protected void configure() {
-        bind(IndexShardGateway.class)
-                .to(indexGateway.shardGatewayClass())
-                .asEagerSingleton();
-
+        bind(IndexShardGateway.class).asEagerSingleton();
         bind(IndexShardGatewayService.class).asEagerSingleton();
     }
 }

+ 0 - 58
src/main/java/org/elasticsearch/index/gateway/local/LocalIndexGateway.java

@@ -1,58 +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.index.gateway.local;
-
-import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.index.AbstractIndexComponent;
-import org.elasticsearch.index.Index;
-import org.elasticsearch.index.gateway.IndexGateway;
-import org.elasticsearch.index.gateway.IndexShardGateway;
-import org.elasticsearch.index.settings.IndexSettings;
-
-/**
- *
- */
-public class LocalIndexGateway extends AbstractIndexComponent implements IndexGateway {
-
-    @Inject
-    public LocalIndexGateway(Index index, @IndexSettings Settings indexSettings) {
-        super(index, indexSettings);
-    }
-
-    @Override
-    public String type() {
-        return "local";
-    }
-
-    @Override
-    public Class<? extends IndexShardGateway> shardGatewayClass() {
-        return LocalIndexShardGateway.class;
-    }
-
-    @Override
-    public String toString() {
-        return "local";
-    }
-
-    @Override
-    public void close() {
-    }
-}

+ 0 - 34
src/main/java/org/elasticsearch/index/gateway/local/LocalIndexGatewayModule.java

@@ -1,34 +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.index.gateway.local;
-
-import org.elasticsearch.common.inject.AbstractModule;
-import org.elasticsearch.index.gateway.IndexGateway;
-
-/**
- *
- */
-public class LocalIndexGatewayModule extends AbstractModule {
-
-    @Override
-    protected void configure() {
-        bind(IndexGateway.class).to(LocalIndexGateway.class).asEagerSingleton();
-    }
-}

+ 0 - 377
src/main/java/org/elasticsearch/index/gateway/local/LocalIndexShardGateway.java

@@ -1,377 +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.index.gateway.local;
-
-import com.google.common.collect.Sets;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.SegmentInfos;
-import org.apache.lucene.util.IOUtils;
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.ExceptionsHelper;
-import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
-import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.lucene.Lucene;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.unit.TimeValue;
-import org.elasticsearch.common.util.concurrent.FutureUtils;
-import org.elasticsearch.index.engine.Engine;
-import org.elasticsearch.index.gateway.IndexShardGateway;
-import org.elasticsearch.index.gateway.IndexShardGatewayRecoveryException;
-import org.elasticsearch.index.IndexService;
-import org.elasticsearch.index.settings.IndexSettings;
-import org.elasticsearch.index.shard.AbstractIndexShardComponent;
-import org.elasticsearch.index.shard.IndexShardState;
-import org.elasticsearch.index.shard.ShardId;
-import org.elasticsearch.index.shard.IndexShard;
-import org.elasticsearch.index.translog.*;
-import org.elasticsearch.indices.recovery.RecoveryState;
-import org.elasticsearch.rest.RestStatus;
-import org.elasticsearch.threadpool.ThreadPool;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.StandardCopyOption;
-import java.util.Arrays;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-
-/**
- *
- */
-public class LocalIndexShardGateway extends AbstractIndexShardComponent implements IndexShardGateway {
-
-    private static final int RECOVERY_TRANSLOG_RENAME_RETRIES = 3;
-
-    private final ThreadPool threadPool;
-    private final MappingUpdatedAction mappingUpdatedAction;
-    private final IndexService indexService;
-    private final IndexShard indexShard;
-
-    private final TimeValue waitForMappingUpdatePostRecovery;
-
-    private final RecoveryState recoveryState = new RecoveryState();
-
-    private volatile ScheduledFuture flushScheduler;
-    private final TimeValue syncInterval;
-
-    @Inject
-    public LocalIndexShardGateway(ShardId shardId, @IndexSettings Settings indexSettings, ThreadPool threadPool, MappingUpdatedAction mappingUpdatedAction,
-                                  IndexService indexService, IndexShard indexShard) {
-        super(shardId, indexSettings);
-        this.threadPool = threadPool;
-        this.mappingUpdatedAction = mappingUpdatedAction;
-        this.indexService = indexService;
-        this.indexShard = indexShard;
-
-        this.waitForMappingUpdatePostRecovery = componentSettings.getAsTime("wait_for_mapping_update_post_recovery", TimeValue.timeValueSeconds(30));
-        syncInterval = componentSettings.getAsTime("sync", TimeValue.timeValueSeconds(5));
-        if (syncInterval.millis() > 0) {
-            this.indexShard.translog().syncOnEachOperation(false);
-            flushScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, new Sync());
-        } else if (syncInterval.millis() == 0) {
-            flushScheduler = null;
-            this.indexShard.translog().syncOnEachOperation(true);
-        } else {
-            flushScheduler = null;
-        }
-    }
-
-    @Override
-    public String toString() {
-        return "local";
-    }
-
-    @Override
-    public RecoveryState recoveryState() {
-        return recoveryState;
-    }
-
-    @Override
-    public void recover(boolean indexShouldExists, RecoveryState recoveryState) throws IndexShardGatewayRecoveryException {
-        recoveryState.getIndex().startTime(System.currentTimeMillis());
-        recoveryState.setStage(RecoveryState.Stage.INDEX);
-        long version = -1;
-        long translogId = -1;
-        indexShard.store().incRef();
-        try {
-            try {
-                indexShard.store().failIfCorrupted();
-                SegmentInfos si = null;
-                try {
-                    si = Lucene.readSegmentInfos(indexShard.store().directory());
-                } catch (Throwable e) {
-                    String files = "_unknown_";
-                    try {
-                        files = Arrays.toString(indexShard.store().directory().listAll());
-                    } catch (Throwable e1) {
-                        files += " (failure=" + ExceptionsHelper.detailedMessage(e1) + ")";
-                    }
-                    if (indexShouldExists) {
-                        throw new IndexShardGatewayRecoveryException(shardId(), "shard allocated for local recovery (post api), should exist, but doesn't, current files: " + files, e);
-                    }
-                }
-                if (si != null) {
-                    if (indexShouldExists) {
-                        version = si.getVersion();
-                        /**
-                         * We generate the translog ID before each lucene commit to ensure that
-                         * we can read the current translog ID safely when we recover. The commits metadata
-                         * therefor contains always the current / active translog ID.
-                         */
-                        if (si.getUserData().containsKey(Translog.TRANSLOG_ID_KEY)) {
-                            translogId = Long.parseLong(si.getUserData().get(Translog.TRANSLOG_ID_KEY));
-                        } else {
-                            translogId = version;
-                        }
-                        logger.trace("using existing shard data, translog id [{}]", translogId);
-                    } else {
-                        // it exists on the directory, but shouldn't exist on the FS, its a leftover (possibly dangling)
-                        // its a "new index create" API, we have to do something, so better to clean it than use same data
-                        logger.trace("cleaning existing shard, shouldn't exists");
-                        IndexWriter writer = new IndexWriter(indexShard.store().directory(), new IndexWriterConfig(Lucene.STANDARD_ANALYZER).setOpenMode(IndexWriterConfig.OpenMode.CREATE));
-                        writer.close();
-                    }
-                }
-            } catch (Throwable e) {
-                throw new IndexShardGatewayRecoveryException(shardId(), "failed to fetch index version after copying it over", e);
-            }
-            recoveryState.getIndex().updateVersion(version);
-            recoveryState.getIndex().time(System.currentTimeMillis() - recoveryState.getIndex().startTime());
-
-            // since we recover from local, just fill the files and size
-            try {
-                int numberOfFiles = 0;
-                long totalSizeInBytes = 0;
-                for (String name : indexShard.store().directory().listAll()) {
-                    numberOfFiles++;
-                    long length =  indexShard.store().directory().fileLength(name);
-                    totalSizeInBytes += length;
-                    recoveryState.getIndex().addFileDetail(name, length, length);
-                }
-                RecoveryState.Index index = recoveryState.getIndex();
-                index.totalFileCount(numberOfFiles);
-                index.totalByteCount(totalSizeInBytes);
-                index.reusedFileCount(numberOfFiles);
-                index.reusedByteCount(totalSizeInBytes);
-                index.recoveredFileCount(numberOfFiles);
-                index.recoveredByteCount(totalSizeInBytes);
-            } catch (Exception e) {
-                // ignore
-            }
-
-            recoveryState.getStart().startTime(System.currentTimeMillis());
-            recoveryState.setStage(RecoveryState.Stage.START);
-            if (translogId == -1) {
-                // no translog files, bail
-                indexShard.postRecovery("post recovery from gateway, no translog for id [" + translogId + "]");
-                // no index, just start the shard and bail
-                recoveryState.getStart().time(System.currentTimeMillis() - recoveryState.getStart().startTime());
-                recoveryState.getStart().checkIndexTime(indexShard.checkIndexTook());
-                return;
-            }
-
-            // move an existing translog, if exists, to "recovering" state, and start reading from it
-            Translog translog = indexShard.translog();
-            final Path translogName = translog.getPath(translogId);
-            final Path recoverTranslogName = translogName.resolveSibling(translogName.getFileName() + ".recovering");
-
-            logger.trace("try recover from translog file {} locations: {}", translogName, Arrays.toString(translog.locations()));
-            Path recoveringTranslogFile = null;
-            for (Path translogLocation : translog.locations()) {
-                final Path tmpRecoveringFile = translogLocation.resolve(recoverTranslogName);
-                if (Files.exists(tmpRecoveringFile) == false) {
-                    Path tmpTranslogFile = translogLocation.resolve(translogName);
-                    if (Files.exists(tmpTranslogFile)) {
-                        logger.trace("Translog file found in {} - renaming", translogLocation);
-                        for (int i = 0; i < RECOVERY_TRANSLOG_RENAME_RETRIES; i++) {
-                            try {
-                                Files.move(tmpTranslogFile, tmpRecoveringFile, StandardCopyOption.ATOMIC_MOVE);
-                                recoveringTranslogFile = tmpRecoveringFile;
-                                logger.trace("Renamed translog from {} to {}", tmpTranslogFile.getFileName(), recoveringTranslogFile.getFileName());
-                                break;
-                            } catch (Exception ex) {
-                                logger.debug("Failed to rename tmp recovery file", ex);
-                            }
-                        }
-                    } else {
-                        logger.trace("Translog file NOT found in {} - continue", translogLocation);
-                    }
-                } else {
-                    recoveringTranslogFile = tmpRecoveringFile;
-                    break;
-                }
-            }
-
-            if (recoveringTranslogFile == null || Files.exists(recoveringTranslogFile) == false) {
-                // no translog to recovery from, start and bail
-                // no translog files, bail
-                indexShard.postRecovery("post recovery from gateway, no translog");
-                // no index, just start the shard and bail
-                recoveryState.getStart().time(System.currentTimeMillis() - recoveryState.getStart().startTime());
-                recoveryState.getStart().checkIndexTime(indexShard.checkIndexTook());
-                return;
-            }
-
-            // recover from the translog file
-            indexShard.performRecoveryPrepareForTranslog();
-            recoveryState.getStart().time(System.currentTimeMillis() - recoveryState.getStart().startTime());
-            recoveryState.getStart().checkIndexTime(indexShard.checkIndexTook());
-
-            recoveryState.getTranslog().startTime(System.currentTimeMillis());
-            recoveryState.setStage(RecoveryState.Stage.TRANSLOG);
-            StreamInput in = null;
-
-            final Set<String> typesToUpdate = Sets.newHashSet();
-            try {
-                logger.trace("recovering translog file: {} length: {}", recoveringTranslogFile, Files.size(recoveringTranslogFile));
-                TranslogStream stream = TranslogStreams.translogStreamFor(recoveringTranslogFile);
-                try {
-                    in = stream.openInput(recoveringTranslogFile);
-                } catch (TruncatedTranslogException e) {
-                    // file is empty or header has been half-written and should be ignored
-                    logger.trace("ignoring truncation exception, the translog is either empty or half-written", e);
-                }
-                while (true) {
-                    if (in == null) {
-                        break;
-                    }
-                    Translog.Operation operation;
-                    try {
-                        if (stream instanceof LegacyTranslogStream) {
-                            in.readInt(); // ignored opSize
-                        }
-                        operation = stream.read(in);
-                    } catch (EOFException e) {
-                        // ignore, not properly written the last op
-                        logger.trace("ignoring translog EOF exception, the last operation was not properly written", e);
-                        break;
-                    } catch (IOException e) {
-                        // ignore, not properly written last op
-                        logger.trace("ignoring translog IO exception, the last operation was not properly written", e);
-                        break;
-                    }
-                    try {
-                        Engine.IndexingOperation potentialIndexOperation = indexShard.performRecoveryOperation(operation);
-                        if (potentialIndexOperation != null && potentialIndexOperation.parsedDoc().mappingsModified()) {
-                            if (!typesToUpdate.contains(potentialIndexOperation.docMapper().type())) {
-                                typesToUpdate.add(potentialIndexOperation.docMapper().type());
-                            }
-                        }
-                        recoveryState.getTranslog().addTranslogOperations(1);
-                    } catch (ElasticsearchException e) {
-                        if (e.status() == RestStatus.BAD_REQUEST) {
-                            // mainly for MapperParsingException and Failure to detect xcontent
-                            logger.info("ignoring recovery of a corrupt translog entry", e);
-                        } else {
-                            throw e;
-                        }
-                    }
-                }
-            } catch (Throwable e) {
-                IOUtils.closeWhileHandlingException(indexShard.translog());
-                throw new IndexShardGatewayRecoveryException(shardId, "failed to recover shard", e);
-            } finally {
-                IOUtils.closeWhileHandlingException(in);
-            }
-            indexShard.performRecoveryFinalization(true);
-
-            try {
-                Files.deleteIfExists(recoveringTranslogFile);
-            } catch (Exception ex) {
-                logger.debug("Failed to delete recovering translog file {}", ex, recoveringTranslogFile);
-            }
-
-            for (final String type : typesToUpdate) {
-                final CountDownLatch latch = new CountDownLatch(1);
-                mappingUpdatedAction.updateMappingOnMaster(indexService.index().name(), indexService.mapperService().documentMapper(type), indexService.indexUUID(), new MappingUpdatedAction.MappingUpdateListener() {
-                    @Override
-                    public void onMappingUpdate() {
-                        latch.countDown();
-                    }
-
-                    @Override
-                    public void onFailure(Throwable t) {
-                        latch.countDown();
-                        logger.debug("failed to send mapping update post recovery to master for [{}]", t, type);
-                    }
-                });
-
-                try {
-                    boolean waited = latch.await(waitForMappingUpdatePostRecovery.millis(), TimeUnit.MILLISECONDS);
-                    if (!waited) {
-                        logger.debug("waited for mapping update on master for [{}], yet timed out");
-                    }
-                } catch (InterruptedException e) {
-                    logger.debug("interrupted while waiting for mapping update");
-                }
-            }
-        } finally {
-            indexShard.store().decRef();
-        }
-
-        recoveryState.getTranslog().time(System.currentTimeMillis() - recoveryState.getTranslog().startTime());
-    }
-
-    @Override
-    public String type() {
-        return "local";
-    }
-
-
-    @Override
-    public void close() {
-        FutureUtils.cancel(flushScheduler);
-    }
-
-    class Sync implements Runnable {
-        @Override
-        public void run() {
-            // don't re-schedule  if its closed..., we are done
-            if (indexShard.state() == IndexShardState.CLOSED) {
-                return;
-            }
-            if (indexShard.state() == IndexShardState.STARTED && indexShard.translog().syncNeeded()) {
-                threadPool.executor(ThreadPool.Names.FLUSH).execute(new Runnable() {
-                    @Override
-                    public void run() {
-                        try {
-                            indexShard.translog().sync();
-                        } catch (Exception e) {
-                            if (indexShard.state() == IndexShardState.STARTED) {
-                                logger.warn("failed to sync translog", e);
-                            }
-                        }
-                        if (indexShard.state() != IndexShardState.CLOSED) {
-                            flushScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, Sync.this);
-                        }
-                    }
-                });
-            } else {
-                flushScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, Sync.this);
-            }
-        }
-    }
-}

+ 2 - 2
src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java

@@ -27,7 +27,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocatio
 import org.elasticsearch.cluster.settings.DynamicSettings;
 import org.elasticsearch.cluster.settings.Validator;
 import org.elasticsearch.common.inject.AbstractModule;
-import org.elasticsearch.gateway.local.LocalGatewayAllocator;
+import org.elasticsearch.gateway.GatewayAllocator;
 import org.elasticsearch.index.engine.internal.InternalEngineHolder;
 import org.elasticsearch.index.indexing.slowlog.ShardSlowLogIndexingService;
 import org.elasticsearch.index.merge.policy.LogByteSizeMergePolicyProvider;
@@ -72,7 +72,7 @@ public class IndexDynamicSettingsModule extends AbstractModule {
         indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_BLOCKS_METADATA);
         indexDynamicSettings.addDynamicSetting(IndicesTTLService.INDEX_TTL_DISABLE_PURGE);
         indexDynamicSettings.addDynamicSetting(IndexShard.INDEX_REFRESH_INTERVAL, Validator.TIME);
-        indexDynamicSettings.addDynamicSetting(LocalGatewayAllocator.INDEX_RECOVERY_INITIAL_SHARDS);
+        indexDynamicSettings.addDynamicSetting(GatewayAllocator.INDEX_RECOVERY_INITIAL_SHARDS);
         indexDynamicSettings.addDynamicSetting(LogByteSizeMergePolicyProvider.INDEX_MERGE_POLICY_MIN_MERGE_SIZE, Validator.BYTES_SIZE);
         indexDynamicSettings.addDynamicSetting(LogByteSizeMergePolicyProvider.INDEX_MERGE_POLICY_MAX_MERGE_SIZE, Validator.BYTES_SIZE);
         indexDynamicSettings.addDynamicSetting(LogByteSizeMergePolicyProvider.INDEX_MERGE_POLICY_MAX_MERGE_DOCS, Validator.POSITIVE_INTEGER);

+ 0 - 5
src/main/java/org/elasticsearch/indices/IndicesService.java

@@ -46,8 +46,6 @@ import org.elasticsearch.index.codec.CodecModule;
 import org.elasticsearch.index.fielddata.IndexFieldDataModule;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
 import org.elasticsearch.index.flush.FlushStats;
-import org.elasticsearch.index.gateway.IndexGateway;
-import org.elasticsearch.index.gateway.IndexGatewayModule;
 import org.elasticsearch.index.get.GetStats;
 import org.elasticsearch.index.indexing.IndexingStats;
 import org.elasticsearch.index.mapper.MapperService;
@@ -317,7 +315,6 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
         modules.add(new MapperServiceModule());
         modules.add(new IndexQueryParserModule(indexSettings));
         modules.add(new IndexAliasesServiceModule());
-        modules.add(new IndexGatewayModule(indexSettings, injector.getInstance(Gateway.class)));
         modules.add(new IndexModule(indexSettings));
 
         Injector indexInjector;
@@ -430,8 +427,6 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
             logger.debug("[{}] closing analysis service (reason [{}])", index, reason);
             indexInjector.getInstance(AnalysisService.class).close();
 
-            logger.debug("[{}] closing index gateway (reason [{}])", index, reason);
-            indexInjector.getInstance(IndexGateway.class).close();
             logger.debug("[{}] closing mapper service (reason [{}])", index, reason);
             indexInjector.getInstance(MapperService.class).close();
             logger.debug("[{}] closing index query parser service (reason [{}])", index, reason);

+ 1 - 1
src/main/java/org/elasticsearch/node/internal/InternalNode.java

@@ -188,7 +188,7 @@ public final class InternalNode implements Node {
             modules.add(new SearchModule());
             modules.add(new ActionModule(false));
             modules.add(new MonitorModule(settings));
-            modules.add(new GatewayModule(settings));
+            modules.add(new GatewayModule());
             modules.add(new NodeClientModule());
             modules.add(new ShapeModule());
             modules.add(new PercolatorModule());

+ 5 - 7
src/test/java/org/elasticsearch/gateway/local/LocalGatewayIndexStateTests.java → src/test/java/org/elasticsearch/gateway/GatewayIndexStateTests.java

@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.elasticsearch.gateway.local;
+package org.elasticsearch.gateway;
 
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
@@ -32,13 +32,11 @@ import org.elasticsearch.common.logging.Loggers;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.XContentFactory;
-import org.elasticsearch.gateway.Gateway;
 import org.elasticsearch.indices.IndexClosedException;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
 import org.elasticsearch.test.InternalTestCluster;
 import org.elasticsearch.test.InternalTestCluster.RestartCallback;
-import org.elasticsearch.test.junit.annotations.TestLogging;
 import org.junit.Test;
 
 import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
@@ -54,9 +52,9 @@ import static org.hamcrest.Matchers.nullValue;
  */
 @ClusterScope(scope = Scope.TEST, numDataNodes = 0)
 @Slow
-public class LocalGatewayIndexStateTests extends ElasticsearchIntegrationTest {
+public class GatewayIndexStateTests extends ElasticsearchIntegrationTest {
 
-    private final ESLogger logger = Loggers.getLogger(LocalGatewayIndexStateTests.class);
+    private final ESLogger logger = Loggers.getLogger(GatewayIndexStateTests.class);
 
     @Test
     public void testMappingMetaDataParsed() throws Exception {
@@ -337,8 +335,8 @@ public class LocalGatewayIndexStateTests extends ElasticsearchIntegrationTest {
     @Test
     public void testDanglingIndicesAutoImportYes() throws Exception {
         Settings settings = settingsBuilder()
-                .put("gateway.type", "local").put("gateway.local.auto_import_dangled", "yes")
-                .put("gateway.local.dangling_timeout", randomIntBetween(0, 120))
+                .put(GatewayMetaState.GATEWAY_AUTO_IMPORT_DANGLED, "yes")
+                .put(GatewayMetaState.GATEWAY_DANGLING_TIMEOUT, randomIntBetween(0, 120))
                 .build();
         logger.info("--> starting two nodes");
 

+ 4 - 4
src/test/java/org/elasticsearch/gateway/local/state/shards/LocalGatewayShardStateTests.java → src/test/java/org/elasticsearch/gateway/GatewayShardStateTests.java

@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.elasticsearch.gateway.local.state.shards;
+package org.elasticsearch.gateway;
 
 import org.elasticsearch.Version;
 import org.elasticsearch.cluster.node.DiscoveryNode;
@@ -31,11 +31,11 @@ import java.util.List;
 import java.util.Map;
 
 
-public class LocalGatewayShardStateTests extends ElasticsearchTestCase {
+public class GatewayShardStateTests extends ElasticsearchTestCase {
 
     public void testWriteShardState() throws Exception {
         try (NodeEnvironment env = newNodeEnvironment()) {
-            LocalGatewayShardsState state = new LocalGatewayShardsState(ImmutableSettings.EMPTY, env, null);
+            GatewayShardsState state = new GatewayShardsState(ImmutableSettings.EMPTY, env, null);
             ShardId id = new ShardId("foo", 1);
             long version = between(1, Integer.MAX_VALUE / 2);
             boolean primary = randomBoolean();
@@ -59,7 +59,7 @@ public class LocalGatewayShardStateTests extends ElasticsearchTestCase {
 
     public void testPersistRoutingNode() throws Exception {
         try (NodeEnvironment env = newNodeEnvironment()) {
-            LocalGatewayShardsState state = new LocalGatewayShardsState(ImmutableSettings.EMPTY, env, null);
+            GatewayShardsState state = new GatewayShardsState(ImmutableSettings.EMPTY, env, null);
             int numShards = between(0, 100);
             List<MutableShardRouting> shards = new ArrayList<>();
             List<MutableShardRouting> active = new ArrayList<>();

+ 13 - 14
src/test/java/org/elasticsearch/gateway/local/state/meta/MetaDataStateFormatTest.java → src/test/java/org/elasticsearch/gateway/MetaDataStateFormatTest.java

@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.elasticsearch.gateway.local.state.meta;
+package org.elasticsearch.gateway;
 
 import com.carrotsearch.randomizedtesting.LifecycleScope;
 import com.google.common.collect.Iterators;
@@ -49,7 +49,6 @@ import org.junit.Test;
 import java.io.Closeable;
 import java.io.InputStream;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.net.URISyntaxException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
@@ -259,7 +258,7 @@ public class MetaDataStateFormatTest extends ElasticsearchTestCase {
     // If the latest version doesn't use the legacy format while previous versions do, then fail hard
     public void testLatestVersionDoesNotUseLegacy() throws IOException {
         final ToXContent.Params params = ToXContent.EMPTY_PARAMS;
-        MetaDataStateFormat<MetaData> format = LocalGatewayMetaState.globalStateFormat(randomFrom(XContentType.values()), params, randomBoolean());
+        MetaDataStateFormat<MetaData> format = GatewayMetaState.globalStateFormat(randomFrom(XContentType.values()), params, randomBoolean());
         final Path[] dirs = new Path[2];
         dirs[0] = newTempDirPath(LifecycleScope.TEST);
         dirs[1] = newTempDirPath(LifecycleScope.TEST);
@@ -269,14 +268,14 @@ public class MetaDataStateFormatTest extends ElasticsearchTestCase {
         final Path dir1 = randomFrom(dirs);
         final int v1 = randomInt(10);
         // write a first state file in the new format
-        format.write(randomMeta(), LocalGatewayMetaState.GLOBAL_STATE_FILE_PREFIX, v1, dir1);
+        format.write(randomMeta(), GatewayMetaState.GLOBAL_STATE_FILE_PREFIX, v1, dir1);
 
         // write older state files in the old format but with a newer version
         final int numLegacyFiles = randomIntBetween(1, 5);
         for (int i = 0; i < numLegacyFiles; ++i) {
             final Path dir2 = randomFrom(dirs);
             final int v2 = v1 + 1 + randomInt(10);
-            try (XContentBuilder xcontentBuilder = XContentFactory.contentBuilder(format.format(), Files.newOutputStream(dir2.resolve(MetaDataStateFormat.STATE_DIR_NAME).resolve(LocalGatewayMetaState.GLOBAL_STATE_FILE_PREFIX + v2)))) {
+            try (XContentBuilder xcontentBuilder = XContentFactory.contentBuilder(format.format(), Files.newOutputStream(dir2.resolve(MetaDataStateFormat.STATE_DIR_NAME).resolve(GatewayMetaState.GLOBAL_STATE_FILE_PREFIX + v2)))) {
                 xcontentBuilder.startObject();
                 MetaData.Builder.toXContent(randomMeta(), xcontentBuilder, params);
                 xcontentBuilder.endObject();
@@ -284,7 +283,7 @@ public class MetaDataStateFormatTest extends ElasticsearchTestCase {
         }
 
         try {
-            MetaDataStateFormat.loadLatestState(logger, format, LocalGatewayMetaState.GLOBAL_STATE_FILE_PATTERN, "foobar", dirs);
+            MetaDataStateFormat.loadLatestState(logger, format, GatewayMetaState.GLOBAL_STATE_FILE_PATTERN, "foobar", dirs);
             fail("latest version can not be read");
         } catch (ElasticsearchIllegalStateException ex) {
             assertThat(ex.getMessage(), startsWith("Could not find a state file to recover from among "));
@@ -294,7 +293,7 @@ public class MetaDataStateFormatTest extends ElasticsearchTestCase {
     // If both the legacy and the new format are available for the latest version, prefer the new format
     public void testPrefersNewerFormat() throws IOException {
         final ToXContent.Params params = ToXContent.EMPTY_PARAMS;
-        MetaDataStateFormat<MetaData> format = LocalGatewayMetaState.globalStateFormat(randomFrom(XContentType.values()), params, randomBoolean());
+        MetaDataStateFormat<MetaData> format = GatewayMetaState.globalStateFormat(randomFrom(XContentType.values()), params, randomBoolean());
         final Path[] dirs = new Path[2];
         dirs[0] = newTempDirPath(LifecycleScope.TEST);
         dirs[1] = newTempDirPath(LifecycleScope.TEST);
@@ -311,16 +310,16 @@ public class MetaDataStateFormatTest extends ElasticsearchTestCase {
         final Path dir2 = randomFrom(dirs);
         MetaData meta2 = randomMeta();
         assertFalse(meta2.uuid().equals(uuid));
-        try (XContentBuilder xcontentBuilder = XContentFactory.contentBuilder(format.format(), Files.newOutputStream(dir2.resolve(MetaDataStateFormat.STATE_DIR_NAME).resolve(LocalGatewayMetaState.GLOBAL_STATE_FILE_PREFIX + v)))) {
+        try (XContentBuilder xcontentBuilder = XContentFactory.contentBuilder(format.format(), Files.newOutputStream(dir2.resolve(MetaDataStateFormat.STATE_DIR_NAME).resolve(GatewayMetaState.GLOBAL_STATE_FILE_PREFIX + v)))) {
             xcontentBuilder.startObject();
             MetaData.Builder.toXContent(randomMeta(), xcontentBuilder, params);
             xcontentBuilder.endObject();
         }
 
         // write a second state file in the new format but with the same version
-        format.write(meta, LocalGatewayMetaState.GLOBAL_STATE_FILE_PREFIX, v, dir1);
+        format.write(meta, GatewayMetaState.GLOBAL_STATE_FILE_PREFIX, v, dir1);
 
-        MetaData state = MetaDataStateFormat.loadLatestState(logger, format, LocalGatewayMetaState.GLOBAL_STATE_FILE_PATTERN, "foobar", dirs);
+        MetaData state = MetaDataStateFormat.loadLatestState(logger, format, GatewayMetaState.GLOBAL_STATE_FILE_PATTERN, "foobar", dirs);
         assertThat(state.uuid(), equalTo(uuid));
     }
 
@@ -335,7 +334,7 @@ public class MetaDataStateFormatTest extends ElasticsearchTestCase {
             meta.add(randomMeta());
         }
         Set<Path> corruptedFiles = new HashSet<>();
-        MetaDataStateFormat<MetaData> format = LocalGatewayMetaState.globalStateFormat(randomFrom(XContentType.values()), params, randomBoolean());
+        MetaDataStateFormat<MetaData> format = GatewayMetaState.globalStateFormat(randomFrom(XContentType.values()), params, randomBoolean());
         for (int i = 0; i < dirs.length; i++) {
             dirs[i] = newTempDirPath(LifecycleScope.TEST);
             Files.createDirectories(dirs[i].resolve(MetaDataStateFormat.STATE_DIR_NAME));
@@ -353,7 +352,7 @@ public class MetaDataStateFormatTest extends ElasticsearchTestCase {
                 }
             }
             for (int j = numLegacy; j < numStates; j++) {
-                format.write(meta.get(j), LocalGatewayMetaState.GLOBAL_STATE_FILE_PREFIX, j, dirs[i]);
+                format.write(meta.get(j), GatewayMetaState.GLOBAL_STATE_FILE_PREFIX, j, dirs[i]);
                 if (randomBoolean() && (j < numStates - 1 || dirs.length > 0 && i != 0)) {  // corrupt a file that we do not necessarily need here....
                     Path file = dirs[i].resolve(MetaDataStateFormat.STATE_DIR_NAME).resolve("global-" + j + ".st");
                     corruptedFiles.add(file);
@@ -364,7 +363,7 @@ public class MetaDataStateFormatTest extends ElasticsearchTestCase {
         }
         List<Path> dirList = Arrays.asList(dirs);
         Collections.shuffle(dirList, getRandom());
-        MetaData loadedMetaData = MetaDataStateFormat.loadLatestState(logger, format, LocalGatewayMetaState.GLOBAL_STATE_FILE_PATTERN, "foobar", dirList.toArray(new Path[0]));
+        MetaData loadedMetaData = MetaDataStateFormat.loadLatestState(logger, format, GatewayMetaState.GLOBAL_STATE_FILE_PATTERN, "foobar", dirList.toArray(new Path[0]));
         MetaData latestMetaData = meta.get(numStates-1);
         assertThat(loadedMetaData.uuid(), not(equalTo("_na_")));
         assertThat(loadedMetaData.uuid(), equalTo(latestMetaData.uuid()));
@@ -388,7 +387,7 @@ public class MetaDataStateFormatTest extends ElasticsearchTestCase {
                 MetaDataStateFormatTest.corruptFile(file, logger);
             }
             try {
-                MetaDataStateFormat.loadLatestState(logger, format, LocalGatewayMetaState.GLOBAL_STATE_FILE_PATTERN, "foobar", dirList.toArray(new Path[0]));
+                MetaDataStateFormat.loadLatestState(logger, format, GatewayMetaState.GLOBAL_STATE_FILE_PATTERN, "foobar", dirList.toArray(new Path[0]));
                 fail("latest version can not be read");
             } catch (ElasticsearchException ex) {
                 assertThat(ex.getCause(), instanceOf(CorruptStateException.class));

+ 2 - 2
src/test/java/org/elasticsearch/gateway/local/QuorumLocalGatewayTests.java → src/test/java/org/elasticsearch/gateway/QuorumGatewayTests.java

@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.elasticsearch.gateway.local;
+package org.elasticsearch.gateway;
 
 import com.google.common.base.Predicate;
 import org.apache.lucene.util.LuceneTestCase.Slow;
@@ -46,7 +46,7 @@ import static org.hamcrest.Matchers.*;
  *
  */
 @ClusterScope(numDataNodes =0, scope= Scope.TEST)
-public class QuorumLocalGatewayTests extends ElasticsearchIntegrationTest {
+public class QuorumGatewayTests extends ElasticsearchIntegrationTest {
 
     @Override
     protected int numberOfReplicas() {

+ 1 - 1
src/test/java/org/elasticsearch/gateway/local/RecoverAfterNodesTests.java → src/test/java/org/elasticsearch/gateway/RecoverAfterNodesTests.java

@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.elasticsearch.gateway.local;
+package org.elasticsearch.gateway;
 
 import com.google.common.collect.ImmutableSet;
 import org.elasticsearch.client.Client;

+ 1 - 1
src/test/java/org/elasticsearch/gateway/local/RecoveryBackwardsCompatibilityTests.java → src/test/java/org/elasticsearch/gateway/RecoveryBackwardsCompatibilityTests.java

@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.elasticsearch.gateway.local;
+package org.elasticsearch.gateway;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.elasticsearch.Version;

+ 2 - 2
src/test/java/org/elasticsearch/gateway/local/SimpleRecoveryLocalGatewayTests.java → src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java

@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.elasticsearch.gateway.local;
+package org.elasticsearch.gateway;
 
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
@@ -53,7 +53,7 @@ import static org.hamcrest.Matchers.*;
  */
 @ClusterScope(numDataNodes = 0, scope = Scope.TEST)
 @Slow
-public class SimpleRecoveryLocalGatewayTests extends ElasticsearchIntegrationTest {
+public class RecoveryFromGatewayTests extends ElasticsearchIntegrationTest {
 
     private ImmutableSettings.Builder settingsBuilder() {
         return ImmutableSettings.settingsBuilder().put("gateway.type", "local");

+ 2 - 2
src/test/java/org/elasticsearch/indices/state/RareClusterStateTests.java

@@ -31,7 +31,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
 import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.gateway.local.LocalGatewayAllocator;
+import org.elasticsearch.gateway.GatewayAllocator;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.junit.Test;
 
@@ -63,7 +63,7 @@ public class RareClusterStateTests extends ElasticsearchIntegrationTest {
         createIndex("a");
         ensureSearchable("a");
         ClusterState current = clusterService().state();
-        LocalGatewayAllocator allocator = internalCluster().getInstance(LocalGatewayAllocator.class);
+        GatewayAllocator allocator = internalCluster().getInstance(GatewayAllocator.class);
 
         AllocationDeciders allocationDeciders = new AllocationDeciders(ImmutableSettings.EMPTY, new AllocationDecider[0]);
         RoutingNodes routingNodes = new RoutingNodes(

+ 2 - 2
src/test/java/org/elasticsearch/indices/warmer/LocalGatewayIndicesWarmerTests.java → src/test/java/org/elasticsearch/indices/warmer/GatewayIndicesWarmerTests.java

@@ -40,9 +40,9 @@ import static org.hamcrest.Matchers.equalTo;
 /**
  */
 @ClusterScope(numDataNodes =0, scope= Scope.TEST)
-public class LocalGatewayIndicesWarmerTests extends ElasticsearchIntegrationTest {
+public class GatewayIndicesWarmerTests extends ElasticsearchIntegrationTest {
 
-    private final ESLogger logger = Loggers.getLogger(LocalGatewayIndicesWarmerTests.class);
+    private final ESLogger logger = Loggers.getLogger(GatewayIndicesWarmerTests.class);
 
     @Test
     public void testStatePersistence() throws Exception {

+ 7 - 2
src/test/java/org/elasticsearch/test/gateway/NoopGatewayAllocator.java

@@ -22,15 +22,20 @@ package org.elasticsearch.test.gateway;
 import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation;
 import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
 import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation;
-import org.elasticsearch.cluster.routing.allocation.allocator.GatewayAllocator;
+import org.elasticsearch.common.settings.ImmutableSettings;
+import org.elasticsearch.gateway.GatewayAllocator;
 
 /**
  * An allocator used for tests that doesn't do anything
  */
-public class NoopGatewayAllocator implements GatewayAllocator {
+public class NoopGatewayAllocator extends GatewayAllocator {
 
     public static final NoopGatewayAllocator INSTANCE = new NoopGatewayAllocator();
 
+    private NoopGatewayAllocator() {
+        super(ImmutableSettings.EMPTY, null, null);
+    }
+
     @Override
     public void applyStartedShards(StartedRerouteAllocation allocation) {
         // noop

+ 0 - 0
src/test/resources/org/elasticsearch/gateway/local/state/meta/global-3.st → src/test/resources/org/elasticsearch/gateway/global-3.st