Skip to content

Commit 462e2c1

Browse files
committed
Ensure pending merges are updated on segment flushes
Due to the default of `async_merge` to `true` we never run the merge policy on a segment flush which prevented the pending merges from being updated and that caused actual pending merges not to contribute to the merge decision. This commit also removes the `index.async.merge` setting is actually misleading since we take care of merges not being excecuted on the indexing threads on a different level (the merge scheduler) since 1.1. This commit also adds an additional check when to run a refresh since soely relying on the dirty flag might leave merges un-refreshed which can cause search slowdowns and higher memory consumption. Closes #5779
1 parent b93c003 commit 462e2c1

File tree

7 files changed

+129
-96
lines changed

7 files changed

+129
-96
lines changed

src/main/java/org/elasticsearch/index/engine/internal/InternalEngine.java

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -674,7 +674,15 @@ protected Searcher newSearcher(String source, IndexSearcher searcher, SearcherMa
674674

675675
@Override
676676
public boolean refreshNeeded() {
677-
return dirty;
677+
try {
678+
// we are either dirty due to a document added or due to a
679+
// finished merge - either way we should refresh
680+
return dirty || !searcherManager.isSearcherCurrent();
681+
} catch (IOException e) {
682+
logger.error("failed to access searcher manager", e);
683+
failEngine(e);
684+
throw new EngineException(shardId, "failed to access searcher manager",e);
685+
}
678686
}
679687

680688
@Override
@@ -706,7 +714,7 @@ public void refresh(Refresh refresh) throws EngineException {
706714
// maybeRefresh will only allow one refresh to execute, and the rest will "pass through",
707715
// but, we want to make sure not to loose ant refresh calls, if one is taking time
708716
synchronized (refreshMutex) {
709-
if (dirty || refresh.force()) {
717+
if (refreshNeeded() || refresh.force()) {
710718
// we set dirty to false, even though the refresh hasn't happened yet
711719
// as the refresh only holds for data indexed before it. Any data indexed during
712720
// the refresh will not be part of it and will set the dirty flag back to true
@@ -926,7 +934,7 @@ private void refreshVersioningTable(long time) {
926934

927935
@Override
928936
public void maybeMerge() throws EngineException {
929-
if (!possibleMergeNeeded) {
937+
if (!possibleMergeNeeded()) {
930938
return;
931939
}
932940
possibleMergeNeeded = false;

src/main/java/org/elasticsearch/index/merge/policy/LogByteSizeMergePolicyProvider.java

Lines changed: 6 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@
2121

2222
import org.apache.lucene.index.LogByteSizeMergePolicy;
2323
import org.apache.lucene.index.MergePolicy;
24-
import org.apache.lucene.index.SegmentInfos;
2524
import org.elasticsearch.ElasticsearchException;
2625
import org.elasticsearch.common.Preconditions;
2726
import org.elasticsearch.common.inject.Inject;
@@ -31,7 +30,6 @@
3130
import org.elasticsearch.index.settings.IndexSettingsService;
3231
import org.elasticsearch.index.store.Store;
3332

34-
import java.io.IOException;
3533
import java.util.Set;
3634
import java.util.concurrent.CopyOnWriteArraySet;
3735

@@ -41,13 +39,14 @@
4139
public class LogByteSizeMergePolicyProvider extends AbstractMergePolicyProvider<LogByteSizeMergePolicy> {
4240

4341
private final IndexSettingsService indexSettingsService;
44-
42+
public static final String MAX_MERGE_BYTE_SIZE_KEY = "index.merge.policy.max_merge_sizes";
43+
public static final String MIN_MERGE_BYTE_SIZE_KEY = "index.merge.policy.min_merge_size";
44+
public static final String MERGE_FACTORY_KEY = "index.merge.policy.merge_factor";
4545
private volatile ByteSizeValue minMergeSize;
4646
private volatile ByteSizeValue maxMergeSize;
4747
private volatile int mergeFactor;
4848
private volatile int maxMergeDocs;
4949
private final boolean calibrateSizeByDeletes;
50-
private boolean asyncMerge;
5150

5251
private final Set<CustomLogByteSizeMergePolicy> policies = new CopyOnWriteArraySet<>();
5352

@@ -63,21 +62,15 @@ public LogByteSizeMergePolicyProvider(Store store, IndexSettingsService indexSet
6362
this.mergeFactor = componentSettings.getAsInt("merge_factor", LogByteSizeMergePolicy.DEFAULT_MERGE_FACTOR);
6463
this.maxMergeDocs = componentSettings.getAsInt("max_merge_docs", LogByteSizeMergePolicy.DEFAULT_MAX_MERGE_DOCS);
6564
this.calibrateSizeByDeletes = componentSettings.getAsBoolean("calibrate_size_by_deletes", true);
66-
this.asyncMerge = indexSettings.getAsBoolean("index.merge.async", true);
67-
logger.debug("using [log_bytes_size] merge policy with merge_factor[{}], min_merge_size[{}], max_merge_size[{}], max_merge_docs[{}], calibrate_size_by_deletes[{}], async_merge[{}]",
68-
mergeFactor, minMergeSize, maxMergeSize, maxMergeDocs, calibrateSizeByDeletes, asyncMerge);
65+
logger.debug("using [log_bytes_size] merge policy with merge_factor[{}], min_merge_size[{}], max_merge_size[{}], max_merge_docs[{}], calibrate_size_by_deletes[{}]",
66+
mergeFactor, minMergeSize, maxMergeSize, maxMergeDocs, calibrateSizeByDeletes);
6967

7068
indexSettingsService.addListener(applySettings);
7169
}
7270

7371
@Override
7472
public LogByteSizeMergePolicy newMergePolicy() {
75-
CustomLogByteSizeMergePolicy mergePolicy;
76-
if (asyncMerge) {
77-
mergePolicy = new EnableMergeLogByteSizeMergePolicy(this);
78-
} else {
79-
mergePolicy = new CustomLogByteSizeMergePolicy(this);
80-
}
73+
final CustomLogByteSizeMergePolicy mergePolicy = new CustomLogByteSizeMergePolicy(this);
8174
mergePolicy.setMinMergeMB(minMergeSize.mbFrac());
8275
mergePolicy.setMaxMergeMB(maxMergeSize.mbFrac());
8376
mergePolicy.setMergeFactor(mergeFactor);
@@ -173,19 +166,4 @@ public MergePolicy clone() {
173166
}
174167
}
175168

176-
public static class EnableMergeLogByteSizeMergePolicy extends CustomLogByteSizeMergePolicy {
177-
178-
public EnableMergeLogByteSizeMergePolicy(LogByteSizeMergePolicyProvider provider) {
179-
super(provider);
180-
}
181-
182-
@Override
183-
public MergeSpecification findMerges(MergeTrigger trigger, SegmentInfos infos) throws IOException {
184-
// we don't enable merges while indexing documents, we do them in the background
185-
if (trigger == MergeTrigger.SEGMENT_FLUSH) {
186-
return null;
187-
}
188-
return super.findMerges(trigger, infos);
189-
}
190-
}
191169
}

src/main/java/org/elasticsearch/index/merge/policy/LogDocMergePolicyProvider.java

Lines changed: 6 additions & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -20,16 +20,13 @@
2020
package org.elasticsearch.index.merge.policy;
2121

2222
import org.apache.lucene.index.LogDocMergePolicy;
23-
import org.apache.lucene.index.MergePolicy;
24-
import org.apache.lucene.index.SegmentInfos;
2523
import org.elasticsearch.ElasticsearchException;
2624
import org.elasticsearch.common.Preconditions;
2725
import org.elasticsearch.common.inject.Inject;
2826
import org.elasticsearch.common.settings.Settings;
2927
import org.elasticsearch.index.settings.IndexSettingsService;
3028
import org.elasticsearch.index.store.Store;
3129

32-
import java.io.IOException;
3330
import java.util.Set;
3431
import java.util.concurrent.CopyOnWriteArraySet;
3532

@@ -39,12 +36,13 @@
3936
public class LogDocMergePolicyProvider extends AbstractMergePolicyProvider<LogDocMergePolicy> {
4037

4138
private final IndexSettingsService indexSettingsService;
42-
39+
public static final String MAX_MERGE_DOCS_KEY = "index.merge.policy.max_merge_docs";
40+
public static final String MIN_MERGE_DOCS_KEY = "index.merge.policy.min_merge_docs";
41+
public static final String MERGE_FACTORY_KEY = "index.merge.policy.merge_factor";
4342
private volatile int minMergeDocs;
4443
private volatile int maxMergeDocs;
4544
private volatile int mergeFactor;
4645
private final boolean calibrateSizeByDeletes;
47-
private boolean asyncMerge;
4846

4947
private final Set<CustomLogDocMergePolicy> policies = new CopyOnWriteArraySet<>();
5048

@@ -60,9 +58,8 @@ public LogDocMergePolicyProvider(Store store, IndexSettingsService indexSettings
6058
this.maxMergeDocs = componentSettings.getAsInt("max_merge_docs", LogDocMergePolicy.DEFAULT_MAX_MERGE_DOCS);
6159
this.mergeFactor = componentSettings.getAsInt("merge_factor", LogDocMergePolicy.DEFAULT_MERGE_FACTOR);
6260
this.calibrateSizeByDeletes = componentSettings.getAsBoolean("calibrate_size_by_deletes", true);
63-
this.asyncMerge = indexSettings.getAsBoolean("index.merge.async", true);
64-
logger.debug("using [log_doc] merge policy with merge_factor[{}], min_merge_docs[{}], max_merge_docs[{}], calibrate_size_by_deletes[{}], async_merge[{}]",
65-
mergeFactor, minMergeDocs, maxMergeDocs, calibrateSizeByDeletes, asyncMerge);
61+
logger.debug("using [log_doc] merge policy with merge_factor[{}], min_merge_docs[{}], max_merge_docs[{}], calibrate_size_by_deletes[{}]",
62+
mergeFactor, minMergeDocs, maxMergeDocs, calibrateSizeByDeletes);
6663

6764
indexSettingsService.addListener(applySettings);
6865
}
@@ -74,12 +71,7 @@ public void close() throws ElasticsearchException {
7471

7572
@Override
7673
public LogDocMergePolicy newMergePolicy() {
77-
CustomLogDocMergePolicy mergePolicy;
78-
if (asyncMerge) {
79-
mergePolicy = new EnableMergeLogDocMergePolicy(this);
80-
} else {
81-
mergePolicy = new CustomLogDocMergePolicy(this);
82-
}
74+
final CustomLogDocMergePolicy mergePolicy = new CustomLogDocMergePolicy(this);
8375
mergePolicy.setMinMergeDocs(minMergeDocs);
8476
mergePolicy.setMaxMergeDocs(maxMergeDocs);
8577
mergePolicy.setMergeFactor(mergeFactor);
@@ -150,27 +142,4 @@ public void close() {
150142
provider.policies.remove(this);
151143
}
152144
}
153-
154-
public static class EnableMergeLogDocMergePolicy extends CustomLogDocMergePolicy {
155-
156-
public EnableMergeLogDocMergePolicy(LogDocMergePolicyProvider provider) {
157-
super(provider);
158-
}
159-
160-
@Override
161-
public MergeSpecification findMerges(MergeTrigger trigger, SegmentInfos infos) throws IOException {
162-
// we don't enable merges while indexing documents, we do them in the background
163-
if (trigger == MergeTrigger.SEGMENT_FLUSH) {
164-
return null;
165-
}
166-
return super.findMerges(trigger, infos);
167-
}
168-
169-
@Override
170-
public MergePolicy clone() {
171-
// Lucene IW makes a clone internally but since we hold on to this instance
172-
// the clone will just be the identity.
173-
return this;
174-
}
175-
}
176145
}

src/main/java/org/elasticsearch/index/merge/policy/TieredMergePolicyProvider.java

Lines changed: 3 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@
2020
package org.elasticsearch.index.merge.policy;
2121

2222
import org.apache.lucene.index.MergePolicy;
23-
import org.apache.lucene.index.SegmentInfos;
2423
import org.apache.lucene.index.TieredMergePolicy;
2524
import org.elasticsearch.ElasticsearchException;
2625
import org.elasticsearch.common.inject.Inject;
@@ -30,7 +29,6 @@
3029
import org.elasticsearch.index.settings.IndexSettingsService;
3130
import org.elasticsearch.index.store.Store;
3231

33-
import java.io.IOException;
3432
import java.util.Set;
3533
import java.util.concurrent.CopyOnWriteArraySet;
3634

@@ -47,7 +45,6 @@ public class TieredMergePolicyProvider extends AbstractMergePolicyProvider<Tiere
4745
private volatile ByteSizeValue maxMergedSegment;
4846
private volatile double segmentsPerTier;
4947
private volatile double reclaimDeletesWeight;
50-
private boolean asyncMerge;
5148

5249
private final ApplySettings applySettings = new ApplySettings();
5350

@@ -57,7 +54,6 @@ public class TieredMergePolicyProvider extends AbstractMergePolicyProvider<Tiere
5754
public TieredMergePolicyProvider(Store store, IndexSettingsService indexSettingsService) {
5855
super(store);
5956
this.indexSettingsService = indexSettingsService;
60-
this.asyncMerge = indexSettings.getAsBoolean("index.merge.async", true);
6157
this.forceMergeDeletesPctAllowed = componentSettings.getAsDouble("expunge_deletes_allowed", 10d); // percentage
6258
this.floorSegment = componentSettings.getAsBytesSize("floor_segment", new ByteSizeValue(2, ByteSizeUnit.MB));
6359
this.maxMergeAtOnce = componentSettings.getAsInt("max_merge_at_once", 10);
@@ -69,8 +65,8 @@ public TieredMergePolicyProvider(Store store, IndexSettingsService indexSettings
6965

7066
fixSettingsIfNeeded();
7167

72-
logger.debug("using [tiered] merge policy with expunge_deletes_allowed[{}], floor_segment[{}], max_merge_at_once[{}], max_merge_at_once_explicit[{}], max_merged_segment[{}], segments_per_tier[{}], reclaim_deletes_weight[{}], async_merge[{}]",
73-
forceMergeDeletesPctAllowed, floorSegment, maxMergeAtOnce, maxMergeAtOnceExplicit, maxMergedSegment, segmentsPerTier, reclaimDeletesWeight, asyncMerge);
68+
logger.debug("using [tiered] merge policy with expunge_deletes_allowed[{}], floor_segment[{}], max_merge_at_once[{}], max_merge_at_once_explicit[{}], max_merged_segment[{}], segments_per_tier[{}], reclaim_deletes_weight[{}]",
69+
forceMergeDeletesPctAllowed, floorSegment, maxMergeAtOnce, maxMergeAtOnceExplicit, maxMergedSegment, segmentsPerTier, reclaimDeletesWeight);
7470

7571
indexSettingsService.addListener(applySettings);
7672
}
@@ -91,12 +87,7 @@ private void fixSettingsIfNeeded() {
9187

9288
@Override
9389
public TieredMergePolicy newMergePolicy() {
94-
CustomTieredMergePolicyProvider mergePolicy;
95-
if (asyncMerge) {
96-
mergePolicy = new EnableMergeTieredMergePolicyProvider(this);
97-
} else {
98-
mergePolicy = new CustomTieredMergePolicyProvider(this);
99-
}
90+
final CustomTieredMergePolicyProvider mergePolicy = new CustomTieredMergePolicyProvider(this);
10091
mergePolicy.setNoCFSRatio(noCFSRatio);
10192
mergePolicy.setForceMergeDeletesPctAllowed(forceMergeDeletesPctAllowed);
10293
mergePolicy.setFloorSegmentMB(floorSegment.mbFrac());
@@ -222,20 +213,4 @@ public MergePolicy clone() {
222213
return this;
223214
}
224215
}
225-
226-
public static class EnableMergeTieredMergePolicyProvider extends CustomTieredMergePolicyProvider {
227-
228-
public EnableMergeTieredMergePolicyProvider(TieredMergePolicyProvider provider) {
229-
super(provider);
230-
}
231-
232-
@Override
233-
public MergePolicy.MergeSpecification findMerges(MergeTrigger trigger, SegmentInfos infos) throws IOException {
234-
// we don't enable merges while indexing documents, we do them in the background
235-
if (trigger == MergeTrigger.SEGMENT_FLUSH) {
236-
return null;
237-
}
238-
return super.findMerges(trigger, infos);
239-
}
240-
}
241216
}

src/test/java/org/elasticsearch/index/engine/internal/InternalEngineIntegrationTest.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -152,4 +152,5 @@ private void assertTotalCompoundSegments(int i, int t, String index) {
152152
assertThat(total, Matchers.equalTo(t));
153153

154154
}
155+
155156
}
Lines changed: 94 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,94 @@
1+
/*
2+
* Licensed to Elasticsearch under one or more contributor
3+
* license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright
5+
* ownership. Elasticsearch licenses this file to you under
6+
* the Apache License, Version 2.0 (the "License"); you may
7+
* not use this file except in compliance with the License.
8+
* You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
package org.elasticsearch.index.engine.internal;
20+
21+
import com.carrotsearch.randomizedtesting.annotations.Nightly;
22+
import com.carrotsearch.randomizedtesting.annotations.Seed;
23+
import com.google.common.base.Predicate;
24+
import org.apache.lucene.index.LogByteSizeMergePolicy;
25+
import org.apache.lucene.util.LuceneTestCase;
26+
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
27+
import org.elasticsearch.action.bulk.BulkRequestBuilder;
28+
import org.elasticsearch.action.bulk.BulkResponse;
29+
import org.elasticsearch.action.index.IndexRequestBuilder;
30+
import org.elasticsearch.client.Requests;
31+
import org.elasticsearch.cluster.metadata.IndexMetaData;
32+
import org.elasticsearch.common.settings.ImmutableSettings;
33+
import org.elasticsearch.index.merge.policy.LogDocMergePolicyProvider;
34+
import org.elasticsearch.test.ElasticsearchIntegrationTest;
35+
import org.hamcrest.Matchers;
36+
import org.junit.Test;
37+
38+
import java.io.IOException;
39+
import java.util.ArrayList;
40+
import java.util.List;
41+
import java.util.concurrent.ExecutionException;
42+
43+
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
44+
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
45+
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
46+
47+
/**
48+
*/
49+
@ElasticsearchIntegrationTest.ClusterScope(numNodes = 1, scope = ElasticsearchIntegrationTest.Scope.SUITE)
50+
public class InternalEngineMergeTests extends ElasticsearchIntegrationTest {
51+
52+
@Test
53+
@LuceneTestCase.Slow
54+
public void testMergesHappening() throws InterruptedException, IOException, ExecutionException {
55+
final int numOfShards = 5;
56+
// some settings to keep num segments low
57+
assertAcked(prepareCreate("test").setSettings(ImmutableSettings.builder()
58+
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, numOfShards)
59+
.put(LogDocMergePolicyProvider.MIN_MERGE_DOCS_KEY, 10)
60+
.put(LogDocMergePolicyProvider.MERGE_FACTORY_KEY, 5)
61+
.put(LogByteSizeMergePolicy.DEFAULT_MIN_MERGE_MB, 0.5)
62+
.build()));
63+
long id = 0;
64+
final int rounds = scaledRandomIntBetween(50, 300);
65+
logger.info("Starting rounds [{}] ", rounds);
66+
for (int i = 0; i < rounds; ++i) {
67+
final int numDocs = scaledRandomIntBetween(100, 1000);
68+
BulkRequestBuilder request = client().prepareBulk();
69+
for (int j = 0; j < numDocs; ++j) {
70+
request.add(Requests.indexRequest("test").type("type1").id(Long.toString(id++)).source(jsonBuilder().startObject().field("l", randomLong()).endObject()));
71+
}
72+
BulkResponse response = request.execute().actionGet();
73+
refresh();
74+
assertNoFailures(response);
75+
IndicesStatsResponse stats = client().admin().indices().prepareStats("test").setSegments(true).setMerge(true).get();
76+
logger.info("index round [{}] - segments {}, total merges {}, current merge {}", i, stats.getPrimaries().getSegments().getCount(), stats.getPrimaries().getMerge().getTotal(), stats.getPrimaries().getMerge().getCurrent());
77+
}
78+
awaitBusy(new Predicate<Object>() {
79+
@Override
80+
public boolean apply(Object input) {
81+
IndicesStatsResponse stats = client().admin().indices().prepareStats().setSegments(true).setMerge(true).get();
82+
logger.info("numshards {}, segments {}, total merges {}, current merge {}", numOfShards, stats.getPrimaries().getSegments().getCount(), stats.getPrimaries().getMerge().getTotal(), stats.getPrimaries().getMerge().getCurrent());
83+
long current = stats.getPrimaries().getMerge().getCurrent();
84+
long count = stats.getPrimaries().getSegments().getCount();
85+
return count < 50 && current == 0;
86+
}
87+
});
88+
IndicesStatsResponse stats = client().admin().indices().prepareStats().setSegments(true).setMerge(true).get();
89+
logger.info("numshards {}, segments {}, total merges {}, current merge {}", numOfShards, stats.getPrimaries().getSegments().getCount(), stats.getPrimaries().getMerge().getTotal(), stats.getPrimaries().getMerge().getCurrent());
90+
long count = stats.getPrimaries().getSegments().getCount();
91+
assertThat(count, Matchers.lessThanOrEqualTo(50l));
92+
}
93+
94+
}

0 commit comments

Comments
 (0)