summaryrefslogtreecommitdiff
path: root/core/src/main/java/org/elasticsearch/indices/ttl/IndicesTTLService.java
blob: e4537b876fa2b37ccdbc06ba92b53ab1c7681c58 (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
/*
 * 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.ttl;

import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.bulk.TransportBulkAction;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.fieldvisitor.FieldsVisitor;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.internal.TTLFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.indices.IndicesService;

import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;


/**
 * A node level service that delete expired docs on node primary shards.
 */
public class IndicesTTLService extends AbstractLifecycleComponent<IndicesTTLService> {

    public static final Setting<TimeValue> INDICES_TTL_INTERVAL_SETTING = Setting.positiveTimeSetting("indices.ttl.interval", TimeValue.timeValueSeconds(60), true, Setting.Scope.CLUSTER);

    private final ClusterService clusterService;
    private final IndicesService indicesService;
    private final TransportBulkAction bulkAction;

    private final int bulkSize;
    private PurgerThread purgerThread;

    @Inject
    public IndicesTTLService(Settings settings, ClusterService clusterService, IndicesService indicesService, ClusterSettings clusterSettings, TransportBulkAction bulkAction) {
        super(settings);
        this.clusterService = clusterService;
        this.indicesService = indicesService;
        TimeValue interval = INDICES_TTL_INTERVAL_SETTING.get(settings);
        this.bulkAction = bulkAction;
        this.bulkSize = this.settings.getAsInt("indices.ttl.bulk_size", 10000);
        this.purgerThread = new PurgerThread(EsExecutors.threadName(settings, "[ttl_expire]"), interval);
        clusterSettings.addSettingsUpdateConsumer(INDICES_TTL_INTERVAL_SETTING, this.purgerThread::resetInterval);
    }

    @Override
    protected void doStart() {
        this.purgerThread.start();
    }

    @Override
    protected void doStop() {
        try {
            this.purgerThread.shutdown();
        } catch (InterruptedException e) {
            // we intentionally do not want to restore the interruption flag, we're about to shutdown anyway
        }
    }

    @Override
    protected void doClose() {
    }

    private class PurgerThread extends Thread {
        private final AtomicBoolean running = new AtomicBoolean(true);
        private final Notifier notifier;
        private final CountDownLatch shutdownLatch = new CountDownLatch(1);


        public PurgerThread(String name, TimeValue interval) {
            super(name);
            setDaemon(true);
            this.notifier = new Notifier(interval);
        }

        public void shutdown() throws InterruptedException {
            if (running.compareAndSet(true, false)) {
                notifier.doNotify();
                shutdownLatch.await();
            }

        }

        public void resetInterval(TimeValue interval) {
            notifier.setTimeout(interval);
        }

        @Override
        public void run() {
            try {
                while (running.get()) {
                    try {
                        List<IndexShard> shardsToPurge = getShardsToPurge();
                        purgeShards(shardsToPurge);
                    } catch (Throwable e) {
                        if (running.get()) {
                            logger.warn("failed to execute ttl purge", e);
                        }
                    }
                    if (running.get()) {
                        notifier.await();
                    }
                }
            } finally {
                shutdownLatch.countDown();
            }
        }

        /**
         * Returns the shards to purge, i.e. the local started primary shards that have ttl enabled and disable_purge to false
         */
        private List<IndexShard> getShardsToPurge() {
            List<IndexShard> shardsToPurge = new ArrayList<>();
            MetaData metaData = clusterService.state().metaData();
            for (IndexService indexService : indicesService) {
                // check the value of disable_purge for this index
                IndexMetaData indexMetaData = metaData.index(indexService.index().getName());
                if (indexMetaData == null) {
                    continue;
                }
                if (indexService.getIndexSettings().isTTLPurgeDisabled()) {
                    continue;
                }

                // check if ttl is enabled for at least one type of this index
                boolean hasTTLEnabled = false;
                for (String type : indexService.mapperService().types()) {
                    DocumentMapper documentType = indexService.mapperService().documentMapper(type);
                    if (documentType.TTLFieldMapper().enabled()) {
                        hasTTLEnabled = true;
                        break;
                    }
                }
                if (hasTTLEnabled) {
                    for (IndexShard indexShard : indexService) {
                        if (indexShard.state() == IndexShardState.STARTED && indexShard.routingEntry().primary() && indexShard.routingEntry().started()) {
                            shardsToPurge.add(indexShard);
                        }
                    }
                }
            }
            return shardsToPurge;
        }

        public TimeValue getInterval() {
            return notifier.getTimeout();
        }
    }

    private void purgeShards(List<IndexShard> shardsToPurge) {
        for (IndexShard shardToPurge : shardsToPurge) {
            Query query = shardToPurge.mapperService().fullName(TTLFieldMapper.NAME).rangeQuery(null, System.currentTimeMillis(), false, true);
            Engine.Searcher searcher = shardToPurge.acquireSearcher("indices_ttl");
            try {
                logger.debug("[{}][{}] purging shard", shardToPurge.routingEntry().index(), shardToPurge.routingEntry().id());
                ExpiredDocsCollector expiredDocsCollector = new ExpiredDocsCollector();
                searcher.searcher().search(query, expiredDocsCollector);
                List<DocToPurge> docsToPurge = expiredDocsCollector.getDocsToPurge();

                BulkRequest bulkRequest = new BulkRequest();
                for (DocToPurge docToPurge : docsToPurge) {

                    bulkRequest.add(new DeleteRequest().index(shardToPurge.routingEntry().getIndexName()).type(docToPurge.type).id(docToPurge.id).version(docToPurge.version).routing(docToPurge.routing));
                    bulkRequest = processBulkIfNeeded(bulkRequest, false);
                }
                processBulkIfNeeded(bulkRequest, true);
            } catch (Exception e) {
                logger.warn("failed to purge", e);
            } finally {
                searcher.close();
            }
        }
    }

    private static class DocToPurge {
        public final String type;
        public final String id;
        public final long version;
        public final String routing;

        public DocToPurge(String type, String id, long version, String routing) {
            this.type = type;
            this.id = id;
            this.version = version;
            this.routing = routing;
        }
    }

    private class ExpiredDocsCollector extends SimpleCollector {
        private LeafReaderContext context;
        private List<DocToPurge> docsToPurge = new ArrayList<>();

        public ExpiredDocsCollector() {
        }

        @Override
        public void setScorer(Scorer scorer) {
        }

        @Override
        public boolean needsScores() {
            return false;
        }

        @Override
        public void collect(int doc) {
            try {
                FieldsVisitor fieldsVisitor = new FieldsVisitor(false);
                context.reader().document(doc, fieldsVisitor);
                Uid uid = fieldsVisitor.uid();
                final long version = Versions.loadVersion(context.reader(), new Term(UidFieldMapper.NAME, uid.toBytesRef()));
                docsToPurge.add(new DocToPurge(uid.type(), uid.id(), version, fieldsVisitor.routing()));
            } catch (Exception e) {
                logger.trace("failed to collect doc", e);
            }
        }

        @Override
        public void doSetNextReader(LeafReaderContext context) throws IOException {
            this.context = context;
        }

        public List<DocToPurge> getDocsToPurge() {
            return this.docsToPurge;
        }
    }

    private BulkRequest processBulkIfNeeded(BulkRequest bulkRequest, boolean force) {
        if ((force && bulkRequest.numberOfActions() > 0) || bulkRequest.numberOfActions() >= bulkSize) {
            try {
                bulkAction.executeBulk(bulkRequest, new ActionListener<BulkResponse>() {
                    @Override
                    public void onResponse(BulkResponse bulkResponse) {
                        if (bulkResponse.hasFailures()) {
                            int failedItems = 0;
                            for (BulkItemResponse response : bulkResponse) {
                                if (response.isFailed()) failedItems++;
                            }
                            if (logger.isTraceEnabled()) {
                                logger.trace("bulk deletion failures for [{}]/[{}] items, failure message: [{}]", failedItems, bulkResponse.getItems().length, bulkResponse.buildFailureMessage());
                            } else {
                                logger.error("bulk deletion failures for [{}]/[{}] items", failedItems, bulkResponse.getItems().length);
                            }
                        } else {
                            logger.trace("bulk deletion took " + bulkResponse.getTookInMillis() + "ms");
                        }
                    }

                    @Override
                    public void onFailure(Throwable e) {
                        if (logger.isTraceEnabled()) {
                            logger.trace("failed to execute bulk", e);
                        } else {
                            logger.warn("failed to execute bulk: [{}]", e.getMessage());
                        }
                    }
                });
            } catch (Exception e) {
                logger.warn("failed to process bulk", e);
            }
            bulkRequest = new BulkRequest();
        }
        return bulkRequest;
    }

    private static final class Notifier {

        private final ReentrantLock lock = new ReentrantLock();
        private final Condition condition = lock.newCondition();
        private volatile TimeValue timeout;

        public Notifier(TimeValue timeout) {
            assert timeout != null;
            this.timeout = timeout;
        }

        public void await() {
            lock.lock();
            try {
                condition.await(timeout.millis(), TimeUnit.MILLISECONDS);
            } catch (InterruptedException e) {
                // we intentionally do not want to restore the interruption flag, we're about to shutdown anyway
            } finally {
                lock.unlock();
            }

        }

        public void setTimeout(TimeValue timeout) {
            assert timeout != null;
            this.timeout = timeout;
            doNotify();
        }

        public TimeValue getTimeout() {
            return timeout;
        }

        public void doNotify() {
            lock.lock();
            try {
                condition.signalAll();
            } finally {
                lock.unlock();
            }
        }
    }
}