summaryrefslogtreecommitdiff
path: root/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java
blob: 59784456e988b1583feab7be076ee3d16fab9896 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
/*
 * 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.flush;

import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.xcontent.XContentType;
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.ShardNotFoundException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.threadpool.ThreadPool;

import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;

public class SyncedFlushSingleNodeTests extends ESSingleNodeTestCase {

    public void testModificationPreventsFlushing() throws InterruptedException {
        createIndex("test");
        client().prepareIndex("test", "test", "1").setSource("{}", XContentType.JSON).get();
        IndexService test = getInstanceFromNode(IndicesService.class).indexService(resolveIndex("test"));
        IndexShard shard = test.getShardOrNull(0);

        SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class);
        final ShardId shardId = shard.shardId();
        final ClusterState state = getInstanceFromNode(ClusterService.class).state();
        final IndexShardRoutingTable shardRoutingTable = flushService.getShardRoutingTable(shardId, state);
        final List<ShardRouting> activeShards = shardRoutingTable.activeShards();
        assertEquals("exactly one active shard", 1, activeShards.size());
        Map<String, Engine.CommitId> commitIds = SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId);
        assertEquals("exactly one commit id", 1, commitIds.size());
        client().prepareIndex("test", "test", "2").setSource("{}", XContentType.JSON).get();
        String syncId = UUIDs.base64UUID();
        SyncedFlushUtil.LatchedListener<ShardsSyncedFlushResult> listener = new SyncedFlushUtil.LatchedListener<>();
        flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, shardRoutingTable.size(), listener);
        listener.latch.await();
        assertNull(listener.error);
        ShardsSyncedFlushResult 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 sync-flush with the old one
        listener = new SyncedFlushUtil.LatchedListener();
        flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, shardRoutingTable.size(), 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());
    }

    public void testSingleShardSuccess() throws InterruptedException {
        createIndex("test");
        client().prepareIndex("test", "test", "1").setSource("{}", XContentType.JSON).get();
        IndexService test = getInstanceFromNode(IndicesService.class).indexService(resolveIndex("test"));
        IndexShard shard = test.getShardOrNull(0);

        SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class);
        final ShardId shardId = shard.shardId();
        SyncedFlushUtil.LatchedListener<ShardsSyncedFlushResult> listener = new SyncedFlushUtil.LatchedListener();
        flushService.attemptSyncedFlush(shardId, listener);
        listener.latch.await();
        assertNull(listener.error);
        ShardsSyncedFlushResult syncedFlushResult = listener.result;
        assertNotNull(syncedFlushResult);
        assertEquals(1, syncedFlushResult.successfulShards());
        assertEquals(1, syncedFlushResult.totalShards());
        SyncedFlushService.ShardSyncedFlushResponse response = syncedFlushResult.shardResponses().values().iterator().next();
        assertTrue(response.success());
    }

    public void testSyncFailsIfOperationIsInFlight() throws InterruptedException, ExecutionException {
        createIndex("test");
        client().prepareIndex("test", "test", "1").setSource("{}", XContentType.JSON).get();
        IndexService test = getInstanceFromNode(IndicesService.class).indexService(resolveIndex("test"));
        IndexShard shard = test.getShardOrNull(0);

        SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class);
        final ShardId shardId = shard.shardId();
        PlainActionFuture<Releasable> fut = new PlainActionFuture<>();
        shard.acquirePrimaryOperationLock(fut, ThreadPool.Names.INDEX);
        try (Releasable operationLock = fut.get()) {
            SyncedFlushUtil.LatchedListener<ShardsSyncedFlushResult> listener = new SyncedFlushUtil.LatchedListener<>();
            flushService.attemptSyncedFlush(shardId, listener);
            listener.latch.await();
            assertNull(listener.error);
            ShardsSyncedFlushResult syncedFlushResult = listener.result;
            assertNotNull(syncedFlushResult);
            assertEquals(0, syncedFlushResult.successfulShards());
            assertNotEquals(0, syncedFlushResult.totalShards());
            assertEquals("[1] ongoing operations on primary", syncedFlushResult.failureReason());
        }
    }

    public void testSyncFailsOnIndexClosedOrMissing() throws InterruptedException {
        createIndex("test");
        IndexService test = getInstanceFromNode(IndicesService.class).indexService(resolveIndex("test"));
        IndexShard shard = test.getShardOrNull(0);

        SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class);
        SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener();
        flushService.attemptSyncedFlush(new ShardId("test", "_na_", 1), listener);
        listener.latch.await();
        assertNotNull(listener.error);
        assertNull(listener.result);
        assertEquals(ShardNotFoundException.class, listener.error.getClass());
        assertEquals("no such shard", 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("index not found", "_na_", 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("{}", XContentType.JSON).get();
        IndexService test = getInstanceFromNode(IndicesService.class).indexService(resolveIndex("test"));
        IndexShard shard = test.getShardOrNull(0);

        SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class);
        final ShardId shardId = shard.shardId();
        final ClusterState state = getInstanceFromNode(ClusterService.class).state();
        final IndexShardRoutingTable shardRoutingTable = flushService.getShardRoutingTable(shardId, state);
        final List<ShardRouting> activeShards = shardRoutingTable.activeShards();
        assertEquals("exactly one active shard", 1, activeShards.size());
        Map<String, Engine.CommitId> commitIds = SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId);
        assertEquals("exactly one commit id", 1, commitIds.size());
        if (randomBoolean()) {
            client().prepareIndex("test", "test", "2").setSource("{}", XContentType.JSON).get();
        }
        client().admin().indices().prepareFlush("test").setForce(true).get();
        String syncId = UUIDs.base64UUID();
        final SyncedFlushUtil.LatchedListener<ShardsSyncedFlushResult> listener = new SyncedFlushUtil.LatchedListener();
        flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, shardRoutingTable.size(), listener);
        listener.latch.await();
        assertNull(listener.error);
        ShardsSyncedFlushResult 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());
    }

    public void testFailWhenCommitIsMissing() throws InterruptedException {
        createIndex("test");
        client().prepareIndex("test", "test", "1").setSource("{}", XContentType.JSON).get();
        IndexService test = getInstanceFromNode(IndicesService.class).indexService(resolveIndex("test"));
        IndexShard shard = test.getShardOrNull(0);

        SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class);
        final ShardId shardId = shard.shardId();
        final ClusterState state = getInstanceFromNode(ClusterService.class).state();
        final IndexShardRoutingTable shardRoutingTable = flushService.getShardRoutingTable(shardId, state);
        final List<ShardRouting> activeShards = shardRoutingTable.activeShards();
        assertEquals("exactly one active shard", 1, activeShards.size());
        Map<String, Engine.CommitId> commitIds =  SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId);
        assertEquals("exactly one commit id", 1, commitIds.size());
        commitIds.clear(); // wipe it...
        String syncId = UUIDs.base64UUID();
        SyncedFlushUtil.LatchedListener<ShardsSyncedFlushResult> listener = new SyncedFlushUtil.LatchedListener();
        flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, shardRoutingTable.size(), listener);
        listener.latch.await();
        assertNull(listener.error);
        ShardsSyncedFlushResult 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());
    }


}