Merge pull request #13135 from jasontedor/fix/eliminate-tuple-as-a-method-parameter

Remove usage of tuple as a method parameter
This commit is contained in:
Jason Tedor 2015-08-27 04:57:26 -04:00
commit a867a950ab
24 changed files with 317 additions and 254 deletions

View File

@ -20,6 +20,7 @@
package org.elasticsearch.action.search;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.type.ScrollIdForNode;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.cluster.ClusterService;
@ -27,7 +28,6 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.CountDown;
@ -69,7 +69,7 @@ public class TransportClearScrollAction extends HandledTransportAction<ClearScro
final DiscoveryNodes nodes;
final CountDown expectedOps;
final ClearScrollRequest request;
final List<Tuple<String, Long>[]> contexts = new ArrayList<>();
final List<ScrollIdForNode[]> contexts = new ArrayList<>();
final ActionListener<ClearScrollResponse> listener;
final AtomicReference<Throwable> expHolder;
final AtomicInteger numberOfFreedSearchContexts = new AtomicInteger(0);
@ -81,7 +81,7 @@ public class TransportClearScrollAction extends HandledTransportAction<ClearScro
expectedOps = nodes.size();
} else {
for (String parsedScrollId : request.getScrollIds()) {
Tuple<String, Long>[] context = parseScrollId(parsedScrollId).getContext();
ScrollIdForNode[] context = parseScrollId(parsedScrollId).getContext();
expectedOps += context.length;
this.contexts.add(context);
}
@ -114,15 +114,15 @@ public class TransportClearScrollAction extends HandledTransportAction<ClearScro
});
}
} else {
for (Tuple<String, Long>[] context : contexts) {
for (Tuple<String, Long> target : context) {
final DiscoveryNode node = nodes.get(target.v1());
for (ScrollIdForNode[] context : contexts) {
for (ScrollIdForNode target : context) {
final DiscoveryNode node = nodes.get(target.getNode());
if (node == null) {
onFreedContext(false);
continue;
}
searchServiceTransportAction.sendFreeContext(node, target.v2(), request, new ActionListener<SearchServiceTransportAction.SearchFreeContextResponse>() {
searchServiceTransportAction.sendFreeContext(node, target.getScrollId(), request, new ActionListener<SearchServiceTransportAction.SearchFreeContextResponse>() {
@Override
public void onResponse(SearchServiceTransportAction.SearchFreeContextResponse freed) {
onFreedContext(freed.isFreed());

View File

@ -19,9 +19,6 @@
package org.elasticsearch.action.search.type;
import org.elasticsearch.Version;
import org.elasticsearch.common.collect.Tuple;
import java.util.Map;
/**
@ -39,11 +36,11 @@ public class ParsedScrollId {
private final String type;
private final Tuple<String, Long>[] context;
private final ScrollIdForNode[] context;
private final Map<String, String> attributes;
public ParsedScrollId(String source, String type, Tuple<String, Long>[] context, Map<String, String> attributes) {
public ParsedScrollId(String source, String type, ScrollIdForNode[] context, Map<String, String> attributes) {
this.source = source;
this.type = type;
this.context = context;
@ -58,7 +55,7 @@ public class ParsedScrollId {
return type;
}
public Tuple<String, Long>[] getContext() {
public ScrollIdForNode[] getContext() {
return context;
}

View File

@ -0,0 +1,38 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.search.type;
public class ScrollIdForNode {
private final String node;
private final long scrollId;
public ScrollIdForNode(String node, long scrollId) {
this.node = node;
this.scrollId = scrollId;
}
public String getNode() {
return node;
}
public long getScrollId() {
return scrollId;
}
}

View File

@ -30,7 +30,6 @@ import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Base64;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.internal.InternalScrollSearchRequest;
@ -103,14 +102,14 @@ public abstract class TransportSearchHelper {
throw new IllegalArgumentException("Malformed scrollId [" + scrollId + "]");
}
@SuppressWarnings({"unchecked"}) Tuple<String, Long>[] context = new Tuple[contextSize];
ScrollIdForNode[] context = new ScrollIdForNode[contextSize];
for (int i = 0; i < contextSize; i++) {
String element = elements[index++];
int sep = element.indexOf(':');
if (sep == -1) {
throw new IllegalArgumentException("Malformed scrollId [" + scrollId + "]");
}
context[i] = new Tuple<>(element.substring(sep + 1), Long.parseLong(element.substring(0, sep)));
context[i] = new ScrollIdForNode(element.substring(sep + 1), Long.parseLong(element.substring(0, sep)));
}
Map<String, String> attributes;
int attributesSize = Integer.parseInt(elements[index++]);

View File

@ -25,7 +25,6 @@ import org.elasticsearch.action.search.*;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
@ -115,15 +114,15 @@ public class TransportSearchScrollQueryAndFetchAction extends AbstractComponent
return;
}
Tuple<String, Long>[] context = scrollId.getContext();
ScrollIdForNode[] context = scrollId.getContext();
for (int i = 0; i < context.length; i++) {
Tuple<String, Long> target = context[i];
DiscoveryNode node = nodes.get(target.v1());
ScrollIdForNode target = context[i];
DiscoveryNode node = nodes.get(target.getNode());
if (node != null) {
executePhase(i, node, target.v2());
executePhase(i, node, target.getScrollId());
} else {
if (logger.isDebugEnabled()) {
logger.debug("Node [" + target.v1() + "] not available for scroll request [" + scrollId.getSource() + "]");
logger.debug("Node [" + target.getNode() + "] not available for scroll request [" + scrollId.getSource() + "]");
}
successfulOps.decrementAndGet();
if (counter.decrementAndGet() == 0) {
@ -132,11 +131,11 @@ public class TransportSearchScrollQueryAndFetchAction extends AbstractComponent
}
}
for (Tuple<String, Long> target : scrollId.getContext()) {
DiscoveryNode node = nodes.get(target.v1());
for (ScrollIdForNode target : scrollId.getContext()) {
DiscoveryNode node = nodes.get(target.getNode());
if (node == null) {
if (logger.isDebugEnabled()) {
logger.debug("Node [" + target.v1() + "] not available for scroll request [" + scrollId.getSource() + "]");
logger.debug("Node [" + target.getNode() + "] not available for scroll request [" + scrollId.getSource() + "]");
}
successfulOps.decrementAndGet();
if (counter.decrementAndGet() == 0) {

View File

@ -26,7 +26,6 @@ import org.elasticsearch.action.search.*;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
@ -124,15 +123,15 @@ public class TransportSearchScrollQueryThenFetchAction extends AbstractComponent
}
final AtomicInteger counter = new AtomicInteger(scrollId.getContext().length);
Tuple<String, Long>[] context = scrollId.getContext();
ScrollIdForNode[] context = scrollId.getContext();
for (int i = 0; i < context.length; i++) {
Tuple<String, Long> target = context[i];
DiscoveryNode node = nodes.get(target.v1());
ScrollIdForNode target = context[i];
DiscoveryNode node = nodes.get(target.getNode());
if (node != null) {
executeQueryPhase(i, counter, node, target.v2());
executeQueryPhase(i, counter, node, target.getScrollId());
} else {
if (logger.isDebugEnabled()) {
logger.debug("Node [" + target.v1() + "] not available for scroll request [" + scrollId.getSource() + "]");
logger.debug("Node [" + target.getNode() + "] not available for scroll request [" + scrollId.getSource() + "]");
}
successfulOps.decrementAndGet();
if (counter.decrementAndGet() == 0) {

View File

@ -28,7 +28,6 @@ import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
@ -125,15 +124,15 @@ public class TransportSearchScrollScanAction extends AbstractComponent {
return;
}
Tuple<String, Long>[] context = scrollId.getContext();
ScrollIdForNode[] context = scrollId.getContext();
for (int i = 0; i < context.length; i++) {
Tuple<String, Long> target = context[i];
DiscoveryNode node = nodes.get(target.v1());
ScrollIdForNode target = context[i];
DiscoveryNode node = nodes.get(target.getNode());
if (node != null) {
executePhase(i, node, target.v2());
executePhase(i, node, target.getScrollId());
} else {
if (logger.isDebugEnabled()) {
logger.debug("Node [" + target.v1() + "] not available for scroll request [" + scrollId.getSource() + "]");
logger.debug("Node [" + target.getScrollId() + "] not available for scroll request [" + scrollId.getSource() + "]");
}
successfulOps.decrementAndGet();
if (counter.decrementAndGet() == 0) {
@ -142,11 +141,11 @@ public class TransportSearchScrollScanAction extends AbstractComponent {
}
}
for (Tuple<String, Long> target : scrollId.getContext()) {
DiscoveryNode node = nodes.get(target.v1());
for (ScrollIdForNode target : scrollId.getContext()) {
DiscoveryNode node = nodes.get(target.getNode());
if (node == null) {
if (logger.isDebugEnabled()) {
logger.debug("Node [" + target.v1() + "] not available for scroll request [" + scrollId.getSource() + "]");
logger.debug("Node [" + target.getNode() + "] not available for scroll request [" + scrollId.getSource() + "]");
}
successfulOps.decrementAndGet();
if (counter.decrementAndGet() == 0) {

View File

@ -22,14 +22,12 @@ package org.elasticsearch.index;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterators;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.*;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
@ -40,7 +38,6 @@ import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
import org.elasticsearch.index.deletionpolicy.DeletionPolicyModule;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.shard.StoreRecoveryService;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.IndexQueryParserService;
import org.elasticsearch.index.settings.IndexSettings;
@ -102,7 +99,25 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
private final NodeEnvironment nodeEnv;
private final IndicesService indicesServices;
private volatile ImmutableMap<Integer, Tuple<IndexShard, Injector>> shards = ImmutableMap.of();
private volatile ImmutableMap<Integer, IndexShardInjectorPair> shards = ImmutableMap.of();
private static class IndexShardInjectorPair {
private final IndexShard indexShard;
private final Injector injector;
public IndexShardInjectorPair(IndexShard indexShard, Injector injector) {
this.indexShard = indexShard;
this.injector = injector;
}
public IndexShard getIndexShard() {
return indexShard;
}
public Injector getInjector() {
return injector;
}
}
private final AtomicBoolean closed = new AtomicBoolean(false);
private final AtomicBoolean deleted = new AtomicBoolean(false);
@ -147,10 +162,10 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
@Override
public Iterator<IndexShard> iterator() {
return Iterators.transform(shards.values().iterator(), new Function<Tuple<IndexShard, Injector>, IndexShard>() {
return Iterators.transform(shards.values().iterator(), new Function<IndexShardInjectorPair, IndexShard>() {
@Override
public IndexShard apply(Tuple<IndexShard, Injector> input) {
return input.v1();
public IndexShard apply(IndexShardInjectorPair input) {
return input.getIndexShard();
}
});
}
@ -164,9 +179,9 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
*/
@Nullable
public IndexShard shard(int shardId) {
Tuple<IndexShard, Injector> indexShardInjectorTuple = shards.get(shardId);
if (indexShardInjectorTuple != null) {
return indexShardInjectorTuple.v1();
IndexShardInjectorPair indexShardInjectorPair = shards.get(shardId);
if (indexShardInjectorPair != null) {
return indexShardInjectorPair.getIndexShard();
}
return null;
}
@ -244,11 +259,11 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
* Return the shard injector for the provided id, or throw an exception if there is no such shard.
*/
public Injector shardInjectorSafe(int shardId) {
Tuple<IndexShard, Injector> tuple = shards.get(shardId);
if (tuple == null) {
IndexShardInjectorPair indexShardInjectorPair = shards.get(shardId);
if (indexShardInjectorPair == null) {
throw new ShardNotFoundException(new ShardId(index, shardId));
}
return tuple.v2();
return indexShardInjectorPair.getInjector();
}
public String indexUUID() {
@ -348,7 +363,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
indicesLifecycle.indexShardStateChanged(indexShard, null, "shard created");
indicesLifecycle.afterIndexShardCreated(indexShard);
shards = newMapBuilder(shards).put(shardId.id(), new Tuple<>(indexShard, shardInjector)).immutableMap();
shards = newMapBuilder(shards).put(shardId.id(), new IndexShardInjectorPair(indexShard, shardInjector)).immutableMap();
success = true;
return indexShard;
} catch (IOException e) {
@ -374,10 +389,10 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
return;
}
logger.debug("[{}] closing... (reason: [{}])", shardId, reason);
HashMap<Integer, Tuple<IndexShard, Injector>> tmpShardsMap = newHashMap(shards);
Tuple<IndexShard, Injector> tuple = tmpShardsMap.remove(shardId);
indexShard = tuple.v1();
shardInjector = tuple.v2();
HashMap<Integer, IndexShardInjectorPair> tmpShardsMap = newHashMap(shards);
IndexShardInjectorPair indexShardInjectorPair = tmpShardsMap.remove(shardId);
indexShard = indexShardInjectorPair.getIndexShard();
shardInjector = indexShardInjectorPair.getInjector();
shards = ImmutableMap.copyOf(tmpShardsMap);
closeShardInjector(reason, sId, shardInjector, indexShard);
logger.debug("[{}] closed (reason: [{}])", shardId, reason);

View File

@ -0,0 +1,38 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
public class DocumentMapperForType {
private final DocumentMapper documentMapper;
private final Mapping mapping;
public DocumentMapperForType(DocumentMapper documentMapper, Mapping mapping) {
this.mapping = mapping;
this.documentMapper = documentMapper;
}
public DocumentMapper getDocumentMapper() {
return documentMapper;
}
public Mapping getMapping() {
return mapping;
}
}

View File

@ -26,24 +26,18 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterators;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.DelegatingAnalyzerWrapper;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.Term;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.*;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchGenerationException;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.Queries;
@ -65,12 +59,7 @@ import org.elasticsearch.script.ScriptService;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.*;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.locks.ReentrantReadWriteLock;
@ -395,16 +384,16 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
* Returns the document mapper created, including a mapping update if the
* type has been dynamically created.
*/
public Tuple<DocumentMapper, Mapping> documentMapperWithAutoCreate(String type) {
public DocumentMapperForType documentMapperWithAutoCreate(String type) {
DocumentMapper mapper = mappers.get(type);
if (mapper != null) {
return Tuple.tuple(mapper, null);
return new DocumentMapperForType(mapper, null);
}
if (!dynamic) {
throw new TypeMissingException(index, type, "trying to auto create mapping, but dynamic mapping is disabled");
}
mapper = parse(type, null, true);
return Tuple.tuple(mapper, mapper.mapping());
return new DocumentMapperForType(mapper, mapper.mapping());
}
/**

View File

@ -21,17 +21,16 @@ package org.elasticsearch.index.query;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.MultiDocValues;
import org.apache.lucene.search.*;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.join.BitDocIdSetFilter;
import org.elasticsearch.common.ParseField;
import org.apache.lucene.search.join.JoinUtil;
import org.apache.lucene.search.join.ScoreMode;
import org.elasticsearch.Version;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentParser;
@ -45,8 +44,8 @@ import org.elasticsearch.index.search.child.ChildrenConstantScoreQuery;
import org.elasticsearch.index.search.child.ChildrenQuery;
import org.elasticsearch.index.search.child.ScoreType;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.fetch.innerhits.InnerHitsSubSearchContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.SubSearchContext;
import java.io.IOException;
@ -82,7 +81,7 @@ public class HasChildQueryParser implements QueryParser {
int maxChildren = 0;
int shortCircuitParentDocSet = 8192;
String queryName = null;
Tuple<String, SubSearchContext> innerHits = null;
InnerHitsSubSearchContext innerHits = null;
String currentFieldName = null;
XContentParser.Token token;
@ -152,8 +151,8 @@ public class HasChildQueryParser implements QueryParser {
if (innerHits != null) {
ParsedQuery parsedQuery = new ParsedQuery(innerQuery, parseContext.copyNamedQueries());
InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.v2(), parsedQuery, null, parseContext.mapperService(), childDocMapper);
String name = innerHits.v1() != null ? innerHits.v1() : childType;
InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.getSubSearchContext(), parsedQuery, null, parseContext.mapperService(), childDocMapper);
String name = innerHits.getName() != null ? innerHits.getName() : childType;
parseContext.addInnerHits(name, parentChildInnerHits);
}

View File

@ -18,15 +18,10 @@
*/
package org.elasticsearch.index.query;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.*;
import org.elasticsearch.Version;
import org.apache.lucene.search.QueryWrapperFilter;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentParser;
@ -39,7 +34,7 @@ import org.elasticsearch.index.search.child.ParentConstantScoreQuery;
import org.elasticsearch.index.search.child.ParentQuery;
import org.elasticsearch.index.search.child.ScoreType;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.internal.SubSearchContext;
import org.elasticsearch.search.fetch.innerhits.InnerHitsSubSearchContext;
import java.io.IOException;
import java.util.HashSet;
@ -73,7 +68,7 @@ public class HasParentQueryParser implements QueryParser {
String parentType = null;
boolean score = false;
String queryName = null;
Tuple<String, SubSearchContext> innerHits = null;
InnerHitsSubSearchContext innerHits = null;
String currentFieldName = null;
XContentParser.Token token;
@ -146,7 +141,7 @@ public class HasParentQueryParser implements QueryParser {
return query;
}
static Query createParentQuery(Query innerQuery, String parentType, boolean score, QueryParseContext parseContext, Tuple<String, SubSearchContext> innerHits) throws IOException {
static Query createParentQuery(Query innerQuery, String parentType, boolean score, QueryParseContext parseContext, InnerHitsSubSearchContext innerHits) throws IOException {
DocumentMapper parentDocMapper = parseContext.mapperService().documentMapper(parentType);
if (parentDocMapper == null) {
throw new QueryParsingException(parseContext, "[has_parent] query configured 'parent_type' [" + parentType
@ -155,8 +150,8 @@ public class HasParentQueryParser implements QueryParser {
if (innerHits != null) {
ParsedQuery parsedQuery = new ParsedQuery(innerQuery, parseContext.copyNamedQueries());
InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.v2(), parsedQuery, null, parseContext.mapperService(), parentDocMapper);
String name = innerHits.v1() != null ? innerHits.v1() : parentType;
InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.getSubSearchContext(), parsedQuery, null, parseContext.mapperService(), parentDocMapper);
String name = innerHits.getName() != null ? innerHits.getName() : parentType;
parseContext.addInnerHits(name, parentChildInnerHits);
}

View File

@ -26,14 +26,13 @@ import org.apache.lucene.search.join.ToParentBlockJoinQuery;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.support.InnerHitsQueryParserHelper;
import org.elasticsearch.index.query.support.NestedInnerQueryParseSupport;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.internal.SubSearchContext;
import org.elasticsearch.search.fetch.innerhits.InnerHitsSubSearchContext;
import java.io.IOException;
@ -120,7 +119,7 @@ public class NestedQueryParser implements QueryParser {
public static class ToBlockJoinQueryBuilder extends NestedInnerQueryParseSupport {
private ScoreMode scoreMode;
private Tuple<String, SubSearchContext> innerHits;
private InnerHitsSubSearchContext innerHits;
public ToBlockJoinQueryBuilder(QueryParseContext parseContext) throws IOException {
super(parseContext);
@ -130,7 +129,7 @@ public class NestedQueryParser implements QueryParser {
this.scoreMode = scoreMode;
}
public void setInnerHits(Tuple<String, SubSearchContext> innerHits) {
public void setInnerHits(InnerHitsSubSearchContext innerHits) {
this.innerHits = innerHits;
}
@ -152,8 +151,8 @@ public class NestedQueryParser implements QueryParser {
if (innerHits != null) {
ParsedQuery parsedQuery = new ParsedQuery(innerQuery, parseContext.copyNamedQueries());
InnerHitsContext.NestedInnerHits nestedInnerHits = new InnerHitsContext.NestedInnerHits(innerHits.v2(), parsedQuery, null, getParentObjectMapper(), nestedObjectMapper);
String name = innerHits.v1() != null ? innerHits.v1() : path;
InnerHitsContext.NestedInnerHits nestedInnerHits = new InnerHitsContext.NestedInnerHits(innerHits.getSubSearchContext(), parsedQuery, null, getParentObjectMapper(), nestedObjectMapper);
String name = innerHits.getName() != null ? innerHits.getName() : path;
parseContext.addInnerHits(name, nestedInnerHits);
}

View File

@ -19,12 +19,12 @@
package org.elasticsearch.index.query.support;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryParsingException;
import org.elasticsearch.search.fetch.fielddata.FieldDataFieldsParseElement;
import org.elasticsearch.search.fetch.innerhits.InnerHitsSubSearchContext;
import org.elasticsearch.search.fetch.script.ScriptFieldsParseElement;
import org.elasticsearch.search.fetch.source.FetchSourceParseElement;
import org.elasticsearch.search.highlight.HighlighterParseElement;
@ -51,7 +51,7 @@ public class InnerHitsQueryParserHelper {
this.fieldDataFieldsParseElement = fieldDataFieldsParseElement;
}
public Tuple<String, SubSearchContext> parse(QueryParseContext parserContext) throws IOException, QueryParsingException {
public InnerHitsSubSearchContext parse(QueryParseContext parserContext) throws IOException, QueryParsingException {
String fieldName = null;
XContentParser.Token token;
String innerHitName = null;
@ -74,7 +74,7 @@ public class InnerHitsQueryParserHelper {
} catch (Exception e) {
throw new QueryParsingException(parserContext, "Failed to parse [_inner_hits]", e);
}
return new Tuple<>(innerHitName, subSearchContext);
return new InnerHitsSubSearchContext(innerHitName, subSearchContext);
}
public static void parseCommonInnerHitOptions(XContentParser parser, XContentParser.Token token, String fieldName, SubSearchContext subSearchContext,

View File

@ -21,17 +21,10 @@ package org.elasticsearch.index.shard;
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.DisjunctionMaxQuery;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.UsageTrackingQueryCachingPolicy;
import org.apache.lucene.search.*;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.ThreadInterruptedException;
@ -48,7 +41,6 @@ import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.logging.ESLogger;
@ -506,13 +498,13 @@ public class IndexShard extends AbstractIndexShardComponent {
}
}
static Engine.Create prepareCreate(Tuple<DocumentMapper, Mapping> docMapper, SourceToParse source, long version, VersionType versionType, Engine.Operation.Origin origin, boolean canHaveDuplicates, boolean autoGeneratedId) {
static Engine.Create prepareCreate(DocumentMapperForType docMapper, SourceToParse source, long version, VersionType versionType, Engine.Operation.Origin origin, boolean canHaveDuplicates, boolean autoGeneratedId) {
long startTime = System.nanoTime();
ParsedDocument doc = docMapper.v1().parse(source);
if (docMapper.v2() != null) {
doc.addDynamicMappingsUpdate(docMapper.v2());
ParsedDocument doc = docMapper.getDocumentMapper().parse(source);
if (docMapper.getMapping() != null) {
doc.addDynamicMappingsUpdate(docMapper.getMapping());
}
return new Engine.Create(docMapper.v1().uidMapper().term(doc.uid().stringValue()), doc, version, versionType, origin, startTime, canHaveDuplicates, autoGeneratedId);
return new Engine.Create(docMapper.getDocumentMapper().uidMapper().term(doc.uid().stringValue()), doc, version, versionType, origin, startTime, canHaveDuplicates, autoGeneratedId);
}
public void create(Engine.Create create) {
@ -540,13 +532,13 @@ public class IndexShard extends AbstractIndexShardComponent {
}
}
static Engine.Index prepareIndex(Tuple<DocumentMapper, Mapping> docMapper, SourceToParse source, long version, VersionType versionType, Engine.Operation.Origin origin, boolean canHaveDuplicates) {
static Engine.Index prepareIndex(DocumentMapperForType docMapper, SourceToParse source, long version, VersionType versionType, Engine.Operation.Origin origin, boolean canHaveDuplicates) {
long startTime = System.nanoTime();
ParsedDocument doc = docMapper.v1().parse(source);
if (docMapper.v2() != null) {
doc.addDynamicMappingsUpdate(docMapper.v2());
ParsedDocument doc = docMapper.getDocumentMapper().parse(source);
if (docMapper.getMapping() != null) {
doc.addDynamicMappingsUpdate(docMapper.getMapping());
}
return new Engine.Index(docMapper.v1().uidMapper().term(doc.uid().stringValue()), doc, version, versionType, origin, startTime, canHaveDuplicates);
return new Engine.Index(docMapper.getDocumentMapper().uidMapper().term(doc.uid().stringValue()), doc, version, versionType, origin, startTime, canHaveDuplicates);
}
/**
@ -573,7 +565,7 @@ public class IndexShard extends AbstractIndexShardComponent {
public Engine.Delete prepareDelete(String type, String id, long version, VersionType versionType, Engine.Operation.Origin origin) {
long startTime = System.nanoTime();
final DocumentMapper documentMapper = docMapper(type).v1();
final DocumentMapper documentMapper = docMapper(type).getDocumentMapper();
return new Engine.Delete(type, id, documentMapper.uidMapper().term(Uid.createUid(type, id)), version, versionType, origin, startTime, false);
}
@ -1389,7 +1381,7 @@ public class IndexShard extends AbstractIndexShardComponent {
return indexSettings.get(IndexMetaData.SETTING_INDEX_UUID, IndexMetaData.INDEX_UUID_NA_VALUE);
}
private Tuple<DocumentMapper, Mapping> docMapper(String type) {
private DocumentMapperForType docMapper(String type) {
return mapperService.documentMapperWithAutoCreate(type);
}

View File

@ -25,7 +25,6 @@ import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.search.Queries;
@ -67,7 +66,7 @@ public class TranslogRecoveryPerformer {
this.indexCache = indexCache;
}
protected Tuple<DocumentMapper, Mapping> docMapper(String type) {
protected DocumentMapperForType docMapper(String type) {
return mapperService.documentMapperWithAutoCreate(type); // protected for testing
}

View File

@ -20,12 +20,7 @@
package org.elasticsearch.index.termvectors;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.*;
import org.apache.lucene.index.memory.MemoryIndex;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.termvectors.TermVectorsFilter;
@ -38,20 +33,13 @@ 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.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.get.GetField;
import org.elasticsearch.index.get.GetResult;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.Uid;
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;
@ -61,14 +49,7 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.dfs.AggregatedDfs;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.*;
import static org.elasticsearch.index.mapper.SourceToParse.source;
@ -308,10 +289,10 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
MapperService mapperService = indexShard.mapperService();
// TODO: make parsing not dynamically create fields not in the original mapping
Tuple<DocumentMapper, Mapping> docMapper = mapperService.documentMapperWithAutoCreate(type);
ParsedDocument parsedDocument = docMapper.v1().parse(source(doc).index(index).type(type).flyweight(true));
if (docMapper.v2() != null) {
parsedDocument.addDynamicMappingsUpdate(docMapper.v2());
DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(type);
ParsedDocument parsedDocument = docMapper.getDocumentMapper().parse(source(doc).index(index).type(type).flyweight(true));
if (docMapper.getMapping() != null) {
parsedDocument.addDynamicMappingsUpdate(docMapper.getMapping());
}
if (parsedDocument.dynamicMappingsUpdate() != null) {
mappingUpdatedAction.updateMappingOnMasterSynchronously(index, type, parsedDocument.dynamicMappingsUpdate());

View File

@ -32,8 +32,10 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.bytes.ReleasablePagedBytesReference;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.*;
import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.logging.ESLogger;
@ -231,7 +233,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
}
})) {
long latestGeneration = -1;
List<Tuple<Path, Long>> filesToUpgrade = new ArrayList<>();
List<PathWithGeneration> filesToUpgrade = new ArrayList<>();
for (Path path : stream) {
Matcher matcher = parseLegacyIdPattern.matcher(path.getFileName().toString());
if (matcher.matches()) {
@ -239,7 +241,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
if (generation >= translogGeneration.translogFileGeneration) {
latestGeneration = Math.max(translogGeneration.translogFileGeneration, generation);
}
filesToUpgrade.add(new Tuple<>(path, generation));
filesToUpgrade.add(new PathWithGeneration(path, generation));
} else {
Matcher strict_matcher = PARSE_STRICT_ID_PATTERN.matcher(path.getFileName().toString());
if (strict_matcher.matches()) {
@ -250,17 +252,17 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
if (latestGeneration < translogGeneration.translogFileGeneration) {
throw new IllegalStateException("latest found translog has a lower generation that the excepcted uncommitted " + translogGeneration.translogFileGeneration + " > " + latestGeneration);
}
CollectionUtil.timSort(filesToUpgrade, new Comparator<Tuple<Path, Long>>() {
CollectionUtil.timSort(filesToUpgrade, new Comparator<PathWithGeneration>() {
@Override
public int compare(Tuple<Path, Long> o1, Tuple<Path, Long> o2) {
long gen1 = o1.v2();
long gen2 = o2.v2();
public int compare(PathWithGeneration o1, PathWithGeneration o2) {
long gen1 = o1.getGeneration();
long gen2 = o2.getGeneration();
return Long.compare(gen1, gen2);
}
});
for (Tuple<Path, Long> pathAndGeneration : filesToUpgrade) {
final Path path = pathAndGeneration.v1();
final long generation = pathAndGeneration.v2();
for (PathWithGeneration pathAndGeneration : filesToUpgrade) {
final Path path = pathAndGeneration.getPath();
final long generation = pathAndGeneration.getGeneration();
final Path target = path.resolveSibling(getFilename(generation));
logger.debug("upgrading translog copy file from {} to {}", path, target);
Files.move(path, target, StandardCopyOption.ATOMIC_MOVE);
@ -1798,4 +1800,21 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
return outstandingViews.size();
}
private static class PathWithGeneration {
private final Path path;
private final long generation;
public PathWithGeneration(Path path, long generation) {
this.path = path;
this.generation = generation;
}
public Path getPath() {
return path;
}
public long getGeneration() {
return generation;
}
}
}

View File

@ -20,12 +20,7 @@
package org.elasticsearch.indices;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.*;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.util.CollectionUtil;
import org.apache.lucene.util.IOUtils;
@ -38,14 +33,8 @@ import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.CreationException;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.inject.Injector;
import org.elasticsearch.common.inject.Injectors;
import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.inject.ModulesBuilder;
import org.elasticsearch.common.inject.*;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
@ -53,12 +42,7 @@ import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.env.ShardLock;
import org.elasticsearch.gateway.MetaDataStateFormat;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexModule;
import org.elasticsearch.index.IndexNameModule;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.LocalNodeIdModule;
import org.elasticsearch.index.*;
import org.elasticsearch.index.aliases.IndexAliasesServiceModule;
import org.elasticsearch.index.analysis.AnalysisModule;
import org.elasticsearch.index.analysis.AnalysisService;
@ -91,11 +75,7 @@ import org.elasticsearch.plugins.PluginsService;
import java.io.Closeable;
import java.io.IOException;
import java.nio.file.Files;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.*;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@ -124,7 +104,26 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
private final NodeEnvironment nodeEnv;
private final TimeValue shardsClosedTimeout;
private volatile Map<String, Tuple<IndexService, Injector>> indices = ImmutableMap.of();
private volatile Map<String, IndexServiceInjectorPair> indices = ImmutableMap.of();
static class IndexServiceInjectorPair {
private final IndexService indexService;
private final Injector injector;
public IndexServiceInjectorPair(IndexService indexService, Injector injector) {
this.indexService = indexService;
this.injector = injector;
}
public IndexService getIndexService() {
return indexService;
}
public Injector getInjector() {
return injector;
}
}
private final Map<Index, List<PendingDelete>> pendingDeletes = new HashMap<>();
private final OldShardsStats oldShardsStats = new OldShardsStats();
@ -229,8 +228,8 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
}
Map<Index, List<IndexShardStats>> statsByShard = Maps.newHashMap();
for (Tuple<IndexService, Injector> value : indices.values()) {
IndexService indexService = value.v1();
for (IndexServiceInjectorPair value : indices.values()) {
IndexService indexService = value.getIndexService();
for (IndexShard indexShard : indexService) {
try {
if (indexShard.routingEntry() == null) {
@ -261,10 +260,10 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
@Override
public Iterator<IndexService> iterator() {
return Iterators.transform(indices.values().iterator(), new Function<Tuple<IndexService, Injector>, IndexService>() {
return Iterators.transform(indices.values().iterator(), new Function<IndexServiceInjectorPair, IndexService>() {
@Override
public IndexService apply(Tuple<IndexService, Injector> input) {
return input.v1();
public IndexService apply(IndexServiceInjectorPair input) {
return input.getIndexService();
}
});
}
@ -279,11 +278,11 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
*/
@Nullable
public IndexService indexService(String index) {
Tuple<IndexService, Injector> indexServiceInjectorTuple = indices.get(index);
if (indexServiceInjectorTuple == null) {
IndexServiceInjectorPair indexServiceInjectorPair = indices.get(index);
if (indexServiceInjectorPair == null) {
return null;
} else {
return indexServiceInjectorTuple.v1();
return indexServiceInjectorPair.getIndexService();
}
}
@ -352,7 +351,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
indicesLifecycle.afterIndexCreated(indexService);
indices = newMapBuilder(indices).put(index.name(), new Tuple<>(indexService, indexInjector)).immutableMap();
indices = newMapBuilder(indices).put(index.name(), new IndexServiceInjectorPair(indexService, indexInjector)).immutableMap();
return indexService;
}
@ -377,10 +376,10 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
}
logger.debug("[{}] closing ... (reason [{}])", index, reason);
Map<String, Tuple<IndexService, Injector>> tmpMap = newHashMap(indices);
Tuple<IndexService, Injector> remove = tmpMap.remove(index);
indexService = remove.v1();
indexInjector = remove.v2();
Map<String, IndexServiceInjectorPair> tmpMap = newHashMap(indices);
IndexServiceInjectorPair remove = tmpMap.remove(index);
indexService = remove.getIndexService();
indexInjector = remove.getInjector();
indices = ImmutableMap.copyOf(tmpMap);
}
@ -488,7 +487,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
synchronized (this) {
String indexName = metaData.index();
if (indices.containsKey(indexName)) {
String localUUid = indices.get(indexName).v1().indexUUID();
String localUUid = indices.get(indexName).getIndexService().indexUUID();
throw new IllegalStateException("Can't delete index store for [" + indexName + "] - it's still part of the indices service [" + localUUid + "] [" + metaData.getIndexUUID() + "]");
}
if (clusterState.metaData().hasIndex(indexName) && (clusterState.nodes().localNode().masterNode() == true)) {
@ -589,9 +588,9 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
* @return true if the index can be deleted on this node
*/
public boolean canDeleteIndexContents(Index index, Settings indexSettings) {
final Tuple<IndexService, Injector> indexServiceInjectorTuple = this.indices.get(index.name());
final IndexServiceInjectorPair indexServiceInjectorPair = this.indices.get(index.name());
if (IndexMetaData.isOnSharedFilesystem(indexSettings) == false) {
if (indexServiceInjectorTuple == null && nodeEnv.hasNodeFile()) {
if (indexServiceInjectorPair == null && nodeEnv.hasNodeFile()) {
return true;
}
} else {
@ -622,10 +621,10 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
}
private boolean canDeleteShardContent(ShardId shardId, @IndexSettings Settings indexSettings) {
final Tuple<IndexService, Injector> indexServiceInjectorTuple = this.indices.get(shardId.getIndex());
final IndexServiceInjectorPair indexServiceInjectorPair = this.indices.get(shardId.getIndex());
if (IndexMetaData.isOnSharedFilesystem(indexSettings) == false) {
if (indexServiceInjectorTuple != null && nodeEnv.hasNodeFile()) {
final IndexService indexService = indexServiceInjectorTuple.v1();
if (indexServiceInjectorPair != null && nodeEnv.hasNodeFile()) {
final IndexService indexService = indexServiceInjectorPair.getIndexService();
return indexService.hasShard(shardId.id()) == false;
} else if (nodeEnv.hasNodeFile()) {
if (NodeEnvironment.hasCustomDataPath(indexSettings)) {

View File

@ -38,7 +38,6 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
@ -275,10 +274,10 @@ public class PercolatorService extends AbstractComponent {
}
MapperService mapperService = documentIndexService.mapperService();
Tuple<DocumentMapper, Mapping> docMapper = mapperService.documentMapperWithAutoCreate(request.documentType());
doc = docMapper.v1().parse(source(parser).index(index).type(request.documentType()).flyweight(true));
if (docMapper.v2() != null) {
doc.addDynamicMappingsUpdate(docMapper.v2());
DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(request.documentType());
doc = docMapper.getDocumentMapper().parse(source(parser).index(index).type(request.documentType()).flyweight(true));
if (docMapper.getMapping() != null) {
doc.addDynamicMappingsUpdate(docMapper.getMapping());
}
if (doc.dynamicMappingsUpdate() != null) {
mappingUpdatedAction.updateMappingOnMasterSynchronously(request.shardId().getIndex(), request.documentType(), doc.dynamicMappingsUpdate());
@ -384,8 +383,8 @@ public class PercolatorService extends AbstractComponent {
try {
parser = XContentFactory.xContent(fetchedDoc).createParser(fetchedDoc);
MapperService mapperService = documentIndexService.mapperService();
Tuple<DocumentMapper, Mapping> docMapper = mapperService.documentMapperWithAutoCreate(type);
doc = docMapper.v1().parse(source(parser).index(index).type(type).flyweight(true));
DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(type);
doc = docMapper.getDocumentMapper().parse(source(parser).index(index).type(type).flyweight(true));
if (context.highlight() != null) {
doc.setSource(fetchedDoc);

View File

@ -0,0 +1,40 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.fetch.innerhits;
import org.elasticsearch.search.internal.SubSearchContext;
public class InnerHitsSubSearchContext {
private final String name;
private final SubSearchContext subSearchContext;
public InnerHitsSubSearchContext(String name, SubSearchContext subSearchContext) {
this.name = name;
this.subSearchContext = subSearchContext;
}
public String getName() {
return name;
}
public SubSearchContext getSubSearchContext() {
return subSearchContext;
}
}

View File

@ -29,15 +29,7 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexDeletionPolicy;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LiveIndexWriterConfig;
import org.apache.lucene.index.LogByteSizeMergePolicy;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.NoMergePolicy;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.index.*;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.TermQuery;
@ -55,7 +47,6 @@ import org.elasticsearch.common.Base64;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.uid.Versions;
@ -69,16 +60,9 @@ import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy;
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
import org.elasticsearch.index.engine.Engine.Searcher;
import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.*;
import org.elasticsearch.index.mapper.Mapper.BuilderContext;
import org.elasticsearch.index.mapper.MapperBuilders;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.mapper.object.RootObjectMapper;
@ -107,13 +91,7 @@ import java.nio.charset.Charset;
import java.nio.file.DirectoryStream;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.*;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.regex.Pattern;
@ -121,12 +99,7 @@ import java.util.regex.Pattern;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY;
import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.hasKey;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.*;
public class InternalEngineTests extends ESTestCase {
@ -1984,8 +1957,8 @@ public class InternalEngineTests extends ESTestCase {
}
@Override
protected Tuple<DocumentMapper, Mapping> docMapper(String type) {
return new Tuple<>(docMapper, mappingUpdate);
protected DocumentMapperForType docMapper(String type) {
return new DocumentMapperForType(docMapper, mappingUpdate);
}
@Override

View File

@ -19,16 +19,11 @@
package org.elasticsearch.index.mapper;
import com.google.common.collect.ImmutableMap;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.*;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.core.IntegerFieldMapper;
import org.elasticsearch.index.mapper.core.StringFieldMapper;
@ -368,7 +363,7 @@ public class DynamicMappingTests extends ESSingleNodeTestCase {
// Even if the dynamic type of our new field is long, we already have a mapping for the same field
// of type string so it should be mapped as a string
DocumentMapper newMapper = indexService.mapperService().documentMapperWithAutoCreate("type2").v1();
DocumentMapper newMapper = indexService.mapperService().documentMapperWithAutoCreate("type2").getDocumentMapper();
Mapper update = parse(newMapper, indexService.mapperService().documentMapperParser(),
XContentFactory.jsonBuilder().startObject().field("my_field1", 42).endObject());
Mapper myField1Mapper = null;

View File

@ -38,7 +38,7 @@ import java.util.Arrays;
import java.util.List;
import java.util.Map;
import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.equalTo;
public class DefaultSourceMappingTests extends ESSingleNodeTestCase {
@ -202,7 +202,7 @@ public class DefaultSourceMappingTests extends ESSingleNodeTestCase {
MapperService mapperService = createIndex("test").mapperService();
mapperService.merge(MapperService.DEFAULT_MAPPING, new CompressedXContent(defaultMapping), true, false);
DocumentMapper mapper = mapperService.documentMapperWithAutoCreate("my_type").v1();
DocumentMapper mapper = mapperService.documentMapperWithAutoCreate("my_type").getDocumentMapper();
assertThat(mapper.type(), equalTo("my_type"));
assertThat(mapper.sourceMapper().enabled(), equalTo(false));
}