diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java index cfa3a435065..98a002cc2fb 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java @@ -73,7 +73,8 @@ public class TransportCreateIndexAction extends TransportMasterNodeAction result = new HashSet<>(count); final int len = chars.length; int start = 0; // starting index in chars of the current substring. diff --git a/core/src/main/java/org/elasticsearch/common/path/PathTrie.java b/core/src/main/java/org/elasticsearch/common/path/PathTrie.java index 0cc1d09c997..3bf2a9b17ee 100644 --- a/core/src/main/java/org/elasticsearch/common/path/PathTrie.java +++ b/core/src/main/java/org/elasticsearch/common/path/PathTrie.java @@ -22,6 +22,8 @@ package org.elasticsearch.common.path; import com.google.common.collect.ImmutableMap; import org.elasticsearch.common.Strings; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder; @@ -195,7 +197,7 @@ public class PathTrie { private void put(Map params, TrieNode node, String value) { if (params != null && node.isNamedWildcard()) { - params.put(node.namedWildcard(), decoder.decode(value)); + params.put(node.namedWildcard(), value); } } } @@ -222,7 +224,7 @@ public class PathTrie { if (path.length() == 0) { return rootValue; } - String[] strings = Strings.splitStringToArray(path, separator); + String[] strings = splitPath(decoder.decode(path)); if (strings.length == 0) { return rootValue; } @@ -233,4 +235,50 @@ public class PathTrie { } return root.retrieve(strings, index, params); } + + /* + Splits up the url path up by '/' and is aware of + index name expressions that appear between '<' and '>'. + */ + String[] splitPath(final String path) { + if (path == null || path.length() == 0) { + return Strings.EMPTY_ARRAY; + } + int count = 1; + boolean splitAllowed = true; + for (int i = 0; i < path.length(); i++) { + final char currentC = path.charAt(i); + if ('<' == currentC) { + splitAllowed = false; + } else if (currentC == '>') { + splitAllowed = true; + } else if (splitAllowed && currentC == separator) { + count++; + } + } + + final List result = new ArrayList<>(count); + final StringBuilder builder = new StringBuilder(); + + splitAllowed = true; + for (int i = 0; i < path.length(); i++) { + final char currentC = path.charAt(i); + if ('<' == currentC) { + splitAllowed = false; + } else if (currentC == '>') { + splitAllowed = true; + } else if (splitAllowed && currentC == separator) { + if (builder.length() > 0) { + result.add(builder.toString()); + builder.setLength(0); + } + continue; + } + builder.append(currentC); + } + if (builder.length() > 0) { + result.add(builder.toString()); + } + return result.toArray(new String[result.size()]); + } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 137226d4dbf..683beacfa51 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -32,6 +32,8 @@ import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.optimize.OptimizeRequest; import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest; +import org.elasticsearch.action.termvectors.TermVectorsRequest; +import org.elasticsearch.action.termvectors.TermVectorsResponse; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -92,7 +94,7 @@ import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.store.StoreStats; import org.elasticsearch.index.suggest.stats.ShardSuggestMetric; import org.elasticsearch.index.suggest.stats.SuggestStats; -import org.elasticsearch.index.termvectors.ShardTermVectorsService; +import org.elasticsearch.index.termvectors.TermVectorsService; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogStats; @@ -118,8 +120,6 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; public class IndexShard extends AbstractIndexShardComponent { @@ -140,7 +140,7 @@ public class IndexShard extends AbstractIndexShardComponent { private final ShardFieldData shardFieldData; private final PercolatorQueriesRegistry percolatorQueriesRegistry; private final ShardPercolateService shardPercolateService; - private final ShardTermVectorsService termVectorsService; + private final TermVectorsService termVectorsService; private final IndexFieldDataService indexFieldDataService; private final IndexService indexService; private final ShardSuggestMetric shardSuggestMetric = new ShardSuggestMetric(); @@ -204,8 +204,8 @@ public class IndexShard extends AbstractIndexShardComponent { @Inject public IndexShard(ShardId shardId, IndexSettingsService indexSettingsService, IndicesLifecycle indicesLifecycle, Store store, StoreRecoveryService storeRecoveryService, ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, - IndicesQueryCache indicesQueryCache, ShardPercolateService shardPercolateService, CodecService codecService, - ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, + IndicesQueryCache indicesQueryCache, CodecService codecService, + TermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, EngineFactory factory, ClusterService clusterService, ShardPath path, BigArrays bigArrays, IndexSearcherWrappingService wrappingService) { super(shardId, indexSettingsService.getSettings()); @@ -229,14 +229,14 @@ public class IndexShard extends AbstractIndexShardComponent { this.indexAliasesService = indexAliasesService; this.indexingService = new ShardIndexingService(shardId, indexSettings); this.getService = new ShardGetService(this, mapperService); - this.termVectorsService = termVectorsService.setIndexShard(this); + this.termVectorsService = termVectorsService; this.searchService = new ShardSearchStats(indexSettings); this.shardWarmerService = new ShardIndexWarmerService(shardId, indexSettings); this.indicesQueryCache = indicesQueryCache; this.shardQueryCache = new ShardRequestCache(shardId, indexSettings); this.shardFieldData = new ShardFieldData(); + this.shardPercolateService = new ShardPercolateService(shardId, indexSettings); this.percolatorQueriesRegistry = new PercolatorQueriesRegistry(shardId, indexSettings, queryParserService, indexingService, indicesLifecycle, mapperService, indexFieldDataService, shardPercolateService); - this.shardPercolateService = shardPercolateService; this.indexFieldDataService = indexFieldDataService; this.indexService = indexService; this.shardBitsetFilterCache = new ShardBitsetFilterCache(shardId, indexSettings); @@ -287,10 +287,6 @@ public class IndexShard extends AbstractIndexShardComponent { return this.getService; } - public ShardTermVectorsService termVectorsService() { - return termVectorsService; - } - public ShardSuggestMetric getSuggestMetric() { return shardSuggestMetric; } @@ -639,6 +635,10 @@ public class IndexShard extends AbstractIndexShardComponent { return segmentsStats; } + public TermVectorsResponse getTermVectors(TermVectorsRequest request) { + return this.termVectorsService.getTermVectors(this, request); + } + public WarmerStats warmerStats() { return shardWarmerService.stats(); } 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 08cf9b27ab8..cd6d3a87337 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java @@ -27,8 +27,6 @@ import org.elasticsearch.index.engine.IndexSearcherWrapper; import org.elasticsearch.index.engine.IndexSearcherWrappingService; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory; -import org.elasticsearch.index.percolator.stats.ShardPercolateService; -import org.elasticsearch.index.termvectors.ShardTermVectorsService; /** * The {@code IndexShardModule} module is responsible for binding the correct @@ -69,8 +67,6 @@ public class IndexShardModule extends AbstractModule { bind(EngineFactory.class).to(engineFactoryImpl); bind(StoreRecoveryService.class).asEagerSingleton(); - bind(ShardPercolateService.class).asEagerSingleton(); - bind(ShardTermVectorsService.class).asEagerSingleton(); bind(IndexSearcherWrappingService.class).asEagerSingleton(); // this injects an empty set in IndexSearcherWrappingService, otherwise guice can't construct IndexSearcherWrappingService Multibinder multibinder diff --git a/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java index 502724e461c..6c45331f826 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java @@ -40,7 +40,7 @@ import org.elasticsearch.index.query.IndexQueryParserService; import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.Store; -import org.elasticsearch.index.termvectors.ShardTermVectorsService; +import org.elasticsearch.index.termvectors.TermVectorsService; import org.elasticsearch.indices.IndicesLifecycle; import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.cache.query.IndicesQueryCache; @@ -62,15 +62,14 @@ public final class ShadowIndexShard extends IndexShard { ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, IndicesQueryCache indicesQueryCache, - ShardPercolateService shardPercolateService, CodecService codecService, - ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, + CodecService codecService, TermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, EngineFactory factory, ClusterService clusterService, ShardPath path, BigArrays bigArrays, IndexSearcherWrappingService wrappingService) throws IOException { super(shardId, indexSettingsService, indicesLifecycle, store, storeRecoveryService, threadPool, mapperService, queryParserService, indexCache, indexAliasesService, - indicesQueryCache, shardPercolateService, codecService, + indicesQueryCache, codecService, termVectorsService, indexFieldDataService, indexService, warmer, deletionPolicy, similarityService, factory, clusterService, path, bigArrays, wrappingService); diff --git a/core/src/main/java/org/elasticsearch/index/termvectors/ShardTermVectorsService.java b/core/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java similarity index 86% rename from core/src/main/java/org/elasticsearch/index/termvectors/ShardTermVectorsService.java rename to core/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java index cb34c1167d5..5b27d327806 100644 --- a/core/src/main/java/org/elasticsearch/index/termvectors/ShardTermVectorsService.java +++ b/core/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java @@ -33,6 +33,7 @@ import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; @@ -42,10 +43,7 @@ import org.elasticsearch.index.get.GetResult; import org.elasticsearch.index.mapper.*; import org.elasticsearch.index.mapper.core.StringFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper; -import org.elasticsearch.index.settings.IndexSettings; -import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.dfs.AggregatedDfs; import java.io.IOException; @@ -56,27 +54,20 @@ import static org.elasticsearch.index.mapper.SourceToParse.source; /** */ -public class ShardTermVectorsService extends AbstractIndexShardComponent { +public class TermVectorsService { - private IndexShard indexShard; private final MappingUpdatedAction mappingUpdatedAction; private final TransportDfsOnlyAction dfsAction; @Inject - public ShardTermVectorsService(ShardId shardId, @IndexSettings Settings indexSettings, MappingUpdatedAction mappingUpdatedAction, TransportDfsOnlyAction dfsAction) { - super(shardId, indexSettings); + public TermVectorsService(MappingUpdatedAction mappingUpdatedAction, TransportDfsOnlyAction dfsAction) { this.mappingUpdatedAction = mappingUpdatedAction; this.dfsAction = dfsAction; } - // sadly, to overcome cyclic dep, we need to do this and inject it ourselves... - public ShardTermVectorsService setIndexShard(IndexShard indexShard) { - this.indexShard = indexShard; - return this; - } - public TermVectorsResponse getTermVectors(TermVectorsRequest request, String concreteIndex) { - final TermVectorsResponse termVectorsResponse = new TermVectorsResponse(concreteIndex, request.type(), request.id()); + public TermVectorsResponse getTermVectors(IndexShard indexShard, TermVectorsRequest request) { + final TermVectorsResponse termVectorsResponse = new TermVectorsResponse(indexShard.shardId().index().name(), request.type(), request.id()); final Term uidTerm = new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(request.type(), request.id())); Engine.GetResult get = indexShard.get(new Engine.Get(request.realtime(), uidTerm).version(request.version()).versionType(request.versionType())); @@ -94,7 +85,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent { /* handle potential wildcards in fields */ if (request.selectedFields() != null) { - handleFieldWildcards(request); + handleFieldWildcards(indexShard, request); } final Engine.Searcher searcher = indexShard.acquireSearcher("term_vector"); @@ -103,7 +94,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent { Versions.DocIdAndVersion docIdAndVersion = get.docIdAndVersion(); /* from an artificial document */ if (request.doc() != null) { - termVectorsByField = generateTermVectorsFromDoc(request, !docFromTranslog); + termVectorsByField = generateTermVectorsFromDoc(indexShard, request, !docFromTranslog); // if no document indexed in shard, take the queried document itself for stats if (topLevelFields == null) { topLevelFields = termVectorsByField; @@ -122,7 +113,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent { } // fields without term vectors if (selectedFields != null) { - termVectorsByField = addGeneratedTermVectors(get, termVectorsByField, request, selectedFields); + termVectorsByField = addGeneratedTermVectors(indexShard, get, termVectorsByField, request, selectedFields); } termVectorsResponse.setDocVersion(docIdAndVersion.version); termVectorsResponse.setExists(true); @@ -158,7 +149,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent { return termVectorsResponse; } - private void handleFieldWildcards(TermVectorsRequest request) { + private void handleFieldWildcards(IndexShard indexShard, TermVectorsRequest request) { Set fieldNames = new HashSet<>(); for (String pattern : request.selectedFields()) { fieldNames.addAll(indexShard.mapperService().simpleMatchToIndexNames(pattern)); @@ -178,7 +169,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent { return true; } - private Fields addGeneratedTermVectors(Engine.GetResult get, Fields termVectorsByField, TermVectorsRequest request, Set selectedFields) throws IOException { + private Fields addGeneratedTermVectors(IndexShard indexShard, Engine.GetResult get, Fields termVectorsByField, TermVectorsRequest request, Set selectedFields) throws IOException { /* only keep valid fields */ Set validFields = new HashSet<>(); for (String field : selectedFields) { @@ -201,7 +192,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent { /* generate term vectors from fetched document fields */ GetResult getResult = indexShard.getService().get( get, request.id(), request.type(), validFields.toArray(Strings.EMPTY_ARRAY), null, false); - Fields generatedTermVectors = generateTermVectors(getResult.getFields().values(), request.offsets(), request.perFieldAnalyzer(), validFields); + Fields generatedTermVectors = generateTermVectors(indexShard, getResult.getFields().values(), request.offsets(), request.perFieldAnalyzer(), validFields); /* merge with existing Fields */ if (termVectorsByField == null) { @@ -211,7 +202,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent { } } - private Analyzer getAnalyzerAtField(String field, @Nullable Map perFieldAnalyzer) { + private Analyzer getAnalyzerAtField(IndexShard indexShard, String field, @Nullable Map perFieldAnalyzer) { MapperService mapperService = indexShard.mapperService(); Analyzer analyzer; if (perFieldAnalyzer != null && perFieldAnalyzer.containsKey(field)) { @@ -235,7 +226,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent { return selectedFields; } - private Fields generateTermVectors(Collection getFields, boolean withOffsets, @Nullable Map perFieldAnalyzer, Set fields) + private Fields generateTermVectors(IndexShard indexShard, Collection getFields, boolean withOffsets, @Nullable Map perFieldAnalyzer, Set fields) throws IOException { /* store document in memory index */ MemoryIndex index = new MemoryIndex(withOffsets); @@ -245,7 +236,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent { // some fields are returned even when not asked for, eg. _timestamp continue; } - Analyzer analyzer = getAnalyzerAtField(field, perFieldAnalyzer); + Analyzer analyzer = getAnalyzerAtField(indexShard, field, perFieldAnalyzer); for (Object text : getField.getValues()) { index.addField(field, text.toString(), analyzer); } @@ -254,9 +245,9 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent { return MultiFields.getFields(index.createSearcher().getIndexReader()); } - private Fields generateTermVectorsFromDoc(TermVectorsRequest request, boolean doAllFields) throws Throwable { + private Fields generateTermVectorsFromDoc(IndexShard indexShard, TermVectorsRequest request, boolean doAllFields) throws Throwable { // parse the document, at the moment we do update the mapping, just like percolate - ParsedDocument parsedDocument = parseDocument(indexShard.shardId().getIndex(), request.type(), request.doc()); + ParsedDocument parsedDocument = parseDocument(indexShard, indexShard.shardId().getIndex(), request.type(), request.doc()); // select the right fields and generate term vectors ParseContext.Document doc = parsedDocument.rootDoc(); @@ -282,10 +273,10 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent { String[] values = doc.getValues(field.name()); getFields.add(new GetField(field.name(), Arrays.asList((Object[]) values))); } - return generateTermVectors(getFields, request.offsets(), request.perFieldAnalyzer(), seenFields); + return generateTermVectors(indexShard, getFields, request.offsets(), request.perFieldAnalyzer(), seenFields); } - private ParsedDocument parseDocument(String index, String type, BytesReference doc) throws Throwable { + private ParsedDocument parseDocument(IndexShard indexShard, String index, String type, BytesReference doc) throws Throwable { MapperService mapperService = indexShard.mapperService(); // TODO: make parsing not dynamically create fields not in the original mapping diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesModule.java b/core/src/main/java/org/elasticsearch/indices/IndicesModule.java index ff248fc034b..ff9bd334d9f 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesModule.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesModule.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.util.ExtensionPoint; import org.elasticsearch.index.query.*; import org.elasticsearch.index.query.functionscore.FunctionScoreQueryParser; import org.elasticsearch.index.query.MoreLikeThisQueryParser; +import org.elasticsearch.index.termvectors.TermVectorsService; import org.elasticsearch.indices.analysis.HunspellService; import org.elasticsearch.indices.analysis.IndicesAnalysisService; import org.elasticsearch.indices.cache.query.IndicesQueryCache; @@ -148,6 +149,7 @@ public class IndicesModule extends AbstractModule { bind(UpdateHelper.class).asEagerSingleton(); bind(MetaDataIndexUpgradeService.class).asEagerSingleton(); bind(IndicesFieldDataCacheListener.class).asEagerSingleton(); + bind(TermVectorsService.class).asEagerSingleton(); } protected void bindQueryParsersExtension() { diff --git a/core/src/main/java/org/elasticsearch/percolator/PercolatorService.java b/core/src/main/java/org/elasticsearch/percolator/PercolatorService.java index e8212e931bd..ba4ccaeb25e 100644 --- a/core/src/main/java/org/elasticsearch/percolator/PercolatorService.java +++ b/core/src/main/java/org/elasticsearch/percolator/PercolatorService.java @@ -65,11 +65,9 @@ import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.SortedBinaryDocValues; -import org.elasticsearch.index.mapper.*; import org.elasticsearch.index.mapper.DocumentMapperForType; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.mapper.internal.UidFieldMapper; diff --git a/core/src/test/java/org/elasticsearch/common/path/PathTrieTests.java b/core/src/test/java/org/elasticsearch/common/path/PathTrieTests.java index d3d6fd3495c..aec4fb24888 100644 --- a/core/src/test/java/org/elasticsearch/common/path/PathTrieTests.java +++ b/core/src/test/java/org/elasticsearch/common/path/PathTrieTests.java @@ -25,6 +25,7 @@ import org.junit.Test; import java.util.HashMap; import java.util.Map; +import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; @@ -33,7 +34,6 @@ import static org.hamcrest.Matchers.nullValue; */ public class PathTrieTests extends ESTestCase { - @Test public void testPath() { PathTrie trie = new PathTrie<>(); trie.insert("/a/b/c", "walla"); @@ -61,14 +61,12 @@ public class PathTrieTests extends ESTestCase { assertThat(params.get("docId"), equalTo("12")); } - @Test public void testEmptyPath() { PathTrie trie = new PathTrie<>(); trie.insert("/", "walla"); assertThat(trie.retrieve(""), equalTo("walla")); } - @Test public void testDifferentNamesOnDifferentPath() { PathTrie trie = new PathTrie<>(); trie.insert("/a/{type}", "test1"); @@ -83,7 +81,6 @@ public class PathTrieTests extends ESTestCase { assertThat(params.get("name"), equalTo("testX")); } - @Test public void testSameNameOnDifferentPath() { PathTrie trie = new PathTrie<>(); trie.insert("/a/c/{name}", "test1"); @@ -98,7 +95,6 @@ public class PathTrieTests extends ESTestCase { assertThat(params.get("name"), equalTo("testX")); } - @Test public void testPreferNonWildcardExecution() { PathTrie trie = new PathTrie<>(); trie.insert("{test}", "test1"); @@ -115,7 +111,6 @@ public class PathTrieTests extends ESTestCase { assertThat(trie.retrieve("/v/x/c", params), equalTo("test6")); } - @Test public void testSamePathConcreteResolution() { PathTrie trie = new PathTrie<>(); trie.insert("{x}/{y}/{z}", "test1"); @@ -132,7 +127,6 @@ public class PathTrieTests extends ESTestCase { assertThat(params.get("k"), equalTo("c")); } - @Test public void testNamedWildcardAndLookupWithWildcard() { PathTrie trie = new PathTrie<>(); trie.insert("x/{test}", "test1"); @@ -161,4 +155,25 @@ public class PathTrieTests extends ESTestCase { assertThat(trie.retrieve("a/*/_endpoint", params), equalTo("test5")); assertThat(params.get("test"), equalTo("*")); } + + public void testSplitPath() { + PathTrie trie = new PathTrie<>(); + assertThat(trie.splitPath("/a/"), arrayContaining("a")); + assertThat(trie.splitPath("/a/b"),arrayContaining("a", "b")); + assertThat(trie.splitPath("/a/b/c"), arrayContaining("a", "b", "c")); + assertThat(trie.splitPath("/a/b/"), arrayContaining("a", "b", "")); + assertThat(trie.splitPath("/a/b//d"), arrayContaining("a", "b", "", "d")); + + assertThat(trie.splitPath("//_search"), arrayContaining("", "_search")); + assertThat(trie.splitPath("//_search"), arrayContaining("", "_search")); + assertThat(trie.splitPath("//_search"), arrayContaining("", "_search")); + assertThat(trie.splitPath("//_search"), arrayContaining("", "_search")); + assertThat(trie.splitPath("//log/_search"), arrayContaining("", "log", "_search")); + + assertThat(trie.splitPath("/,/_search"), arrayContaining(",", "_search")); + assertThat(trie.splitPath("/,/_search"), arrayContaining(",", "_search")); + assertThat(trie.splitPath("/,/_search"), arrayContaining(",", "_search")); + assertThat(trie.splitPath("/,/_search"), arrayContaining(",", "_search")); + } + } diff --git a/core/src/test/java/org/elasticsearch/indices/DateMathIndexExpressionsIntegrationIT.java b/core/src/test/java/org/elasticsearch/indices/DateMathIndexExpressionsIntegrationIT.java index 6868137dcda..792f14bce1e 100644 --- a/core/src/test/java/org/elasticsearch/indices/DateMathIndexExpressionsIntegrationIT.java +++ b/core/src/test/java/org/elasticsearch/indices/DateMathIndexExpressionsIntegrationIT.java @@ -23,12 +23,15 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -51,8 +54,8 @@ public class DateMathIndexExpressionsIntegrationIT extends ESIntegTestCase { refresh(); SearchResponse searchResponse = client().prepareSearch(dateMathExp1, dateMathExp2, dateMathExp3).get(); - ElasticsearchAssertions.assertHitCount(searchResponse, 3); - ElasticsearchAssertions.assertSearchHits(searchResponse, "1", "2", "3"); + assertHitCount(searchResponse, 3); + assertSearchHits(searchResponse, "1", "2", "3"); GetResponse getResponse = client().prepareGet(dateMathExp1, "type", "1").get(); assertThat(getResponse.isExists(), is(true)); @@ -84,4 +87,45 @@ public class DateMathIndexExpressionsIntegrationIT extends ESIntegTestCase { assertThat(deleteResponse.getId(), equalTo("3")); } + public void testAutoCreateIndexWithDateMathExpression() throws Exception { + DateTime now = new DateTime(DateTimeZone.UTC); + String index1 = ".marvel-" + DateTimeFormat.forPattern("YYYY.MM.dd").print(now); + String index2 = ".marvel-" + DateTimeFormat.forPattern("YYYY.MM.dd").print(now.minusDays(1)); + String index3 = ".marvel-" + DateTimeFormat.forPattern("YYYY.MM.dd").print(now.minusDays(2)); + + String dateMathExp1 = "<.marvel-{now/d}>"; + String dateMathExp2 = "<.marvel-{now/d-1d}>"; + String dateMathExp3 = "<.marvel-{now/d-2d}>"; + client().prepareIndex(dateMathExp1, "type", "1").setSource("{}").get(); + client().prepareIndex(dateMathExp2, "type", "2").setSource("{}").get(); + client().prepareIndex(dateMathExp3, "type", "3").setSource("{}").get(); + refresh(); + + SearchResponse searchResponse = client().prepareSearch(dateMathExp1, dateMathExp2, dateMathExp3).get(); + assertHitCount(searchResponse, 3); + assertSearchHits(searchResponse, "1", "2", "3"); + + IndicesStatsResponse indicesStatsResponse = client().admin().indices().prepareStats(dateMathExp1, dateMathExp2, dateMathExp3).get(); + assertThat(indicesStatsResponse.getIndex(index1), notNullValue()); + assertThat(indicesStatsResponse.getIndex(index2), notNullValue()); + assertThat(indicesStatsResponse.getIndex(index3), notNullValue()); + } + + public void testCreateIndexWithDateMathExpression() throws Exception { + DateTime now = new DateTime(DateTimeZone.UTC); + String index1 = ".marvel-" + DateTimeFormat.forPattern("YYYY.MM.dd").print(now); + String index2 = ".marvel-" + DateTimeFormat.forPattern("YYYY.MM.dd").print(now.minusDays(1)); + String index3 = ".marvel-" + DateTimeFormat.forPattern("YYYY.MM.dd").print(now.minusDays(2)); + + String dateMathExp1 = "<.marvel-{now/d}>"; + String dateMathExp2 = "<.marvel-{now/d-1d}>"; + String dateMathExp3 = "<.marvel-{now/d-2d}>"; + createIndex(dateMathExp1, dateMathExp2, dateMathExp3); + + ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + assertThat(clusterState.metaData().index(index1), notNullValue()); + assertThat(clusterState.metaData().index(index2), notNullValue()); + assertThat(clusterState.metaData().index(index3), notNullValue()); + } + } diff --git a/core/src/test/java/org/elasticsearch/search/fetch/FetchSubPhasePluginIT.java b/core/src/test/java/org/elasticsearch/search/fetch/FetchSubPhasePluginIT.java index e07bb73d6a6..6280f4af524 100644 --- a/core/src/test/java/org/elasticsearch/search/fetch/FetchSubPhasePluginIT.java +++ b/core/src/test/java/org/elasticsearch/search/fetch/FetchSubPhasePluginIT.java @@ -28,8 +28,10 @@ import org.elasticsearch.action.termvectors.TermVectorsRequest; import org.elasticsearch.action.termvectors.TermVectorsResponse; import org.elasticsearch.common.Priority; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.termvectors.TermVectorsService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.SearchModule; @@ -166,7 +168,7 @@ public class FetchSubPhasePluginIT extends ESIntegTestCase { hitField = new InternalSearchHitField(NAMES[0], new ArrayList<>(1)); hitContext.hit().fields().put(NAMES[0], hitField); } - TermVectorsResponse termVector = context.indexShard().termVectorsService().getTermVectors(new TermVectorsRequest(context.indexShard().indexService().index().getName(), hitContext.hit().type(), hitContext.hit().id()), context.indexShard().indexService().index().getName()); + TermVectorsResponse termVector = context.indexShard().getTermVectors(new TermVectorsRequest(context.indexShard().indexService().index().getName(), hitContext.hit().type(), hitContext.hit().id())); try { Map tv = new HashMap<>(); TermsEnum terms = termVector.getFields().terms(field).iterator(); diff --git a/docs/reference/indices/put-mapping.asciidoc b/docs/reference/indices/put-mapping.asciidoc index 25b6ce33917..7d6e6587786 100644 --- a/docs/reference/indices/put-mapping.asciidoc +++ b/docs/reference/indices/put-mapping.asciidoc @@ -189,7 +189,7 @@ PUT my_index/_mapping/type_one?update_all_types <3> ----------------------------------- // AUTOSENSE <1> Create an index with two types, both of which contain a `text` field which have the same mapping. -<2> Tring to update the `search_analyzer` just for `type_one` throws an exception like `"Merge failed with failures..."`. +<2> Trying to update the `search_analyzer` just for `type_one` throws an exception like `"Merge failed with failures..."`. <3> Adding the `update_all_types` parameter updates the `text` field in `type_one` and `type_two`. diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/80_date_math_index_names.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/80_date_math_index_names.yaml new file mode 100644 index 00000000000..233b41c6cf7 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/80_date_math_index_names.yaml @@ -0,0 +1,7 @@ +--- +"Missing index with catch": + + - do: + catch: /index=logstash-\d{4}\.\d{2}\.\d{2}/ + search: + index: