Merge branch 'master' into feature/query-refactoring

This commit is contained in:
javanna 2015-09-25 13:53:06 +02:00 committed by Luca Cavanna
commit 3ac4da5f84
17 changed files with 177 additions and 67 deletions

View File

@ -73,7 +73,8 @@ public class TransportCreateIndexAction extends TransportMasterNodeAction<Create
cause = "api"; 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()) .ackTimeout(request.timeout()).masterNodeTimeout(request.masterNodeTimeout())
.settings(request.settings()).mappings(request.mappings()) .settings(request.settings()).mappings(request.mappings())
.aliases(request.aliases()).customs(request.customs()); .aliases(request.aliases()).customs(request.customs());

View File

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

View File

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

View File

@ -219,6 +219,15 @@ public class IndexNameExpressionResolver extends AbstractComponent {
return state.metaData().getAliasAndIndexLookup().containsKey(resolvedAliasOrIndex); 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 * Iterates through the list of indices and selects the effective list of filtering aliases for the
* given index. * given index.

View File

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

View File

@ -22,6 +22,8 @@ package org.elasticsearch.common.path;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import java.util.ArrayList;
import java.util.List;
import java.util.Map; import java.util.Map;
import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder; 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) { private void put(Map<String, String> params, TrieNode<T> node, String value) {
if (params != null && node.isNamedWildcard()) { 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) { if (path.length() == 0) {
return rootValue; return rootValue;
} }
String[] strings = Strings.splitStringToArray(path, separator); String[] strings = splitPath(decoder.decode(path));
if (strings.length == 0) { if (strings.length == 0) {
return rootValue; return rootValue;
} }
@ -233,4 +235,50 @@ public class PathTrie<T> {
} }
return root.retrieve(strings, index, params); 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()]);
}
} }

View File

@ -32,6 +32,8 @@ import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.optimize.OptimizeRequest; import org.elasticsearch.action.admin.indices.optimize.OptimizeRequest;
import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest; 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.ClusterService;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; 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.store.StoreStats;
import org.elasticsearch.index.suggest.stats.ShardSuggestMetric; import org.elasticsearch.index.suggest.stats.ShardSuggestMetric;
import org.elasticsearch.index.suggest.stats.SuggestStats; 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.Translog;
import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.index.translog.TranslogStats; import org.elasticsearch.index.translog.TranslogStats;
@ -118,8 +120,6 @@ import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
public class IndexShard extends AbstractIndexShardComponent { public class IndexShard extends AbstractIndexShardComponent {
@ -140,7 +140,7 @@ public class IndexShard extends AbstractIndexShardComponent {
private final ShardFieldData shardFieldData; private final ShardFieldData shardFieldData;
private final PercolatorQueriesRegistry percolatorQueriesRegistry; private final PercolatorQueriesRegistry percolatorQueriesRegistry;
private final ShardPercolateService shardPercolateService; private final ShardPercolateService shardPercolateService;
private final ShardTermVectorsService termVectorsService; private final TermVectorsService termVectorsService;
private final IndexFieldDataService indexFieldDataService; private final IndexFieldDataService indexFieldDataService;
private final IndexService indexService; private final IndexService indexService;
private final ShardSuggestMetric shardSuggestMetric = new ShardSuggestMetric(); private final ShardSuggestMetric shardSuggestMetric = new ShardSuggestMetric();
@ -204,8 +204,8 @@ public class IndexShard extends AbstractIndexShardComponent {
@Inject @Inject
public IndexShard(ShardId shardId, IndexSettingsService indexSettingsService, IndicesLifecycle indicesLifecycle, Store store, StoreRecoveryService storeRecoveryService, public IndexShard(ShardId shardId, IndexSettingsService indexSettingsService, IndicesLifecycle indicesLifecycle, Store store, StoreRecoveryService storeRecoveryService,
ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService,
IndicesQueryCache indicesQueryCache, ShardPercolateService shardPercolateService, CodecService codecService, IndicesQueryCache indicesQueryCache, CodecService codecService,
ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, TermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService,
@Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, EngineFactory factory, @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, EngineFactory factory,
ClusterService clusterService, ShardPath path, BigArrays bigArrays, IndexSearcherWrappingService wrappingService) { ClusterService clusterService, ShardPath path, BigArrays bigArrays, IndexSearcherWrappingService wrappingService) {
super(shardId, indexSettingsService.getSettings()); super(shardId, indexSettingsService.getSettings());
@ -229,14 +229,14 @@ public class IndexShard extends AbstractIndexShardComponent {
this.indexAliasesService = indexAliasesService; this.indexAliasesService = indexAliasesService;
this.indexingService = new ShardIndexingService(shardId, indexSettings); this.indexingService = new ShardIndexingService(shardId, indexSettings);
this.getService = new ShardGetService(this, mapperService); this.getService = new ShardGetService(this, mapperService);
this.termVectorsService = termVectorsService.setIndexShard(this); this.termVectorsService = termVectorsService;
this.searchService = new ShardSearchStats(indexSettings); this.searchService = new ShardSearchStats(indexSettings);
this.shardWarmerService = new ShardIndexWarmerService(shardId, indexSettings); this.shardWarmerService = new ShardIndexWarmerService(shardId, indexSettings);
this.indicesQueryCache = indicesQueryCache; this.indicesQueryCache = indicesQueryCache;
this.shardQueryCache = new ShardRequestCache(shardId, indexSettings); this.shardQueryCache = new ShardRequestCache(shardId, indexSettings);
this.shardFieldData = new ShardFieldData(); this.shardFieldData = new ShardFieldData();
this.shardPercolateService = new ShardPercolateService(shardId, indexSettings);
this.percolatorQueriesRegistry = new PercolatorQueriesRegistry(shardId, indexSettings, queryParserService, indexingService, indicesLifecycle, mapperService, indexFieldDataService, shardPercolateService); this.percolatorQueriesRegistry = new PercolatorQueriesRegistry(shardId, indexSettings, queryParserService, indexingService, indicesLifecycle, mapperService, indexFieldDataService, shardPercolateService);
this.shardPercolateService = shardPercolateService;
this.indexFieldDataService = indexFieldDataService; this.indexFieldDataService = indexFieldDataService;
this.indexService = indexService; this.indexService = indexService;
this.shardBitsetFilterCache = new ShardBitsetFilterCache(shardId, indexSettings); this.shardBitsetFilterCache = new ShardBitsetFilterCache(shardId, indexSettings);
@ -287,10 +287,6 @@ public class IndexShard extends AbstractIndexShardComponent {
return this.getService; return this.getService;
} }
public ShardTermVectorsService termVectorsService() {
return termVectorsService;
}
public ShardSuggestMetric getSuggestMetric() { public ShardSuggestMetric getSuggestMetric() {
return shardSuggestMetric; return shardSuggestMetric;
} }
@ -639,6 +635,10 @@ public class IndexShard extends AbstractIndexShardComponent {
return segmentsStats; return segmentsStats;
} }
public TermVectorsResponse getTermVectors(TermVectorsRequest request) {
return this.termVectorsService.getTermVectors(this, request);
}
public WarmerStats warmerStats() { public WarmerStats warmerStats() {
return shardWarmerService.stats(); return shardWarmerService.stats();
} }

View File

@ -27,8 +27,6 @@ import org.elasticsearch.index.engine.IndexSearcherWrapper;
import org.elasticsearch.index.engine.IndexSearcherWrappingService; import org.elasticsearch.index.engine.IndexSearcherWrappingService;
import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.engine.InternalEngineFactory; 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 * 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(EngineFactory.class).to(engineFactoryImpl);
bind(StoreRecoveryService.class).asEagerSingleton(); bind(StoreRecoveryService.class).asEagerSingleton();
bind(ShardPercolateService.class).asEagerSingleton();
bind(ShardTermVectorsService.class).asEagerSingleton();
bind(IndexSearcherWrappingService.class).asEagerSingleton(); bind(IndexSearcherWrappingService.class).asEagerSingleton();
// this injects an empty set in IndexSearcherWrappingService, otherwise guice can't construct IndexSearcherWrappingService // this injects an empty set in IndexSearcherWrappingService, otherwise guice can't construct IndexSearcherWrappingService
Multibinder<IndexSearcherWrapper> multibinder Multibinder<IndexSearcherWrapper> multibinder

View File

@ -40,7 +40,7 @@ import org.elasticsearch.index.query.IndexQueryParserService;
import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.Store; 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.IndicesLifecycle;
import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.cache.query.IndicesQueryCache;
@ -62,15 +62,14 @@ public final class ShadowIndexShard extends IndexShard {
ThreadPool threadPool, MapperService mapperService, ThreadPool threadPool, MapperService mapperService,
IndexQueryParserService queryParserService, IndexCache indexCache, IndexQueryParserService queryParserService, IndexCache indexCache,
IndexAliasesService indexAliasesService, IndicesQueryCache indicesQueryCache, IndexAliasesService indexAliasesService, IndicesQueryCache indicesQueryCache,
ShardPercolateService shardPercolateService, CodecService codecService, CodecService codecService, TermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService,
ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService,
IndexService indexService, @Nullable IndicesWarmer warmer, IndexService indexService, @Nullable IndicesWarmer warmer,
SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService,
EngineFactory factory, ClusterService clusterService, EngineFactory factory, ClusterService clusterService,
ShardPath path, BigArrays bigArrays, IndexSearcherWrappingService wrappingService) throws IOException { ShardPath path, BigArrays bigArrays, IndexSearcherWrappingService wrappingService) throws IOException {
super(shardId, indexSettingsService, indicesLifecycle, store, storeRecoveryService, super(shardId, indexSettingsService, indicesLifecycle, store, storeRecoveryService,
threadPool, mapperService, queryParserService, indexCache, indexAliasesService, threadPool, mapperService, queryParserService, indexCache, indexAliasesService,
indicesQueryCache, shardPercolateService, codecService, indicesQueryCache, codecService,
termVectorsService, indexFieldDataService, indexService, termVectorsService, indexFieldDataService, indexService,
warmer, deletionPolicy, similarityService, warmer, deletionPolicy, similarityService,
factory, clusterService, path, bigArrays, wrappingService); factory, clusterService, path, bigArrays, wrappingService);

View File

@ -33,6 +33,7 @@ import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.settings.Settings; 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.*;
import org.elasticsearch.index.mapper.core.StringFieldMapper; import org.elasticsearch.index.mapper.core.StringFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper; 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.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.dfs.AggregatedDfs; import org.elasticsearch.search.dfs.AggregatedDfs;
import java.io.IOException; 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 MappingUpdatedAction mappingUpdatedAction;
private final TransportDfsOnlyAction dfsAction; private final TransportDfsOnlyAction dfsAction;
@Inject @Inject
public ShardTermVectorsService(ShardId shardId, @IndexSettings Settings indexSettings, MappingUpdatedAction mappingUpdatedAction, TransportDfsOnlyAction dfsAction) { public TermVectorsService(MappingUpdatedAction mappingUpdatedAction, TransportDfsOnlyAction dfsAction) {
super(shardId, indexSettings);
this.mappingUpdatedAction = mappingUpdatedAction; this.mappingUpdatedAction = mappingUpdatedAction;
this.dfsAction = dfsAction; 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) { public TermVectorsResponse getTermVectors(IndexShard indexShard, TermVectorsRequest request) {
final TermVectorsResponse termVectorsResponse = new TermVectorsResponse(concreteIndex, request.type(), request.id()); 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())); 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())); 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 */ /* handle potential wildcards in fields */
if (request.selectedFields() != null) { if (request.selectedFields() != null) {
handleFieldWildcards(request); handleFieldWildcards(indexShard, request);
} }
final Engine.Searcher searcher = indexShard.acquireSearcher("term_vector"); final Engine.Searcher searcher = indexShard.acquireSearcher("term_vector");
@ -103,7 +94,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
Versions.DocIdAndVersion docIdAndVersion = get.docIdAndVersion(); Versions.DocIdAndVersion docIdAndVersion = get.docIdAndVersion();
/* from an artificial document */ /* from an artificial document */
if (request.doc() != null) { 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 no document indexed in shard, take the queried document itself for stats
if (topLevelFields == null) { if (topLevelFields == null) {
topLevelFields = termVectorsByField; topLevelFields = termVectorsByField;
@ -122,7 +113,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
} }
// fields without term vectors // fields without term vectors
if (selectedFields != null) { if (selectedFields != null) {
termVectorsByField = addGeneratedTermVectors(get, termVectorsByField, request, selectedFields); termVectorsByField = addGeneratedTermVectors(indexShard, get, termVectorsByField, request, selectedFields);
} }
termVectorsResponse.setDocVersion(docIdAndVersion.version); termVectorsResponse.setDocVersion(docIdAndVersion.version);
termVectorsResponse.setExists(true); termVectorsResponse.setExists(true);
@ -158,7 +149,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
return termVectorsResponse; return termVectorsResponse;
} }
private void handleFieldWildcards(TermVectorsRequest request) { private void handleFieldWildcards(IndexShard indexShard, TermVectorsRequest request) {
Set<String> fieldNames = new HashSet<>(); Set<String> fieldNames = new HashSet<>();
for (String pattern : request.selectedFields()) { for (String pattern : request.selectedFields()) {
fieldNames.addAll(indexShard.mapperService().simpleMatchToIndexNames(pattern)); fieldNames.addAll(indexShard.mapperService().simpleMatchToIndexNames(pattern));
@ -178,7 +169,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
return true; 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 */ /* only keep valid fields */
Set<String> validFields = new HashSet<>(); Set<String> validFields = new HashSet<>();
for (String field : selectedFields) { for (String field : selectedFields) {
@ -201,7 +192,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
/* generate term vectors from fetched document fields */ /* generate term vectors from fetched document fields */
GetResult getResult = indexShard.getService().get( GetResult getResult = indexShard.getService().get(
get, request.id(), request.type(), validFields.toArray(Strings.EMPTY_ARRAY), null, false); 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 */ /* merge with existing Fields */
if (termVectorsByField == null) { 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(); MapperService mapperService = indexShard.mapperService();
Analyzer analyzer; Analyzer analyzer;
if (perFieldAnalyzer != null && perFieldAnalyzer.containsKey(field)) { if (perFieldAnalyzer != null && perFieldAnalyzer.containsKey(field)) {
@ -235,7 +226,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
return selectedFields; 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 { throws IOException {
/* store document in memory index */ /* store document in memory index */
MemoryIndex index = new MemoryIndex(withOffsets); 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 // some fields are returned even when not asked for, eg. _timestamp
continue; continue;
} }
Analyzer analyzer = getAnalyzerAtField(field, perFieldAnalyzer); Analyzer analyzer = getAnalyzerAtField(indexShard, field, perFieldAnalyzer);
for (Object text : getField.getValues()) { for (Object text : getField.getValues()) {
index.addField(field, text.toString(), analyzer); index.addField(field, text.toString(), analyzer);
} }
@ -254,9 +245,9 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
return MultiFields.getFields(index.createSearcher().getIndexReader()); 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 // 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 // select the right fields and generate term vectors
ParseContext.Document doc = parsedDocument.rootDoc(); ParseContext.Document doc = parsedDocument.rootDoc();
@ -282,10 +273,10 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
String[] values = doc.getValues(field.name()); String[] values = doc.getValues(field.name());
getFields.add(new GetField(field.name(), Arrays.asList((Object[]) values))); 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(); MapperService mapperService = indexShard.mapperService();
// TODO: make parsing not dynamically create fields not in the original mapping // TODO: make parsing not dynamically create fields not in the original mapping

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.util.ExtensionPoint;
import org.elasticsearch.index.query.*; import org.elasticsearch.index.query.*;
import org.elasticsearch.index.query.functionscore.FunctionScoreQueryParser; import org.elasticsearch.index.query.functionscore.FunctionScoreQueryParser;
import org.elasticsearch.index.query.MoreLikeThisQueryParser; import org.elasticsearch.index.query.MoreLikeThisQueryParser;
import org.elasticsearch.index.termvectors.TermVectorsService;
import org.elasticsearch.indices.analysis.HunspellService; import org.elasticsearch.indices.analysis.HunspellService;
import org.elasticsearch.indices.analysis.IndicesAnalysisService; import org.elasticsearch.indices.analysis.IndicesAnalysisService;
import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.cache.query.IndicesQueryCache;
@ -148,6 +149,7 @@ public class IndicesModule extends AbstractModule {
bind(UpdateHelper.class).asEagerSingleton(); bind(UpdateHelper.class).asEagerSingleton();
bind(MetaDataIndexUpgradeService.class).asEagerSingleton(); bind(MetaDataIndexUpgradeService.class).asEagerSingleton();
bind(IndicesFieldDataCacheListener.class).asEagerSingleton(); bind(IndicesFieldDataCacheListener.class).asEagerSingleton();
bind(TermVectorsService.class).asEagerSingleton();
} }
protected void bindQueryParsersExtension() { protected void bindQueryParsersExtension() {

View File

@ -65,11 +65,9 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues; import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.mapper.*;
import org.elasticsearch.index.mapper.DocumentMapperForType; import org.elasticsearch.index.mapper.DocumentMapperForType;
import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.internal.UidFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper;

View File

@ -25,6 +25,7 @@ import org.junit.Test;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import static org.hamcrest.Matchers.arrayContaining;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.nullValue;
@ -33,7 +34,6 @@ import static org.hamcrest.Matchers.nullValue;
*/ */
public class PathTrieTests extends ESTestCase { public class PathTrieTests extends ESTestCase {
@Test
public void testPath() { public void testPath() {
PathTrie<String> trie = new PathTrie<>(); PathTrie<String> trie = new PathTrie<>();
trie.insert("/a/b/c", "walla"); trie.insert("/a/b/c", "walla");
@ -61,14 +61,12 @@ public class PathTrieTests extends ESTestCase {
assertThat(params.get("docId"), equalTo("12")); assertThat(params.get("docId"), equalTo("12"));
} }
@Test
public void testEmptyPath() { public void testEmptyPath() {
PathTrie<String> trie = new PathTrie<>(); PathTrie<String> trie = new PathTrie<>();
trie.insert("/", "walla"); trie.insert("/", "walla");
assertThat(trie.retrieve(""), equalTo("walla")); assertThat(trie.retrieve(""), equalTo("walla"));
} }
@Test
public void testDifferentNamesOnDifferentPath() { public void testDifferentNamesOnDifferentPath() {
PathTrie<String> trie = new PathTrie<>(); PathTrie<String> trie = new PathTrie<>();
trie.insert("/a/{type}", "test1"); trie.insert("/a/{type}", "test1");
@ -83,7 +81,6 @@ public class PathTrieTests extends ESTestCase {
assertThat(params.get("name"), equalTo("testX")); assertThat(params.get("name"), equalTo("testX"));
} }
@Test
public void testSameNameOnDifferentPath() { public void testSameNameOnDifferentPath() {
PathTrie<String> trie = new PathTrie<>(); PathTrie<String> trie = new PathTrie<>();
trie.insert("/a/c/{name}", "test1"); trie.insert("/a/c/{name}", "test1");
@ -98,7 +95,6 @@ public class PathTrieTests extends ESTestCase {
assertThat(params.get("name"), equalTo("testX")); assertThat(params.get("name"), equalTo("testX"));
} }
@Test
public void testPreferNonWildcardExecution() { public void testPreferNonWildcardExecution() {
PathTrie<String> trie = new PathTrie<>(); PathTrie<String> trie = new PathTrie<>();
trie.insert("{test}", "test1"); trie.insert("{test}", "test1");
@ -115,7 +111,6 @@ public class PathTrieTests extends ESTestCase {
assertThat(trie.retrieve("/v/x/c", params), equalTo("test6")); assertThat(trie.retrieve("/v/x/c", params), equalTo("test6"));
} }
@Test
public void testSamePathConcreteResolution() { public void testSamePathConcreteResolution() {
PathTrie<String> trie = new PathTrie<>(); PathTrie<String> trie = new PathTrie<>();
trie.insert("{x}/{y}/{z}", "test1"); trie.insert("{x}/{y}/{z}", "test1");
@ -132,7 +127,6 @@ public class PathTrieTests extends ESTestCase {
assertThat(params.get("k"), equalTo("c")); assertThat(params.get("k"), equalTo("c"));
} }
@Test
public void testNamedWildcardAndLookupWithWildcard() { public void testNamedWildcardAndLookupWithWildcard() {
PathTrie<String> trie = new PathTrie<>(); PathTrie<String> trie = new PathTrie<>();
trie.insert("x/{test}", "test1"); trie.insert("x/{test}", "test1");
@ -161,4 +155,25 @@ public class PathTrieTests extends ESTestCase {
assertThat(trie.retrieve("a/*/_endpoint", params), equalTo("test5")); assertThat(trie.retrieve("a/*/_endpoint", params), equalTo("test5"));
assertThat(params.get("test"), equalTo("*")); 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"));
}
} }

View File

@ -23,12 +23,15 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.DateTimeZone; import org.joda.time.DateTimeZone;
import org.joda.time.format.DateTimeFormat; 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.equalTo;
import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.notNullValue;
@ -51,8 +54,8 @@ public class DateMathIndexExpressionsIntegrationIT extends ESIntegTestCase {
refresh(); refresh();
SearchResponse searchResponse = client().prepareSearch(dateMathExp1, dateMathExp2, dateMathExp3).get(); SearchResponse searchResponse = client().prepareSearch(dateMathExp1, dateMathExp2, dateMathExp3).get();
ElasticsearchAssertions.assertHitCount(searchResponse, 3); assertHitCount(searchResponse, 3);
ElasticsearchAssertions.assertSearchHits(searchResponse, "1", "2", "3"); assertSearchHits(searchResponse, "1", "2", "3");
GetResponse getResponse = client().prepareGet(dateMathExp1, "type", "1").get(); GetResponse getResponse = client().prepareGet(dateMathExp1, "type", "1").get();
assertThat(getResponse.isExists(), is(true)); assertThat(getResponse.isExists(), is(true));
@ -84,4 +87,45 @@ public class DateMathIndexExpressionsIntegrationIT extends ESIntegTestCase {
assertThat(deleteResponse.getId(), equalTo("3")); 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());
}
} }

View File

@ -28,8 +28,10 @@ import org.elasticsearch.action.termvectors.TermVectorsRequest;
import org.elasticsearch.action.termvectors.TermVectorsResponse; import org.elasticsearch.action.termvectors.TermVectorsResponse;
import org.elasticsearch.common.Priority; import org.elasticsearch.common.Priority;
import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.termvectors.TermVectorsService;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.SearchModule;
@ -166,7 +168,7 @@ public class FetchSubPhasePluginIT extends ESIntegTestCase {
hitField = new InternalSearchHitField(NAMES[0], new ArrayList<>(1)); hitField = new InternalSearchHitField(NAMES[0], new ArrayList<>(1));
hitContext.hit().fields().put(NAMES[0], hitField); 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 { try {
Map<String, Integer> tv = new HashMap<>(); Map<String, Integer> tv = new HashMap<>();
TermsEnum terms = termVector.getFields().terms(field).iterator(); TermsEnum terms = termVector.getFields().terms(field).iterator();

View File

@ -189,7 +189,7 @@ PUT my_index/_mapping/type_one?update_all_types <3>
----------------------------------- -----------------------------------
// AUTOSENSE // AUTOSENSE
<1> Create an index with two types, both of which contain a `text` field which have the same mapping. <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`. <3> Adding the `update_all_types` parameter updates the `text` field in `type_one` and `type_two`.

View File

@ -0,0 +1,7 @@
---
"Missing index with catch":
- do:
catch: /index=logstash-\d{4}\.\d{2}\.\d{2}/
search:
index: <logstash-{now/M}>