summaryrefslogtreecommitdiff
path: root/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java
blob: d5c10dddc3653dde844bff981705aae38a088f91 (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
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
/*
 * Licensed to Elasticsearch under one or more contributor
 * license agreements. See the NOTICE file distributed with
 * this work for additional information regarding copyright
 * ownership. Elasticsearch licenses this file to you under
 * the Apache License, Version 2.0 (the "License"); you may
 * not use this file except in compliance with the License.
 * You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing,
 * software distributed under the License is distributed on an
 * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 * KIND, either express or implied.  See the License for the
 * specific language governing permissions and limitations
 * under the License.
 */

package org.elasticsearch.index.shard;

import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
import org.apache.lucene.index.SnapshotDeletionPolicy;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.InternalEngine;
import org.elasticsearch.index.engine.InternalEngineTests.TranslogHandler;
import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.test.DummyShardLock;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.threadpool.ThreadPool.Cancellable;
import org.elasticsearch.threadpool.ThreadPool.Names;
import org.junit.After;
import org.junit.Before;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;

import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;

/**
 * Tests how {@linkplain RefreshListeners} interacts with {@linkplain InternalEngine}.
 */
public class RefreshListenersTests extends ESTestCase {
    private RefreshListeners listeners;
    private Engine engine;
    private volatile int maxListeners;
    private ThreadPool threadPool;
    private Store store;

    @Before
    public void setupListeners() throws Exception {
        // Setup dependencies of the listeners
        maxListeners = randomIntBetween(1, 1000);
        listeners = new RefreshListeners(
                () -> maxListeners,
                () -> engine.refresh("too-many-listeners"),
                // Immediately run listeners rather than adding them to the listener thread pool like IndexShard does to simplify the test.
                Runnable::run,
                logger
                );

        // Now setup the InternalEngine which is much more complicated because we aren't mocking anything
        threadPool = new TestThreadPool(getTestName());
        IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("index", Settings.EMPTY);
        ShardId shardId = new ShardId(new Index("index", "_na_"), 1);
        Directory directory = newDirectory();
        DirectoryService directoryService = new DirectoryService(shardId, indexSettings) {
            @Override
            public Directory newDirectory() throws IOException {
                return directory;
            }
        };
        store = new Store(shardId, indexSettings, directoryService, new DummyShardLock(shardId));
        IndexWriterConfig iwc = newIndexWriterConfig();
        TranslogConfig translogConfig = new TranslogConfig(shardId, createTempDir("translog"), indexSettings,
                BigArrays.NON_RECYCLING_INSTANCE);
        Engine.EventListener eventListener = new Engine.EventListener() {
            @Override
            public void onFailedEngine(String reason, @Nullable Exception e) {
                // we don't need to notify anybody in this test
            }
        };
        TranslogHandler translogHandler = new TranslogHandler(xContentRegistry(), shardId.getIndexName(), logger);
        EngineConfig config = new EngineConfig(EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG, shardId, threadPool, indexSettings, null,
                store, new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()), newMergePolicy(), iwc.getAnalyzer(),
                iwc.getSimilarity(), new CodecService(null, logger), eventListener, translogHandler,
                IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig,
                TimeValue.timeValueMinutes(5), listeners, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP);
        engine = new InternalEngine(config);
        listeners.setTranslog(engine.getTranslog());
    }

    @After
    public void tearDownListeners() throws Exception {
        IOUtils.close(engine, store);
        terminate(threadPool);
    }

    public void testBeforeRefresh() throws Exception {
        assertEquals(0, listeners.pendingCount());
        Engine.IndexResult index = index("1");
        DummyRefreshListener listener = new DummyRefreshListener();
        assertFalse(listeners.addOrNotify(index.getTranslogLocation(), listener));
        assertNull(listener.forcedRefresh.get());
        assertEquals(1, listeners.pendingCount());
        engine.refresh("I said so");
        assertFalse(listener.forcedRefresh.get());
        listener.assertNoError();
        assertEquals(0, listeners.pendingCount());
    }

    public void testAfterRefresh() throws Exception {
        assertEquals(0, listeners.pendingCount());
        Engine.IndexResult index = index("1");
        engine.refresh("I said so");
        if (randomBoolean()) {
            index(randomFrom("1" /* same document */, "2" /* different document */));
            if (randomBoolean()) {
                engine.refresh("I said so");
            }
        }
        DummyRefreshListener listener = new DummyRefreshListener();
        assertTrue(listeners.addOrNotify(index.getTranslogLocation(), listener));
        assertFalse(listener.forcedRefresh.get());
        listener.assertNoError();
        assertEquals(0, listeners.pendingCount());
    }

    public void testTooMany() throws Exception {
        assertEquals(0, listeners.pendingCount());
        assertFalse(listeners.refreshNeeded());
        Engine.IndexResult index = index("1");

        // Fill the listener slots
        List<DummyRefreshListener> nonForcedListeners = new ArrayList<>(maxListeners);
        for (int i = 0; i < maxListeners; i++) {
            DummyRefreshListener listener = new DummyRefreshListener();
            nonForcedListeners.add(listener);
            listeners.addOrNotify(index.getTranslogLocation(), listener);
            assertTrue(listeners.refreshNeeded());
        }

        // We shouldn't have called any of them
        for (DummyRefreshListener listener : nonForcedListeners) {
            assertNull("Called listener too early!", listener.forcedRefresh.get());
        }
        assertEquals(maxListeners, listeners.pendingCount());

        // Add one more listener which should cause a refresh.
        DummyRefreshListener forcingListener = new DummyRefreshListener();
        listeners.addOrNotify(index.getTranslogLocation(), forcingListener);
        assertTrue("Forced listener wasn't forced?", forcingListener.forcedRefresh.get());
        forcingListener.assertNoError();

        // That forces all the listeners through. It would be on the listener ThreadPool but we've made all of those execute immediately.
        for (DummyRefreshListener listener : nonForcedListeners) {
            assertEquals("Expected listener called with unforced refresh!", Boolean.FALSE, listener.forcedRefresh.get());
            listener.assertNoError();
        }
        assertFalse(listeners.refreshNeeded());
        assertEquals(0, listeners.pendingCount());
    }

    public void testClose() throws Exception {
        assertEquals(0, listeners.pendingCount());
        Engine.IndexResult refreshedOperation = index("1");
        engine.refresh("I said so");
        Engine.IndexResult unrefreshedOperation = index("1");
        {
            /* Closing flushed pending listeners as though they were refreshed. Since this can only happen when the index is closed and no
             * longer useful there doesn't seem much point in sending the listener some kind of "I'm closed now, go away" enum value. */
            DummyRefreshListener listener = new DummyRefreshListener();
            assertFalse(listeners.addOrNotify(unrefreshedOperation.getTranslogLocation(), listener));
            assertNull(listener.forcedRefresh.get());
            listeners.close();
            assertFalse(listener.forcedRefresh.get());
            listener.assertNoError();
            assertFalse(listeners.refreshNeeded());
            assertEquals(0, listeners.pendingCount());
        }
        {
            // If you add a listener for an already refreshed location then it'll just fire even if closed
            DummyRefreshListener listener = new DummyRefreshListener();
            assertTrue(listeners.addOrNotify(refreshedOperation.getTranslogLocation(), listener));
            assertFalse(listener.forcedRefresh.get());
            listener.assertNoError();
            assertFalse(listeners.refreshNeeded());
            assertEquals(0, listeners.pendingCount());
        }
        {
            // But adding a listener to a non-refreshed location will fail
            DummyRefreshListener listener = new DummyRefreshListener();
            Exception e = expectThrows(IllegalStateException.class, () ->
                listeners.addOrNotify(unrefreshedOperation.getTranslogLocation(), listener));
            assertEquals("can't wait for refresh on a closed index", e.getMessage());
            assertNull(listener.forcedRefresh.get());
            assertFalse(listeners.refreshNeeded());
            assertEquals(0, listeners.pendingCount());
        }
    }

    /**
     * Attempts to add a listener at the same time as a refresh occurs by having a background thread force a refresh as fast as it can while
     * adding listeners. This can catch the situation where a refresh happens right as the listener is being added such that the listener
     * misses the refresh and has to catch the next one. If the listener wasn't able to properly catch the next one then this would fail.
     */
    public void testConcurrentRefresh() throws Exception {
        AtomicBoolean run = new AtomicBoolean(true);
        Thread refresher = new Thread(() -> {
            while (run.get()) {
                engine.refresh("test");
            }
        });
        refresher.start();
        try {
            for (int i = 0; i < 1000; i++) {
                Engine.IndexResult index = index("1");
                DummyRefreshListener listener = new DummyRefreshListener();
                boolean immediate = listeners.addOrNotify(index.getTranslogLocation(), listener);
                if (immediate) {
                    assertNotNull(listener.forcedRefresh.get());
                } else {
                    assertBusy(() -> assertNotNull(listener.forcedRefresh.get()));
                }
                assertFalse(listener.forcedRefresh.get());
                listener.assertNoError();
            }
        } finally {
            run.set(false);
            refresher.join();
        }
    }

    /**
     * Uses a bunch of threads to index, wait for refresh, and non-realtime get documents to validate that they are visible after waiting
     * regardless of what crazy sequence of events causes the refresh listener to fire.
     */
    public void testLotsOfThreads() throws Exception {
        int threadCount = between(3, 10);
        maxListeners = between(1, threadCount * 2);

        // This thread just refreshes every once in a while to cause trouble.
        Cancellable refresher = threadPool.scheduleWithFixedDelay(() -> engine.refresh("because test"), timeValueMillis(100), Names.SAME);

        // These threads add and block until the refresh makes the change visible and then do a non-realtime get.
        Thread[] indexers = new Thread[threadCount];
        for (int thread = 0; thread < threadCount; thread++) {
            final String threadId = String.format(Locale.ROOT, "%04d", thread);
            indexers[thread] = new Thread(() -> {
                for (int iteration = 1; iteration <= 50; iteration++) {
                    try {
                        String testFieldValue = String.format(Locale.ROOT, "%s%04d", threadId, iteration);
                        Engine.IndexResult index = index(threadId, testFieldValue);
                        assertEquals(iteration, index.getVersion());

                        DummyRefreshListener listener = new DummyRefreshListener();
                        listeners.addOrNotify(index.getTranslogLocation(), listener);
                        assertBusy(() -> assertNotNull("listener never called", listener.forcedRefresh.get()));
                        if (threadCount < maxListeners) {
                            assertFalse(listener.forcedRefresh.get());
                        }
                        listener.assertNoError();

                        Engine.Get get = new Engine.Get(false, new Term("_uid",  Uid.createUid("test", threadId)));
                        try (Engine.GetResult getResult = engine.get(get)) {
                            assertTrue("document not found", getResult.exists());
                            assertEquals(iteration, getResult.version());
                            SingleFieldsVisitor visitor = new SingleFieldsVisitor("test");
                            getResult.docIdAndVersion().context.reader().document(getResult.docIdAndVersion().docId, visitor);
                            assertEquals(Arrays.asList(testFieldValue), visitor.fields().get("test"));
                        }
                    } catch (Exception t) {
                        throw new RuntimeException("failure on the [" + iteration + "] iteration of thread [" + threadId + "]", t);
                    }
                }
            });
            indexers[thread].start();
        }

        for (Thread indexer: indexers) {
            indexer.join();
        }
        refresher.cancel();
    }

    private Engine.IndexResult index(String id) throws IOException {
        return index(id, "test");
    }

    private Engine.IndexResult index(String id, String testFieldValue) throws IOException {
        String type = "test";
        String uid = type + ":" + id;
        Document document = new Document();
        document.add(new TextField("test", testFieldValue, Field.Store.YES));
        Field uidField = new Field("_uid", Uid.createUid(type, id), UidFieldMapper.Defaults.FIELD_TYPE);
        Field versionField = new NumericDocValuesField("_version", Versions.MATCH_ANY);
        SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID();
        document.add(uidField);
        document.add(versionField);
        document.add(seqID.seqNo);
        document.add(seqID.seqNoDocValue);
        document.add(seqID.primaryTerm);
        BytesReference source = new BytesArray(new byte[] { 1 });
        ParsedDocument doc = new ParsedDocument(versionField, seqID, id, type, null, Arrays.asList(document), source, null);
        Engine.Index index = new Engine.Index(new Term("_uid", doc.uid()), doc);
        return engine.index(index);
    }

    private static class DummyRefreshListener implements Consumer<Boolean> {
        /**
         * When the listener is called this captures it's only argument.
         */
        final AtomicReference<Boolean> forcedRefresh = new AtomicReference<>();
        private volatile Exception error;

        @Override
        public void accept(Boolean forcedRefresh) {
            try {
                Boolean oldValue = this.forcedRefresh.getAndSet(forcedRefresh);
                assertNull("Listener called twice", oldValue);
            } catch (Exception e) {
                error = e;
            }
        }

        public void assertNoError() {
            if (error != null) {
                throw new RuntimeException(error);
            }
        }
    }
}