diff --git a/core/src/main/java/org/elasticsearch/common/compress/CompressedXContent.java b/core/src/main/java/org/elasticsearch/common/compress/CompressedXContent.java index 09ced0e29b2d3..462b91aeef051 100644 --- a/core/src/main/java/org/elasticsearch/common/compress/CompressedXContent.java +++ b/core/src/main/java/org/elasticsearch/common/compress/CompressedXContent.java @@ -25,9 +25,16 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; 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.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentType; import java.io.IOException; +import java.io.OutputStream; import java.util.Arrays; +import java.util.zip.CRC32; +import java.util.zip.CheckedOutputStream; /** * Similar class to the {@link String} class except that it internally stores @@ -37,23 +44,79 @@ */ public final class CompressedXContent { + private static int crc32(BytesReference data) { + OutputStream dummy = new OutputStream() { + @Override + public void write(int b) throws IOException { + // no-op + } + @Override + public void write(byte[] b, int off, int len) throws IOException { + // no-op + } + }; + CRC32 crc32 = new CRC32(); + try { + data.writeTo(new CheckedOutputStream(dummy, crc32)); + } catch (IOException bogus) { + // cannot happen + throw new Error(bogus); + } + return (int) crc32.getValue(); + } + private final byte[] bytes; - private int hashCode; + private final int crc32; + + // Used for serialization + private CompressedXContent(byte[] compressed, int crc32) { + this.bytes = compressed; + this.crc32 = crc32; + assertConsistent(); + } + /** + * Create a {@link CompressedXContent} out of a {@link ToXContent} instance. + */ + public CompressedXContent(ToXContent xcontent, XContentType type, ToXContent.Params params) throws IOException { + BytesStreamOutput bStream = new BytesStreamOutput(); + OutputStream compressedStream = CompressorFactory.defaultCompressor().streamOutput(bStream); + CRC32 crc32 = new CRC32(); + OutputStream checkedStream = new CheckedOutputStream(compressedStream, crc32); + try (XContentBuilder builder = XContentFactory.contentBuilder(type, checkedStream)) { + builder.startObject(); + xcontent.toXContent(builder, params); + builder.endObject(); + } + this.bytes = bStream.bytes().toBytes(); + this.crc32 = (int) crc32.getValue(); + assertConsistent(); + } + + /** + * Create a {@link CompressedXContent} out of a serialized {@link ToXContent} + * that may already be compressed. + */ public CompressedXContent(BytesReference data) throws IOException { Compressor compressor = CompressorFactory.compressor(data); if (compressor != null) { // already compressed... this.bytes = data.toBytes(); + this.crc32 = crc32(new BytesArray(uncompressed())); } else { BytesStreamOutput out = new BytesStreamOutput(); - try (StreamOutput compressedOutput = CompressorFactory.defaultCompressor().streamOutput(out)) { + try (OutputStream compressedOutput = CompressorFactory.defaultCompressor().streamOutput(out)) { data.writeTo(compressedOutput); } this.bytes = out.bytes().toBytes(); - assert CompressorFactory.compressor(new BytesArray(bytes)) != null; + this.crc32 = crc32(data); } + assertConsistent(); + } + private void assertConsistent() { + assert CompressorFactory.compressor(new BytesArray(bytes)) != null; + assert this.crc32 == crc32(new BytesArray(uncompressed())); } public CompressedXContent(byte[] data) throws IOException { @@ -88,12 +151,14 @@ public String string() throws IOException { } public static CompressedXContent readCompressedString(StreamInput in) throws IOException { - byte[] bytes = new byte[in.readVInt()]; - in.readBytes(bytes, 0, bytes.length); - return new CompressedXContent(bytes); + int crc32 = in.readInt(); + byte[] compressed = new byte[in.readVInt()]; + in.readBytes(compressed, 0, compressed.length); + return new CompressedXContent(compressed, crc32); } public void writeTo(StreamOutput out) throws IOException { + out.writeInt(crc32); out.writeVInt(bytes.length); out.writeBytes(bytes); } @@ -109,19 +174,16 @@ public boolean equals(Object o) { return true; } + if (crc32 != that.crc32) { + return false; + } + return Arrays.equals(uncompressed(), that.uncompressed()); } @Override public int hashCode() { - if (hashCode == 0) { - int h = Arrays.hashCode(uncompressed()); - if (h == 0) { - h = 1; - } - hashCode = h; - } - return hashCode; + return crc32; } @Override diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index e6ff7f232f2bf..988251b9e833f 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -36,37 +36,23 @@ import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; -import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache; import org.elasticsearch.index.cache.filter.ShardFilterCache; -import org.elasticsearch.index.cache.query.ShardQueryCache; import org.elasticsearch.index.deletionpolicy.DeletionPolicyModule; import org.elasticsearch.index.fielddata.IndexFieldDataService; -import org.elasticsearch.index.fielddata.ShardFieldData; -import org.elasticsearch.index.gateway.IndexShardGateway; import org.elasticsearch.index.gateway.IndexShardGatewayService; -import org.elasticsearch.index.get.ShardGetService; -import org.elasticsearch.index.indexing.ShardIndexingService; -import org.elasticsearch.index.indexing.slowlog.ShardSlowLogIndexingService; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.merge.policy.MergePolicyModule; import org.elasticsearch.index.merge.policy.MergePolicyProvider; -import org.elasticsearch.index.merge.scheduler.MergeSchedulerModule; import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider; import org.elasticsearch.index.percolator.PercolatorQueriesRegistry; -import org.elasticsearch.index.percolator.stats.ShardPercolateService; import org.elasticsearch.index.query.IndexQueryParserService; -import org.elasticsearch.index.search.slowlog.ShardSlowLogSearchService; -import org.elasticsearch.index.search.stats.ShardSearchService; import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.shard.*; import org.elasticsearch.index.similarity.SimilarityService; -import org.elasticsearch.index.snapshots.IndexShardSnapshotAndRestoreService; import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreModule; -import org.elasticsearch.index.suggest.stats.ShardSuggestService; -import org.elasticsearch.index.termvectors.ShardTermVectorsService; import org.elasticsearch.index.translog.TranslogService; import org.elasticsearch.indices.IndicesLifecycle; import org.elasticsearch.indices.IndicesService; @@ -313,7 +299,6 @@ public synchronized IndexShard createShard(int sShardId, boolean primary) { new StoreCloseListener(shardId, canDeleteShardContent, shardFilterCache), path)); modules.add(new DeletionPolicyModule(indexSettings)); modules.add(new MergePolicyModule(indexSettings)); - modules.add(new MergeSchedulerModule(indexSettings)); try { shardInjector = modules.createChildInjector(injector); } catch (CreationException e) { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index 01d4b78a87fd8..825d0ddc62884 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -36,15 +36,12 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.compress.CompressedXContent; -import org.elasticsearch.common.compress.CompressorFactory; -import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.text.StringAndBytesText; import org.elasticsearch.common.text.Text; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.mapper.Mapping.SourceTransform; import org.elasticsearch.index.mapper.internal.AllFieldMapper; @@ -481,13 +478,7 @@ public MergeResult merge(Mapping mapping, boolean simulate) { private void refreshSource() throws ElasticsearchGenerationException { try { - BytesStreamOutput bStream = new BytesStreamOutput(); - try (XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, CompressorFactory.defaultCompressor().streamOutput(bStream))) { - builder.startObject(); - toXContent(builder, ToXContent.EMPTY_PARAMS); - builder.endObject(); - } - mappingSource = new CompressedXContent(bStream.bytes()); + mappingSource = new CompressedXContent(this, XContentType.JSON, ToXContent.EMPTY_PARAMS); } catch (Exception e) { throw new ElasticsearchGenerationException("failed to serialize source for type [" + type + "]", e); } diff --git a/core/src/main/java/org/elasticsearch/index/merge/scheduler/MergeSchedulerModule.java b/core/src/main/java/org/elasticsearch/index/merge/scheduler/MergeSchedulerModule.java deleted file mode 100644 index 01cf42c3b853c..0000000000000 --- a/core/src/main/java/org/elasticsearch/index/merge/scheduler/MergeSchedulerModule.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.merge.scheduler; - -import org.elasticsearch.common.inject.AbstractModule; -import org.elasticsearch.common.settings.Settings; - -/** - * - */ -public class MergeSchedulerModule extends AbstractModule { - - public static final String MERGE_SCHEDULER_TYPE_KEY = "index.merge.scheduler.type"; - public static final Class DEFAULT = ConcurrentMergeSchedulerProvider.class; - - private final Settings settings; - - public MergeSchedulerModule(Settings settings) { - this.settings = settings; - } - - @Override - protected void configure() { - bind(MergeSchedulerProvider.class) - .to(settings.getAsClass(MERGE_SCHEDULER_TYPE_KEY, DEFAULT, "org.elasticsearch.index.merge.scheduler.", "MergeSchedulerProvider")) - .asEagerSingleton(); - } -} diff --git a/core/src/main/java/org/elasticsearch/index/query/ExistsQueryParser.java b/core/src/main/java/org/elasticsearch/index/query/ExistsQueryParser.java index 45015bebcf3ea..c7eda929003c2 100644 --- a/core/src/main/java/org/elasticsearch/index/query/ExistsQueryParser.java +++ b/core/src/main/java/org/elasticsearch/index/query/ExistsQueryParser.java @@ -77,6 +77,10 @@ public Query parse(QueryParseContext parseContext) throws IOException, QueryPars public static Query newFilter(QueryParseContext parseContext, String fieldPattern, String queryName) { final FieldNamesFieldMapper.FieldNamesFieldType fieldNamesFieldType = (FieldNamesFieldMapper.FieldNamesFieldType)parseContext.mapperService().fullName(FieldNamesFieldMapper.NAME); + if (fieldNamesFieldType == null) { + // can only happen when no types exist, so no docs exist either + return Queries.newMatchNoDocsQuery(); + } MapperService.SmartNameObjectMapper smartNameObjectMapper = parseContext.smartObjectMapper(fieldPattern); if (smartNameObjectMapper != null && smartNameObjectMapper.hasMapper()) { diff --git a/core/src/main/java/org/elasticsearch/index/query/MissingQueryParser.java b/core/src/main/java/org/elasticsearch/index/query/MissingQueryParser.java index 701b1dc6f06e2..19500930b9d7a 100644 --- a/core/src/main/java/org/elasticsearch/index/query/MissingQueryParser.java +++ b/core/src/main/java/org/elasticsearch/index/query/MissingQueryParser.java @@ -90,6 +90,11 @@ public static Query newFilter(QueryParseContext parseContext, String fieldPatter } final FieldNamesFieldMapper.FieldNamesFieldType fieldNamesFieldType = (FieldNamesFieldMapper.FieldNamesFieldType)parseContext.mapperService().fullName(FieldNamesFieldMapper.NAME); + if (fieldNamesFieldType == null) { + // can only happen when no types exist, so no docs exist either + return Queries.newMatchNoDocsQuery(); + } + MapperService.SmartNameObjectMapper smartNameObjectMapper = parseContext.smartObjectMapper(fieldPattern); if (smartNameObjectMapper != null && smartNameObjectMapper.hasMapper()) { // automatic make the object mapper pattern diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java index 672b63bfb1c40..a83b6a117d429 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java @@ -34,6 +34,8 @@ import org.elasticsearch.index.get.ShardGetService; import org.elasticsearch.index.indexing.ShardIndexingService; import org.elasticsearch.index.indexing.slowlog.ShardSlowLogIndexingService; +import org.elasticsearch.index.merge.scheduler.ConcurrentMergeSchedulerProvider; +import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider; import org.elasticsearch.index.percolator.PercolatorQueriesRegistry; import org.elasticsearch.index.percolator.stats.ShardPercolateService; import org.elasticsearch.index.search.slowlog.ShardSlowLogSearchService; @@ -90,7 +92,7 @@ protected void configure() { } bind(EngineFactory.class).to(settings.getAsClass(ENGINE_FACTORY, DEFAULT_ENGINE_FACTORY_CLASS, ENGINE_PREFIX, ENGINE_SUFFIX)); - + bind(MergeSchedulerProvider.class).to(ConcurrentMergeSchedulerProvider.class).asEagerSingleton(); bind(ShardIndexWarmerService.class).asEagerSingleton(); bind(ShardIndexingService.class).asEagerSingleton(); bind(ShardSlowLogIndexingService.class).asEagerSingleton(); diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 18f08c929ee45..22bd433697444 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -306,7 +306,8 @@ public void messageReceived(final RecoveryTranslogOperationsRequest request, fin try { recoveryStatus.indexShard().performBatchRecovery(request.operations()); } catch (TranslogRecoveryPerformer.BatchOperationException exception) { - if (ExceptionsHelper.unwrapCause(exception) instanceof MapperException == false) { + MapperException mapperException = (MapperException) ExceptionsHelper.unwrap(exception, MapperException.class); + if (mapperException == null) { throw exception; } // in very rare cases a translog replay from primary is processed before a mapping update on this node diff --git a/core/src/main/java/org/elasticsearch/repositories/Repository.java b/core/src/main/java/org/elasticsearch/repositories/Repository.java index f2d2386bbfbeb..2bcdccb5565c0 100644 --- a/core/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/core/src/main/java/org/elasticsearch/repositories/Repository.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.repositories; -import com.google.common.collect.ImmutableList; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.SnapshotId; import org.elasticsearch.common.component.LifecycleComponent; @@ -59,11 +58,11 @@ public interface Repository extends LifecycleComponent { *

* The returned meta data contains global metadata as well as metadata for all indices listed in the indices parameter. * - * @param snapshotId snapshot ID + * @param snapshot snapshot * @param indices list of indices * @return information about snapshot */ - MetaData readSnapshotMetaData(SnapshotId snapshotId, List indices) throws IOException; + MetaData readSnapshotMetaData(SnapshotId snapshotId, Snapshot snapshot, List indices) throws IOException; /** * Returns the list of snapshots currently stored in the repository diff --git a/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index d08d6fbcb8727..9f37b8c88cfc6 100644 --- a/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -21,13 +21,13 @@ import com.fasterxml.jackson.core.JsonParseException; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.io.ByteStreams; import org.apache.lucene.store.RateLimiter; import org.apache.lucene.util.IOUtils; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.SnapshotId; @@ -137,7 +137,11 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent indices, Meta } // Write Global MetaData // TODO: Check if metadata needs to be written - try (StreamOutput output = compressIfNeeded(snapshotsBlobContainer.createOutput(metaDataBlobName(snapshotId)))) { + try (StreamOutput output = compressIfNeeded(snapshotsBlobContainer.createOutput(metaDataBlobName(snapshotId, false)))) { writeGlobalMetaData(metaData, output); } for (String index : indices) { @@ -270,8 +274,10 @@ public void initializeSnapshot(SnapshotId snapshotId, List indices, Meta @Override public void deleteSnapshot(SnapshotId snapshotId) { List indices = Collections.EMPTY_LIST; + Snapshot snapshot = null; try { - indices = readSnapshot(snapshotId).indices(); + snapshot = readSnapshot(snapshotId); + indices = snapshot.indices(); } catch (SnapshotMissingException ex) { throw ex; } catch (SnapshotException | ElasticsearchParseException ex) { @@ -279,7 +285,15 @@ public void deleteSnapshot(SnapshotId snapshotId) { } MetaData metaData = null; try { - metaData = readSnapshotMetaData(snapshotId, indices, true); + if (snapshot != null) { + metaData = readSnapshotMetaData(snapshotId, snapshot.version(), indices, true); + } else { + try { + metaData = readSnapshotMetaData(snapshotId, false, indices, true); + } catch (IOException ex) { + metaData = readSnapshotMetaData(snapshotId, true, indices, true); + } + } } catch (IOException | SnapshotException ex) { logger.warn("cannot read metadata for snapshot [{}]", ex, snapshotId); } @@ -287,7 +301,13 @@ public void deleteSnapshot(SnapshotId snapshotId) { String blobName = snapshotBlobName(snapshotId); // Delete snapshot file first so we wouldn't end up with partially deleted snapshot that looks OK snapshotsBlobContainer.deleteBlob(blobName); - snapshotsBlobContainer.deleteBlob(metaDataBlobName(snapshotId)); + if (snapshot != null) { + snapshotsBlobContainer.deleteBlob(metaDataBlobName(snapshotId, legacyMetaData(snapshot.version()))); + } else { + // We don't know which version was the snapshot created with - try deleting both current and legacy metadata + snapshotsBlobContainer.deleteBlob(metaDataBlobName(snapshotId, true)); + snapshotsBlobContainer.deleteBlob(metaDataBlobName(snapshotId, false)); + } // Delete snapshot from the snapshot list List snapshotIds = snapshots(); if (snapshotIds.contains(snapshotId)) { @@ -402,8 +422,8 @@ public List snapshots() { * {@inheritDoc} */ @Override - public MetaData readSnapshotMetaData(SnapshotId snapshotId, List indices) throws IOException { - return readSnapshotMetaData(snapshotId, indices, false); + public MetaData readSnapshotMetaData(SnapshotId snapshotId, Snapshot snapshot, List indices) throws IOException { + return readSnapshotMetaData(snapshotId, snapshot.version(), indices, false); } /** @@ -422,11 +442,14 @@ public Snapshot readSnapshot(SnapshotId snapshotId) { } } - private MetaData readSnapshotMetaData(SnapshotId snapshotId, List indices, boolean ignoreIndexErrors) throws IOException { + private MetaData readSnapshotMetaData(SnapshotId snapshotId, Version snapshotVersion, List indices, boolean ignoreIndexErrors) throws IOException { + return readSnapshotMetaData(snapshotId, legacyMetaData(snapshotVersion), indices, ignoreIndexErrors); + } + + private MetaData readSnapshotMetaData(SnapshotId snapshotId, boolean legacy, List indices, boolean ignoreIndexErrors) throws IOException { MetaData metaData; - try (InputStream blob = snapshotsBlobContainer.openInput(metaDataBlobName(snapshotId))) { - byte[] data = ByteStreams.toByteArray(blob); - metaData = readMetaData(data); + try (InputStream blob = snapshotsBlobContainer.openInput(metaDataBlobName(snapshotId, legacy))) { + metaData = readMetaData(ByteStreams.toByteArray(blob)); } catch (FileNotFoundException | NoSuchFileException ex) { throw new SnapshotMissingException(snapshotId, ex); } catch (IOException ex) { @@ -554,10 +577,24 @@ private String tempSnapshotBlobName(SnapshotId snapshotId) { * Returns name of metadata blob * * @param snapshotId snapshot id + * @param legacy true if legacy (pre-2.0.0) format should be used * @return name of metadata blob */ - private String metaDataBlobName(SnapshotId snapshotId) { - return METADATA_PREFIX + snapshotId.getSnapshot(); + private String metaDataBlobName(SnapshotId snapshotId, boolean legacy) { + if (legacy) { + return LEGACY_METADATA_PREFIX + snapshotId.getSnapshot(); + } else { + return METADATA_PREFIX + snapshotId.getSnapshot() + METADATA_SUFFIX; + } + } + + /** + * In v2.0.0 we changed the matadata file format + * @param version + * @return true if legacy version should be used false otherwise + */ + private boolean legacyMetaData(Version version) { + return version.before(Version.V_2_0_0); } /** diff --git a/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 9f880365c1dea..7f39a2770ea80 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -157,7 +157,7 @@ public void restoreSnapshot(final RestoreRequest request, final ActionListener filteredIndices = SnapshotUtils.filterIndices(snapshot.indices(), request.indices(), request.indicesOptions()); - MetaData metaDataIn = repository.readSnapshotMetaData(snapshotId, filteredIndices); + MetaData metaDataIn = repository.readSnapshotMetaData(snapshotId, snapshot, filteredIndices); final MetaData metaData; if (snapshot.version().before(Version.V_2_0_0)) { diff --git a/core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index aa8b34e83fc2a..4281c42b430e5 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -500,7 +500,7 @@ public ImmutableMap snapshotShards(SnapshotId Repository repository = repositoriesService.repository(snapshotId.getRepository()); IndexShardRepository indexShardRepository = repositoriesService.indexShardRepository(snapshotId.getRepository()); Snapshot snapshot = repository.readSnapshot(snapshotId); - MetaData metaData = repository.readSnapshotMetaData(snapshotId, snapshot.indices()); + MetaData metaData = repository.readSnapshotMetaData(snapshotId, snapshot, snapshot.indices()); for (String index : snapshot.indices()) { IndexMetaData indexMetaData = metaData.indices().get(index); if (indexMetaData != null) { @@ -836,16 +836,20 @@ private void processIndexShardSnapshots(ClusterChangedEvent event) { for (Map.Entry shard : entry.shards().entrySet()) { IndexShardSnapshotStatus snapshotStatus = snapshotShards.shards.get(shard.getKey()); if (snapshotStatus != null) { - if (snapshotStatus.stage() == IndexShardSnapshotStatus.Stage.STARTED) { - snapshotStatus.abort(); - } else if (snapshotStatus.stage() == IndexShardSnapshotStatus.Stage.DONE) { - logger.debug("[{}] trying to cancel snapshot on the shard [{}] that is already done, updating status on the master", entry.snapshotId(), shard.getKey()); - updateIndexShardSnapshotStatus(new UpdateIndexShardSnapshotStatusRequest(entry.snapshotId(), shard.getKey(), - new ShardSnapshotStatus(event.state().nodes().localNodeId(), SnapshotMetaData.State.SUCCESS))); - } else if (snapshotStatus.stage() == IndexShardSnapshotStatus.Stage.FAILURE) { - logger.debug("[{}] trying to cancel snapshot on the shard [{}] that has already failed, updating status on the master", entry.snapshotId(), shard.getKey()); - updateIndexShardSnapshotStatus(new UpdateIndexShardSnapshotStatusRequest(entry.snapshotId(), shard.getKey(), - new ShardSnapshotStatus(event.state().nodes().localNodeId(), State.FAILED, snapshotStatus.failure()))); + switch (snapshotStatus.stage()) { + case STARTED: + snapshotStatus.abort(); + break; + case DONE: + logger.debug("[{}] trying to cancel snapshot on the shard [{}] that is already done, updating status on the master", entry.snapshotId(), shard.getKey()); + updateIndexShardSnapshotStatus(new UpdateIndexShardSnapshotStatusRequest(entry.snapshotId(), shard.getKey(), + new ShardSnapshotStatus(event.state().nodes().localNodeId(), SnapshotMetaData.State.SUCCESS))); + break; + case FAILURE: + logger.debug("[{}] trying to cancel snapshot on the shard [{}] that has already failed, updating status on the master", entry.snapshotId(), shard.getKey()); + updateIndexShardSnapshotStatus(new UpdateIndexShardSnapshotStatusRequest(entry.snapshotId(), shard.getKey(), + new ShardSnapshotStatus(event.state().nodes().localNodeId(), State.FAILED, snapshotStatus.failure()))); + break; } } } diff --git a/core/src/test/java/org/elasticsearch/action/termvectors/GetTermVectorsTests.java b/core/src/test/java/org/elasticsearch/action/termvectors/GetTermVectorsTests.java index 7eec3a65d22e0..12467ecef5925 100644 --- a/core/src/test/java/org/elasticsearch/action/termvectors/GetTermVectorsTests.java +++ b/core/src/test/java/org/elasticsearch/action/termvectors/GetTermVectorsTests.java @@ -917,7 +917,7 @@ public void testArtificialNonExistingField() throws Exception { assertThat(resp.isExists(), equalTo(true)); checkBrownFoxTermVector(resp.getFields(), "field1", false); // we should have created a mapping for this field - waitForMappingOnMaster("test", "type1", "non_existing"); + assertMappingOnMaster("test", "type1", "non_existing"); // and return the generated term vectors checkBrownFoxTermVector(resp.getFields(), "non_existing", false); } diff --git a/core/src/test/java/org/elasticsearch/aliases/IndexAliasesTests.java b/core/src/test/java/org/elasticsearch/aliases/IndexAliasesTests.java index 461c3272b340d..63e6771b01af4 100644 --- a/core/src/test/java/org/elasticsearch/aliases/IndexAliasesTests.java +++ b/core/src/test/java/org/elasticsearch/aliases/IndexAliasesTests.java @@ -303,13 +303,9 @@ public void testSearchingFilteringAliasesMultipleIndices() throws Exception { logger.info("--> creating indices"); createIndex("test1", "test2", "test3"); - client().admin().indices().preparePutMapping("test1", "test2", "test3") + assertAcked(client().admin().indices().preparePutMapping("test1", "test2", "test3") .setType("type1") - .setSource("name", "type=string") - .get(); - waitForConcreteMappingsOnAll("test1", "type1", "name"); - waitForConcreteMappingsOnAll("test2", "type1", "name"); - waitForConcreteMappingsOnAll("test3", "type1", "name"); + .setSource("name", "type=string")); ensureGreen(); @@ -553,14 +549,8 @@ public void testIndicesGetAliases() throws Exception { createIndex("foobarbaz"); createIndex("bazbar"); - client().admin().indices().preparePutMapping("foobar", "test", "test123", "foobarbaz", "bazbar") - .setType("type").setSource("field", "type=string").get(); - waitForConcreteMappingsOnAll("foobar", "type", "field"); - waitForConcreteMappingsOnAll("test", "type", "field"); - waitForConcreteMappingsOnAll("test123", "type", "field"); - waitForConcreteMappingsOnAll("foobarbaz", "type", "field"); - waitForConcreteMappingsOnAll("bazbar", "type", "field"); - + assertAcked(client().admin().indices().preparePutMapping("foobar", "test", "test123", "foobarbaz", "bazbar") + .setType("type").setSource("field", "type=string")); ensureGreen(); logger.info("--> creating aliases [alias1, alias2]"); diff --git a/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateTests.java b/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateTests.java index 3123edb4c5f1c..186e05da604ff 100644 --- a/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateTests.java @@ -104,14 +104,10 @@ public void testSimpleOpenClose() throws Exception { assertThat(stateResponse.getState().routingTable().index("test").shardsWithState(ShardRoutingState.STARTED).size(), equalTo(test.totalNumShards)); logger.info("--> indexing a simple document"); - client().prepareIndex("test", "type1", "1").setSource("field1", "value1").execute().actionGet(); - - // we need this until we have https://github.com/elasticsearch/elasticsearch/issues/8688 - // the test rarely fails else because the master does not apply the new mapping quick enough and it is lost - waitForConcreteMappingsOnAll("test", "type1", "field1"); + client().prepareIndex("test", "type1", "1").setSource("field1", "value1").get(); logger.info("--> closing test index..."); - client().admin().indices().prepareClose("test").execute().actionGet(); + client().admin().indices().prepareClose("test").get(); stateResponse = client().admin().cluster().prepareState().execute().actionGet(); assertThat(stateResponse.getState().metaData().index("test").state(), equalTo(IndexMetaData.State.CLOSE)); diff --git a/core/src/test/java/org/elasticsearch/gateway/MetaDataWriteDataNodesTests.java b/core/src/test/java/org/elasticsearch/gateway/MetaDataWriteDataNodesTests.java index 71143159cc4e4..30608cf1a8d0d 100644 --- a/core/src/test/java/org/elasticsearch/gateway/MetaDataWriteDataNodesTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/MetaDataWriteDataNodesTests.java @@ -55,7 +55,6 @@ public void testMetaWrittenAlsoOnDataNode() throws Exception { String redNode = startDataNode("red"); assertAcked(prepareCreate("test").setSettings(Settings.builder().put("index.number_of_replicas", 0))); index("test", "doc", "1", jsonBuilder().startObject().field("text", "some text").endObject()); - waitForConcreteMappingsOnAll("test", "doc", "text"); ensureGreen("test"); assertIndexInMetaState(redNode, "test"); assertIndexInMetaState(masterNodeName, "test"); @@ -63,8 +62,6 @@ public void testMetaWrittenAlsoOnDataNode() throws Exception { ((InternalTestCluster) cluster()).stopCurrentMasterNode(); String newMasterNode = startMasterNode(); ensureGreen("test"); - // wait for mapping also on master becasue then we can be sure the state was written - waitForConcreteMappingsOnAll("test", "doc", "text"); // check for meta data assertIndexInMetaState(redNode, "test"); assertIndexInMetaState(newMasterNode, "test"); @@ -85,8 +82,6 @@ public void testMetaWrittenOnlyForIndicesOnNodesThatHaveAShard() throws Exceptio assertAcked(prepareCreate("red_index").setSettings(Settings.builder().put("index.number_of_replicas", 0).put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "color", "red"))); index("red_index", "doc", "1", jsonBuilder().startObject().field("text", "some text").endObject()); ensureGreen(); - waitForConcreteMappingsOnAll("blue_index", "doc", "text"); - waitForConcreteMappingsOnAll("red_index", "doc", "text"); assertIndexNotInMetaState(blueNode, "red_index"); assertIndexNotInMetaState(redNode, "blue_index"); assertIndexInMetaState(blueNode, "blue_index"); @@ -151,8 +146,6 @@ public void testMetaIsRemovedIfAllShardsFromIndexRemoved() throws Exception { assertIndexInMetaState(blueNode, "red_index"); assertIndexInMetaState(masterNode, "red_index"); assertIndexInMetaState(masterNode, "blue_index"); - waitForConcreteMappingsOnAll("blue_index", "doc", "text"); - waitForConcreteMappingsOnAll("red_index", "doc", "text"); //at this point the blue_index is on red node and the red_index on blue node // now, when we start red and master node again but without data folder, the red index should be gone but the blue index should initialize fine @@ -188,7 +181,6 @@ public void testMetaWrittenWhenIndexIsClosed() throws Exception { assertIndexInMetaState(redNode, "red_index"); assertIndexInMetaState(masterNode, "red_index"); - waitForConcreteMappingsOnAll("red_index", "doc", "text"); client().admin().indices().prepareClose("red_index").get(); // close the index ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().get(); @@ -252,8 +244,6 @@ public void testMetaWrittenWhenIndexIsClosedAndMetaUpdated() throws Exception { assertIndexInMetaState(redNode, "red_index"); assertIndexInMetaState(masterNode, "red_index"); - waitForConcreteMappingsOnAll("red_index", "doc", "text"); - logger.info("--> close red_index"); client().admin().indices().prepareClose("red_index").get(); // close the index diff --git a/core/src/test/java/org/elasticsearch/indices/mapping/ConcurrentDynamicTemplateTests.java b/core/src/test/java/org/elasticsearch/indices/mapping/ConcurrentDynamicTemplateTests.java index 28bcde323d296..673bd140d01b4 100644 --- a/core/src/test/java/org/elasticsearch/indices/mapping/ConcurrentDynamicTemplateTests.java +++ b/core/src/test/java/org/elasticsearch/indices/mapping/ConcurrentDynamicTemplateTests.java @@ -89,29 +89,4 @@ public void onFailure(Throwable e) { } } - - @Test - public void testDynamicMappingIntroductionPropagatesToAll() throws Exception { - int numDocs = randomIntBetween(100, 1000); - int numberOfFields = scaledRandomIntBetween(1, 50); - Set fieldsIdx = Sets.newHashSet(); - IndexRequestBuilder[] builders = new IndexRequestBuilder[numDocs]; - - createIndex("idx"); - ensureGreen("idx"); - for (int i = 0; i < numDocs; ++i) { - int fieldIdx = i % numberOfFields; - fieldsIdx.add(fieldIdx); - builders[i] = client().prepareIndex("idx", "type").setSource(jsonBuilder() - .startObject() - .field("str_value_" + fieldIdx, "s" + i) - .field("l_value_" + fieldIdx, i) - .field("d_value_" + fieldIdx, (double)i + 0.01) - .endObject()); - } - indexRandom(false, builders); - for (Integer fieldIdx : fieldsIdx) { - waitForConcreteMappingsOnAll("idx", "type", "str_value_" + fieldIdx, "l_value_" + fieldIdx, "d_value_" + fieldIdx); - } - } } \ No newline at end of file diff --git a/core/src/test/java/org/elasticsearch/indices/mapping/SimpleGetFieldMappingsTests.java b/core/src/test/java/org/elasticsearch/indices/mapping/SimpleGetFieldMappingsTests.java index 119157bcfc1ce..0e160189bfb22 100644 --- a/core/src/test/java/org/elasticsearch/indices/mapping/SimpleGetFieldMappingsTests.java +++ b/core/src/test/java/org/elasticsearch/indices/mapping/SimpleGetFieldMappingsTests.java @@ -133,7 +133,6 @@ public void simpleGetFieldMappingsWithDefaults() throws Exception { client().prepareIndex("test", "type", "1").setSource("num", 1).get(); ensureYellow(); - waitForConcreteMappingsOnAll("test", "type", "num"); // for num, we need to wait... GetFieldMappingsResponse response = client().admin().indices().prepareGetFieldMappings().setFields("num", "field1", "obj.subfield").includeDefaults(true).get(); diff --git a/core/src/test/java/org/elasticsearch/indices/mapping/UpdateMappingIntegrationTests.java b/core/src/test/java/org/elasticsearch/indices/mapping/UpdateMappingIntegrationTests.java index 36026e95b525e..e9aef353582a0 100644 --- a/core/src/test/java/org/elasticsearch/indices/mapping/UpdateMappingIntegrationTests.java +++ b/core/src/test/java/org/elasticsearch/indices/mapping/UpdateMappingIntegrationTests.java @@ -39,10 +39,7 @@ import org.hamcrest.Matchers; import org.junit.Test; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicBoolean; @@ -86,7 +83,7 @@ public void dynamicUpdates() throws Exception { for (int rec = 0; rec < recCount; rec++) { String type = "type" + (rec % numberOfTypes); String fieldName = "field_" + type + "_" + rec; - waitForConcreteMappingsOnAll("test", type, fieldName); + assertConcreteMappingsOnAll("test", type, fieldName); } } diff --git a/core/src/test/java/org/elasticsearch/indices/settings/UpdateSettingsTests.java b/core/src/test/java/org/elasticsearch/indices/settings/UpdateSettingsTests.java index e1ca345b68dcd..58bcffd37e22d 100644 --- a/core/src/test/java/org/elasticsearch/indices/settings/UpdateSettingsTests.java +++ b/core/src/test/java/org/elasticsearch/indices/settings/UpdateSettingsTests.java @@ -33,7 +33,6 @@ import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.merge.policy.TieredMergePolicyProvider; import org.elasticsearch.index.merge.scheduler.ConcurrentMergeSchedulerProvider; -import org.elasticsearch.index.merge.scheduler.MergeSchedulerModule; import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.Store; import org.elasticsearch.test.ElasticsearchIntegrationTest; @@ -366,7 +365,6 @@ public void testUpdateMergeMaxThreadCount() { .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, "0") .put(TieredMergePolicyProvider.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE, "2") .put(TieredMergePolicyProvider.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER, "2") - .put(MergeSchedulerModule.MERGE_SCHEDULER_TYPE_KEY, ConcurrentMergeSchedulerProvider.class) .put(ConcurrentMergeSchedulerProvider.MAX_THREAD_COUNT, "10000") .put(ConcurrentMergeSchedulerProvider.MAX_MERGE_COUNT, "10000") )); diff --git a/core/src/test/java/org/elasticsearch/percolator/PercolatorTests.java b/core/src/test/java/org/elasticsearch/percolator/PercolatorTests.java index 07ef51f32c17f..1168642a27313 100644 --- a/core/src/test/java/org/elasticsearch/percolator/PercolatorTests.java +++ b/core/src/test/java/org/elasticsearch/percolator/PercolatorTests.java @@ -109,7 +109,6 @@ public void testSimple1() throws Exception { logger.info("--> Add dummy doc"); client().prepareIndex("test", "type", "1").setSource("field1", "value").execute().actionGet(); - waitForConcreteMappingsOnAll("test", "type", "field1"); logger.info("--> register a queries"); client().prepareIndex("test", PercolatorService.TYPE_NAME, "1") @@ -198,7 +197,6 @@ public void testSimple2() throws Exception { .execute().actionGet(); assertMatchCount(response, 0l); assertThat(response.getMatches(), emptyArray()); - waitForConcreteMappingsOnAll("test", "type1", "field1", "field2"); // add first query... client().prepareIndex("test", PercolatorService.TYPE_NAME, "test1") @@ -278,12 +276,11 @@ public void testPercolateQueriesWithRouting() throws Exception { } @Test - public void percolateOnRecreatedIndex() throws Exception { + public void storePeroclateQueriesOnRecreatedIndex() throws Exception { createIndex("test"); ensureGreen(); client().prepareIndex("my-queries-index", "test", "1").setSource("field1", "value1").execute().actionGet(); - waitForConcreteMappingsOnAll("my-queries-index", "test", "field1"); logger.info("--> register a query"); client().prepareIndex("my-queries-index", PercolatorService.TYPE_NAME, "kuku1") .setSource(jsonBuilder().startObject() @@ -298,7 +295,6 @@ public void percolateOnRecreatedIndex() throws Exception { ensureGreen(); client().prepareIndex("my-queries-index", "test", "1").setSource("field1", "value1").execute().actionGet(); - waitForConcreteMappingsOnAll("my-queries-index", "test", "field1"); logger.info("--> register a query"); client().prepareIndex("my-queries-index", PercolatorService.TYPE_NAME, "kuku2") .setSource(jsonBuilder().startObject() @@ -995,7 +991,6 @@ public void testCountPercolation() throws Exception { logger.info("--> Add dummy doc"); client().prepareIndex("test", "type", "1").setSource("field1", "value").execute().actionGet(); - waitForConcreteMappingsOnAll("test", "type", "field1"); logger.info("--> register a queries"); client().prepareIndex("test", PercolatorService.TYPE_NAME, "1") @@ -1724,7 +1719,6 @@ public void testPercolationWithDynamicTemplates() throws Exception { assertMatchCount(percolateResponse, 0l); assertThat(percolateResponse.getMatches(), arrayWithSize(0)); - waitForConcreteMappingsOnAll("idx", "type", "custom.color"); // The previous percolate request introduced the custom.color field, so now we register the query again // and the field name `color` will be resolved to `custom.color` field in mapping via smart field mapping resolving. @@ -1762,7 +1756,7 @@ public void testUpdateMappingDynamicallyWhilePercolating() throws Exception { assertMatchCount(response, 0l); assertThat(response.getMatches(), arrayWithSize(0)); - waitForMappingOnMaster("test", "type1"); + assertMappingOnMaster("test", "type1"); GetMappingsResponse mappingsResponse = client().admin().indices().prepareGetMappings("test").get(); assertThat(mappingsResponse.getMappings().get("test"), notNullValue()); diff --git a/core/src/test/java/org/elasticsearch/percolator/RecoveryPercolatorTests.java b/core/src/test/java/org/elasticsearch/percolator/RecoveryPercolatorTests.java index 7679ff6724bdb..131f72853d603 100644 --- a/core/src/test/java/org/elasticsearch/percolator/RecoveryPercolatorTests.java +++ b/core/src/test/java/org/elasticsearch/percolator/RecoveryPercolatorTests.java @@ -186,7 +186,6 @@ public void testLoadingPercolateQueriesDuringCloseAndOpen() throws Exception { logger.info("--> Add dummy docs"); client().prepareIndex("test", "type1", "1").setSource("field1", 0).get(); client().prepareIndex("test", "type2", "1").setSource("field1", "0").get(); - waitForConcreteMappingsOnAll("test", "type1", "field1"); logger.info("--> register a queries"); for (int i = 1; i <= 100; i++) { @@ -199,7 +198,6 @@ public void testLoadingPercolateQueriesDuringCloseAndOpen() throws Exception { .endObject()) .get(); } - waitForConcreteMappingsOnAll("test", PercolatorService.TYPE_NAME); logger.info("--> Percolate doc with field1=95"); PercolateResponse response = client().preparePercolate() diff --git a/core/src/test/java/org/elasticsearch/search/query/ExistsMissingTests.java b/core/src/test/java/org/elasticsearch/search/query/ExistsMissingTests.java index a33e7c44a9785..3a2c2fcefcc07 100644 --- a/core/src/test/java/org/elasticsearch/search/query/ExistsMissingTests.java +++ b/core/src/test/java/org/elasticsearch/search/query/ExistsMissingTests.java @@ -45,6 +45,16 @@ public class ExistsMissingTests extends ElasticsearchIntegrationTest { + // TODO: move this to a unit test somewhere... + public void testEmptyIndex() throws Exception { + createIndex("test"); + ensureYellow("test"); + SearchResponse resp = client().prepareSearch("test").setQuery(QueryBuilders.existsQuery("foo")).execute().actionGet(); + assertSearchResponse(resp); + resp = client().prepareSearch("test").setQuery(QueryBuilders.missingQuery("foo")).execute().actionGet(); + assertSearchResponse(resp); + } + public void testExistsMissing() throws Exception { XContentBuilder mapping = XContentBuilder.builder(JsonXContent.jsonXContent) .startObject() diff --git a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreTests.java b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreTests.java index b9fb2bf1fe45f..285baa1e60b25 100644 --- a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreTests.java +++ b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreTests.java @@ -196,9 +196,6 @@ public void singleGetAfterRestoreTest() throws Exception { String docId = Integer.toString(randomInt()); index(indexName, typeName, docId, "value", expectedValue); - // TODO: Remove after dynamic mapping flushing is implemented - waitForConcreteMappingsOnAll(indexName, typeName, "value"); - logger.info("--> creating repository"); assertAcked(client.admin().cluster().preparePutRepository(repoName) .setType("fs").setSettings(Settings.settingsBuilder() @@ -873,7 +870,7 @@ public void deleteSnapshotWithMissingMetadataTest() throws Exception { assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); logger.info("--> delete index metadata and shard metadata"); - Path metadata = repo.resolve("metadata-test-snap-1"); + Path metadata = repo.resolve("meta-test-snap-1.dat"); Files.delete(metadata); logger.info("--> delete snapshot"); diff --git a/core/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java b/core/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java index 621d4fc4d01a0..0494c47bb84b7 100644 --- a/core/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java +++ b/core/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java @@ -97,14 +97,12 @@ import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.DocumentMapper; -import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType.Loading; import org.elasticsearch.index.mapper.internal.SizeFieldMapper; import org.elasticsearch.index.mapper.internal.TimestampFieldMapper; import org.elasticsearch.index.merge.policy.*; import org.elasticsearch.index.merge.scheduler.ConcurrentMergeSchedulerProvider; -import org.elasticsearch.index.merge.scheduler.MergeSchedulerModule; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogService; @@ -546,7 +544,6 @@ private static Settings.Builder setRandomMerge(Random random, Settings.Builder b switch (random.nextInt(4)) { case 3: - builder.put(MergeSchedulerModule.MERGE_SCHEDULER_TYPE_KEY, ConcurrentMergeSchedulerProvider.class); final int maxThreadCount = RandomInts.randomIntBetween(random, 1, 4); final int maxMergeCount = RandomInts.randomIntBetween(random, maxThreadCount, maxThreadCount + 4); builder.put(ConcurrentMergeSchedulerProvider.MAX_MERGE_COUNT, maxMergeCount); @@ -866,56 +863,44 @@ public void run() { * Waits till a (pattern) field name mappings concretely exists on all nodes. Note, this waits for the current * started shards and checks for concrete mappings. */ - public void waitForConcreteMappingsOnAll(final String index, final String type, final String... fieldNames) throws Exception { - assertBusy(new Runnable() { - @Override - public void run() { - Set nodes = internalCluster().nodesInclude(index); - assertThat(nodes, Matchers.not(Matchers.emptyIterable())); - for (String node : nodes) { - IndicesService indicesService = internalCluster().getInstance(IndicesService.class, node); - IndexService indexService = indicesService.indexService(index); - assertThat("index service doesn't exists on " + node, indexService, notNullValue()); - DocumentMapper documentMapper = indexService.mapperService().documentMapper(type); - assertThat("document mapper doesn't exists on " + node, documentMapper, notNullValue()); - for (String fieldName : fieldNames) { - Collection matches = documentMapper.mappers().simpleMatchToFullName(fieldName); - assertThat("field " + fieldName + " doesn't exists on " + node, matches, Matchers.not(emptyIterable())); - } - } + public void assertConcreteMappingsOnAll(final String index, final String type, final String... fieldNames) throws Exception { + Set nodes = internalCluster().nodesInclude(index); + assertThat(nodes, Matchers.not(Matchers.emptyIterable())); + for (String node : nodes) { + IndicesService indicesService = internalCluster().getInstance(IndicesService.class, node); + IndexService indexService = indicesService.indexService(index); + assertThat("index service doesn't exists on " + node, indexService, notNullValue()); + DocumentMapper documentMapper = indexService.mapperService().documentMapper(type); + assertThat("document mapper doesn't exists on " + node, documentMapper, notNullValue()); + for (String fieldName : fieldNames) { + Collection matches = documentMapper.mappers().simpleMatchToFullName(fieldName); + assertThat("field " + fieldName + " doesn't exists on " + node, matches, Matchers.not(emptyIterable())); } - }); - waitForMappingOnMaster(index, type, fieldNames); + } + assertMappingOnMaster(index, type, fieldNames); } /** * Waits for the given mapping type to exists on the master node. */ - public void waitForMappingOnMaster(final String index, final String type, final String... fieldNames) throws Exception { - assertBusy(new Callable() { - @Override - public Object call() throws Exception { - GetMappingsResponse response = client().admin().indices().prepareGetMappings(index).setTypes(type).get(); - ImmutableOpenMap mappings = response.getMappings().get(index); - assertThat(mappings, notNullValue()); - MappingMetaData mappingMetaData = mappings.get(type); - assertThat(mappingMetaData, notNullValue()); - - Map mappingSource = mappingMetaData.getSourceAsMap(); - assertFalse(mappingSource.isEmpty()); - assertTrue(mappingSource.containsKey("properties")); - - for (String fieldName : fieldNames) { - Map mappingProperties = (Map) mappingSource.get("properties"); - if (fieldName.indexOf('.') != -1) { - fieldName = fieldName.replace(".", ".properties."); - } - assertThat("field " + fieldName + " doesn't exists in mapping " + mappingMetaData.source().string(), XContentMapValues.extractValue(fieldName, mappingProperties), notNullValue()); - } - - return null; + public void assertMappingOnMaster(final String index, final String type, final String... fieldNames) throws Exception { + GetMappingsResponse response = client().admin().indices().prepareGetMappings(index).setTypes(type).get(); + ImmutableOpenMap mappings = response.getMappings().get(index); + assertThat(mappings, notNullValue()); + MappingMetaData mappingMetaData = mappings.get(type); + assertThat(mappingMetaData, notNullValue()); + + Map mappingSource = mappingMetaData.getSourceAsMap(); + assertFalse(mappingSource.isEmpty()); + assertTrue(mappingSource.containsKey("properties")); + + for (String fieldName : fieldNames) { + Map mappingProperties = (Map) mappingSource.get("properties"); + if (fieldName.indexOf('.') != -1) { + fieldName = fieldName.replace(".", ".properties."); } - }); + assertThat("field " + fieldName + " doesn't exists in mapping " + mappingMetaData.source().string(), XContentMapValues.extractValue(fieldName, mappingProperties), notNullValue()); + } } /** diff --git a/docs/reference/index-modules/merge.asciidoc b/docs/reference/index-modules/merge.asciidoc index f6eb479a979a4..c648a34b8a6db 100644 --- a/docs/reference/index-modules/merge.asciidoc +++ b/docs/reference/index-modules/merge.asciidoc @@ -199,9 +199,3 @@ indexing will see the throttle move higher to allow merges to keep up with ongoing indexing. This is a dynamic setting (you can <>). -[float] -==== SerialMergeScheduler - -This is accepted for backwards compatibility, but just uses -ConcurrentMergeScheduler with index.merge.scheduler.max_thread_count -set to 1 so that only 1 merge may run at a time. diff --git a/docs/reference/query-dsl/bool-query.asciidoc b/docs/reference/query-dsl/bool-query.asciidoc index 9f0c3246c0798..8f2fdb0c99ee6 100644 --- a/docs/reference/query-dsl/bool-query.asciidoc +++ b/docs/reference/query-dsl/bool-query.asciidoc @@ -26,8 +26,12 @@ parameter. documents. |======================================================================= -IMPORTANT: If this query is used in a filter context and it has `should` +[IMPORTANT] +.Bool query in filter context +======================================================================== +If this query is used in a filter context and it has `should` clauses then at least one `should` clause is required to match. +======================================================================== The bool query also supports `disable_coord` parameter (defaults to `false`). Basically the coord similarity computes a score factor based diff --git a/plugins/cloud-azure/src/test/java/org/elasticsearch/discovery/azure/AzureTwoStartedNodesTest.java b/plugins/cloud-azure/src/test/java/org/elasticsearch/discovery/azure/AzureTwoStartedNodesTest.java index 890fa2d15c593..b65be4c69d0f0 100644 --- a/plugins/cloud-azure/src/test/java/org/elasticsearch/discovery/azure/AzureTwoStartedNodesTest.java +++ b/plugins/cloud-azure/src/test/java/org/elasticsearch/discovery/azure/AzureTwoStartedNodesTest.java @@ -41,6 +41,7 @@ public AzureTwoStartedNodesTest() { } @Test + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/11533") public void two_nodes_should_run_using_private_ip() { Settings.Builder settings = Settings.settingsBuilder() .put(Management.SERVICE_NAME, "dummy")