diff --git a/docs/reference/indices/seal.asciidoc b/docs/reference/indices/seal.asciidoc new file mode 100644 index 00000000000..583888617b5 --- /dev/null +++ b/docs/reference/indices/seal.asciidoc @@ -0,0 +1,78 @@ +[[indices-seal]] +== Seal + +The seal API allows to flush one or more indices and adds a marker to primaries and replicas if there are no pending write operations. +The seal marker is used during recovery after a node restarts. If a replica is allocated on a node which already has a shard copy with the same seal as the primary then no files will be copied during recovery. +Sealing is a best effort operation. If write operations are ongoing while the sealing is in progress then writing the seal might fail on some copies. + +A seal marks a point in time snapshot (a low level lucene commit). This mark can be used to decide if the initial, rather resource heavy, recovery phase where segments or event the entire lucene index is copied over the network can +be skipped. If the indices on both sides of the recover have the same seal no segment files need to be copied and transaction log replay can start immediately. The seal breaks as soon as the shard issues a new lucene commit, uncommitted operations +in the transaction log do not break the seal until they are committed. + +[source,js] +-------------------------------------------------- +$ curl -XPOST 'http://localhost:9200/twitter/_seal' +-------------------------------------------------- + +The response contains details about for which shards a seal was written and the reason in case of failure. +Response in case all copies of a shard successfully wrote the seal: + +[source,js] +-------------------------------------------------- +{ + "twitter": [ + { + "shard_id": 0, + "responses": { + "5wjOIntuRqy9F_7JRrrLwA": "success", + "M2iCBe-nS5yaInE8volfSg": "success" + }, + "message": "success" + } +} +-------------------------------------------------- + + +Response in case some copies of a shard failed: +[source,js] +-------------------------------------------------- +{ + "twitter": [ + { + "shard_id": 0, + "responses": { + "M2iCBe-nS5yaInE8volfSg": "pending operations", + "5wjOIntuRqy9F_7JRrrLwA": "success" + }, + "message": "failed on some copies" + } +} +-------------------------------------------------- + + +Response in case all copies of a shard failed: +[source,js] +-------------------------------------------------- +{ + "twitter": [ + { + "shard_id": 0, + "message": "operation counter on primary is non zero [2]" + } +} +-------------------------------------------------- + + +[float] +[[seal-multi-index]] +=== Multi Index + +The seal API can be applied to more than one index with a single call, +or even on `_all` the indices. + +[source,js] +-------------------------------------------------- +$ curl -XPOST 'http://localhost:9200/kimchy,elasticsearch/_seal' + +$ curl -XPOST 'http://localhost:9200/_seal' +-------------------------------------------------- diff --git a/rest-api-spec/api/indices.seal.json b/rest-api-spec/api/indices.seal.json new file mode 100644 index 00000000000..3377d6e88a1 --- /dev/null +++ b/rest-api-spec/api/indices.seal.json @@ -0,0 +1,17 @@ +{ + "indices.seal": { + "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-seal.html", + "methods": ["POST", "GET"], + "url": { + "path": "/_seal", + "paths": ["/_seal", "/{index}/_seal"], + "parts": { + "index": { + "type" : "list", + "description" : "A comma-separated list of index names; use `_all` or empty string for all indices" + } + } + }, + "body": null + } +} diff --git a/rest-api-spec/test/indices.seal/10_basic.yaml b/rest-api-spec/test/indices.seal/10_basic.yaml new file mode 100644 index 00000000000..2291cf3d647 --- /dev/null +++ b/rest-api-spec/test/indices.seal/10_basic.yaml @@ -0,0 +1,16 @@ +--- +"Index seal rest test": +- do: + indices.create: + index: testing + +- do: + cluster.health: + wait_for_status: green +- do: + indices.seal: + index: testing +- do: + indices.stats: {level: shards} + +- is_true: indices.testing.shards.0.0.commit.user_data.sync_id diff --git a/src/main/java/org/elasticsearch/action/ActionModule.java b/src/main/java/org/elasticsearch/action/ActionModule.java index c529a3e876f..1c273f67b5b 100644 --- a/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/src/main/java/org/elasticsearch/action/ActionModule.java @@ -103,6 +103,8 @@ import org.elasticsearch.action.admin.indices.settings.put.TransportUpdateSettin import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsAction; import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction; import org.elasticsearch.action.admin.indices.stats.TransportIndicesStatsAction; +import org.elasticsearch.action.admin.indices.seal.SealIndicesAction; +import org.elasticsearch.action.admin.indices.seal.TransportSealIndicesAction; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateAction; import org.elasticsearch.action.admin.indices.template.delete.TransportDeleteIndexTemplateAction; import org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesAction; @@ -254,6 +256,7 @@ public class ActionModule extends AbstractModule { registerAction(ValidateQueryAction.INSTANCE, TransportValidateQueryAction.class); registerAction(RefreshAction.INSTANCE, TransportRefreshAction.class); registerAction(FlushAction.INSTANCE, TransportFlushAction.class); + registerAction(SealIndicesAction.INSTANCE, TransportSealIndicesAction.class); registerAction(OptimizeAction.INSTANCE, TransportOptimizeAction.class); registerAction(ClearIndicesCacheAction.INSTANCE, TransportClearIndicesCacheAction.class); registerAction(PutWarmerAction.INSTANCE, TransportPutWarmerAction.class); diff --git a/src/main/java/org/elasticsearch/action/LatchedActionListener.java b/src/main/java/org/elasticsearch/action/LatchedActionListener.java index fa17ae5c8fe..fb0fd81a7be 100644 --- a/src/main/java/org/elasticsearch/action/LatchedActionListener.java +++ b/src/main/java/org/elasticsearch/action/LatchedActionListener.java @@ -25,7 +25,7 @@ import java.util.concurrent.CountDownLatch; * An action listener that allows passing in a {@link CountDownLatch} that * will be counted down after onResponse or onFailure is called */ -public final class LatchedActionListener implements ActionListener { +public class LatchedActionListener implements ActionListener { private final ActionListener delegate; private final CountDownLatch latch; diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesAction.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesAction.java new file mode 100644 index 00000000000..fbb01b05abe --- /dev/null +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesAction.java @@ -0,0 +1,45 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.admin.indices.seal; + +import org.elasticsearch.action.Action; +import org.elasticsearch.client.ElasticsearchClient; + +/** + */ +public class SealIndicesAction extends Action { + + public static final SealIndicesAction INSTANCE = new SealIndicesAction(); + public static final String NAME = "indices:admin/seal"; + + private SealIndicesAction() { + super(NAME); + } + + @Override + public SealIndicesResponse newResponse() { + return new SealIndicesResponse(); + } + + @Override + public SealIndicesRequestBuilder newRequestBuilder(ElasticsearchClient client) { + return new SealIndicesRequestBuilder(client, this); + } +} diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequest.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequest.java new file mode 100644 index 00000000000..42cdc51ed32 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequest.java @@ -0,0 +1,49 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.admin.indices.seal; + +import org.elasticsearch.action.support.broadcast.BroadcastOperationRequest; + +import java.util.Arrays; + +/** + * A request to seal one or more indices. + */ +public class SealIndicesRequest extends BroadcastOperationRequest { + + SealIndicesRequest() { + } + + /** + * Constructs a seal request against one or more indices. If nothing is provided, all indices will + * be sealed. + */ + public SealIndicesRequest(String... indices) { + super(indices); + } + + @Override + public String toString() { + return "SealIndicesRequest{" + + "indices=" + Arrays.toString(indices) + + ", indicesOptions=" + indicesOptions() + + '}'; + } +} diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequestBuilder.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequestBuilder.java new file mode 100644 index 00000000000..a424ab3fc3b --- /dev/null +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequestBuilder.java @@ -0,0 +1,38 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.admin.indices.seal; + +import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.client.ElasticsearchClient; + +/** + * + */ +public class SealIndicesRequestBuilder extends ActionRequestBuilder { + + public SealIndicesRequestBuilder(ElasticsearchClient client, SealIndicesAction action) { + super(client, action, new SealIndicesRequest()); + } + + public SealIndicesRequestBuilder indices(String ... indices) { + request.indices(indices); + return this; + } +} diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java new file mode 100644 index 00000000000..1dfd47795a5 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java @@ -0,0 +1,171 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.admin.indices.seal; + +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.indices.SyncedFlushService; +import org.elasticsearch.rest.RestStatus; + +import java.io.IOException; +import java.util.*; + +/** + * A response to a seal action on several indices. + */ +public class SealIndicesResponse extends ActionResponse implements ToXContent { + + final private Set results; + + private RestStatus restStatus; + + SealIndicesResponse() { + results = new HashSet<>(); + } + + SealIndicesResponse(Set results) { + this.results = results; + if (allShardsFailed()) { + restStatus = RestStatus.CONFLICT; + } else if (someShardsFailed()) { + restStatus = RestStatus.PARTIAL_CONTENT; + } else { + restStatus = RestStatus.OK; + } + } + + public RestStatus status() { + return restStatus; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + int size = in.readVInt(); + results.clear(); + for (int i = 0; i < size; i++) { + SyncedFlushService.SyncedFlushResult syncedFlushResult = new SyncedFlushService.SyncedFlushResult(); + syncedFlushResult.readFrom(in); + results.add(syncedFlushResult); + } + restStatus = RestStatus.readFrom(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeVInt(results.size()); + for (SyncedFlushService.SyncedFlushResult syncedFlushResult : results) { + syncedFlushResult.writeTo(out); + } + RestStatus.writeTo(out, restStatus); + } + + public Set results() { + return results; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + Map> allResults = new HashMap<>(); + + // first, sort everything by index and shard id + for (SyncedFlushService.SyncedFlushResult result : results) { + String indexName = result.getShardId().index().name(); + int shardId = result.getShardId().getId(); + + if (allResults.get(indexName) == null) { + // no results yet for this index + allResults.put(indexName, new TreeMap()); + } + if (result.shardResponses().size() > 0) { + Map shardResponses = new HashMap<>(); + for (Map.Entry shardResponse : result.shardResponses().entrySet()) { + shardResponses.put(shardResponse.getKey(), shardResponse.getValue()); + } + allResults.get(indexName).put(shardId, shardResponses); + } else { + allResults.get(indexName).put(shardId, result.failureReason()); + } + } + for (Map.Entry> result : allResults.entrySet()) { + builder.startArray(result.getKey()); + for (Map.Entry shardResponse : result.getValue().entrySet()) { + builder.startObject(); + builder.field("shard_id", shardResponse.getKey()); + if (shardResponse.getValue() instanceof Map) { + builder.startObject("responses"); + Map results = (Map) shardResponse.getValue(); + boolean success = true; + for (Map.Entry shardCopy : results.entrySet()) { + builder.field(shardCopy.getKey().currentNodeId(), shardCopy.getValue().success() ? "success" : shardCopy.getValue().failureReason()); + if (shardCopy.getValue().success() == false) { + success = false; + } + } + builder.endObject(); + builder.field("message", success ? "success" : "failed on some copies"); + + } else { + builder.field("message", shardResponse.getValue()); // must be a string + } + builder.endObject(); + } + builder.endArray(); + } + return builder; + } + + public boolean allShardsFailed() { + for (SyncedFlushService.SyncedFlushResult result : results) { + if (result.success()) { + return false; + } + if (result.shardResponses().size() > 0) { + for (Map.Entry shardResponse : result.shardResponses().entrySet()) { + if (shardResponse.getValue().success()) { + return false; + } + } + } + } + return true; + } + + public boolean someShardsFailed() { + for (SyncedFlushService.SyncedFlushResult result : results) { + if (result.success() == false) { + return true; + } + if (result.shardResponses().size() > 0) { + for (Map.Entry shardResponse : result.shardResponses().entrySet()) { + if (shardResponse.getValue().success() == false) { + return true; + } + } + } + } + return false; + } +} diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java new file mode 100644 index 00000000000..61d5e53c32f --- /dev/null +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java @@ -0,0 +1,100 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.admin.indices.seal; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.cluster.routing.ShardIterator; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesLifecycle; +import org.elasticsearch.indices.SyncedFlushService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.*; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; + +/** + */ +public class TransportSealIndicesAction extends HandledTransportAction { + + + final private SyncedFlushService syncedFlushService; + final private ClusterService clusterService; + + @Inject + public TransportSealIndicesAction(Settings settings, ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, SyncedFlushService syncedFlushService, ClusterService clusterService) { + super(settings, SealIndicesAction.NAME, threadPool, transportService, actionFilters, SealIndicesRequest.class); + this.syncedFlushService = syncedFlushService; + this.clusterService = clusterService; + } + + @Override + protected void doExecute(final SealIndicesRequest request, final ActionListener listener) { + ClusterState state = clusterService.state(); + String[] concreteIndices = state.metaData().concreteIndices(request.indicesOptions(), request.indices()); + GroupShardsIterator primaries = state.routingTable().activePrimaryShardsGrouped(concreteIndices, true); + final Set results = ConcurrentCollections.newConcurrentSet(); + + final CountDown countDown = new CountDown(primaries.size()); + + for (final ShardIterator shard : primaries) { + if (shard.size() == 0) { + results.add(new SyncedFlushService.SyncedFlushResult(shard.shardId(), "no active primary available")); + if (countDown.countDown()) { + listener.onResponse(new SealIndicesResponse(results)); + } + } else { + final ShardId shardId = shard.shardId(); + syncedFlushService.attemptSyncedFlush(shardId, new ActionListener() { + @Override + public void onResponse(SyncedFlushService.SyncedFlushResult syncedFlushResult) { + results.add(syncedFlushResult); + if (countDown.countDown()) { + listener.onResponse(new SealIndicesResponse(results)); + } + } + + @Override + public void onFailure(Throwable e) { + logger.debug("{} unexpected error while executing synced flush", shardId); + results.add(new SyncedFlushService.SyncedFlushResult(shardId, e.getMessage())); + if (countDown.countDown()) { + listener.onResponse(new SealIndicesResponse(results)); + } + } + }); + } + } + + } +} diff --git a/src/main/java/org/elasticsearch/client/IndicesAdminClient.java b/src/main/java/org/elasticsearch/client/IndicesAdminClient.java index 239f48e3b60..c54aaece7f4 100644 --- a/src/main/java/org/elasticsearch/client/IndicesAdminClient.java +++ b/src/main/java/org/elasticsearch/client/IndicesAdminClient.java @@ -84,6 +84,9 @@ import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRespons import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequestBuilder; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; +import org.elasticsearch.action.admin.indices.seal.SealIndicesRequest; +import org.elasticsearch.action.admin.indices.seal.SealIndicesRequestBuilder; +import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequest; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequestBuilder; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateResponse; @@ -352,13 +355,34 @@ public interface IndicesAdminClient extends ElasticsearchClient { * @param listener A listener to be notified with a result * @see org.elasticsearch.client.Requests#flushRequest(String...) */ - void flush(FlushRequest request, ActionListener listener); + void flush(FlushRequest request, ActionListener listener); /** * Explicitly flush one or more indices (releasing memory from the node). */ FlushRequestBuilder prepareFlush(String... indices); + /** + * Explicitly sync flush one or more indices + * + * @param request The seal indices request + * @return A result future + */ + ActionFuture sealIndices(SealIndicesRequest request); + + /** + * Explicitly sync flush one or more indices + * + * @param request The seal indices request + * @param listener A listener to be notified with a result + */ + void sealIndices(SealIndicesRequest request, ActionListener listener); + + /** + * Explicitly seal one or more indices + */ + SealIndicesRequestBuilder prepareSealIndices(String... indices); + /** * Explicitly optimize one or more indices into a the number of segments. * @@ -729,4 +753,5 @@ public interface IndicesAdminClient extends ElasticsearchClient { * @see #getSettings(org.elasticsearch.action.admin.indices.settings.get.GetSettingsRequest) */ GetSettingsRequestBuilder prepareGetSettings(String... indices); + } diff --git a/src/main/java/org/elasticsearch/client/support/AbstractClient.java b/src/main/java/org/elasticsearch/client/support/AbstractClient.java index 703b03f0dc5..18b0a377ac0 100644 --- a/src/main/java/org/elasticsearch/client/support/AbstractClient.java +++ b/src/main/java/org/elasticsearch/client/support/AbstractClient.java @@ -180,6 +180,10 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequestBuilder; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; +import org.elasticsearch.action.admin.indices.seal.SealIndicesAction; +import org.elasticsearch.action.admin.indices.seal.SealIndicesRequest; +import org.elasticsearch.action.admin.indices.seal.SealIndicesRequestBuilder; +import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateAction; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequest; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequestBuilder; @@ -1323,6 +1327,21 @@ public abstract class AbstractClient extends AbstractComponent implements Client return new FlushRequestBuilder(this, FlushAction.INSTANCE).setIndices(indices); } + @Override + public ActionFuture sealIndices(SealIndicesRequest request) { + return execute(SealIndicesAction.INSTANCE, request); + } + + @Override + public void sealIndices(SealIndicesRequest request, ActionListener listener) { + execute(SealIndicesAction.INSTANCE, request, listener); + } + + @Override + public SealIndicesRequestBuilder prepareSealIndices(String... indices) { + return new SealIndicesRequestBuilder(this, SealIndicesAction.INSTANCE).indices(indices); + } + @Override public void getMappings(GetMappingsRequest request, ActionListener listener) { execute(GetMappingsAction.INSTANCE, request, listener); diff --git a/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java b/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java index 25a8bac2f88..a42a33ee0b7 100644 --- a/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java +++ b/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java @@ -197,7 +197,7 @@ public class RoutingTable implements Iterable, Diffableextra shard iterator will be added for relocating shards. The extra * iterator contains a single ShardRouting pointing at the relocating target */ diff --git a/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java b/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java index 82322b518c1..a17e849b205 100644 --- a/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java +++ b/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java @@ -438,7 +438,7 @@ public abstract class StreamInput extends InputStream { public int[] readIntArray() throws IOException { int length = readVInt(); int[] values = new int[length]; - for(int i=0; i lastSizeMatched) { - lastSizeMatched = sizeMatched; - lastDiscoNodeMatched = discoNode; + String primarySyncId = primaryNodeStore.syncId(); + String replicaSyncId = storeFilesMetaData.syncId(); + // see if we have a sync id we can make use of + if (replicaSyncId != null && replicaSyncId.equals(primarySyncId)) { + logger.trace("{}: node [{}] has same sync id {} as primary", shard, discoNode.name(), replicaSyncId); lastNodeMatched = node; + lastSizeMatched = Long.MAX_VALUE; + lastDiscoNodeMatched = discoNode; + } else { + for (StoreFileMetaData storeFileMetaData : storeFilesMetaData) { + String metaDataFileName = storeFileMetaData.name(); + if (primaryNodeStore.fileExists(metaDataFileName) && primaryNodeStore.file(metaDataFileName).isSame(storeFileMetaData)) { + sizeMatched += storeFileMetaData.length(); + } + } + logger.trace("{}: node [{}] has [{}/{}] bytes of re-usable data", + shard, discoNode.name(), new ByteSizeValue(sizeMatched), sizeMatched); + if (sizeMatched > lastSizeMatched) { + lastSizeMatched = sizeMatched; + lastDiscoNodeMatched = discoNode; + lastNodeMatched = node; + } } } } diff --git a/src/main/java/org/elasticsearch/index/engine/CommitStats.java b/src/main/java/org/elasticsearch/index/engine/CommitStats.java index de0474c5d7e..6e5d26c8e81 100644 --- a/src/main/java/org/elasticsearch/index/engine/CommitStats.java +++ b/src/main/java/org/elasticsearch/index/engine/CommitStats.java @@ -43,7 +43,9 @@ public final class CommitStats implements Streamable, ToXContent { userData = MapBuilder.newMapBuilder().putAll(segmentInfos.getUserData()).immutableMap(); // lucene calls the current generation, last generation. generation = segmentInfos.getLastGeneration(); - id = Base64.encodeBytes(segmentInfos.getId()); + if (segmentInfos.getId() != null) { // id is only written starting with Lucene 5.0 + id = Base64.encodeBytes(segmentInfos.getId()); + } } private CommitStats() { diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index c2f7f63a76a..3a2f1eba93d 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -37,8 +37,12 @@ import org.apache.lucene.search.join.BitDocIdSetFilter; import org.apache.lucene.util.Accountable; import org.apache.lucene.util.Accountables; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.common.Base64; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.ESLogger; @@ -77,6 +81,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; */ public abstract class Engine implements Closeable { + public static final String SYNC_COMMIT_ID = "sync_id"; + protected final ShardId shardId; protected final ESLogger logger; protected final EngineConfig engineConfig; @@ -218,6 +224,21 @@ public abstract class Engine implements Closeable { @Deprecated public abstract void delete(DeleteByQuery delete) throws EngineException; + /** + * Attempts to do a special commit where the given syncID is put into the commit data. The attempt + * succeeds if there are not pending writes in lucene and the current point is equal to the expected one. + * @param syncId id of this sync + * @param expectedCommitId the expected value of + * @return true if the sync commit was made, false o.w. + */ + public abstract SyncedFlushResult syncFlush(String syncId, CommitId expectedCommitId) throws EngineException; + + public enum SyncedFlushResult { + SUCCESS, + COMMIT_MISMATCH, + PENDING_OPERATIONS + } + final protected GetResult getFromSearcher(Get get) throws EngineException { final Searcher searcher = acquireSearcher("get"); final Versions.DocIdAndVersion docIdAndVersion; @@ -439,16 +460,19 @@ public abstract class Engine implements Closeable { * @param force if true a lucene commit is executed even if no changes need to be committed. * @param waitIfOngoing if true this call will block until all currently running flushes have finished. * Otherwise this call will return without blocking. + * @return the commit Id for the resulting commit */ - public abstract void flush(boolean force, boolean waitIfOngoing) throws EngineException; + public abstract CommitId flush(boolean force, boolean waitIfOngoing) throws EngineException; /** * Flushes the state of the engine including the transaction log, clearing memory and persisting * documents in the lucene index to disk including a potentially heavy and durable fsync operation. * This operation is not going to block if another flush operation is currently running and won't write * a lucene commit if nothing needs to be committed. + * + * @return the commit Id for the resulting commit */ - public abstract void flush() throws EngineException; + public abstract CommitId flush() throws EngineException; /** * Optimizes to 1 segment @@ -1100,4 +1124,55 @@ public abstract class Engine implements Closeable { * @return */ public abstract boolean hasUncommittedChanges(); + + public static class CommitId implements Writeable { + + private final byte[] id; + + public CommitId(byte[] id) { + assert id != null; + this.id = Arrays.copyOf(id, id.length); + } + + public CommitId(StreamInput in) throws IOException { + assert in != null; + this.id = in.readByteArray(); + } + + @Override + public String toString() { + return Base64.encodeBytes(id); + } + + @Override + public CommitId readFrom(StreamInput in) throws IOException { + return new CommitId(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeByteArray(id); + } + + public boolean idsEqual(byte[] id) { + return Arrays.equals(id, this.id); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + CommitId commitId = (CommitId) o; + + if (!Arrays.equals(id, commitId.id)) return false; + + return true; + } + + @Override + public int hashCode() { + return Arrays.hashCode(id); + } + } } diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index ac3fb1b1d74..8504340857d 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -77,7 +77,9 @@ public class InternalEngine extends Engine { private final FailEngineOnMergeFailure mergeSchedulerFailureListener; private final MergeSchedulerListener mergeSchedulerListener; - /** When we last pruned expired tombstones from versionMap.deletes: */ + /** + * When we last pruned expired tombstones from versionMap.deletes: + */ private volatile long lastDeleteVersionPruneTimeMSec; private final ShardIndexingService indexingService; @@ -152,7 +154,7 @@ public class InternalEngine extends Engine { try { if (skipInitialTranslogRecovery) { // make sure we point at the latest translog from now on.. - commitIndexWriter(writer, translog); + commitIndexWriter(writer, translog, lastCommittedSegmentInfos.getUserData().get(SYNC_COMMIT_ID)); } else { recoverFromTranslog(engineConfig, translogGeneration); } @@ -234,10 +236,12 @@ public class InternalEngine extends Engine { // flush if we recovered something or if we have references to older translogs // note: if opsRecovered == 0 and we have older translogs it means they are corrupted or 0 length. - if (opsRecovered > 0 || translog.isCurrent(translogGeneration) == false) { + if (opsRecovered > 0) { logger.trace("flushing post recovery from translog. ops recovered [{}]. committed translog id [{}]. current id [{}]", opsRecovered, translogGeneration == null ? null : translogGeneration.translogFileGeneration, translog.currentFileGeneration()); flush(true, true); + } else if (translog.isCurrent(translogGeneration) == false){ + commitIndexWriter(indexWriter, translog, lastCommittedSegmentInfos.getUserData().get(Engine.SYNC_COMMIT_ID)); } } @@ -674,13 +678,47 @@ public class InternalEngine extends Engine { } @Override - public void flush() throws EngineException { - flush(false, false); + public SyncedFlushResult syncFlush(String syncId, CommitId expectedCommitId) throws EngineException { + // best effort attempt before we acquire locks + ensureOpen(); + if (indexWriter.hasUncommittedChanges()) { + logger.trace("can't sync commit [{}]. have pending changes", syncId); + return SyncedFlushResult.PENDING_OPERATIONS; + } + if (expectedCommitId.idsEqual(lastCommittedSegmentInfos.getId()) == false) { + logger.trace("can't sync commit [{}]. current commit id is not equal to expected.", syncId); + return SyncedFlushResult.COMMIT_MISMATCH; + } + try (ReleasableLock lock = writeLock.acquire()) { + ensureOpen(); + if (indexWriter.hasUncommittedChanges()) { + logger.trace("can't sync commit [{}]. have pending changes", syncId); + return SyncedFlushResult.PENDING_OPERATIONS; + } + if (expectedCommitId.idsEqual(lastCommittedSegmentInfos.getId()) == false) { + logger.trace("can't sync commit [{}]. current commit id is not equal to expected.", syncId); + return SyncedFlushResult.COMMIT_MISMATCH; + } + logger.trace("starting sync commit [{}]", syncId); + commitIndexWriter(indexWriter, translog, syncId); + logger.debug("successfully sync committed. sync id [{}].", syncId); + lastCommittedSegmentInfos = store.readLastCommittedSegmentsInfo(); + return SyncedFlushResult.SUCCESS; + } catch (IOException ex) { + maybeFailEngine("sync commit", ex); + throw new EngineException(shardId, "failed to sync commit", ex); + } } @Override - public void flush(boolean force, boolean waitIfOngoing) throws EngineException { + public CommitId flush() throws EngineException { + return flush(false, false); + } + + @Override + public CommitId flush(boolean force, boolean waitIfOngoing) throws EngineException { ensureOpen(); + final byte[] newCommitId; /* * Unfortunately the lock order is important here. We have to acquire the readlock first otherwise * if we are flushing at the end of the recovery while holding the write lock we can deadlock if: @@ -736,6 +774,7 @@ public class InternalEngine extends Engine { } finally { store.decRef(); } + newCommitId = lastCommittedSegmentInfos.getId(); } catch (FlushFailedEngineException ex) { maybeFailEngine("flush", ex); throw ex; @@ -748,6 +787,7 @@ public class InternalEngine extends Engine { if (engineConfig.isEnableGcDeletes()) { pruneDeletedTombstones(); } + return new CommitId(newCommitId); } private void pruneDeletedTombstones() { @@ -981,7 +1021,8 @@ public class InternalEngine extends Engine { boolean verbose = false; try { verbose = Boolean.parseBoolean(System.getProperty("tests.verbose")); - } catch (Throwable ignore) {} + } catch (Throwable ignore) { + } iwc.setInfoStream(verbose ? InfoStream.getDefault() : new LoggerInfoStream(logger)); iwc.setMergeScheduler(mergeScheduler.newMergeScheduler()); MergePolicy mergePolicy = mergePolicyProvider.getMergePolicy(); @@ -1031,10 +1072,8 @@ public class InternalEngine extends Engine { throw ex; } } - /** Extended SearcherFactory that warms the segments if needed when acquiring a new searcher */ final static class SearchFactory extends EngineSearcherFactory { - private final IndicesWarmer warmer; private final ShardId shardId; private final ESLogger logger; @@ -1164,14 +1203,16 @@ public class InternalEngine extends Engine { } } - - private void commitIndexWriter(IndexWriter writer, Translog translog) throws IOException { + private void commitIndexWriter(IndexWriter writer, Translog translog, String syncId) throws IOException { try { Translog.TranslogGeneration translogGeneration = translog.getGeneration(); - logger.trace("committing writer with translog id [{}] ", translogGeneration.translogFileGeneration); + logger.trace("committing writer with translog id [{}] and sync id [{}] ", translogGeneration.translogFileGeneration, syncId); Map commitData = new HashMap<>(2); commitData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(translogGeneration.translogFileGeneration)); commitData.put(Translog.TRANSLOG_UUID_KEY, translogGeneration.translogUUID); + if (syncId != null) { + commitData.put(Engine.SYNC_COMMIT_ID, syncId); + } indexWriter.setCommitData(commitData); writer.commit(); } catch (Throwable ex) { @@ -1179,4 +1220,8 @@ public class InternalEngine extends Engine { throw ex; } } + + private void commitIndexWriter(IndexWriter writer, Translog translog) throws IOException { + commitIndexWriter(writer, translog, null); + } } diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index d967d3f8b8d..e09acaa0e28 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -125,12 +125,17 @@ public class ShadowEngine extends Engine { } @Override - public void flush() throws EngineException { - flush(false, false); + public SyncedFlushResult syncFlush(String syncId, CommitId expectedCommitId) { + throw new UnsupportedOperationException(shardId + " sync commit operation not allowed on shadow engine"); } @Override - public void flush(boolean force, boolean waitIfOngoing) throws EngineException { + public CommitId flush() throws EngineException { + return flush(false, false); + } + + @Override + public CommitId flush(boolean force, boolean waitIfOngoing) throws EngineException { logger.trace("skipping FLUSH on shadow engine"); // reread the last committed segment infos refresh("flush"); @@ -154,6 +159,7 @@ public class ShadowEngine extends Engine { } finally { store.decRef(); } + return new CommitId(lastCommittedSegmentInfos.getId()); } @Override diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index b39e9665811..c5a4209ab84 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -689,17 +689,28 @@ public class IndexShard extends AbstractIndexShardComponent { return completionStats; } - public void flush(FlushRequest request) { + public Engine.SyncedFlushResult syncFlush(String syncId, Engine.CommitId expectedCommitId) { + verifyStartedOrRecovering(); + logger.trace("trying to sync flush. sync id [{}]. expected commit id [{}]]", syncId, expectedCommitId); + return engine().syncFlush(syncId, expectedCommitId); + } + + public Engine.CommitId flush(FlushRequest request) throws ElasticsearchException { + boolean waitIfOngoing = request.waitIfOngoing(); + boolean force = request.force(); + if (logger.isTraceEnabled()) { + logger.trace("flush with {}", request); + } // we allows flush while recovering, since we allow for operations to happen // while recovering, and we want to keep the translog at bay (up to deletes, which // we don't gc). verifyStartedOrRecovering(); - if (logger.isTraceEnabled()) { - logger.trace("flush with {}", request); - } + long time = System.nanoTime(); - engine().flush(request.force(), request.waitIfOngoing()); + Engine.CommitId commitId = engine().flush(force, waitIfOngoing); flushMetric.inc(System.nanoTime() - time); + return commitId; + } public void optimize(OptimizeRequest optimize) { @@ -999,6 +1010,7 @@ public class IndexShard extends AbstractIndexShardComponent { public void markAsInactive() { updateBufferSize(EngineConfig.INACTIVE_SHARD_INDEXING_BUFFER, TranslogConfig.INACTIVE_SHARD_TRANSLOG_BUFFER); + indicesLifecycle.onShardInactive(this); } public final boolean isFlushOnClose() { @@ -1387,4 +1399,5 @@ public class IndexShard extends AbstractIndexShardComponent { return defaultValue; } } + } diff --git a/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardRepository.java b/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardRepository.java index d8c906886b0..707e77dbefe 100644 --- a/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardRepository.java +++ b/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardRepository.java @@ -186,7 +186,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements @Override public void verify(String seed) { - BlobContainer testBlobContainer = blobStore.blobContainer(basePath);; + BlobContainer testBlobContainer = blobStore.blobContainer(basePath); DiscoveryNode localNode = clusterService.localNode(); if (testBlobContainer.blobExists(testBlobPrefix(seed) + "-master")) { try (OutputStream outputStream = testBlobContainer.createOutput(testBlobPrefix(seed) + "-" + localNode.getId())) { @@ -232,7 +232,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements * Serializes snapshot to JSON * * @param snapshot snapshot - * @param stream the stream to output the snapshot JSON represetation to + * @param stream the stream to output the snapshot JSON represetation to * @throws IOException if an IOException occurs */ public static void writeSnapshot(BlobStoreIndexShardSnapshot snapshot, OutputStream stream) throws IOException { @@ -247,7 +247,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements * @param stream JSON * @return snapshot * @throws IOException if an IOException occurs - * */ + */ public static BlobStoreIndexShardSnapshot readSnapshot(InputStream stream) throws IOException { try (XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(stream)) { parser.nextToken(); @@ -314,7 +314,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements public BlobStoreIndexShardSnapshot loadSnapshot() { BlobStoreIndexShardSnapshot snapshot; try (InputStream stream = blobContainer.openInput(snapshotBlobName(snapshotId))) { - snapshot = readSnapshot(stream); + snapshot = readSnapshot(stream); } catch (IOException ex) { throw new IndexShardRestoreFailedException(shardId, "failed to read shard snapshot file", ex); } @@ -472,7 +472,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements // don't have this hash we try to read that hash from the blob store // in a bwc compatible way. maybeRecalculateMetadataHash(blobContainer, fileInfo, metadata); - } catch (Throwable e) { + } catch (Throwable e) { logger.warn("{} Can't calculate hash from blob for file [{}] [{}]", e, shardId, fileInfo.physicalName(), fileInfo.metadata()); } if (fileInfo == null || !fileInfo.isSame(md) || !snapshotFileExistsInBlobs(fileInfo, blobs)) { @@ -550,7 +550,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements try (IndexInput indexInput = store.openVerifyingInput(file, IOContext.READONCE, fileInfo.metadata())) { for (int i = 0; i < fileInfo.numberOfParts(); i++) { final InputStreamIndexInput inputStreamIndexInput = new InputStreamIndexInput(indexInput, fileInfo.partBytes()); - InputStream inputStream = snapshotRateLimiter == null ? inputStreamIndexInput : new RateLimitingInputStream(inputStreamIndexInput, snapshotRateLimiter, snapshotThrottleListener); + InputStream inputStream = snapshotRateLimiter == null ? inputStreamIndexInput : new RateLimitingInputStream(inputStreamIndexInput, snapshotRateLimiter, snapshotThrottleListener); inputStream = new AbortableInputStream(inputStream, fileInfo.physicalName()); try (OutputStream output = blobContainer.createOutput(fileInfo.partName(i))) { int len; @@ -727,14 +727,14 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements // don't have this hash we try to read that hash from the blob store // in a bwc compatible way. maybeRecalculateMetadataHash(blobContainer, fileInfo, recoveryTargetMetadata); - } catch (Throwable e) { + } catch (Throwable e) { // if the index is broken we might not be able to read it logger.warn("{} Can't calculate hash from blog for file [{}] [{}]", e, shardId, fileInfo.physicalName(), fileInfo.metadata()); } snapshotMetaData.put(fileInfo.metadata().name(), fileInfo.metadata()); fileInfos.put(fileInfo.metadata().name(), fileInfo); } - final Store.MetadataSnapshot sourceMetaData = new Store.MetadataSnapshot(snapshotMetaData); + final Store.MetadataSnapshot sourceMetaData = new Store.MetadataSnapshot(snapshotMetaData, Collections.EMPTY_MAP, 0); final Store.RecoveryDiff diff = sourceMetaData.recoveryDiff(recoveryTargetMetadata); for (StoreFileMetaData md : diff.identical) { FileInfo fileInfo = fileInfos.get(md.name()); @@ -804,8 +804,8 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements try (final IndexOutput indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), fileInfo.metadata(), IOContext.DEFAULT)) { final byte[] buffer = new byte[BUFFER_SIZE]; int length; - while((length=stream.read(buffer))>0){ - indexOutput.writeBytes(buffer,0,length); + while ((length = stream.read(buffer)) > 0) { + indexOutput.writeBytes(buffer, 0, length); recoveryState.getIndex().addRecoveredBytesToFile(fileInfo.name(), length); if (restoreRateLimiter != null) { rateLimiterListener.onRestorePause(restoreRateLimiter.pause(length)); @@ -838,7 +838,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements } } - + public interface RateLimiterListener { void onRestorePause(long nanos); diff --git a/src/main/java/org/elasticsearch/index/store/Store.java b/src/main/java/org/elasticsearch/index/store/Store.java index 5342ae870f3..edcd930b8d6 100644 --- a/src/main/java/org/elasticsearch/index/store/Store.java +++ b/src/main/java/org/elasticsearch/index/store/Store.java @@ -28,13 +28,14 @@ import org.apache.lucene.store.*; import org.apache.lucene.util.*; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; +import org.apache.lucene.util.Version; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.Lucene; @@ -46,6 +47,7 @@ import org.elasticsearch.common.util.SingleObjectCache; import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.common.util.concurrent.RefCounted; import org.elasticsearch.env.ShardLock; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.ShardId; @@ -166,10 +168,10 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref * Returns a new MetadataSnapshot for the latest commit in this store or * an empty snapshot if no index exists or can not be opened. * - * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an - * unexpected exception when opening the index reading the segments file. - * @throws IndexFormatTooOldException if the lucene index is too old to be opened. - * @throws IndexFormatTooNewException if the lucene index is too new to be opened. + * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an + * unexpected exception when opening the index reading the segments file. + * @throws IndexFormatTooOldException if the lucene index is too old to be opened. + * @throws IndexFormatTooNewException if the lucene index is too new to be opened. */ public MetadataSnapshot getMetadataOrEmpty() throws IOException { try { @@ -185,13 +187,13 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref /** * Returns a new MetadataSnapshot for the latest commit in this store. * - * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an - * unexpected exception when opening the index reading the segments file. - * @throws IndexFormatTooOldException if the lucene index is too old to be opened. - * @throws IndexFormatTooNewException if the lucene index is too new to be opened. - * @throws FileNotFoundException if one or more files referenced by a commit are not present. - * @throws NoSuchFileException if one or more files referenced by a commit are not present. - * @throws IndexNotFoundException if no index / valid commit-point can be found in this store + * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an + * unexpected exception when opening the index reading the segments file. + * @throws IndexFormatTooOldException if the lucene index is too old to be opened. + * @throws IndexFormatTooNewException if the lucene index is too new to be opened. + * @throws FileNotFoundException if one or more files referenced by a commit are not present. + * @throws NoSuchFileException if one or more files referenced by a commit are not present. + * @throws IndexNotFoundException if no index / valid commit-point can be found in this store */ public MetadataSnapshot getMetadata() throws IOException { return getMetadata(null); @@ -201,13 +203,13 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref * Returns a new MetadataSnapshot for the given commit. If the given commit is null * the latest commit point is used. * - * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an - * unexpected exception when opening the index reading the segments file. - * @throws IndexFormatTooOldException if the lucene index is too old to be opened. - * @throws IndexFormatTooNewException if the lucene index is too new to be opened. - * @throws FileNotFoundException if one or more files referenced by a commit are not present. - * @throws NoSuchFileException if one or more files referenced by a commit are not present. - * @throws IndexNotFoundException if the commit point can't be found in this store + * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an + * unexpected exception when opening the index reading the segments file. + * @throws IndexFormatTooOldException if the lucene index is too old to be opened. + * @throws IndexFormatTooNewException if the lucene index is too new to be opened. + * @throws FileNotFoundException if one or more files referenced by a commit are not present. + * @throws NoSuchFileException if one or more files referenced by a commit are not present. + * @throws IndexNotFoundException if the commit point can't be found in this store */ public MetadataSnapshot getMetadata(IndexCommit commit) throws IOException { ensureOpen(); @@ -363,7 +365,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref * @throws IOException if the index we try to read is corrupted */ public static MetadataSnapshot readMetadataSnapshot(Path indexLocation, ESLogger logger) throws IOException { - try (Directory dir = new SimpleFSDirectory(indexLocation)){ + try (Directory dir = new SimpleFSDirectory(indexLocation)) { failIfCorrupted(dir, new ShardId("", 1)); return new MetadataSnapshot(null, dir, logger); } catch (IndexNotFoundException ex) { @@ -433,7 +435,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref public boolean checkIntegrityNoException(StoreFileMetaData md) { return checkIntegrityNoException(md, directory()); } - + public static boolean checkIntegrityNoException(StoreFileMetaData md, Directory directory) { try { checkIntegrity(md, directory); @@ -454,7 +456,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref // throw exception if metadata is inconsistent if (!checksum.equals(md.checksum())) { throw new CorruptIndexException("inconsistent metadata: lucene checksum=" + checksum + - ", metadata checksum=" + md.checksum(), input); + ", metadata checksum=" + md.checksum(), input); } } else if (md.hasLegacyChecksum()) { // legacy checksum verification - no footer that we need to omit in the checksum! @@ -472,7 +474,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref String adler32 = Store.digestToString(checksum.getValue()); if (!adler32.equals(md.checksum())) { throw new CorruptIndexException("checksum failed (hardware problem?) : expected=" + md.checksum() + - " actual=" + adler32, input); + " actual=" + adler32, input); } } } @@ -530,7 +532,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref * * @param reason the reason for this cleanup operation logged for each deleted file * @param sourceMetaData the metadata used for cleanup. all files in this metadata should be kept around. - * @throws IOException if an IOException occurs + * @throws IOException if an IOException occurs * @throws IllegalStateException if the latest snapshot in this store differs from the given one after the cleanup. */ public void cleanupAndVerify(String reason, MetadataSnapshot sourceMetaData) throws IOException { @@ -549,7 +551,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref // FNF should not happen since we hold a write lock? } catch (IOException ex) { if (existingFile.startsWith(IndexFileNames.SEGMENTS) - || existingFile.equals(IndexFileNames.OLD_SEGMENTS_GEN)) { + || existingFile.equals(IndexFileNames.OLD_SEGMENTS_GEN)) { // TODO do we need to also fail this if we can't delete the pending commit file? // if one of those files can't be deleted we better fail the cleanup otherwise we might leave an old commit point around? throw new IllegalStateException("Can't delete " + existingFile + " - cleanup failed", ex); @@ -656,32 +658,87 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref * * @see StoreFileMetaData */ - public final static class MetadataSnapshot implements Iterable, Streamable { + public final static class MetadataSnapshot implements Iterable, Writeable { private static final ESLogger logger = Loggers.getLogger(MetadataSnapshot.class); private static final Version FIRST_LUCENE_CHECKSUM_VERSION = Version.LUCENE_4_8; - private Map metadata; + private final ImmutableMap metadata; public static final MetadataSnapshot EMPTY = new MetadataSnapshot(); - public MetadataSnapshot(Map metadata) { - this.metadata = metadata; + private final ImmutableMap commitUserData; + + private final long numDocs; + + public MetadataSnapshot(Map metadata, Map commitUserData, long numDocs) { + ImmutableMap.Builder metaDataBuilder = ImmutableMap.builder(); + this.metadata = metaDataBuilder.putAll(metadata).build(); + ImmutableMap.Builder commitUserDataBuilder = ImmutableMap.builder(); + this.commitUserData = commitUserDataBuilder.putAll(commitUserData).build(); + this.numDocs = numDocs; } MetadataSnapshot() { - this.metadata = Collections.emptyMap(); + metadata = ImmutableMap.of(); + commitUserData = ImmutableMap.of(); + numDocs = 0; } MetadataSnapshot(IndexCommit commit, Directory directory, ESLogger logger) throws IOException { - metadata = buildMetadata(commit, directory, logger); + LoadedMetadata loadedMetadata = loadMetadata(commit, directory, logger); + metadata = loadedMetadata.fileMetadata; + commitUserData = loadedMetadata.userData; + numDocs = loadedMetadata.numDocs; assert metadata.isEmpty() || numSegmentFiles() == 1 : "numSegmentFiles: " + numSegmentFiles(); } - ImmutableMap buildMetadata(IndexCommit commit, Directory directory, ESLogger logger) throws IOException { + public MetadataSnapshot(StreamInput in) throws IOException { + final int size = in.readVInt(); + final ImmutableMap.Builder metadataBuilder = ImmutableMap.builder(); + for (int i = 0; i < size; i++) { + StoreFileMetaData meta = StoreFileMetaData.readStoreFileMetaData(in); + metadataBuilder.put(meta.name(), meta); + } + final ImmutableMap.Builder commitUserDataBuilder = ImmutableMap.builder(); + int num = in.readVInt(); + for (int i = num; i > 0; i--) { + commitUserDataBuilder.put(in.readString(), in.readString()); + } + + this.commitUserData = commitUserDataBuilder.build(); + this.metadata = metadataBuilder.build(); + this.numDocs = in.readLong(); + assert metadata.isEmpty() || numSegmentFiles() == 1 : "numSegmentFiles: " + numSegmentFiles(); + } + + /** + * Returns the number of documents in this store snapshot + */ + public long getNumDocs() { + return numDocs; + } + + static class LoadedMetadata { + final ImmutableMap fileMetadata; + final ImmutableMap userData; + final long numDocs; + + LoadedMetadata(ImmutableMap fileMetadata, ImmutableMap userData, long numDocs) { + this.fileMetadata = fileMetadata; + this.userData = userData; + this.numDocs = numDocs; + } + } + + static LoadedMetadata loadMetadata(IndexCommit commit, Directory directory, ESLogger logger) throws IOException { + long numDocs; ImmutableMap.Builder builder = ImmutableMap.builder(); Map checksumMap = readLegacyChecksums(directory).v1(); + ImmutableMap.Builder commitUserDataBuilder = ImmutableMap.builder(); try { final SegmentInfos segmentCommitInfos = Store.readSegmentsInfo(commit, directory); + numDocs = Lucene.getNumDocs(segmentCommitInfos); + commitUserDataBuilder.putAll(segmentCommitInfos.getUserData()); Version maxVersion = Version.LUCENE_4_0; // we don't know which version was used to write so we take the max version. for (SegmentCommitInfo info : segmentCommitInfos) { final Version version = info.info.getVersion(); @@ -734,7 +791,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref throw ex; } - return builder.build(); + return new LoadedMetadata(builder.build(), commitUserDataBuilder.build(), numDocs); } /** @@ -955,30 +1012,22 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref return metadata.size(); } - public static MetadataSnapshot read(StreamInput in) throws IOException { - MetadataSnapshot storeFileMetaDatas = new MetadataSnapshot(); - storeFileMetaDatas.readFrom(in); - return storeFileMetaDatas; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - int size = in.readVInt(); - ImmutableMap.Builder builder = ImmutableMap.builder(); - for (int i = 0; i < size; i++) { - StoreFileMetaData meta = StoreFileMetaData.readStoreFileMetaData(in); - builder.put(meta.name(), meta); - } - this.metadata = builder.build(); - assert metadata.isEmpty() || numSegmentFiles() == 1 : "numSegmentFiles: " + numSegmentFiles(); - } - @Override public void writeTo(StreamOutput out) throws IOException { out.writeVInt(this.metadata.size()); for (StoreFileMetaData meta : this) { meta.writeTo(out); } + out.writeVInt(commitUserData.size()); + for (Map.Entry entry : commitUserData.entrySet()) { + out.writeString(entry.getKey()); + out.writeString(entry.getValue()); + } + out.writeLong(numDocs); + } + + public Map getCommitUserData() { + return commitUserData; } /** @@ -1010,6 +1059,20 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref } return count; } + + /** + * Returns the sync id of the commit point that this MetadataSnapshot represents. + * + * @return sync id if exists, else null + */ + public String getSyncId() { + return commitUserData.get(Engine.SYNC_COMMIT_ID); + } + + @Override + public MetadataSnapshot readFrom(StreamInput in) throws IOException { + return new MetadataSnapshot(in); + } } /** @@ -1358,7 +1421,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref } @Override - protected StoreStats refresh() { + protected StoreStats refresh() { try { return new StoreStats(estimateSize(directory), directoryService.throttleTimeInNanos()); } catch (IOException ex) { diff --git a/src/main/java/org/elasticsearch/indices/IndicesLifecycle.java b/src/main/java/org/elasticsearch/indices/IndicesLifecycle.java index 003ba9f9264..39de271bfe8 100644 --- a/src/main/java/org/elasticsearch/indices/IndicesLifecycle.java +++ b/src/main/java/org/elasticsearch/indices/IndicesLifecycle.java @@ -201,6 +201,15 @@ public interface IndicesLifecycle { public void beforeIndexDeleted(IndexService indexService) { } + + /** + * Called when a shard is marked as inactive + * + * @param indexShard The shard that was marked inactive + */ + public void onShardInactive(IndexShard indexShard) { + + } } } diff --git a/src/main/java/org/elasticsearch/indices/IndicesModule.java b/src/main/java/org/elasticsearch/indices/IndicesModule.java index 102fa1854d4..785d2af7eca 100644 --- a/src/main/java/org/elasticsearch/indices/IndicesModule.java +++ b/src/main/java/org/elasticsearch/indices/IndicesModule.java @@ -20,7 +20,6 @@ package org.elasticsearch.indices; import com.google.common.collect.ImmutableList; - import org.elasticsearch.action.update.UpdateHelper; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.Module; @@ -69,6 +68,7 @@ public class IndicesModule extends AbstractModule implements SpawnModules { bind(IndicesStore.class).asEagerSingleton(); bind(IndicesClusterStateService.class).asEagerSingleton(); bind(IndexingMemoryController.class).asEagerSingleton(); + bind(SyncedFlushService.class).asEagerSingleton(); bind(IndicesFilterCache.class).asEagerSingleton(); bind(IndicesQueryCache.class).asEagerSingleton(); bind(IndicesFieldDataCache.class).asEagerSingleton(); diff --git a/src/main/java/org/elasticsearch/indices/InternalIndicesLifecycle.java b/src/main/java/org/elasticsearch/indices/InternalIndicesLifecycle.java index 8e7e6527bef..046acb7f625 100644 --- a/src/main/java/org/elasticsearch/indices/InternalIndicesLifecycle.java +++ b/src/main/java/org/elasticsearch/indices/InternalIndicesLifecycle.java @@ -245,4 +245,15 @@ public class InternalIndicesLifecycle extends AbstractComponent implements Indic } } } + + public void onShardInactive(IndexShard indexShard) { + for (Listener listener : listeners) { + try { + listener.onShardInactive(indexShard); + } catch (Throwable t) { + logger.warn("{} failed to invoke on shard inactive callback", t, indexShard.shardId()); + throw t; + } + } + } } diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java new file mode 100644 index 00000000000..1821da0e2bc --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -0,0 +1,751 @@ +/* + * 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.indices; + +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.ImmutableShardRouting; +import org.elasticsearch.cluster.routing.IndexRoutingTable; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexShardMissingException; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.shard.IllegalIndexShardStateException; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardException; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.*; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +public class SyncedFlushService extends AbstractComponent { + + private static final String PRE_SYNCED_FLUSH_ACTION_NAME = "internal:indices/flush/synced/pre"; + private static final String SYNCED_FLUSH_ACTION_NAME = "internal:indices/flush/synced/sync"; + private static final String IN_FLIGHT_OPS_ACTION_NAME = "internal:indices/flush/synced/in_flight"; + + private final IndicesService indicesService; + private final ClusterService clusterService; + private final TransportService transportService; + + @Inject + public SyncedFlushService(Settings settings, IndicesService indicesService, ClusterService clusterService, TransportService transportService) { + super(settings); + this.indicesService = indicesService; + this.clusterService = clusterService; + this.transportService = transportService; + + transportService.registerRequestHandler(PRE_SYNCED_FLUSH_ACTION_NAME, PreSyncedFlushRequest.class, ThreadPool.Names.FLUSH, new PreSyncedFlushTransportHandler()); + transportService.registerRequestHandler(SYNCED_FLUSH_ACTION_NAME, SyncedFlushRequest.class, ThreadPool.Names.FLUSH, new SyncedFlushTransportHandler()); + transportService.registerRequestHandler(IN_FLIGHT_OPS_ACTION_NAME, InFlightOpsRequest.class, ThreadPool.Names.SAME, new InFlightOpCountTransportHandler()); + indicesService.indicesLifecycle().addListener(new IndicesLifecycle.Listener() { + @Override + public void onShardInactive(final IndexShard indexShard) { + // we only want to call sync flush once, so only trigger it when we are on a primary + if (indexShard.routingEntry().primary()) { + attemptSyncedFlush(indexShard.shardId(), new ActionListener() { + @Override + public void onResponse(SyncedFlushResult syncedFlushResult) { + logger.debug("{} sync flush on inactive shard returned successfully for sync_id: {}", syncedFlushResult.getShardId(), syncedFlushResult.syncId()); + } + + @Override + public void onFailure(Throwable e) { + logger.debug("{} sync flush on inactive shard failed", e, indexShard.shardId()); + } + }); + } + } + }); + } + + /* + * Tries to flush all copies of a shard and write a sync id to it. + * After a synced flush two shard copies may only contain the same sync id if they contain the same documents. + * To ensure this, synced flush works in three steps: + * 1. Flush all shard copies and gather the commit ids for each copy after the flush + * 2. Ensure that there are no ongoing indexing operations on the primary + * 3. Perform an additional flush on each shard copy that writes the sync id + * + * Step 3 is only executed on a shard if + * a) the shard has no uncommitted changes since the last flush + * b) the last flush was the one executed in 1 (use the collected commit id to verify this) + * + * This alone is not enough to ensure that all copies contain the same documents. Without step 2 a sync id would be written for inconsistent copies in the following scenario: + * + * Write operation has completed on a primary and is being sent to replicas. The write request does not reach the replicas until sync flush is finished. + * Step 1 is executed. After the flush the commit points on primary contains a write operation that the replica does not have. + * Step 3 will be executed on primary and replica as well because there are no uncommitted changes on primary (the first flush committed them) and there are no uncommitted + * changes on the replica (the write operation has not reached the replica yet). + * + * Step 2 detects this scenario and fails the whole synced flush if a write operation is ongoing on the primary. + * Together with the conditions for step 3 (same commit id and no uncommitted changes) this guarantees that a snc id will only + * be written on a primary if no write operation was executed between step 1 and step 3 and sync id will only be written on + * the replica if it contains the same changes that the primary contains. + * + * Synced flush is a best effort operation. The sync id may be written on all, some or none of the copies. + **/ + public void attemptSyncedFlush(final ShardId shardId, final ActionListener actionListener) { + try { + final ClusterState state = clusterService.state(); + final IndexShardRoutingTable shardRoutingTable = getActiveShardRoutings(shardId, state); + final List activeShards = shardRoutingTable.activeShards(); + final ActionListener> commitIdsListener = new ActionListener>() { + @Override + public void onResponse(final Map commitIds) { + if (commitIds.isEmpty()) { + actionListener.onResponse(new SyncedFlushResult(shardId, "all shards failed to commit on pre-sync")); + } + final ActionListener inflightOpsListener = new ActionListener() { + @Override + public void onResponse(InFlightOpsResponse response) { + final int inflight = response.opCount(); + assert inflight >= -1; + if (inflight != 1) { // 1 means that there are no write operations are in flight (>1) and the shard is not closed (0). + actionListener.onResponse(new SyncedFlushResult(shardId, "operation counter on primary is non zero [" + inflight + "]")); + } else { + // 3. now send the sync request to all the shards + String syncId = Strings.base64UUID(); + sendSyncRequests(syncId, activeShards, state, commitIds, shardId, actionListener); + } + } + + @Override + public void onFailure(Throwable e) { + actionListener.onFailure(e); + } + }; + // 2. fetch in flight operations + getInflightOpsCount(shardId, state, shardRoutingTable, inflightOpsListener); + } + + @Override + public void onFailure(Throwable e) { + actionListener.onFailure(e); + } + }; + + // 1. send pre-sync flushes to all replicas + sendPreSyncRequests(activeShards, state, shardId, commitIdsListener); + } catch (Throwable t) { + actionListener.onFailure(t); + } + } + + final IndexShardRoutingTable getActiveShardRoutings(ShardId shardId, ClusterState state) { + final IndexRoutingTable indexRoutingTable = state.routingTable().index(shardId.index().name()); + if (indexRoutingTable == null) { + IndexMetaData index = state.getMetaData().index(shardId.index().getName()); + if (index != null && index.state() == IndexMetaData.State.CLOSE) { + throw new IndexClosedException(shardId.index()); + } + throw new IndexMissingException(shardId.index()); + } + final IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(shardId.id()); + if (shardRoutingTable == null) { + throw new IndexShardMissingException(shardId); + } + return shardRoutingTable; + } + + /** + * returns the number of inflight operations on primary. -1 upon error. + */ + protected void getInflightOpsCount(final ShardId shardId, ClusterState state, IndexShardRoutingTable shardRoutingTable, final ActionListener listener) { + final ShardRouting primaryShard = shardRoutingTable.primaryShard(); + final DiscoveryNode primaryNode = state.nodes().get(primaryShard.currentNodeId()); + if (primaryNode == null) { + logger.trace("{} failed to resolve node for primary shard {}, skipping sync", shardId, primaryShard); + listener.onResponse(new InFlightOpsResponse(-1)); + return; + } + logger.trace("{} retrieving in flight operation count", shardId); + transportService.sendRequest(primaryNode, IN_FLIGHT_OPS_ACTION_NAME, new InFlightOpsRequest(shardId), + new BaseTransportResponseHandler() { + @Override + public InFlightOpsResponse newInstance() { + return new InFlightOpsResponse(); + } + + @Override + public void handleResponse(InFlightOpsResponse response) { + listener.onResponse(response); + } + + @Override + public void handleException(TransportException exp) { + logger.debug("{} unexpected error while retrieving inflight op count", shardId); + listener.onFailure(exp); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + }); + } + + + void sendSyncRequests(final String syncId, final List shards, ClusterState state, Map expectedCommitIds, final ShardId shardId, final ActionListener listener) { + final CountDown countDown = new CountDown(shards.size()); + final Map results = ConcurrentCollections.newConcurrentMap(); + for (final ShardRouting shard : shards) { + final DiscoveryNode node = state.nodes().get(shard.currentNodeId()); + if (node == null) { + logger.trace("{} is assigned to an unknown node. skipping for sync id [{}]. shard routing {}", shardId, syncId, shard); + results.put(shard, new SyncedFlushResponse("unknown node")); + contDownAndSendResponseIfDone(syncId, shards, shardId, listener, countDown, results); + continue; + } + final Engine.CommitId expectedCommitId = expectedCommitIds.get(shard.currentNodeId()); + if (expectedCommitId == null) { + logger.trace("{} can't resolve expected commit id for {}, skipping for sync id [{}]. shard routing {}", shardId, syncId, shard); + results.put(shard, new SyncedFlushResponse("no commit id from pre-sync flush")); + contDownAndSendResponseIfDone(syncId, shards, shardId, listener, countDown, results); + continue; + } + logger.trace("{} sending synced flush request to {}. sync id [{}].", shardId, shard, syncId); + transportService.sendRequest(node, SYNCED_FLUSH_ACTION_NAME, new SyncedFlushRequest(shard.shardId(), syncId, expectedCommitId), + new BaseTransportResponseHandler() { + @Override + public SyncedFlushResponse newInstance() { + return new SyncedFlushResponse(); + } + + @Override + public void handleResponse(SyncedFlushResponse response) { + SyncedFlushResponse existing = results.put(shard, response); + assert existing == null : "got two answers for node [" + node + "]"; + // count after the assert so we won't decrement twice in handleException + contDownAndSendResponseIfDone(syncId, shards, shardId, listener, countDown, results); + } + + @Override + public void handleException(TransportException exp) { + logger.trace("{} error while performing synced flush on [{}], skipping", exp, shardId, shard); + results.put(shard, new SyncedFlushResponse(exp.getMessage())); + contDownAndSendResponseIfDone(syncId, shards, shardId, listener, countDown, results); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + }); + } + + } + + private void contDownAndSendResponseIfDone(String syncId, List shards, ShardId shardId, ActionListener listener, CountDown countDown, Map results) { + if (countDown.countDown()) { + assert results.size() == shards.size(); + listener.onResponse(new SyncedFlushResult(shardId, syncId, results)); + } + } + + /** + * send presync requests to all started copies of the given shard + */ + void sendPreSyncRequests(final List shards, final ClusterState state, final ShardId shardId, final ActionListener> listener) { + final CountDown countDown = new CountDown(shards.size()); + final ConcurrentMap commitIds = ConcurrentCollections.newConcurrentMap(); + for (final ShardRouting shard : shards) { + logger.trace("{} sending pre-synced flush request to {}", shardId, shard); + final DiscoveryNode node = state.nodes().get(shard.currentNodeId()); + if (node == null) { + logger.trace("{} shard routing {} refers to an unknown node. skipping.", shardId, shard); + if(countDown.countDown()) { + listener.onResponse(commitIds); + } + continue; + } + transportService.sendRequest(node, PRE_SYNCED_FLUSH_ACTION_NAME, new PreSyncedFlushRequest(shard.shardId()), new BaseTransportResponseHandler() { + @Override + public PreSyncedFlushResponse newInstance() { + return new PreSyncedFlushResponse(); + } + + @Override + public void handleResponse(PreSyncedFlushResponse response) { + Engine.CommitId existing = commitIds.putIfAbsent(node.id(), response.commitId()); + assert existing == null : "got two answers for node [" + node + "]"; + // count after the assert so we won't decrement twice in handleException + if(countDown.countDown()) { + listener.onResponse(commitIds); + } + } + + @Override + public void handleException(TransportException exp) { + logger.trace("{} error while performing pre synced flush on [{}], skipping", shardId, exp, shard); + if(countDown.countDown()) { + listener.onResponse(commitIds); + } + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + }); + } + } + + private PreSyncedFlushResponse performPreSyncedFlush(PreSyncedFlushRequest request) { + IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).shardSafe(request.shardId().id()); + FlushRequest flushRequest = new FlushRequest().force(false).waitIfOngoing(true); + logger.trace("{} performing pre sync flush", request.shardId()); + Engine.CommitId commitId = indexShard.flush(flushRequest); + logger.trace("{} pre sync flush done. commit id {}", request.shardId(), commitId); + return new PreSyncedFlushResponse(commitId); + } + + private SyncedFlushResponse performSyncedFlush(SyncedFlushRequest request) { + IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); + IndexShard indexShard = indexService.shardSafe(request.shardId().id()); + logger.trace("{} performing sync flush. sync id [{}], expected commit id {}", request.shardId(), request.syncId(), request.expectedCommitId()); + Engine.SyncedFlushResult result = indexShard.syncFlush(request.syncId(), request.expectedCommitId()); + logger.trace("{} sync flush done. sync id [{}], result [{}]", request.shardId(), request.syncId(), result); + switch (result) { + case SUCCESS: + return new SyncedFlushResponse(); + case COMMIT_MISMATCH: + return new SyncedFlushResponse("commit has changed"); + case PENDING_OPERATIONS: + return new SyncedFlushResponse("pending operations"); + default: + throw new ElasticsearchException("unknown synced flush result [" + result + "]"); + } + } + + private InFlightOpsResponse performInFlightOps(InFlightOpsRequest request) { + IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); + IndexShard indexShard = indexService.shardSafe(request.shardId().id()); + if (indexShard.routingEntry().primary() == false) { + throw new IndexShardException(request.shardId(), "expected a primary shard"); + } + int opCount = indexShard.getOperationsCount(); + logger.trace("{} in flight operations sampled at [{}]", request.shardId(), opCount); + return new InFlightOpsResponse(opCount); + } + + /** + * Result for all copies of a shard + */ + public static class SyncedFlushResult extends TransportResponse { + private String failureReason; + private Map shardResponses; + private String syncId; + private ShardId shardId; + + public SyncedFlushResult() { + } + + public ShardId getShardId() { + return shardId; + } + + /** + * failure constructor + */ + public SyncedFlushResult(ShardId shardId, String failureReason) { + this.syncId = null; + this.failureReason = failureReason; + this.shardResponses = ImmutableMap.of(); + this.shardId = shardId; + } + + /** + * success constructor + */ + public SyncedFlushResult(ShardId shardId, String syncId, Map shardResponses) { + this.failureReason = null; + ImmutableMap.Builder builder = ImmutableMap.builder(); + this.shardResponses = builder.putAll(shardResponses).build(); + this.syncId = syncId; + this.shardId = shardId; + } + + /** + * @return true if one or more shard copies was successful, false if all failed before step three of synced flush + */ + public boolean success() { + return syncId != null; + } + + /** + * @return the reason for the failure if synced flush failed before step three of synced flush + */ + public String failureReason() { + return failureReason; + } + + public String syncId() { + return syncId; + } + + /** + * @return total number of shards for which a sync attempt was made + */ + public int totalShards() { + return shardResponses.size(); + } + + /** + * @return total number of successful shards + */ + public int successfulShards() { + int i = 0; + for (SyncedFlushResponse result : shardResponses.values()) { + if (result.success()) { + i++; + } + } + return i; + } + + /** + * @return Individual responses for each shard copy with a detailed failure message if the copy failed to perform the synced flush. + * Empty if synced flush failed before step three. + */ + public Map shardResponses() { + return shardResponses; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeOptionalString(failureReason); + out.writeOptionalString(syncId); + out.writeVInt(shardResponses.size()); + for (Map.Entry result : shardResponses.entrySet()) { + result.getKey().writeTo(out); + result.getValue().writeTo(out); + } + shardId.writeTo(out); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + failureReason = in.readOptionalString(); + syncId = in.readOptionalString(); + int size = in.readVInt(); + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (int i = 0; i < size; i++) { + ImmutableShardRouting shardRouting = ImmutableShardRouting.readShardRoutingEntry(in); + SyncedFlushResponse syncedFlushRsponse = new SyncedFlushResponse(); + syncedFlushRsponse.readFrom(in); + builder.put(shardRouting, syncedFlushRsponse); + } + shardResponses = builder.build(); + shardId = ShardId.readShardId(in); + } + + public ShardId shardId() { + return shardId; + } + } + + final static class PreSyncedFlushRequest extends TransportRequest { + private ShardId shardId; + + PreSyncedFlushRequest() { + } + + public PreSyncedFlushRequest(ShardId shardId) { + this.shardId = shardId; + } + + @Override + public String toString() { + return "PreSyncedFlushRequest{" + + "shardId=" + shardId + + '}'; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + shardId.writeTo(out); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + this.shardId = ShardId.readShardId(in); + } + + public ShardId shardId() { + return shardId; + } + } + + /** + * Response for first step of synced flush (flush) for one shard copy + */ + final static class PreSyncedFlushResponse extends TransportResponse { + + Engine.CommitId commitId; + + PreSyncedFlushResponse() { + } + + PreSyncedFlushResponse(Engine.CommitId commitId) { + this.commitId = commitId; + } + + public Engine.CommitId commitId() { + return commitId; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + commitId = new Engine.CommitId(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + commitId.writeTo(out); + } + } + + static final class SyncedFlushRequest extends TransportRequest { + + private String syncId; + private Engine.CommitId expectedCommitId; + private ShardId shardId; + + public SyncedFlushRequest() { + } + + public SyncedFlushRequest(ShardId shardId, String syncId, Engine.CommitId expectedCommitId) { + this.expectedCommitId = expectedCommitId; + this.shardId = shardId; + this.syncId = syncId; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + shardId = ShardId.readShardId(in); + expectedCommitId = new Engine.CommitId(in); + syncId = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + shardId.writeTo(out); + expectedCommitId.writeTo(out); + out.writeString(syncId); + } + + public ShardId shardId() { + return shardId; + } + + public String syncId() { + return syncId; + } + + public Engine.CommitId expectedCommitId() { + return expectedCommitId; + } + + @Override + public String toString() { + return "SyncedFlushRequest{" + + "shardId=" + shardId + + ",syncId='" + syncId + '\'' + + '}'; + } + } + + /** + * Response for third step of synced flush (writing the sync id) for one shard copy + */ + public static final class SyncedFlushResponse extends TransportResponse { + + /** + * a non null value indicates a failure to sync flush. null means success + */ + String failureReason; + + public SyncedFlushResponse() { + failureReason = null; + } + + public SyncedFlushResponse(String failureReason) { + this.failureReason = failureReason; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + failureReason = in.readOptionalString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeOptionalString(failureReason); + } + + public boolean success() { + return failureReason == null; + } + + public String failureReason() { + return failureReason; + } + + @Override + public String toString() { + return "SyncedFlushResponse{" + + "success=" + success() + + ", failureReason='" + failureReason + '\'' + + '}'; + } + } + + + static final class InFlightOpsRequest extends TransportRequest { + + private ShardId shardId; + + public InFlightOpsRequest() { + } + + public InFlightOpsRequest(ShardId shardId) { + this.shardId = shardId; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + shardId = ShardId.readShardId(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + shardId.writeTo(out); + } + + public ShardId shardId() { + return shardId; + } + + @Override + public String toString() { + return "InFlightOpsRequest{" + + "shardId=" + shardId + + '}'; + } + } + + /** + * Response for second step of synced flush (check operations in flight) + */ + static final class InFlightOpsResponse extends TransportResponse { + + int opCount; + + public InFlightOpsResponse() { + } + + public InFlightOpsResponse(int opCount) { + this.opCount = opCount; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + opCount = in.readVInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeVInt(opCount); + } + + public int opCount() { + return opCount; + } + + @Override + public String toString() { + return "InFlightOpsResponse{" + + "opCount=" + opCount + + '}'; + } + } + + private final class PreSyncedFlushTransportHandler implements TransportRequestHandler { + + @Override + public void messageReceived(PreSyncedFlushRequest request, TransportChannel channel) throws Exception { + channel.sendResponse(performPreSyncedFlush(request)); + } + } + + private final class SyncedFlushTransportHandler implements TransportRequestHandler { + + @Override + public void messageReceived(SyncedFlushRequest request, TransportChannel channel) throws Exception { + channel.sendResponse(performSyncedFlush(request)); + } + } + + private final class InFlightOpCountTransportHandler implements TransportRequestHandler { + + @Override + public void messageReceived(InFlightOpsRequest request, TransportChannel channel) throws Exception { + channel.sendResponse(performInFlightOps(request)); + } + } + +} diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryCleanFilesRequest.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryCleanFilesRequest.java index 0ff00d7c008..b0d224c41ef 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryCleanFilesRequest.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryCleanFilesRequest.java @@ -61,7 +61,7 @@ class RecoveryCleanFilesRequest extends TransportRequest { super.readFrom(in); recoveryId = in.readLong(); shardId = ShardId.readShardId(in); - snapshotFiles = Store.MetadataSnapshot.read(in); + snapshotFiles = new Store.MetadataSnapshot(in); totalTranslogOps = in.readVInt(); } diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 86cc680b56a..b131f11fbc0 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -57,7 +57,6 @@ import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; -import java.io.Closeable; import java.io.IOException; import java.util.Comparator; import java.util.List; @@ -115,7 +114,9 @@ public class RecoverySourceHandler { this.response = new RecoveryResponse(); } - /** performs the recovery from the local engine to the target */ + /** + * performs the recovery from the local engine to the target + */ public RecoveryResponse recoverToTarget() { final Engine engine = shard.engine(); assert engine.getTranslog() != null : "translog must not be null"; @@ -186,246 +187,260 @@ public class RecoverySourceHandler { // Generate a "diff" of all the identical, different, and missing // segment files on the target node, using the existing files on // the source node - final Store.RecoveryDiff diff = recoverySourceMetadata.recoveryDiff(new Store.MetadataSnapshot(request.existingFiles())); - for (StoreFileMetaData md : diff.identical) { - response.phase1ExistingFileNames.add(md.name()); - response.phase1ExistingFileSizes.add(md.length()); - existingTotalSize += md.length(); - if (logger.isTraceEnabled()) { - logger.trace("[{}][{}] recovery [phase1] to {}: not recovering [{}], exists in local store and has checksum [{}], size [{}]", - indexName, shardId, request.targetNode(), md.name(), md.checksum(), md.length()); + String recoverySourceSyncId = recoverySourceMetadata.getSyncId(); + String recoveryTargetSyncId = request.metadataSnapshot().getSyncId(); + final boolean recoverWithSyncId = recoverySourceSyncId != null && + recoverySourceSyncId.equals(recoveryTargetSyncId); + if (recoverWithSyncId) { + final long numDocsTarget = request.metadataSnapshot().getNumDocs(); + final long numDocsSource = recoverySourceMetadata.getNumDocs(); + if (numDocsTarget != numDocsSource) { + throw new IllegalStateException("try to recover " + request.shardId() + " from primary shard with sync id but number of docs differ: " + numDocsTarget + " (" + request.sourceNode().getName() + ", primary) vs " + numDocsSource + "(" + request.targetNode().getName() + ")"); } - totalSize += md.length(); - } - for (StoreFileMetaData md : Iterables.concat(diff.different, diff.missing)) { - if (request.existingFiles().containsKey(md.name())) { - logger.trace("[{}][{}] recovery [phase1] to {}: recovering [{}], exists in local store, but is different: remote [{}], local [{}]", - indexName, shardId, request.targetNode(), md.name(), request.existingFiles().get(md.name()), md); - } else { - logger.trace("[{}][{}] recovery [phase1] to {}: recovering [{}], does not exists in remote", - indexName, shardId, request.targetNode(), md.name()); + // we shortcut recovery here because we have nothing to copy. but we must still start the engine on the target. + // so we don't return here + } else { + final Store.RecoveryDiff diff = recoverySourceMetadata.recoveryDiff(request.metadataSnapshot()); + for (StoreFileMetaData md : diff.identical) { + response.phase1ExistingFileNames.add(md.name()); + response.phase1ExistingFileSizes.add(md.length()); + existingTotalSize += md.length(); + if (logger.isTraceEnabled()) { + logger.trace("[{}][{}] recovery [phase1] to {}: not recovering [{}], exists in local store and has checksum [{}], size [{}]", + indexName, shardId, request.targetNode(), md.name(), md.checksum(), md.length()); + } + totalSize += md.length(); } - response.phase1FileNames.add(md.name()); - response.phase1FileSizes.add(md.length()); - totalSize += md.length(); - } - response.phase1TotalSize = totalSize; - response.phase1ExistingTotalSize = existingTotalSize; - - logger.trace("[{}][{}] recovery [phase1] to {}: recovering_files [{}] with total_size [{}], reusing_files [{}] with total_size [{}]", - indexName, shardId, request.targetNode(), response.phase1FileNames.size(), - new ByteSizeValue(totalSize), response.phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSize)); - cancellableThreads.execute(new Interruptable() { - @Override - public void run() throws InterruptedException { - RecoveryFilesInfoRequest recoveryInfoFilesRequest = new RecoveryFilesInfoRequest(request.recoveryId(), request.shardId(), - response.phase1FileNames, response.phase1FileSizes, response.phase1ExistingFileNames, response.phase1ExistingFileSizes, - translogView.totalOperations()); - transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILES_INFO, recoveryInfoFilesRequest, - TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), - EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); - } - }); - - - // This latch will be used to wait until all files have been transferred to the target node - final CountDownLatch latch = new CountDownLatch(response.phase1FileNames.size()); - final CopyOnWriteArrayList exceptions = new CopyOnWriteArrayList<>(); - final AtomicReference corruptedEngine = new AtomicReference<>(); - int fileIndex = 0; - ThreadPoolExecutor pool; - - // How many bytes we've copied since we last called RateLimiter.pause - final AtomicLong bytesSinceLastPause = new AtomicLong(); - - for (final String name : response.phase1FileNames) { - long fileSize = response.phase1FileSizes.get(fileIndex); - - // Files are split into two categories, files that are "small" - // (under 5mb) and other files. Small files are transferred - // using a separate thread pool dedicated to small files. - // - // The idea behind this is that while we are transferring an - // older, large index, a user may create a new index, but that - // index will not be able to recover until the large index - // finishes, by using two different thread pools we can allow - // tiny files (like segments for a brand new index) to be - // recovered while ongoing large segment recoveries are - // happening. It also allows these pools to be configured - // separately. - if (fileSize > RecoverySettings.SMALL_FILE_CUTOFF_BYTES) { - pool = recoverySettings.concurrentStreamPool(); - } else { - pool = recoverySettings.concurrentSmallFileStreamPool(); + for (StoreFileMetaData md : Iterables.concat(diff.different, diff.missing)) { + if (request.metadataSnapshot().asMap().containsKey(md.name())) { + logger.trace("[{}][{}] recovery [phase1] to {}: recovering [{}], exists in local store, but is different: remote [{}], local [{}]", + indexName, shardId, request.targetNode(), md.name(), request.metadataSnapshot().asMap().get(md.name()), md); + } else { + logger.trace("[{}][{}] recovery [phase1] to {}: recovering [{}], does not exists in remote", + indexName, shardId, request.targetNode(), md.name()); + } + response.phase1FileNames.add(md.name()); + response.phase1FileSizes.add(md.length()); + totalSize += md.length(); } - pool.execute(new AbstractRunnable() { + response.phase1TotalSize = totalSize; + response.phase1ExistingTotalSize = existingTotalSize; + + logger.trace("[{}][{}] recovery [phase1] to {}: recovering_files [{}] with total_size [{}], reusing_files [{}] with total_size [{}]", + indexName, shardId, request.targetNode(), response.phase1FileNames.size(), + new ByteSizeValue(totalSize), response.phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSize)); + cancellableThreads.execute(new Interruptable() { @Override - public void onFailure(Throwable t) { - // we either got rejected or the store can't be incremented / we are canceled - logger.debug("Failed to transfer file [" + name + "] on recovery"); + public void run() throws InterruptedException { + RecoveryFilesInfoRequest recoveryInfoFilesRequest = new RecoveryFilesInfoRequest(request.recoveryId(), request.shardId(), + response.phase1FileNames, response.phase1FileSizes, response.phase1ExistingFileNames, response.phase1ExistingFileSizes, + translogView.totalOperations()); + transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILES_INFO, recoveryInfoFilesRequest, + TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), + EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); + } + }); + + // This latch will be used to wait until all files have been transferred to the target node + final CountDownLatch latch = new CountDownLatch(response.phase1FileNames.size()); + final CopyOnWriteArrayList exceptions = new CopyOnWriteArrayList<>(); + final AtomicReference corruptedEngine = new AtomicReference<>(); + int fileIndex = 0; + ThreadPoolExecutor pool; + + // How many bytes we've copied since we last called RateLimiter.pause + final AtomicLong bytesSinceLastPause = new AtomicLong(); + + for (final String name : response.phase1FileNames) { + long fileSize = response.phase1FileSizes.get(fileIndex); + + // Files are split into two categories, files that are "small" + // (under 5mb) and other files. Small files are transferred + // using a separate thread pool dedicated to small files. + // + // The idea behind this is that while we are transferring an + // older, large index, a user may create a new index, but that + // index will not be able to recover until the large index + // finishes, by using two different thread pools we can allow + // tiny files (like segments for a brand new index) to be + // recovered while ongoing large segment recoveries are + // happening. It also allows these pools to be configured + // separately. + if (fileSize > RecoverySettings.SMALL_FILE_CUTOFF_BYTES) { + pool = recoverySettings.concurrentStreamPool(); + } else { + pool = recoverySettings.concurrentSmallFileStreamPool(); } + pool.execute(new AbstractRunnable() { + @Override + public void onFailure(Throwable t) { + // we either got rejected or the store can't be incremented / we are canceled + logger.debug("Failed to transfer file [" + name + "] on recovery"); + } + + @Override + public void onAfter() { + // Signify this file has completed by decrementing the latch + latch.countDown(); + } + + @Override + protected void doRun() { + cancellableThreads.checkForCancel(); + store.incRef(); + final StoreFileMetaData md = recoverySourceMetadata.get(name); + try (final IndexInput indexInput = store.directory().openInput(name, IOContext.READONCE)) { + final int BUFFER_SIZE = (int) recoverySettings.fileChunkSize().bytes(); + final byte[] buf = new byte[BUFFER_SIZE]; + boolean shouldCompressRequest = recoverySettings.compress(); + if (CompressorFactory.isCompressed(indexInput)) { + shouldCompressRequest = false; + } + + final long len = indexInput.length(); + long readCount = 0; + final TransportRequestOptions requestOptions = TransportRequestOptions.options() + .withCompress(shouldCompressRequest) + .withType(TransportRequestOptions.Type.RECOVERY) + .withTimeout(recoverySettings.internalActionTimeout()); + + while (readCount < len) { + if (shard.state() == IndexShardState.CLOSED) { // check if the shard got closed on us + throw new IndexShardClosedException(shard.shardId()); + } + int toRead = readCount + BUFFER_SIZE > len ? (int) (len - readCount) : BUFFER_SIZE; + final long position = indexInput.getFilePointer(); + + // Pause using the rate limiter, if desired, to throttle the recovery + RateLimiter rl = recoverySettings.rateLimiter(); + long throttleTimeInNanos = 0; + if (rl != null) { + long bytes = bytesSinceLastPause.addAndGet(toRead); + if (bytes > rl.getMinPauseCheckBytes()) { + // Time to pause + bytesSinceLastPause.addAndGet(-bytes); + throttleTimeInNanos = rl.pause(bytes); + shard.recoveryStats().addThrottleTime(throttleTimeInNanos); + } + } + indexInput.readBytes(buf, 0, toRead, false); + final BytesArray content = new BytesArray(buf, 0, toRead); + readCount += toRead; + final boolean lastChunk = readCount == len; + final RecoveryFileChunkRequest fileChunkRequest = new RecoveryFileChunkRequest(request.recoveryId(), request.shardId(), md, position, + content, lastChunk, translogView.totalOperations(), throttleTimeInNanos); + cancellableThreads.execute(new Interruptable() { + @Override + public void run() throws InterruptedException { + // Actually send the file chunk to the target node, waiting for it to complete + transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILE_CHUNK, + fileChunkRequest, requestOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); + } + }); + + } + } catch (Throwable e) { + final Throwable corruptIndexException; + if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(e)) != null) { + if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! + logger.warn("{} Corrupted file detected {} checksum mismatch", shard.shardId(), md); + if (corruptedEngine.compareAndSet(null, corruptIndexException) == false) { + // if we are not the first exception, add ourselves as suppressed to the main one: + corruptedEngine.get().addSuppressed(e); + } + } else { // corruption has happened on the way to replica + RemoteTransportException exception = new RemoteTransportException("File corruption occurred on recovery but checksums are ok", null); + exception.addSuppressed(e); + exceptions.add(0, exception); // last exception first + logger.warn("{} Remote file corruption on node {}, recovering {}. local checksum OK", + corruptIndexException, shard.shardId(), request.targetNode(), md); + + } + } else { + exceptions.add(0, e); // last exceptions first + } + } finally { + store.decRef(); + + } + } + }); + fileIndex++; + } + + cancellableThreads.execute(new Interruptable() { @Override - public void onAfter() { - // Signify this file has completed by decrementing the latch - latch.countDown(); + public void run() throws InterruptedException { + // Wait for all files that need to be transferred to finish transferring + latch.await(); } + }); + if (corruptedEngine.get() != null) { + shard.engine().failEngine("recovery", corruptedEngine.get()); + throw corruptedEngine.get(); + } else { + ExceptionsHelper.rethrowAndSuppress(exceptions); + } + + cancellableThreads.execute(new Interruptable() { @Override - protected void doRun() { - cancellableThreads.checkForCancel(); - store.incRef(); - final StoreFileMetaData md = recoverySourceMetadata.get(name); - try (final IndexInput indexInput = store.directory().openInput(name, IOContext.READONCE)) { - final int BUFFER_SIZE = (int) recoverySettings.fileChunkSize().bytes(); - final byte[] buf = new byte[BUFFER_SIZE]; - boolean shouldCompressRequest = recoverySettings.compress(); - if (CompressorFactory.isCompressed(indexInput)) { - shouldCompressRequest = false; - } - - final long len = indexInput.length(); - long readCount = 0; - final TransportRequestOptions requestOptions = TransportRequestOptions.options() - .withCompress(shouldCompressRequest) - .withType(TransportRequestOptions.Type.RECOVERY) - .withTimeout(recoverySettings.internalActionTimeout()); - - while (readCount < len) { - if (shard.state() == IndexShardState.CLOSED) { // check if the shard got closed on us - throw new IndexShardClosedException(shard.shardId()); - } - int toRead = readCount + BUFFER_SIZE > len ? (int) (len - readCount) : BUFFER_SIZE; - final long position = indexInput.getFilePointer(); - - // Pause using the rate limiter, if desired, to throttle the recovery - RateLimiter rl = recoverySettings.rateLimiter(); - long throttleTimeInNanos = 0; - if (rl != null) { - long bytes = bytesSinceLastPause.addAndGet(toRead); - if (bytes > rl.getMinPauseCheckBytes()) { - // Time to pause - bytesSinceLastPause.addAndGet(-bytes); - throttleTimeInNanos = rl.pause(bytes); - shard.recoveryStats().addThrottleTime(throttleTimeInNanos); + public void run() throws InterruptedException { + // Send the CLEAN_FILES request, which takes all of the files that + // were transferred and renames them from their temporary file + // names to the actual file names. It also writes checksums for + // the files after they have been renamed. + // + // Once the files have been renamed, any other files that are not + // related to this recovery (out of date segments, for example) + // are deleted + try { + transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.CLEAN_FILES, + new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), recoverySourceMetadata, translogView.totalOperations()), + TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), + EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); + } catch (RemoteTransportException remoteException) { + final IOException corruptIndexException; + // we realized that after the index was copied and we wanted to finalize the recovery + // the index was corrupted: + // - maybe due to a broken segments file on an empty index (transferred with no checksum) + // - maybe due to old segments without checksums or length only checks + if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(remoteException)) != null) { + try { + final Store.MetadataSnapshot recoverySourceMetadata = store.getMetadata(snapshot); + StoreFileMetaData[] metadata = Iterables.toArray(recoverySourceMetadata, StoreFileMetaData.class); + ArrayUtil.timSort(metadata, new Comparator() { + @Override + public int compare(StoreFileMetaData o1, StoreFileMetaData o2) { + return Long.compare(o1.length(), o2.length()); // check small files first + } + }); + for (StoreFileMetaData md : metadata) { + logger.debug("{} checking integrity for file {} after remove corruption exception", shard.shardId(), md); + if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! + logger.warn("{} Corrupted file detected {} checksum mismatch", shard.shardId(), md); + throw corruptIndexException; + } } + } catch (IOException ex) { + remoteException.addSuppressed(ex); + throw remoteException; } - indexInput.readBytes(buf, 0, toRead, false); - final BytesArray content = new BytesArray(buf, 0, toRead); - readCount += toRead; - final boolean lastChunk = readCount == len; - final RecoveryFileChunkRequest fileChunkRequest = new RecoveryFileChunkRequest(request.recoveryId(), request.shardId(), md, position, - content, lastChunk, translogView.totalOperations(), throttleTimeInNanos); - cancellableThreads.execute(new Interruptable() { - @Override - public void run() throws InterruptedException { - // Actually send the file chunk to the target node, waiting for it to complete - transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILE_CHUNK, - fileChunkRequest, requestOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); - } - }); - - } - } catch (Throwable e) { - final Throwable corruptIndexException; - if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(e)) != null) { - if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! - logger.warn("{} Corrupted file detected {} checksum mismatch", shard.shardId(), md); - if (corruptedEngine.compareAndSet(null, corruptIndexException) == false) { - // if we are not the first exception, add ourselves as suppressed to the main one: - corruptedEngine.get().addSuppressed(e); - } - } else { // corruption has happened on the way to replica - RemoteTransportException exception = new RemoteTransportException("File corruption occurred on recovery but checksums are ok", null); - exception.addSuppressed(e); - exceptions.add(0, exception); // last exception first - logger.warn("{} Remote file corruption on node {}, recovering {}. local checksum OK", - corruptIndexException, shard.shardId(), request.targetNode(), md); - - } + // corruption has happened on the way to replica + RemoteTransportException exception = new RemoteTransportException("File corruption occurred on recovery but checksums are ok", null); + exception.addSuppressed(remoteException); + logger.warn("{} Remote file corruption during finalization on node {}, recovering {}. local checksum OK", + corruptIndexException, shard.shardId(), request.targetNode()); } else { - exceptions.add(0, e); // last exceptions first + throw remoteException; } - } finally { - store.decRef(); - } } }); - fileIndex++; } - cancellableThreads.execute(new Interruptable() { - @Override - public void run() throws InterruptedException { - // Wait for all files that need to be transferred to finish transferring - latch.await(); - } - }); - - if (corruptedEngine.get() != null) { - shard.engine().failEngine("recovery", corruptedEngine.get()); - throw corruptedEngine.get(); - } else { - ExceptionsHelper.rethrowAndSuppress(exceptions); - } - - cancellableThreads.execute(new Interruptable() { - @Override - public void run() throws InterruptedException { - // Send the CLEAN_FILES request, which takes all of the files that - // were transferred and renames them from their temporary file - // names to the actual file names. It also writes checksums for - // the files after they have been renamed. - // - // Once the files have been renamed, any other files that are not - // related to this recovery (out of date segments, for example) - // are deleted - try { - transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.CLEAN_FILES, - new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), recoverySourceMetadata, translogView.totalOperations()), - TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), - EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); - } catch (RemoteTransportException remoteException) { - final IOException corruptIndexException; - // we realized that after the index was copied and we wanted to finalize the recovery - // the index was corrupted: - // - maybe due to a broken segments file on an empty index (transferred with no checksum) - // - maybe due to old segments without checksums or length only checks - if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(remoteException)) != null) { - try { - final Store.MetadataSnapshot recoverySourceMetadata = store.getMetadata(snapshot); - StoreFileMetaData[] metadata = Iterables.toArray(recoverySourceMetadata, StoreFileMetaData.class); - ArrayUtil.timSort(metadata, new Comparator() { - @Override - public int compare(StoreFileMetaData o1, StoreFileMetaData o2) { - return Long.compare(o1.length(), o2.length()); // check small files first - } - }); - for (StoreFileMetaData md : metadata) { - logger.debug("{} checking integrity for file {} after remove corruption exception", shard.shardId(), md); - if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! - logger.warn("{} Corrupted file detected {} checksum mismatch", shard.shardId(), md); - throw corruptIndexException; - } - } - } catch (IOException ex) { - remoteException.addSuppressed(ex); - throw remoteException; - } - // corruption has happened on the way to replica - RemoteTransportException exception = new RemoteTransportException("File corruption occurred on recovery but checksums are ok", null); - exception.addSuppressed(remoteException); - logger.warn("{} Remote file corruption during finalization on node {}, recovering {}. local checksum OK", - corruptIndexException, shard.shardId(), request.targetNode()); - } else { - throw remoteException; - } - } - } - }); - prepareTargetForTranslog(translogView); logger.trace("[{}][{}] recovery [phase1] to {}: took [{}]", indexName, shardId, request.targetNode(), stopWatch.totalTime()); diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java index ad664f95ee4..54494987b9b 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -47,16 +48,24 @@ public class RecoveryState implements ToXContent, Streamable { public static enum Stage { INIT((byte) 0), - /** recovery of lucene files, either reusing local ones are copying new ones */ + /** + * recovery of lucene files, either reusing local ones are copying new ones + */ INDEX((byte) 1), - /** potentially running check index */ + /** + * potentially running check index + */ VERIFY_INDEX((byte) 2), - /** starting up the engine, replaying the translog */ + /** + * starting up the engine, replaying the translog + */ TRANSLOG((byte) 3), - /** performing final task after all translog ops have been done */ + /** + * performing final task after all translog ops have been done + */ FINALIZE((byte) 4), DONE((byte) 5); @@ -497,7 +506,9 @@ public class RecoveryState implements ToXContent, Streamable { assert total == UNKNOWN || total >= recovered : "total, if known, should be > recovered. total [" + total + "], recovered [" + recovered + "]"; } - /** returns the total number of translog operations recovered so far */ + /** + * returns the total number of translog operations recovered so far + */ public synchronized int recoveredOperations() { return recovered; } @@ -590,22 +601,30 @@ public class RecoveryState implements ToXContent, Streamable { recovered += bytes; } - /** file name * */ + /** + * file name * + */ public String name() { return name; } - /** file length * */ + /** + * file length * + */ public long length() { return length; } - /** number of bytes recovered for this file (so far). 0 if the file is reused * */ + /** + * number of bytes recovered for this file (so far). 0 if the file is reused * + */ public long recovered() { return recovered; } - /** returns true if the file is reused from a local copy */ + /** + * returns true if the file is reused from a local copy + */ public boolean reused() { return reused; } @@ -732,12 +751,16 @@ public class RecoveryState implements ToXContent, Streamable { return TimeValue.timeValueNanos(targetThrottleTimeInNanos); } - /** total number of files that are part of this recovery, both re-used and recovered */ + /** + * total number of files that are part of this recovery, both re-used and recovered + */ public synchronized int totalFileCount() { return fileDetails.size(); } - /** total number of files to be recovered (potentially not yet done) */ + /** + * total number of files to be recovered (potentially not yet done) + */ public synchronized int totalRecoverFiles() { int total = 0; for (File file : fileDetails.values()) { @@ -749,7 +772,9 @@ public class RecoveryState implements ToXContent, Streamable { } - /** number of file that were recovered (excluding on ongoing files) */ + /** + * number of file that were recovered (excluding on ongoing files) + */ public synchronized int recoveredFileCount() { int count = 0; for (File file : fileDetails.values()) { @@ -760,7 +785,9 @@ public class RecoveryState implements ToXContent, Streamable { return count; } - /** percent of recovered (i.e., not reused) files out of the total files to be recovered */ + /** + * percent of recovered (i.e., not reused) files out of the total files to be recovered + */ public synchronized float recoveredFilesPercent() { int total = 0; int recovered = 0; @@ -783,7 +810,9 @@ public class RecoveryState implements ToXContent, Streamable { } } - /** total number of bytes in th shard */ + /** + * total number of bytes in th shard + */ public synchronized long totalBytes() { long total = 0; for (File file : fileDetails.values()) { @@ -792,7 +821,9 @@ public class RecoveryState implements ToXContent, Streamable { return total; } - /** total number of bytes recovered so far, including both existing and reused */ + /** + * total number of bytes recovered so far, including both existing and reused + */ public synchronized long recoveredBytes() { long recovered = 0; for (File file : fileDetails.values()) { @@ -801,7 +832,9 @@ public class RecoveryState implements ToXContent, Streamable { return recovered; } - /** total bytes of files to be recovered (potentially not yet done) */ + /** + * total bytes of files to be recovered (potentially not yet done) + */ public synchronized long totalRecoverBytes() { long total = 0; for (File file : fileDetails.values()) { @@ -822,7 +855,9 @@ public class RecoveryState implements ToXContent, Streamable { return total; } - /** percent of bytes recovered out of total files bytes *to be* recovered */ + /** + * percent of bytes recovered out of total files bytes *to be* recovered + */ public synchronized float recoveredBytesPercent() { long total = 0; long recovered = 0; diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 0b0f65b3f4d..31cfb7a7671 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -40,6 +40,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.index.IndexShardMissingException; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.shard.IllegalIndexShardStateException; @@ -158,12 +159,12 @@ public class RecoveryTarget extends AbstractComponent { assert recoveryStatus.sourceNode() != null : "can't do a recovery without a source node"; logger.trace("collecting local files for {}", recoveryStatus); - Map existingFiles; + Store.MetadataSnapshot metadataSnapshot = null; try { - existingFiles = recoveryStatus.store().getMetadataOrEmpty().asMap(); + metadataSnapshot = recoveryStatus.store().getMetadataOrEmpty(); } catch (IOException e) { logger.warn("error while listing local files, recover as if there are none", e); - existingFiles = Store.MetadataSnapshot.EMPTY.asMap(); + metadataSnapshot = Store.MetadataSnapshot.EMPTY; } catch (Exception e) { // this will be logged as warning later on... logger.trace("unexpected error while listing local files, failing recovery", e); @@ -172,7 +173,7 @@ public class RecoveryTarget extends AbstractComponent { return; } final StartRecoveryRequest request = new StartRecoveryRequest(recoveryStatus.shardId(), recoveryStatus.sourceNode(), clusterService.localNode(), - false, existingFiles, recoveryStatus.state().getType(), recoveryStatus.recoveryId()); + false, metadataSnapshot, recoveryStatus.state().getType(), recoveryStatus.recoveryId()); final AtomicReference responseHolder = new AtomicReference<>(); try { diff --git a/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java b/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java index 88c14d7f0ce..b5fa3ee6047 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java +++ b/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java @@ -25,6 +25,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.transport.TransportRequest; @@ -46,7 +47,7 @@ public class StartRecoveryRequest extends TransportRequest { private boolean markAsRelocated; - private Map existingFiles; + private Store.MetadataSnapshot metadataSnapshot; private RecoveryState.Type recoveryType; @@ -57,20 +58,19 @@ public class StartRecoveryRequest extends TransportRequest { * Start recovery request. * * @param shardId - * @param sourceNode The node to recover from - * @param targetNode The node to recover to + * @param sourceNode The node to recover from + * @param targetNode The node to recover to * @param markAsRelocated - * @param existingFiles + * @param metadataSnapshot */ - public StartRecoveryRequest(ShardId shardId, DiscoveryNode sourceNode, DiscoveryNode targetNode, boolean markAsRelocated, Map existingFiles, RecoveryState.Type recoveryType, long recoveryId) { + public StartRecoveryRequest(ShardId shardId, DiscoveryNode sourceNode, DiscoveryNode targetNode, boolean markAsRelocated, Store.MetadataSnapshot metadataSnapshot, RecoveryState.Type recoveryType, long recoveryId) { this.recoveryId = recoveryId; this.shardId = shardId; this.sourceNode = sourceNode; this.targetNode = targetNode; this.markAsRelocated = markAsRelocated; - this.existingFiles = existingFiles; this.recoveryType = recoveryType; + this.metadataSnapshot = metadataSnapshot; } public long recoveryId() { @@ -93,14 +93,14 @@ public class StartRecoveryRequest extends TransportRequest { return markAsRelocated; } - public Map existingFiles() { - return existingFiles; - } - public RecoveryState.Type recoveryType() { return recoveryType; } + public Store.MetadataSnapshot metadataSnapshot() { + return metadataSnapshot; + } + @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); @@ -109,13 +109,9 @@ public class StartRecoveryRequest extends TransportRequest { sourceNode = DiscoveryNode.readNode(in); targetNode = DiscoveryNode.readNode(in); markAsRelocated = in.readBoolean(); - int size = in.readVInt(); - existingFiles = Maps.newHashMapWithExpectedSize(size); - for (int i = 0; i < size; i++) { - StoreFileMetaData md = StoreFileMetaData.readStoreFileMetaData(in); - existingFiles.put(md.name(), md); - } + metadataSnapshot = new Store.MetadataSnapshot(in); recoveryType = RecoveryState.Type.fromId(in.readByte()); + } @Override @@ -126,10 +122,8 @@ public class StartRecoveryRequest extends TransportRequest { sourceNode.writeTo(out); targetNode.writeTo(out); out.writeBoolean(markAsRelocated); - out.writeVInt(existingFiles.size()); - for (StoreFileMetaData md : existingFiles.values()) { - md.writeTo(out); - } + metadataSnapshot.writeTo(out); out.writeByte(recoveryType.id()); } + } diff --git a/src/main/java/org/elasticsearch/indices/store/TransportNodesListShardStoreMetaData.java b/src/main/java/org/elasticsearch/indices/store/TransportNodesListShardStoreMetaData.java index c384b4dd88f..c13c4748356 100644 --- a/src/main/java/org/elasticsearch/indices/store/TransportNodesListShardStoreMetaData.java +++ b/src/main/java/org/elasticsearch/indices/store/TransportNodesListShardStoreMetaData.java @@ -40,6 +40,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; @@ -145,7 +146,7 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio store.incRef(); try { exists = true; - return new StoreFilesMetaData(true, shardId, store.getMetadataOrEmpty().asMap()); + return new StoreFilesMetaData(true, shardId, store.getMetadataOrEmpty()); } finally { store.decRef(); } @@ -154,17 +155,17 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio // try and see if we an list unallocated IndexMetaData metaData = clusterService.state().metaData().index(shardId.index().name()); if (metaData == null) { - return new StoreFilesMetaData(false, shardId, ImmutableMap.of()); + return new StoreFilesMetaData(false, shardId, Store.MetadataSnapshot.EMPTY); } String storeType = metaData.settings().get(IndexStoreModule.STORE_TYPE, "fs"); if (!storeType.contains("fs")) { - return new StoreFilesMetaData(false, shardId, ImmutableMap.of()); + return new StoreFilesMetaData(false, shardId, Store.MetadataSnapshot.EMPTY); } final ShardPath shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, metaData.settings()); if (shardPath == null) { - return new StoreFilesMetaData(false, shardId, ImmutableMap.of()); + return new StoreFilesMetaData(false, shardId, Store.MetadataSnapshot.EMPTY); } - return new StoreFilesMetaData(false, shardId, Store.readMetadataSnapshot(shardPath.resolveIndex(), logger).asMap()); + return new StoreFilesMetaData(false, shardId, Store.readMetadataSnapshot(shardPath.resolveIndex(), logger)); } finally { TimeValue took = new TimeValue(System.nanoTime() - startTimeNS, TimeUnit.NANOSECONDS); if (exists) { @@ -181,17 +182,18 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio } public static class StoreFilesMetaData implements Iterable, Streamable { + // here also trasmit sync id, else recovery will not use sync id because of stupid gateway allocator every now and then... private boolean allocated; private ShardId shardId; - private Map files; + Store.MetadataSnapshot metadataSnapshot; StoreFilesMetaData() { } - public StoreFilesMetaData(boolean allocated, ShardId shardId, Map files) { + public StoreFilesMetaData(boolean allocated, ShardId shardId, Store.MetadataSnapshot metadataSnapshot) { this.allocated = allocated; this.shardId = shardId; - this.files = files; + this.metadataSnapshot = metadataSnapshot; } public boolean allocated() { @@ -204,15 +206,15 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio @Override public Iterator iterator() { - return files.values().iterator(); + return metadataSnapshot.iterator(); } public boolean fileExists(String name) { - return files.containsKey(name); + return metadataSnapshot.asMap().containsKey(name); } public StoreFileMetaData file(String name) { - return files.get(name); + return metadataSnapshot.asMap().get(name); } public static StoreFilesMetaData readStoreFilesMetaData(StreamInput in) throws IOException { @@ -225,22 +227,18 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio public void readFrom(StreamInput in) throws IOException { allocated = in.readBoolean(); shardId = ShardId.readShardId(in); - int size = in.readVInt(); - files = Maps.newHashMapWithExpectedSize(size); - for (int i = 0; i < size; i++) { - StoreFileMetaData md = StoreFileMetaData.readStoreFileMetaData(in); - files.put(md.name(), md); - } + this.metadataSnapshot = new Store.MetadataSnapshot(in); } @Override public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(allocated); shardId.writeTo(out); - out.writeVInt(files.size()); - for (StoreFileMetaData md : files.values()) { - md.writeTo(out); - } + metadataSnapshot.writeTo(out); + } + + public String syncId() { + return metadataSnapshot.getSyncId(); } } diff --git a/src/main/java/org/elasticsearch/rest/action/RestActionModule.java b/src/main/java/org/elasticsearch/rest/action/RestActionModule.java index 59c28553622..a3864d5886e 100644 --- a/src/main/java/org/elasticsearch/rest/action/RestActionModule.java +++ b/src/main/java/org/elasticsearch/rest/action/RestActionModule.java @@ -23,6 +23,9 @@ import com.google.common.collect.Lists; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.multibindings.Multibinder; import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.action.admin.indices.seal.RestSealIndicesAction; +import org.elasticsearch.rest.action.admin.indices.upgrade.RestUpgradeAction; +import org.elasticsearch.rest.action.admin.cluster.repositories.verify.RestVerifyRepositoryAction; import org.elasticsearch.rest.action.admin.cluster.health.RestClusterHealthAction; import org.elasticsearch.rest.action.admin.cluster.node.hotthreads.RestNodesHotThreadsAction; import org.elasticsearch.rest.action.admin.cluster.node.info.RestNodesInfoAction; @@ -30,7 +33,6 @@ import org.elasticsearch.rest.action.admin.cluster.node.stats.RestNodesStatsActi import org.elasticsearch.rest.action.admin.cluster.repositories.delete.RestDeleteRepositoryAction; import org.elasticsearch.rest.action.admin.cluster.repositories.get.RestGetRepositoriesAction; import org.elasticsearch.rest.action.admin.cluster.repositories.put.RestPutRepositoryAction; -import org.elasticsearch.rest.action.admin.cluster.repositories.verify.RestVerifyRepositoryAction; import org.elasticsearch.rest.action.admin.cluster.reroute.RestClusterRerouteAction; import org.elasticsearch.rest.action.admin.cluster.settings.RestClusterGetSettingsAction; import org.elasticsearch.rest.action.admin.cluster.settings.RestClusterUpdateSettingsAction; @@ -73,7 +75,6 @@ import org.elasticsearch.rest.action.admin.indices.template.delete.RestDeleteInd import org.elasticsearch.rest.action.admin.indices.template.get.RestGetIndexTemplateAction; import org.elasticsearch.rest.action.admin.indices.template.head.RestHeadIndexTemplateAction; import org.elasticsearch.rest.action.admin.indices.template.put.RestPutIndexTemplateAction; -import org.elasticsearch.rest.action.admin.indices.upgrade.RestUpgradeAction; import org.elasticsearch.rest.action.admin.indices.validate.query.RestValidateQueryAction; import org.elasticsearch.rest.action.admin.indices.warmer.delete.RestDeleteWarmerAction; import org.elasticsearch.rest.action.admin.indices.warmer.get.RestGetWarmerAction; @@ -182,6 +183,7 @@ public class RestActionModule extends AbstractModule { bind(RestRefreshAction.class).asEagerSingleton(); bind(RestFlushAction.class).asEagerSingleton(); + bind(RestSealIndicesAction.class).asEagerSingleton(); bind(RestOptimizeAction.class).asEagerSingleton(); bind(RestUpgradeAction.class).asEagerSingleton(); bind(RestClearIndicesCacheAction.class).asEagerSingleton(); diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/seal/RestSealIndicesAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/seal/RestSealIndicesAction.java new file mode 100644 index 00000000000..a2007ca703d --- /dev/null +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/seal/RestSealIndicesAction.java @@ -0,0 +1,66 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.rest.action.admin.indices.seal; + +import org.elasticsearch.action.admin.indices.seal.SealIndicesAction; +import org.elasticsearch.action.admin.indices.seal.SealIndicesRequest; +import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.rest.*; +import org.elasticsearch.rest.action.support.RestBuilderListener; + +import static org.elasticsearch.rest.RestRequest.Method.GET; +import static org.elasticsearch.rest.RestRequest.Method.POST; + +/** + * + */ +public class RestSealIndicesAction extends BaseRestHandler { + + @Inject + public RestSealIndicesAction(Settings settings, RestController controller, Client client) { + super(settings, controller, client); + controller.registerHandler(POST, "/_seal", this); + controller.registerHandler(POST, "/{index}/_seal", this); + + controller.registerHandler(GET, "/_seal", this); + controller.registerHandler(GET, "/{index}/_seal", this); + } + + @Override + public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) { + String[] indices = Strings.splitStringByCommaToArray(request.param("index")); + SealIndicesRequest sealIndicesRequest = new SealIndicesRequest(indices); + client.admin().indices().execute(SealIndicesAction.INSTANCE, sealIndicesRequest, new RestBuilderListener(channel) { + @Override + public RestResponse buildResponse(SealIndicesResponse response, XContentBuilder builder) throws Exception { + builder.startObject(); + builder = response.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + return new BytesRestResponse(response.status(), builder); + } + }); + } +} diff --git a/src/test/java/org/elasticsearch/action/admin/indices/seal/SealIndicesTests.java b/src/test/java/org/elasticsearch/action/admin/indices/seal/SealIndicesTests.java new file mode 100644 index 00000000000..13c376eb250 --- /dev/null +++ b/src/test/java/org/elasticsearch/action/admin/indices/seal/SealIndicesTests.java @@ -0,0 +1,116 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.admin.indices.seal; + +import org.elasticsearch.cluster.routing.ImmutableShardRouting; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.SyncedFlushService; +import org.elasticsearch.test.ElasticsearchTestCase; + +import java.io.IOException; +import java.util.*; + +import static org.elasticsearch.test.XContentTestUtils.convertToMap; +import static org.hamcrest.Matchers.equalTo; + +public class SealIndicesTests extends ElasticsearchTestCase { + + public void testSealIndicesResponseStreaming() throws IOException { + + Set shardResults = new HashSet<>(); + // add one result where one shard failed and one succeeded + SyncedFlushService.SyncedFlushResult syncedFlushResult = createSyncedFlushResult(0, "test"); + shardResults.add(syncedFlushResult); + // add one result where all failed + syncedFlushResult = new SyncedFlushService.SyncedFlushResult(new ShardId("test", 1), "all failed :("); + shardResults.add(syncedFlushResult); + SealIndicesResponse sealIndicesResponse = new SealIndicesResponse(shardResults); + BytesStreamOutput out = new BytesStreamOutput(); + sealIndicesResponse.writeTo(out); + out.close(); + StreamInput in = StreamInput.wrap(out.bytes()); + SealIndicesResponse readResponse = new SealIndicesResponse(); + readResponse.readFrom(in); + Map asMap = convertToMap(readResponse); + assertResponse(asMap); + } + + public void testXContentResponse() throws IOException { + + Set shardResults = new HashSet<>(); + // add one result where one shard failed and one succeeded + SyncedFlushService.SyncedFlushResult syncedFlushResult = createSyncedFlushResult(0, "test"); + shardResults.add(syncedFlushResult); + // add one result where all failed + syncedFlushResult = new SyncedFlushService.SyncedFlushResult(new ShardId("test", 1), "all failed :("); + shardResults.add(syncedFlushResult); + SealIndicesResponse sealIndicesResponse = new SealIndicesResponse(shardResults); + Map asMap = convertToMap(sealIndicesResponse); + assertResponse(asMap); + } + + protected void assertResponse(Map asMap) { + assertNotNull(asMap.get("test")); + assertThat((Integer) (((HashMap) ((ArrayList) asMap.get("test")).get(0)).get("shard_id")), equalTo(0)); + assertThat((String) (((HashMap) ((ArrayList) asMap.get("test")).get(0)).get("message")), equalTo("failed on some copies")); + HashMap shardResponses = (HashMap) ((HashMap) ((ArrayList) asMap.get("test")).get(0)).get("responses"); + assertThat(shardResponses.get("node_1"), equalTo("failed for some reason")); + assertThat(shardResponses.get("node_2"), equalTo("success")); + HashMap failedShard = (HashMap) (((ArrayList) asMap.get("test")).get(1)); + assertThat((Integer) (failedShard.get("shard_id")), equalTo(1)); + assertThat((String) (failedShard.get("message")), equalTo("all failed :(")); + } + + public void testXContentResponseSortsShards() throws IOException { + Set shardResults = new HashSet<>(); + // add one result where one shard failed and one succeeded + SyncedFlushService.SyncedFlushResult syncedFlushResult; + for (int i = 100000; i >= 0; i--) { + if (randomBoolean()) { + syncedFlushResult = createSyncedFlushResult(i, "test"); + shardResults.add(syncedFlushResult); + } else { + syncedFlushResult = new SyncedFlushService.SyncedFlushResult(new ShardId("test", i), "all failed :("); + shardResults.add(syncedFlushResult); + } + } + SealIndicesResponse sealIndicesResponse = new SealIndicesResponse(shardResults); + Map asMap = convertToMap(sealIndicesResponse); + assertNotNull(asMap.get("test")); + for (int i = 0; i < 100000; i++) { + assertThat((Integer) (((HashMap) ((ArrayList) asMap.get("test")).get(i)).get("shard_id")), equalTo(i)); + } + } + + protected SyncedFlushService.SyncedFlushResult createSyncedFlushResult(int shardId, String index) { + Map responses = new HashMap<>(); + ImmutableShardRouting shardRouting = new ImmutableShardRouting(index, shardId, "node_1", false, ShardRoutingState.RELOCATING, 2); + SyncedFlushService.SyncedFlushResponse syncedFlushResponse = new SyncedFlushService.SyncedFlushResponse("failed for some reason"); + responses.put(shardRouting, syncedFlushResponse); + shardRouting = new ImmutableShardRouting(index, shardId, "node_2", false, ShardRoutingState.RELOCATING, 2); + syncedFlushResponse = new SyncedFlushService.SyncedFlushResponse(); + responses.put(shardRouting, syncedFlushResponse); + return new SyncedFlushService.SyncedFlushResult(new ShardId(index, shardId), "some_sync_id", responses); + } +} diff --git a/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTest.java b/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTest.java index 4a945f7e92f..f6f6c6cc38e 100644 --- a/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTest.java +++ b/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTest.java @@ -34,13 +34,12 @@ import org.elasticsearch.test.ElasticsearchAllocationTestCase; import org.junit.Before; import org.junit.Test; -import static org.hamcrest.Matchers.nullValue; - import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; -public class RoutingTableTest extends ElasticsearchAllocationTestCase { +public class RoutingTableTest extends ElasticsearchAllocationTestCase { private static final String TEST_INDEX_1 = "test1"; private static final String TEST_INDEX_2 = "test2"; @@ -72,9 +71,9 @@ public class RoutingTableTest extends ElasticsearchAllocationTestCase { .build(); this.testRoutingTable = new RoutingTable.Builder() - .add(new IndexRoutingTable.Builder(TEST_INDEX_1).initializeAsNew(metaData.index(TEST_INDEX_1)).build()) - .add(new IndexRoutingTable.Builder(TEST_INDEX_2).initializeAsNew(metaData.index(TEST_INDEX_2)).build()) - .build(); + .add(new IndexRoutingTable.Builder(TEST_INDEX_1).initializeAsNew(metaData.index(TEST_INDEX_1)).build()) + .add(new IndexRoutingTable.Builder(TEST_INDEX_2).initializeAsNew(metaData.index(TEST_INDEX_2)).build()) + .build(); this.clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(testRoutingTable).build(); } @@ -82,10 +81,10 @@ public class RoutingTableTest extends ElasticsearchAllocationTestCase { * puts primary shard routings into initializing state */ private void initPrimaries() { - logger.info("adding " + (this.numberOfReplicas + 1) + " nodes and performing rerouting"); + logger.info("adding " + (this.numberOfReplicas + 1) + " nodes and performing rerouting"); Builder discoBuilder = DiscoveryNodes.builder(); - for (int i=0; i deleteTerms = new ArrayList<>(); + for (int i = 0; i < numDocsToIndex; i++) { + doc = new Document(); + doc.add(new TextField("id", "extra_" + i, random().nextBoolean() ? Field.Store.YES : Field.Store.NO)); + deleteTerms.add(new Term("id", "extra_" + i)); + writer.addDocument(doc); + } + int numDocsToDelete = randomIntBetween(0, numDocsToIndex); + Collections.shuffle(deleteTerms, random()); + for (int i = 0; i < numDocsToDelete; i++) { + Term remove = deleteTerms.remove(0); + writer.deleteDocuments(remove); + } + writer.commit(); + segmentCommitInfos = Lucene.readSegmentInfos(dir); + assertEquals(2 + deleteTerms.size(), Lucene.getNumDocs(segmentCommitInfos)); + writer.close(); + dir.close(); } } diff --git a/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java b/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java index b072d8c2ced..93ead8ac862 100644 --- a/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java +++ b/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java @@ -22,6 +22,8 @@ package org.elasticsearch.gateway; import org.apache.lucene.util.LuceneTestCase.Slow; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.recovery.ShardRecoveryResponse; +import org.elasticsearch.action.admin.indices.stats.IndexStats; +import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; @@ -29,11 +31,16 @@ import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocation import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.SyncedFlushService; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope; import org.elasticsearch.test.InternalTestCluster.RestartCallback; +import org.elasticsearch.indices.SyncedFlushUtil; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSDirectoryService; import org.junit.Test; @@ -46,10 +53,7 @@ import static org.elasticsearch.index.query.QueryBuilders.termQuery; import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.*; /** * @@ -345,11 +349,13 @@ public class RecoveryFromGatewayTests extends ElasticsearchIntegrationTest { @Test @Slow + @TestLogging("gateway:TRACE,indices.recovery:TRACE,index.engine:TRACE") public void testReusePeerRecovery() throws Exception { final Settings settings = settingsBuilder() .put("action.admin.cluster.node.shutdown.delay", "10ms") .put(MockFSDirectoryService.CHECK_INDEX_ON_CLOSE, false) .put("gateway.recover_after_nodes", 4) + .put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CONCURRENT_RECOVERIES, 4) .put(MockFSDirectoryService.CRASH_INDEX, false).build(); @@ -376,8 +382,31 @@ public class RecoveryFromGatewayTests extends ElasticsearchIntegrationTest { client().admin().indices().prepareOptimize("test").setMaxNumSegments(100).get(); // just wait for merges client().admin().indices().prepareFlush().setWaitIfOngoing(true).setForce(true).get(); - logger.info("--> disabling allocation while the cluster is shut down"); + boolean useSyncIds = randomBoolean(); + if (useSyncIds == false) { + logger.info("--> disabling allocation while the cluster is shut down"); + // Disable allocations while we are closing nodes + client().admin().cluster().prepareUpdateSettings() + .setTransientSettings(settingsBuilder() + .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, EnableAllocationDecider.Allocation.NONE)) + .get(); + logger.info("--> full cluster restart"); + internalCluster().fullRestart(); + + logger.info("--> waiting for cluster to return to green after first shutdown"); + ensureGreen(); + } else { + logger.info("--> trying to sync flush"); + int numShards = Integer.parseInt(client().admin().indices().prepareGetSettings("test").get().getSetting("test", "index.number_of_shards")); + SyncedFlushService syncedFlushService = internalCluster().getInstance(SyncedFlushService.class); + for (int i = 0; i < numShards; i++) { + assertTrue(SyncedFlushUtil.attemptSyncedFlush(syncedFlushService, new ShardId("test", i)).success()); + } + assertSyncIdsNotNull(); + } + + logger.info("--> disabling allocation while the cluster is shut down", useSyncIds ? "" : " a second time"); // Disable allocations while we are closing nodes client().admin().cluster().prepareUpdateSettings() .setTransientSettings(settingsBuilder() @@ -386,49 +415,51 @@ public class RecoveryFromGatewayTests extends ElasticsearchIntegrationTest { logger.info("--> full cluster restart"); internalCluster().fullRestart(); - logger.info("--> waiting for cluster to return to green after first shutdown"); - ensureGreen(); - - logger.info("--> disabling allocation while the cluster is shut down second time"); - // Disable allocations while we are closing nodes - client().admin().cluster().prepareUpdateSettings() - .setTransientSettings(settingsBuilder() - .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, EnableAllocationDecider.Allocation.NONE)) - .get(); - logger.info("--> full cluster restart"); - internalCluster().fullRestart(); - - logger.info("--> waiting for cluster to return to green after second shutdown"); + logger.info("--> waiting for cluster to return to green after {}shutdown", useSyncIds ? "" : "second "); ensureGreen(); + if (useSyncIds) { + assertSyncIdsNotNull(); + } RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").get(); for (ShardRecoveryResponse response : recoveryResponse.shardResponses().get("test")) { RecoveryState recoveryState = response.recoveryState(); long recovered = 0; for (RecoveryState.File file : recoveryState.getIndex().fileDetails()) { - if (file.name().startsWith("segments")) { - recovered += file.length(); - } + if (file.name().startsWith("segments")) { + recovered += file.length(); + } } - if (!recoveryState.getPrimary()) { + if (!recoveryState.getPrimary() && (useSyncIds == false)) { logger.info("--> replica shard {} recovered from {} to {}, recovered {}, reuse {}", response.getShardId(), recoveryState.getSourceNode().name(), recoveryState.getTargetNode().name(), recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); assertThat("no bytes should be recovered", recoveryState.getIndex().recoveredBytes(), equalTo(recovered)); assertThat("data should have been reused", recoveryState.getIndex().reusedBytes(), greaterThan(0l)); // we have to recover the segments file since we commit the translog ID on engine startup - assertThat("all bytes should be reused except of the segments file", recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes()-recovered)); + assertThat("all bytes should be reused except of the segments file", recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes() - recovered)); assertThat("no files should be recovered except of the segments file", recoveryState.getIndex().recoveredFileCount(), equalTo(1)); - assertThat("all files should be reused except of the segments file", recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount()-1)); + assertThat("all files should be reused except of the segments file", recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount() - 1)); assertThat("> 0 files should be reused", recoveryState.getIndex().reusedFileCount(), greaterThan(0)); } else { + if (useSyncIds && !recoveryState.getPrimary()) { + logger.info("--> replica shard {} recovered from {} to {} using sync id, recovered {}, reuse {}", + response.getShardId(), recoveryState.getSourceNode().name(), recoveryState.getTargetNode().name(), + recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); + } assertThat(recoveryState.getIndex().recoveredBytes(), equalTo(0l)); assertThat(recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes())); assertThat(recoveryState.getIndex().recoveredFileCount(), equalTo(0)); assertThat(recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount())); } } + } + public void assertSyncIdsNotNull() { + IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNotNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } } @Test diff --git a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 04334b68a53..fa8c0f3be70 100644 --- a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -40,6 +40,7 @@ import org.apache.lucene.store.MockDirectoryWrapper; import org.apache.lucene.util.IOUtils; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.Base64; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -690,6 +691,69 @@ public class InternalEngineTests extends ElasticsearchTestCase { searchResult.close(); } + public void testSyncedFlush() throws IOException { + final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); + engine.create(new Engine.Create(null, newUid("1"), doc)); + Engine.CommitId commitID = engine.flush(); + assertThat(commitID, equalTo(new Engine.CommitId(store.readLastCommittedSegmentsInfo().getId()))); + byte[] wrongBytes = Base64.decode(commitID.toString()); + wrongBytes[0] = (byte) ~wrongBytes[0]; + Engine.CommitId wrongId = new Engine.CommitId(wrongBytes); + assertEquals("should fail to sync flush with wrong id (but no docs)", engine.syncFlush(syncId + "1", wrongId), + Engine.SyncedFlushResult.COMMIT_MISMATCH); + engine.create(new Engine.Create(null, newUid("2"), doc)); + assertEquals("should fail to sync flush with right id but pending doc", engine.syncFlush(syncId + "2", commitID), + Engine.SyncedFlushResult.PENDING_OPERATIONS); + commitID = engine.flush(); + assertEquals("should succeed to flush commit with right id and no pending doc", engine.syncFlush(syncId, commitID), + Engine.SyncedFlushResult.SUCCESS); + assertEquals(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + } + + public void testSycnedFlushSurvivesEngineRestart() throws IOException { + final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); + engine.create(new Engine.Create(null, newUid("1"), doc)); + final Engine.CommitId commitID = engine.flush(); + assertEquals("should succeed to flush commit with right id and no pending doc", engine.syncFlush(syncId, commitID), + Engine.SyncedFlushResult.SUCCESS); + assertEquals(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + EngineConfig config = engine.config(); + if (randomBoolean()) { + engine.close(); + } else { + engine.flushAndClose(); + } + engine = new InternalEngine(config, randomBoolean()); + assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + } + + public void testSycnedFlushVanishesOnReplay() throws IOException { + final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); + engine.create(new Engine.Create(null, newUid("1"), doc)); + final Engine.CommitId commitID = engine.flush(); + assertEquals("should succeed to flush commit with right id and no pending doc", engine.syncFlush(syncId, commitID), + Engine.SyncedFlushResult.SUCCESS); + assertEquals(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + doc = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), new BytesArray("{}"), null); + engine.create(new Engine.Create(null, newUid("2"), doc)); + EngineConfig config = engine.config(); + engine.close(); + final MockDirectoryWrapper directory = DirectoryUtils.getLeaf(store.directory(), MockDirectoryWrapper.class); + if (directory != null) { + // since we rollback the IW we are writing the same segment files again after starting IW but MDW prevents + // this so we have to disable the check explicitly + directory.setPreventDoubleWrite(false); + } + engine = new InternalEngine(config, false); + assertNull("Sync ID must be gone since we have a document to replay", engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + @Test public void testVersioningNewCreate() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); @@ -1811,8 +1875,6 @@ public class InternalEngineTests extends ElasticsearchTestCase { translog.close(); EngineConfig config = engine.config(); - Path translogPath = config.getTranslogConfig().getTranslogPath(); - /* create a TranslogConfig that has been created with a different UUID */ TranslogConfig translogConfig = new TranslogConfig(shardId, translog.location(), config.getIndexSettings(), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool); @@ -1833,5 +1895,4 @@ public class InternalEngineTests extends ElasticsearchTestCase { assertThat(topDocs.totalHits, equalTo(numDocs)); } } - } diff --git a/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 03efbad3e5d..0ecc99d923b 100644 --- a/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.index.shard; +import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.MutableShardRouting; @@ -28,6 +29,7 @@ import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.ShardLock; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.indices.IndicesService; @@ -41,6 +43,8 @@ import java.util.HashSet; import java.util.Set; import java.util.concurrent.ExecutionException; +import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; +import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; @@ -257,8 +261,25 @@ public class IndexShardTests extends ElasticsearchSingleNodeTest { indexShard.decrementOperationCounter(); indexShard.decrementOperationCounter(); assertEquals(1, indexShard.getOperationsCount()); + } - + @Test + public void testMarkAsInactiveTriggersSyncedFlush() throws Exception { + assertAcked(client().admin().indices().prepareCreate("test") + .setSettings(SETTING_NUMBER_OF_SHARDS, 1, SETTING_NUMBER_OF_REPLICAS, 0)); + client().prepareIndex("test", "test").setSource("{}").get(); + ensureGreen("test"); + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + indicesService.indexService("test").shard(0).markAsInactive(); + assertBusy(new Runnable() { // should be very very quick + @Override + public void run() { + IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + assertNotNull(indexStats.getShards()[0].getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + }); + IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + assertNotNull(indexStats.getShards()[0].getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); } public static ShardStateMetaData load(ESLogger logger, Path... shardPaths) throws IOException { diff --git a/src/test/java/org/elasticsearch/index/store/StoreTest.java b/src/test/java/org/elasticsearch/index/store/StoreTest.java index df2adbe2d5b..f17862150f6 100644 --- a/src/test/java/org/elasticsearch/index/store/StoreTest.java +++ b/src/test/java/org/elasticsearch/index/store/StoreTest.java @@ -32,26 +32,37 @@ import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.Version; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.common.io.stream.InputStreamStreamInput; +import org.elasticsearch.common.io.stream.OutputStreamStreamOutput; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.env.ShardLock; import org.elasticsearch.index.Index; +import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy; +import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.TranslogConfig; +import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ElasticsearchTestCase; import org.hamcrest.Matchers; import org.junit.Test; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.FileNotFoundException; import java.io.IOException; -import java.nio.file.Files; import java.nio.file.NoSuchFileException; import java.util.*; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.zip.Adler32; +import static org.elasticsearch.common.settings.ImmutableSettings.Builder.EMPTY_SETTINGS; +import static org.elasticsearch.test.VersionUtils.randomVersion; import static org.hamcrest.Matchers.*; public class StoreTest extends ElasticsearchTestCase { @@ -181,6 +192,7 @@ public class StoreTest extends ElasticsearchTestCase { public SegmentInfo read(Directory directory, String segmentName, byte[] segmentID, IOContext context) throws IOException { return segmentInfoFormat.read(directory, segmentName, segmentID, context); } + // this sucks it's a full copy of Lucene50SegmentInfoFormat but hey I couldn't find a way to make it write 4_5_0 versions // somebody was too paranoid when implementing this. ey rmuir, was that you? - go fix it :P @Override @@ -537,7 +549,7 @@ public class StoreTest extends ElasticsearchTestCase { } final long luceneChecksum; final long adler32LegacyChecksum = adler32.getValue(); - try(IndexInput indexInput = dir.openInput("lucene_checksum.bin", IOContext.DEFAULT)) { + try (IndexInput indexInput = dir.openInput("lucene_checksum.bin", IOContext.DEFAULT)) { assertEquals(luceneFileLength, indexInput.length()); luceneChecksum = CodecUtil.retrieveChecksum(indexInput); } @@ -552,8 +564,8 @@ public class StoreTest extends ElasticsearchTestCase { } { // negative check - wrong checksum - StoreFileMetaData lucene = new StoreFileMetaData("lucene_checksum.bin", luceneFileLength, Store.digestToString(luceneChecksum+1), Version.LUCENE_4_8_0); - StoreFileMetaData legacy = new StoreFileMetaData("legacy.bin", legacyFileLength, Store.digestToString(adler32LegacyChecksum+1)); + StoreFileMetaData lucene = new StoreFileMetaData("lucene_checksum.bin", luceneFileLength, Store.digestToString(luceneChecksum + 1), Version.LUCENE_4_8_0); + StoreFileMetaData legacy = new StoreFileMetaData("legacy.bin", legacyFileLength, Store.digestToString(adler32LegacyChecksum + 1)); assertTrue(legacy.hasLegacyChecksum()); assertFalse(lucene.hasLegacyChecksum()); assertFalse(Store.checkIntegrityNoException(lucene, dir)); @@ -561,8 +573,8 @@ public class StoreTest extends ElasticsearchTestCase { } { // negative check - wrong length - StoreFileMetaData lucene = new StoreFileMetaData("lucene_checksum.bin", luceneFileLength+1, Store.digestToString(luceneChecksum), Version.LUCENE_4_8_0); - StoreFileMetaData legacy = new StoreFileMetaData("legacy.bin", legacyFileLength+1, Store.digestToString(adler32LegacyChecksum)); + StoreFileMetaData lucene = new StoreFileMetaData("lucene_checksum.bin", luceneFileLength + 1, Store.digestToString(luceneChecksum), Version.LUCENE_4_8_0); + StoreFileMetaData legacy = new StoreFileMetaData("legacy.bin", legacyFileLength + 1, Store.digestToString(adler32LegacyChecksum)); assertTrue(legacy.hasLegacyChecksum()); assertFalse(lucene.hasLegacyChecksum()); assertFalse(Store.checkIntegrityNoException(lucene, dir)); @@ -617,19 +629,19 @@ public class StoreTest extends ElasticsearchTestCase { IOUtils.close(dir); } - private void readIndexInputFullyWithRandomSeeks(IndexInput indexInput) throws IOException{ + private void readIndexInputFullyWithRandomSeeks(IndexInput indexInput) throws IOException { BytesRef ref = new BytesRef(scaledRandomIntBetween(1, 1024)); long pos = 0; while (pos < indexInput.length()) { assertEquals(pos, indexInput.getFilePointer()); int op = random().nextInt(5); - if (op == 0 ) { - int shift = 100 - randomIntBetween(0, 200); - pos = Math.min(indexInput.length() - 1, Math.max(0, pos + shift)); + if (op == 0) { + int shift = 100 - randomIntBetween(0, 200); + pos = Math.min(indexInput.length() - 1, Math.max(0, pos + shift)); indexInput.seek(pos); } else if (op == 1) { indexInput.readByte(); - pos ++; + pos++; } else { int min = (int) Math.min(indexInput.length() - pos, ref.bytes.length); indexInput.readBytes(ref.bytes, ref.offset, min); @@ -674,16 +686,18 @@ public class StoreTest extends ElasticsearchTestCase { public LuceneManagedDirectoryService(Random random) { this(random, true); } + public LuceneManagedDirectoryService(Random random, boolean preventDoubleWrite) { super(new ShardId("fake", 1), ImmutableSettings.EMPTY); - dir = StoreTest.newDirectory(random); - if (dir instanceof MockDirectoryWrapper) { - ((MockDirectoryWrapper)dir).setPreventDoubleWrite(preventDoubleWrite); - // TODO: fix this test to handle virus checker - ((MockDirectoryWrapper)dir).setEnableVirusScanner(false); - } + dir = StoreTest.newDirectory(random); + if (dir instanceof MockDirectoryWrapper) { + ((MockDirectoryWrapper) dir).setPreventDoubleWrite(preventDoubleWrite); + // TODO: fix this test to handle virus checker + ((MockDirectoryWrapper) dir).setEnableVirusScanner(false); + } this.random = random; } + @Override public Directory newDirectory() throws IOException { return dir; @@ -712,11 +726,11 @@ public class StoreTest extends ElasticsearchTestCase { @Test public void testRecoveryDiffWithLegacyCommit() { Map metaDataMap = new HashMap<>(); - metaDataMap.put("segments_1", new StoreFileMetaData("segments_1", 50, null, null, new BytesRef(new byte[] {1}))); + metaDataMap.put("segments_1", new StoreFileMetaData("segments_1", 50, null, null, new BytesRef(new byte[]{1}))); metaDataMap.put("_0_1.del", new StoreFileMetaData("_0_1.del", 42, "foobarbaz", null, new BytesRef())); - Store.MetadataSnapshot first = new Store.MetadataSnapshot(metaDataMap); + Store.MetadataSnapshot first = new Store.MetadataSnapshot(metaDataMap, Collections.EMPTY_MAP, 0); - Store.MetadataSnapshot second = new Store.MetadataSnapshot(metaDataMap); + Store.MetadataSnapshot second = new Store.MetadataSnapshot(metaDataMap, Collections.EMPTY_MAP, 0); Store.RecoveryDiff recoveryDiff = first.recoveryDiff(second); assertEquals(recoveryDiff.toString(), recoveryDiff.different.size(), 2); } @@ -761,7 +775,7 @@ public class StoreTest extends ElasticsearchTestCase { store.close(); } long time = new Date().getTime(); - while(time == new Date().getTime()) { + while (time == new Date().getTime()) { Thread.sleep(10); // bump the time } Store.MetadataSnapshot second; @@ -828,7 +842,7 @@ public class StoreTest extends ElasticsearchTestCase { } Store.RecoveryDiff afterDeleteDiff = metadata.recoveryDiff(second); if (delFile != null) { - assertThat(afterDeleteDiff.identical.size(), equalTo(metadata.size()-2)); // segments_N + del file + assertThat(afterDeleteDiff.identical.size(), equalTo(metadata.size() - 2)); // segments_N + del file assertThat(afterDeleteDiff.different.size(), equalTo(0)); assertThat(afterDeleteDiff.missing.size(), equalTo(2)); } else { @@ -857,7 +871,7 @@ public class StoreTest extends ElasticsearchTestCase { Store.MetadataSnapshot newCommitMetaData = store.getMetadata(); Store.RecoveryDiff newCommitDiff = newCommitMetaData.recoveryDiff(metadata); if (delFile != null) { - assertThat(newCommitDiff.identical.size(), equalTo(newCommitMetaData.size()-5)); // segments_N, del file, cfs, cfe, si for the new segment + assertThat(newCommitDiff.identical.size(), equalTo(newCommitMetaData.size() - 5)); // segments_N, del file, cfs, cfe, si for the new segment assertThat(newCommitDiff.different.size(), equalTo(1)); // the del file must be different assertThat(newCommitDiff.different.get(0).name(), endsWith(".liv")); assertThat(newCommitDiff.missing.size(), equalTo(4)); // segments_N,cfs, cfe, si for the new segment @@ -884,7 +898,7 @@ public class StoreTest extends ElasticsearchTestCase { int docs = 1 + random().nextInt(100); int numCommits = 0; for (int i = 0; i < docs; i++) { - if (i > 0 && randomIntBetween(0, 10 ) == 0) { + if (i > 0 && randomIntBetween(0, 10) == 0) { writer.commit(); numCommits++; } @@ -949,7 +963,7 @@ public class StoreTest extends ElasticsearchTestCase { assertTrue(firstMeta.contains(file) || Store.isChecksum(file) || file.equals("write.lock")); if (Store.isChecksum(file)) { numChecksums++; - } else if (secondMeta.contains(file) == false) { + } else if (secondMeta.contains(file) == false) { numNotFound++; } @@ -968,7 +982,7 @@ public class StoreTest extends ElasticsearchTestCase { assertTrue(file, secondMeta.contains(file) || Store.isChecksum(file) || file.equals("write.lock")); if (Store.isChecksum(file)) { numChecksums++; - } else if (firstMeta.contains(file) == false) { + } else if (firstMeta.contains(file) == false) { numNotFound++; } @@ -986,7 +1000,7 @@ public class StoreTest extends ElasticsearchTestCase { Map metaDataMap = new HashMap<>(); metaDataMap.put("segments_1", new StoreFileMetaData("segments_1", 50, null, null, new BytesRef(new byte[]{1}))); metaDataMap.put("_0_1.del", new StoreFileMetaData("_0_1.del", 42, "foobarbaz", null, new BytesRef())); - Store.MetadataSnapshot snapshot = new Store.MetadataSnapshot(metaDataMap); + Store.MetadataSnapshot snapshot = new Store.MetadataSnapshot(metaDataMap, Collections.EMPTY_MAP, 0); final ShardId shardId = new ShardId(new Index("index"), 1); DirectoryService directoryService = new LuceneManagedDirectoryService(random()); @@ -1010,7 +1024,7 @@ public class StoreTest extends ElasticsearchTestCase { final AtomicInteger count = new AtomicInteger(0); final ShardLock lock = new DummyShardLock(shardId); - Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, lock , new Store.OnClose() { + Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, lock, new Store.OnClose() { @Override public void handle(ShardLock theLock) { assertEquals(shardId, theLock.getShardId()); @@ -1086,4 +1100,95 @@ public class StoreTest extends ElasticsearchTestCase { } return numNonExtra; } + + @Test + public void testMetadataSnapshotStreaming() throws Exception { + + Store.MetadataSnapshot outMetadataSnapshot = createMetaDataSnapshot(); + org.elasticsearch.Version targetNodeVersion = randomVersion(random()); + + ByteArrayOutputStream outBuffer = new ByteArrayOutputStream(); + OutputStreamStreamOutput out = new OutputStreamStreamOutput(outBuffer); + out.setVersion(targetNodeVersion); + outMetadataSnapshot.writeTo(out); + + ByteArrayInputStream inBuffer = new ByteArrayInputStream(outBuffer.toByteArray()); + InputStreamStreamInput in = new InputStreamStreamInput(inBuffer); + in.setVersion(targetNodeVersion); + Store.MetadataSnapshot inMetadataSnapshot = new Store.MetadataSnapshot(in); + Map origEntries = new HashMap<>(); + origEntries.putAll(outMetadataSnapshot.asMap()); + for (Map.Entry entry : inMetadataSnapshot.asMap().entrySet()) { + assertThat(entry.getValue().name(), equalTo(origEntries.remove(entry.getKey()).name())); + } + assertThat(origEntries.size(), equalTo(0)); + assertThat(inMetadataSnapshot.getCommitUserData(), equalTo(outMetadataSnapshot.getCommitUserData())); + } + + protected Store.MetadataSnapshot createMetaDataSnapshot() { + StoreFileMetaData storeFileMetaData1 = new StoreFileMetaData("segments", 1); + StoreFileMetaData storeFileMetaData2 = new StoreFileMetaData("no_segments", 1); + Map storeFileMetaDataMap = new HashMap<>(); + storeFileMetaDataMap.put(storeFileMetaData1.name(), storeFileMetaData1); + storeFileMetaDataMap.put(storeFileMetaData2.name(), storeFileMetaData2); + Map commitUserData = new HashMap<>(); + commitUserData.put("userdata_1", "test"); + commitUserData.put("userdata_2", "test"); + return new Store.MetadataSnapshot(storeFileMetaDataMap, commitUserData, 0); + } + + @Test + public void testUserDataRead() throws IOException { + final ShardId shardId = new ShardId(new Index("index"), 1); + DirectoryService directoryService = new LuceneManagedDirectoryService(random()); + Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, new DummyShardLock(shardId)); + IndexWriterConfig config = newIndexWriterConfig(random(), new MockAnalyzer(random())).setCodec(TestUtil.getDefaultCodec()); + SnapshotDeletionPolicy deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastDeletionPolicy(shardId, EMPTY_SETTINGS)); + config.setIndexDeletionPolicy(deletionPolicy); + IndexWriter writer = new IndexWriter(store.directory(), config); + Document doc = new Document(); + doc.add(new TextField("id", "1", Field.Store.NO)); + writer.addDocument(doc); + Map commitData = new HashMap<>(2); + String syncId = "a sync id"; + String translogId = "a translog id"; + commitData.put(Engine.SYNC_COMMIT_ID, syncId); + commitData.put(Translog.TRANSLOG_GENERATION_KEY, translogId); + writer.setCommitData(commitData); + writer.commit(); + writer.close(); + Store.MetadataSnapshot metadata; + if (randomBoolean()) { + metadata = store.getMetadata(); + } else { + metadata = store.getMetadata(deletionPolicy.snapshot()); + } + assertFalse(metadata.asMap().isEmpty()); + // do not check for correct files, we have enough tests for that above + assertThat(metadata.getCommitUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncId)); + assertThat(metadata.getCommitUserData().get(Translog.TRANSLOG_GENERATION_KEY), equalTo(translogId)); + TestUtil.checkIndex(store.directory()); + assertDeleteContent(store, directoryService); + IOUtils.close(store); + } + + @Test + public void testStreamStoreFilesMetaData() throws Exception { + Store.MetadataSnapshot metadataSnapshot = createMetaDataSnapshot(); + TransportNodesListShardStoreMetaData.StoreFilesMetaData outStoreFileMetaData = new TransportNodesListShardStoreMetaData.StoreFilesMetaData(randomBoolean(), new ShardId("test", 0),metadataSnapshot); + ByteArrayOutputStream outBuffer = new ByteArrayOutputStream(); + OutputStreamStreamOutput out = new OutputStreamStreamOutput(outBuffer); + org.elasticsearch.Version targetNodeVersion = randomVersion(random()); + out.setVersion(targetNodeVersion); + outStoreFileMetaData.writeTo(out); + ByteArrayInputStream inBuffer = new ByteArrayInputStream(outBuffer.toByteArray()); + InputStreamStreamInput in = new InputStreamStreamInput(inBuffer); + in.setVersion(targetNodeVersion); + TransportNodesListShardStoreMetaData.StoreFilesMetaData inStoreFileMetaData = TransportNodesListShardStoreMetaData.StoreFilesMetaData.readStoreFilesMetaData(in); + Iterator outFiles = outStoreFileMetaData.iterator(); + for (StoreFileMetaData inFile : inStoreFileMetaData) { + assertThat(inFile.name(), equalTo(outFiles.next().name())); + } + assertThat(outStoreFileMetaData.syncId(), equalTo(inStoreFileMetaData.syncId())); + } } diff --git a/src/test/java/org/elasticsearch/indices/FlushTest.java b/src/test/java/org/elasticsearch/indices/FlushTest.java index 161ad6c0553..65fb2f6816b 100644 --- a/src/test/java/org/elasticsearch/indices/FlushTest.java +++ b/src/test/java/org/elasticsearch/indices/FlushTest.java @@ -20,17 +20,30 @@ package org.elasticsearch.indices; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushResponse; +import org.elasticsearch.action.admin.indices.stats.IndexStats; +import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ElasticsearchIntegrationTest; -import org.elasticsearch.test.store.MockFSDirectoryService; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.junit.Test; +import java.io.IOException; import java.util.Arrays; +import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.MatcherAssert.assertThat; +import static java.lang.Thread.sleep; import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; @@ -72,4 +85,140 @@ public class FlushTest extends ElasticsearchIntegrationTest { assertThat(errors, emptyIterable()); } } + + @TestLogging("indices:TRACE") + public void testSyncedFlush() throws ExecutionException, InterruptedException, IOException { + internalCluster().ensureAtLeastNumDataNodes(2); + prepareCreate("test").setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).get(); + ensureGreen(); + + IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + + SyncedFlushService.SyncedFlushResult result = SyncedFlushUtil.attemptSyncedFlush(internalCluster().getInstance(SyncedFlushService.class), new ShardId("test", 0)); + assertTrue(result.success()); + assertThat(result.totalShards(), equalTo(indexStats.getShards().length)); + assertThat(result.successfulShards(), equalTo(indexStats.getShards().length)); + + indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + String syncId = result.syncId(); + for (ShardStats shardStats : indexStats.getShards()) { + final String shardSyncId = shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID); + assertThat(shardSyncId, equalTo(syncId)); + } + + // now, start new node and relocate a shard there and see if sync id still there + String newNodeName = internalCluster().startNode(); + ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + ShardRouting shardRouting = clusterState.getRoutingTable().index("test").shard(0).iterator().next(); + String currentNodeName = clusterState.nodes().resolveNode(shardRouting.currentNodeId()).name(); + assertFalse(currentNodeName.equals(newNodeName)); + internalCluster().client().admin().cluster().prepareReroute().add(new MoveAllocationCommand(new ShardId("test", 0), currentNodeName, newNodeName)).get(); + + client().admin().cluster().prepareHealth() + .setWaitForRelocatingShards(0) + .get(); + indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNotNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + + client().admin().indices().prepareUpdateSettings("test").setSettings(ImmutableSettings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).build()).get(); + ensureGreen("test"); + indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNotNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + client().admin().indices().prepareUpdateSettings("test").setSettings(ImmutableSettings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, internalCluster().numDataNodes() - 1).build()).get(); + ensureGreen("test"); + indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNotNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + } + + @TestLogging("indices:TRACE") + public void testSyncedFlushWithApi() throws ExecutionException, InterruptedException, IOException { + + createIndex("test"); + ensureGreen(); + + IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + logger.info("--> trying sync flush"); + SealIndicesResponse sealIndicesResponse = client().admin().indices().prepareSealIndices("test").get(); + logger.info("--> sync flush done"); + indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNotNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + } + + @TestLogging("indices:TRACE") + public void testSyncedFlushWithApiAndConcurrentIndexing() throws Exception { + + internalCluster().ensureAtLeastNumDataNodes(3); + createIndex("test"); + + client().admin().indices().prepareUpdateSettings("test").setSettings( + ImmutableSettings.builder().put("index.translog.disable_flush", true).put("index.refresh_interval", -1).put("index.number_of_replicas", internalCluster().numDataNodes() - 1)) + .get(); + ensureGreen(); + final AtomicBoolean stop = new AtomicBoolean(false); + final AtomicInteger numDocs = new AtomicInteger(0); + Thread indexingThread = new Thread() { + @Override + public void run() { + while (stop.get() == false) { + client().prepareIndex().setIndex("test").setType("doc").setSource("{}").get(); + numDocs.incrementAndGet(); + } + } + }; + indexingThread.start(); + + IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + logger.info("--> trying sync flush"); + SealIndicesResponse sealIndicesResponse = client().admin().indices().prepareSealIndices("test").get(); + logger.info("--> sync flush done"); + stop.set(true); + indexingThread.join(); + indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertFlushResponseEqualsShardStats(shardStats, sealIndicesResponse); + } + refresh(); + assertThat(client().prepareCount().get().getCount(), equalTo((long) numDocs.get())); + logger.info("indexed {} docs", client().prepareCount().get().getCount()); + logClusterState(); + internalCluster().fullRestart(); + ensureGreen(); + assertThat(client().prepareCount().get().getCount(), equalTo((long) numDocs.get())); + } + + private void assertFlushResponseEqualsShardStats(ShardStats shardStats, SealIndicesResponse sealIndicesResponse) { + + for (SyncedFlushService.SyncedFlushResult shardResult : sealIndicesResponse.results()) { + if (shardStats.getShardRouting().getId() == shardResult.shardId().getId()) { + for (Map.Entry singleResponse : shardResult.shardResponses().entrySet()) { + if (singleResponse.getKey().currentNodeId().equals(shardStats.getShardRouting().currentNodeId())) { + if (singleResponse.getValue().success()) { + assertNotNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + logger.info("sync flushed {} on node {}", singleResponse.getKey().shardId(), singleResponse.getKey().currentNodeId()); + } else { + assertNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + logger.info("sync flush failed for {} on node {}", singleResponse.getKey().shardId(), singleResponse.getKey().currentNodeId()); + } + } + } + } + } + } } diff --git a/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java index afa9e0e6cf4..910902b7b4b 100644 --- a/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -86,5 +86,5 @@ public class IndicesLifecycleListenerSingleNodeTests extends ElasticsearchSingle assertAcked(client().admin().indices().prepareDelete("test").get()); assertEquals(7, counter.get()); } - + } diff --git a/src/test/java/org/elasticsearch/indices/SealTests.java b/src/test/java/org/elasticsearch/indices/SealTests.java new file mode 100644 index 00000000000..723f699d8cd --- /dev/null +++ b/src/test/java/org/elasticsearch/indices/SealTests.java @@ -0,0 +1,49 @@ +/* + * 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.indices; + +import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.junit.Test; + +import static java.lang.Thread.sleep; +import static org.hamcrest.Matchers.equalTo; + +@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.TEST, numDataNodes = 0) +public class SealTests extends ElasticsearchIntegrationTest { + + @Test + public void testUnallocatedShardsDoesNotHang() throws InterruptedException { + ImmutableSettings.Builder settingsBuilder = ImmutableSettings.builder() + .put("node.data", false) + .put("node.master", true) + .put("path.data", createTempDir().toString()); + internalCluster().startNode(settingsBuilder.build()); + // create an index but because no data nodes are available no shards will be allocated + createIndex("test"); + // this should not hang but instead immediately return with empty result set + SealIndicesResponse sealIndicesResponse = client().admin().indices().prepareSealIndices("test").get(); + // just to make sure the test actually tests the right thing + int numShards = client().admin().indices().prepareGetSettings("test").get().getIndexToSettings().get("test").getAsInt(IndexMetaData.SETTING_NUMBER_OF_SHARDS, -1); + assertThat(sealIndicesResponse.results().size(), equalTo(numShards)); + assertThat(sealIndicesResponse.results().iterator().next().failureReason(), equalTo("no active primary available")); + } +} diff --git a/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java b/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java new file mode 100644 index 00000000000..348c5753c83 --- /dev/null +++ b/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java @@ -0,0 +1,228 @@ +/* + * 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.indices; + +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.Strings; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.test.ElasticsearchSingleNodeTest; +import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; + +import java.util.List; +import java.util.Map; + +/** + */ +public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { + + public void testModificationPreventsSealing() throws InterruptedException { + createIndex("test"); + client().prepareIndex("test", "test", "1").setSource("{}").get(); + IndexService test = getInstanceFromNode(IndicesService.class).indexService("test"); + IndexShard shard = test.shard(0); + + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); + final ShardId shardId = shard.shardId(); + final ClusterState state = getInstanceFromNode(ClusterService.class).state(); + final IndexShardRoutingTable shardRoutingTable = flushService.getActiveShardRoutings(shardId, state); + final List activeShards = shardRoutingTable.activeShards(); + assertEquals("exactly one active shard", 1, activeShards.size()); + Map commitIds = SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId); + assertEquals("exactly one commit id", 1, commitIds.size()); + client().prepareIndex("test", "test", "2").setSource("{}").get(); + String syncId = Strings.base64UUID(); + SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); + flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId,listener); + listener.latch.await(); + assertNull(listener.error); + SyncedFlushService.SyncedFlushResult syncedFlushResult = listener.result; + assertNotNull(syncedFlushResult); + assertEquals(0, syncedFlushResult.successfulShards()); + assertEquals(1, syncedFlushResult.totalShards()); + assertEquals(syncId, syncedFlushResult.syncId()); + assertNotNull(syncedFlushResult.shardResponses().get(activeShards.get(0))); + assertFalse(syncedFlushResult.shardResponses().get(activeShards.get(0)).success()); + assertEquals("pending operations", syncedFlushResult.shardResponses().get(activeShards.get(0)).failureReason()); + + SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId); // pull another commit and make sure we can't seal with the old one + listener = new SyncedFlushUtil.LatchedListener(); + flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId,listener); + listener.latch.await(); + assertNull(listener.error); + syncedFlushResult = listener.result; + assertNotNull(syncedFlushResult); + assertEquals(0, syncedFlushResult.successfulShards()); + assertEquals(1, syncedFlushResult.totalShards()); + assertEquals(syncId, syncedFlushResult.syncId()); + assertNotNull(syncedFlushResult.shardResponses().get(activeShards.get(0))); + assertFalse(syncedFlushResult.shardResponses().get(activeShards.get(0)).success()); + assertEquals("commit has changed", syncedFlushResult.shardResponses().get(activeShards.get(0)).failureReason()); + ElasticsearchAssertions.assertVersionSerializable(syncedFlushResult); + } + + public void testSingleShardSuccess() throws InterruptedException { + createIndex("test"); + client().prepareIndex("test", "test", "1").setSource("{}").get(); + IndexService test = getInstanceFromNode(IndicesService.class).indexService("test"); + IndexShard shard = test.shard(0); + + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); + final ShardId shardId = shard.shardId(); + SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); + flushService.attemptSyncedFlush(shardId, listener); + listener.latch.await(); + assertNull(listener.error); + SyncedFlushService.SyncedFlushResult syncedFlushResult = listener.result; + assertNotNull(syncedFlushResult); + assertEquals(1, syncedFlushResult.successfulShards()); + assertEquals(1, syncedFlushResult.totalShards()); + SyncedFlushService.SyncedFlushResponse response = syncedFlushResult.shardResponses().values().iterator().next(); + assertTrue(response.success()); + ElasticsearchAssertions.assertVersionSerializable(syncedFlushResult); + } + + public void testSyncFailsIfOperationIsInFlight() throws InterruptedException { + createIndex("test"); + client().prepareIndex("test", "test", "1").setSource("{}").get(); + IndexService test = getInstanceFromNode(IndicesService.class).indexService("test"); + IndexShard shard = test.shard(0); + + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); + final ShardId shardId = shard.shardId(); + shard.incrementOperationCounter(); + try { + SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); + flushService.attemptSyncedFlush(shardId, listener); + listener.latch.await(); + assertNull(listener.error); + SyncedFlushService.SyncedFlushResult syncedFlushResult = listener.result; + assertNotNull(syncedFlushResult); + assertEquals(0, syncedFlushResult.successfulShards()); + assertEquals(0, syncedFlushResult.totalShards()); + assertEquals("operation counter on primary is non zero [2]", syncedFlushResult.failureReason()); + ElasticsearchAssertions.assertVersionSerializable(syncedFlushResult); + } finally { + shard.decrementOperationCounter(); + } + } + + public void testSyncFailsOnIndexClosedOrMissing() throws InterruptedException { + createIndex("test"); + IndexService test = getInstanceFromNode(IndicesService.class).indexService("test"); + IndexShard shard = test.shard(0); + + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); + SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); + flushService.attemptSyncedFlush(new ShardId("test", 1), listener); + listener.latch.await(); + assertNotNull(listener.error); + assertNull(listener.result); + assertEquals("missing", listener.error.getMessage()); + + final ShardId shardId = shard.shardId(); + + client().admin().indices().prepareClose("test").get(); + listener = new SyncedFlushUtil.LatchedListener(); + flushService.attemptSyncedFlush(shardId, listener); + listener.latch.await(); + assertNotNull(listener.error); + assertNull(listener.result); + assertEquals("closed", listener.error.getMessage()); + + listener = new SyncedFlushUtil.LatchedListener(); + flushService.attemptSyncedFlush(new ShardId("nosuchindex", 0), listener); + listener.latch.await(); + assertNotNull(listener.error); + assertNull(listener.result); + assertEquals("no such index", listener.error.getMessage()); + } + + public void testFailAfterIntermediateCommit() throws InterruptedException { + createIndex("test"); + client().prepareIndex("test", "test", "1").setSource("{}").get(); + IndexService test = getInstanceFromNode(IndicesService.class).indexService("test"); + IndexShard shard = test.shard(0); + + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); + final ShardId shardId = shard.shardId(); + final ClusterState state = getInstanceFromNode(ClusterService.class).state(); + final IndexShardRoutingTable shardRoutingTable = flushService.getActiveShardRoutings(shardId, state); + final List activeShards = shardRoutingTable.activeShards(); + assertEquals("exactly one active shard", 1, activeShards.size()); + Map commitIds = SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId); + assertEquals("exactly one commit id", 1, commitIds.size()); + if (randomBoolean()) { + client().prepareIndex("test", "test", "2").setSource("{}").get(); + } + client().admin().indices().prepareFlush("test").setForce(true).get(); + String syncId = Strings.base64UUID(); + final SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); + flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, listener); + listener.latch.await(); + assertNull(listener.error); + SyncedFlushService.SyncedFlushResult syncedFlushResult = listener.result; + assertNotNull(syncedFlushResult); + assertEquals(0, syncedFlushResult.successfulShards()); + assertEquals(1, syncedFlushResult.totalShards()); + assertEquals(syncId, syncedFlushResult.syncId()); + assertNotNull(syncedFlushResult.shardResponses().get(activeShards.get(0))); + assertFalse(syncedFlushResult.shardResponses().get(activeShards.get(0)).success()); + assertEquals("commit has changed", syncedFlushResult.shardResponses().get(activeShards.get(0)).failureReason()); + ElasticsearchAssertions.assertVersionSerializable(syncedFlushResult); + } + + public void testFailWhenCommitIsMissing() throws InterruptedException { + createIndex("test"); + client().prepareIndex("test", "test", "1").setSource("{}").get(); + IndexService test = getInstanceFromNode(IndicesService.class).indexService("test"); + IndexShard shard = test.shard(0); + + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); + final ShardId shardId = shard.shardId(); + final ClusterState state = getInstanceFromNode(ClusterService.class).state(); + final IndexShardRoutingTable shardRoutingTable = flushService.getActiveShardRoutings(shardId, state); + final List activeShards = shardRoutingTable.activeShards(); + assertEquals("exactly one active shard", 1, activeShards.size()); + Map commitIds = SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId); + assertEquals("exactly one commit id", 1, commitIds.size()); + commitIds.clear(); // wipe it... + String syncId = Strings.base64UUID(); + SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); + flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, listener); + listener.latch.await(); + assertNull(listener.error); + SyncedFlushService.SyncedFlushResult syncedFlushResult = listener.result; + assertNotNull(syncedFlushResult); + assertEquals(0, syncedFlushResult.successfulShards()); + assertEquals(1, syncedFlushResult.totalShards()); + assertEquals(syncId, syncedFlushResult.syncId()); + assertNotNull(syncedFlushResult.shardResponses().get(activeShards.get(0))); + assertFalse(syncedFlushResult.shardResponses().get(activeShards.get(0)).success()); + assertEquals("no commit id from pre-sync flush", syncedFlushResult.shardResponses().get(activeShards.get(0)).failureReason()); + ElasticsearchAssertions.assertVersionSerializable(syncedFlushResult); + } + + +} diff --git a/src/test/java/org/elasticsearch/indices/SyncedFlushUtil.java b/src/test/java/org/elasticsearch/indices/SyncedFlushUtil.java new file mode 100644 index 00000000000..e16c85b4b7e --- /dev/null +++ b/src/test/java/org/elasticsearch/indices/SyncedFlushUtil.java @@ -0,0 +1,93 @@ +/* + * 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.indices; + +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.LatchedActionListener; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.SyncedFlushService; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +/** Utils for SyncedFlush */ +public class SyncedFlushUtil { + + private SyncedFlushUtil() { + + } + + /** + * Blocking version of {@link SyncedFlushService#attemptSyncedFlush(ShardId, ActionListener)} + */ + public static SyncedFlushService.SyncedFlushResult attemptSyncedFlush(SyncedFlushService service, ShardId shardId) { + LatchedListener listener = new LatchedListener(); + service.attemptSyncedFlush(shardId, listener); + try { + listener.latch.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + if (listener.error != null) { + throw ExceptionsHelper.convertToElastic(listener.error); + } + return listener.result; + } + + public static final class LatchedListener implements ActionListener { + public volatile T result; + public volatile Throwable error; + public final CountDownLatch latch = new CountDownLatch(1); + + @Override + public void onResponse(T result) { + this.result = result; + latch.countDown(); + } + + @Override + public void onFailure(Throwable e) { + error = e; + latch.countDown(); + } + } + + /** + * Blocking version of {@link SyncedFlushService#sendPreSyncRequests(List, ClusterState, ShardId, ActionListener)} + */ + public static Map sendPreSyncRequests(SyncedFlushService service, List activeShards, ClusterState state, ShardId shardId) { + LatchedListener> listener = new LatchedListener<>(); + service.sendPreSyncRequests(activeShards, state, shardId, listener); + try { + listener.latch.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + if (listener.error != null) { + throw ExceptionsHelper.convertToElastic(listener.error); + } + return listener.result; + } + +} diff --git a/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTest.java b/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTest.java index c15d1d8b552..4a1586e5c45 100644 --- a/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTest.java +++ b/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTest.java @@ -19,12 +19,14 @@ package org.elasticsearch.indices.recovery; +import org.apache.lucene.index.IndexFileNames; import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.InputStreamStreamInput; import org.elasticsearch.common.io.stream.OutputStreamStreamOutput; import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; @@ -32,6 +34,8 @@ import org.junit.Test; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import static org.elasticsearch.test.VersionUtils.randomVersion; import static org.hamcrest.Matchers.equalTo; @@ -49,7 +53,7 @@ public class StartRecoveryRequestTest extends ElasticsearchTestCase { new DiscoveryNode("a", new LocalTransportAddress("1"), targetNodeVersion), new DiscoveryNode("b", new LocalTransportAddress("1"), targetNodeVersion), true, - Collections.emptyMap(), + Store.MetadataSnapshot.EMPTY, RecoveryState.Type.RELOCATION, 1l @@ -69,11 +73,9 @@ public class StartRecoveryRequestTest extends ElasticsearchTestCase { assertThat(outRequest.sourceNode(), equalTo(inRequest.sourceNode())); assertThat(outRequest.targetNode(), equalTo(inRequest.targetNode())); assertThat(outRequest.markAsRelocated(), equalTo(inRequest.markAsRelocated())); - assertThat(outRequest.existingFiles(), equalTo(inRequest.existingFiles())); + assertThat(outRequest.metadataSnapshot().asMap(), equalTo(inRequest.metadataSnapshot().asMap())); assertThat(outRequest.recoveryId(), equalTo(inRequest.recoveryId())); assertThat(outRequest.recoveryType(), equalTo(inRequest.recoveryType())); } - - } diff --git a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java index a3afb48e344..f1fc137dfc8 100644 --- a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java +++ b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java @@ -49,6 +49,7 @@ import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; import org.elasticsearch.action.admin.indices.optimize.OptimizeResponse; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.admin.indices.segments.IndicesSegmentResponse; +import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; @@ -1518,8 +1519,13 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase client().admin().indices().prepareRefresh(indices).setIndicesOptions(IndicesOptions.lenientExpandOpen()).execute( new LatchedActionListener(newLatch(inFlightAsyncOperations))); } else if (maybeFlush && rarely()) { - client().admin().indices().prepareFlush(indices).setIndicesOptions(IndicesOptions.lenientExpandOpen()).execute( - new LatchedActionListener(newLatch(inFlightAsyncOperations))); + if (randomBoolean()) { + client().admin().indices().prepareFlush(indices).setIndicesOptions(IndicesOptions.lenientExpandOpen()).execute( + new LatchedActionListener(newLatch(inFlightAsyncOperations))); + } else { + client().admin().indices().prepareSealIndices(indices).execute( + new LatchedActionListener(newLatch(inFlightAsyncOperations))); + } } else if (rarely()) { client().admin().indices().prepareOptimize(indices).setIndicesOptions(IndicesOptions.lenientExpandOpen()).setMaxNumSegments(between(1, 10)).setFlush(maybeFlush && randomBoolean()).execute( new LatchedActionListener(newLatch(inFlightAsyncOperations))); diff --git a/src/test/java/org/elasticsearch/test/InternalTestCluster.java b/src/test/java/org/elasticsearch/test/InternalTestCluster.java index cf0a52df92f..bfdf951ccc6 100644 --- a/src/test/java/org/elasticsearch/test/InternalTestCluster.java +++ b/src/test/java/org/elasticsearch/test/InternalTestCluster.java @@ -74,6 +74,8 @@ import org.elasticsearch.index.cache.filter.FilterCacheModule; import org.elasticsearch.index.cache.filter.FilterCacheModule.FilterCacheSettings; import org.elasticsearch.index.cache.filter.index.IndexFilterCache; import org.elasticsearch.index.cache.filter.none.NoneFilterCache; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.EngineClosedException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardModule; import org.elasticsearch.index.shard.IndexShardState; @@ -978,6 +980,34 @@ public final class InternalTestCluster extends TestCluster { // and not all docs have been purged after the test) and inherit from // ElasticsearchIntegrationTest must override beforeIndexDeletion() to avoid failures. assertShardIndexCounter(); + //check that shards that have same sync id also contain same number of documents + assertSameSyncIdSameDocs(); + + } + + private void assertSameSyncIdSameDocs() { + Map docsOnShards = new HashMap<>(); + final Collection nodesAndClients = nodes.values(); + for (NodeAndClient nodeAndClient : nodesAndClients) { + IndicesService indexServices = getInstance(IndicesService.class, nodeAndClient.name); + for (IndexService indexService : indexServices) { + for (IndexShard indexShard : indexService) { + try { + String syncId = indexShard.engine().commitStats().getUserData().get(Engine.SYNC_COMMIT_ID); + if (syncId != null) { + long liveDocsOnShard = indexShard.docStats().getCount() - indexShard.docStats().getDeleted(); + if (docsOnShards.get(syncId) != null) { + assertThat("sync id is equal but number of docs does not match on node " + nodeAndClient.name + ". expected " + docsOnShards.get(syncId) + " but got " + liveDocsOnShard, docsOnShards.get(syncId), equalTo(liveDocsOnShard)); + } else { + docsOnShards.put(syncId, liveDocsOnShard); + } + } + } catch (EngineClosedException e) { + // nothing to do, shard is closed + } + } + } + } } private void assertShardIndexCounter() {