Prechádzať zdrojové kódy

Merge branch 'master' into feature/query-refactoring

javanna 10 rokov pred
rodič
commit
3ac4da5f84
17 zmenil súbory, kde vykonal 177 pridanie a 67 odobranie
  1. 2 1
      core/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java
  2. 1 2
      core/src/main/java/org/elasticsearch/action/termvectors/TransportShardMultiTermsVectorAction.java
  3. 1 1
      core/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java
  4. 9 0
      core/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java
  5. 0 1
      core/src/main/java/org/elasticsearch/common/Strings.java
  6. 50 2
      core/src/main/java/org/elasticsearch/common/path/PathTrie.java
  7. 12 12
      core/src/main/java/org/elasticsearch/index/shard/IndexShard.java
  8. 0 4
      core/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java
  9. 3 4
      core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java
  10. 18 27
      core/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java
  11. 2 0
      core/src/main/java/org/elasticsearch/indices/IndicesModule.java
  12. 0 2
      core/src/main/java/org/elasticsearch/percolator/PercolatorService.java
  13. 22 7
      core/src/test/java/org/elasticsearch/common/path/PathTrieTests.java
  14. 46 2
      core/src/test/java/org/elasticsearch/indices/DateMathIndexExpressionsIntegrationIT.java
  15. 3 1
      core/src/test/java/org/elasticsearch/search/fetch/FetchSubPhasePluginIT.java
  16. 1 1
      docs/reference/indices/put-mapping.asciidoc
  17. 7 0
      rest-api-spec/src/main/resources/rest-api-spec/test/search/80_date_math_index_names.yaml

+ 2 - 1
core/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java

@@ -73,7 +73,8 @@ public class TransportCreateIndexAction extends TransportMasterNodeAction<Create
             cause = "api";
         }
 
-        final CreateIndexClusterStateUpdateRequest updateRequest = new CreateIndexClusterStateUpdateRequest(request, cause, request.index(), request.updateAllTypes())
+        final String indexName = indexNameExpressionResolver.resolveDateMathExpression(request.index());
+        final CreateIndexClusterStateUpdateRequest updateRequest = new CreateIndexClusterStateUpdateRequest(request, cause, indexName, request.updateAllTypes())
                 .ackTimeout(request.timeout()).masterNodeTimeout(request.masterNodeTimeout())
                 .settings(request.settings()).mappings(request.mappings())
                 .aliases(request.aliases()).customs(request.customs());

+ 1 - 2
core/src/main/java/org/elasticsearch/action/termvectors/TransportShardMultiTermsVectorAction.java

@@ -20,7 +20,6 @@
 package org.elasticsearch.action.termvectors;
 
 import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.ExceptionsHelper;
 import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.action.support.TransportActions;
 import org.elasticsearch.action.support.single.shard.TransportSingleShardAction;
@@ -81,7 +80,7 @@ public class TransportShardMultiTermsVectorAction extends TransportSingleShardAc
             try {
                 IndexService indexService = indicesService.indexServiceSafe(request.index());
                 IndexShard indexShard = indexService.shardSafe(shardId.id());
-                TermVectorsResponse termVectorsResponse = indexShard.termVectorsService().getTermVectors(termVectorsRequest, shardId.getIndex());
+                TermVectorsResponse termVectorsResponse = indexShard.getTermVectors(termVectorsRequest);
                 termVectorsResponse.updateTookInMillis(termVectorsRequest.startTime());
                 response.add(request.locations.get(i), termVectorsResponse);
             } catch (Throwable t) {

+ 1 - 1
core/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java

@@ -83,7 +83,7 @@ public class TransportTermVectorsAction extends TransportSingleShardAction<TermV
     protected TermVectorsResponse shardOperation(TermVectorsRequest request, ShardId shardId) {
         IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
         IndexShard indexShard = indexService.shardSafe(shardId.id());
-        TermVectorsResponse response = indexShard.termVectorsService().getTermVectors(request, shardId.getIndex());
+        TermVectorsResponse response = indexShard.getTermVectors(request);
         response.updateTookInMillis(request.startTime());
         return response;
     }

+ 9 - 0
core/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java

@@ -219,6 +219,15 @@ public class IndexNameExpressionResolver extends AbstractComponent {
         return state.metaData().getAliasAndIndexLookup().containsKey(resolvedAliasOrIndex);
     }
 
+    /**
+     * @return If the specified string is data math expression then this method returns the resolved expression.
+     */
+    public String resolveDateMathExpression(String dateExpression) {
+        // The data math expression resolver doesn't rely on cluster state or indices options, because
+        // it just resolves the date math to an actual date.
+        return dateMathExpressionResolver.resolveExpression(dateExpression, new Context(null, null));
+    }
+
     /**
      * Iterates through the list of indices and selects the effective list of filtering aliases for the
      * given index.

+ 0 - 1
core/src/main/java/org/elasticsearch/common/Strings.java

@@ -556,7 +556,6 @@ public class Strings {
                 count++;
             }
         }
-        // TODO (MvG): No push: hppc or jcf?
         final Set<String> result = new HashSet<>(count);
         final int len = chars.length;
         int start = 0;  // starting index in chars of the current substring.

+ 50 - 2
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<T> {
 
         private void put(Map<String, String> params, TrieNode<T> 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<T> {
         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<T> {
         }
         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<String> 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()]);
+    }
 }

+ 12 - 12
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();
     }

+ 0 - 4
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<IndexSearcherWrapper> multibinder

+ 3 - 4
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);

+ 18 - 27
core/src/main/java/org/elasticsearch/index/termvectors/ShardTermVectorsService.java → 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<String> 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<String> selectedFields) throws IOException {
+    private Fields addGeneratedTermVectors(IndexShard indexShard, Engine.GetResult get, Fields termVectorsByField, TermVectorsRequest request, Set<String> selectedFields) throws IOException {
         /* only keep valid fields */
         Set<String> 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<String, String> perFieldAnalyzer) {
+    private Analyzer getAnalyzerAtField(IndexShard indexShard, String field, @Nullable Map<String, String> 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<GetField> getFields, boolean withOffsets, @Nullable Map<String, String> perFieldAnalyzer, Set<String> fields)
+    private Fields generateTermVectors(IndexShard indexShard, Collection<GetField> getFields, boolean withOffsets, @Nullable Map<String, String> perFieldAnalyzer, Set<String> 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

+ 2 - 0
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() {

+ 0 - 2
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;

+ 22 - 7
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<String> 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<String> trie = new PathTrie<>();
         trie.insert("/", "walla");
         assertThat(trie.retrieve(""), equalTo("walla"));
     }
 
-    @Test
     public void testDifferentNamesOnDifferentPath() {
         PathTrie<String> 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<String> 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<String> 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<String> 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<String> 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<String> 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/<c/d>"), arrayContaining("a", "b", "<c/d>"));
+        assertThat(trie.splitPath("/a/b/<c/d>/d"), arrayContaining("a", "b", "<c/d>", "d"));
+
+        assertThat(trie.splitPath("/<logstash-{now}>/_search"), arrayContaining("<logstash-{now}>", "_search"));
+        assertThat(trie.splitPath("/<logstash-{now/d}>/_search"), arrayContaining("<logstash-{now/d}>", "_search"));
+        assertThat(trie.splitPath("/<logstash-{now/M{YYYY.MM}}>/_search"), arrayContaining("<logstash-{now/M{YYYY.MM}}>", "_search"));
+        assertThat(trie.splitPath("/<logstash-{now/M{YYYY.MM}}>/_search"), arrayContaining("<logstash-{now/M{YYYY.MM}}>", "_search"));
+        assertThat(trie.splitPath("/<logstash-{now/M{YYYY.MM|UTC}}>/log/_search"), arrayContaining("<logstash-{now/M{YYYY.MM|UTC}}>", "log", "_search"));
+
+        assertThat(trie.splitPath("/<logstash-{now/M}>,<logstash-{now/M-1M}>/_search"), arrayContaining("<logstash-{now/M}>,<logstash-{now/M-1M}>", "_search"));
+        assertThat(trie.splitPath("/<logstash-{now/M}>,<logstash-{now/M-1M}>/_search"), arrayContaining("<logstash-{now/M}>,<logstash-{now/M-1M}>", "_search"));
+        assertThat(trie.splitPath("/<logstash-{now/M{YYYY.MM}}>,<logstash-{now/M-1M{YYYY.MM}}>/_search"), arrayContaining("<logstash-{now/M{YYYY.MM}}>,<logstash-{now/M-1M{YYYY.MM}}>", "_search"));
+        assertThat(trie.splitPath("/<logstash-{now/M{YYYY.MM|UTC}}>,<logstash-{now/M-1M{YYYY.MM|UTC}}>/_search"), arrayContaining("<logstash-{now/M{YYYY.MM|UTC}}>,<logstash-{now/M-1M{YYYY.MM|UTC}}>", "_search"));
+    }
+
 }

+ 46 - 2
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());
+    }
+
 }

+ 3 - 1
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<String, Integer> tv = new HashMap<>();
                 TermsEnum terms = termVector.getFields().terms(field).iterator();

+ 1 - 1
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`.
 
 

+ 7 - 0
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: <logstash-{now/M}>