Merge branch 'master' into fix/bwc_creation

This commit is contained in:
Ryan Ernst 2015-08-27 10:16:45 -07:00
commit 448d3498b1
55 changed files with 729 additions and 332 deletions

View File

@ -106,7 +106,7 @@ public class TransportClusterStatsAction extends TransportNodesAction<ClusterSta
for (IndexShard indexShard : indexService) {
if (indexShard.routingEntry() != null && indexShard.routingEntry().active()) {
// only report on fully started shards
shardsStats.add(new ShardStats(indexShard, indexShard.routingEntry(), SHARD_STATS_FLAGS));
shardsStats.add(new ShardStats(indexShard, SHARD_STATS_FLAGS));
}
}
}

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentBuilderString;
import org.elasticsearch.index.engine.CommitStats;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardPath;
import java.io.IOException;
@ -37,20 +38,23 @@ import static org.elasticsearch.cluster.routing.ShardRouting.readShardRoutingEnt
/**
*/
public class ShardStats extends BroadcastShardResponse implements ToXContent {
private ShardRouting shardRouting;
CommonStats commonStats;
private CommonStats commonStats;
@Nullable
CommitStats commitStats;
private CommitStats commitStats;
private String dataPath;
private String statePath;
private boolean isCustomDataPath;
ShardStats() {
}
public ShardStats(IndexShard indexShard, ShardRouting shardRouting, CommonStatsFlags flags) {
public ShardStats(IndexShard indexShard, CommonStatsFlags flags) {
super(indexShard.shardId());
this.shardRouting = shardRouting;
this.shardRouting = indexShard.routingEntry();
this.dataPath = indexShard.shardPath().getRootDataPath().toString();
this.statePath = indexShard.shardPath().getRootStatePath().toString();
this.isCustomDataPath = indexShard.shardPath().isCustomDataPath();
this.commonStats = new CommonStats(indexShard, flags);
this.commitStats = indexShard.commitStats();
}
@ -70,6 +74,18 @@ public class ShardStats extends BroadcastShardResponse implements ToXContent {
return this.commitStats;
}
public String getDataPath() {
return dataPath;
}
public String getStatePath() {
return statePath;
}
public boolean isCustomDataPath() {
return isCustomDataPath;
}
public static ShardStats readShardStats(StreamInput in) throws IOException {
ShardStats stats = new ShardStats();
stats.readFrom(in);
@ -82,6 +98,9 @@ public class ShardStats extends BroadcastShardResponse implements ToXContent {
shardRouting = readShardRoutingEntry(in);
commonStats = CommonStats.readCommonStats(in);
commitStats = CommitStats.readOptionalCommitStatsFrom(in);
statePath = in.readString();
dataPath = in.readString();
isCustomDataPath = in.readBoolean();
}
@Override
@ -90,6 +109,9 @@ public class ShardStats extends BroadcastShardResponse implements ToXContent {
shardRouting.writeTo(out);
commonStats.writeTo(out);
out.writeOptionalStreamable(commitStats);
out.writeString(statePath);
out.writeString(dataPath);
out.writeBoolean(isCustomDataPath);
}
@Override
@ -105,12 +127,21 @@ public class ShardStats extends BroadcastShardResponse implements ToXContent {
if (commitStats != null) {
commitStats.toXContent(builder, params);
}
builder.startObject(Fields.SHARD_PATH);
builder.field(Fields.STATE_PATH, statePath);
builder.field(Fields.DATA_PATH, dataPath);
builder.field(Fields.IS_CUSTOM_DATA_PATH, isCustomDataPath);
builder.endObject();
return builder;
}
static final class Fields {
static final XContentBuilderString ROUTING = new XContentBuilderString("routing");
static final XContentBuilderString STATE = new XContentBuilderString("state");
static final XContentBuilderString STATE_PATH = new XContentBuilderString("state_path");
static final XContentBuilderString DATA_PATH = new XContentBuilderString("data_path");
static final XContentBuilderString IS_CUSTOM_DATA_PATH = new XContentBuilderString("is_custom_data_path");
static final XContentBuilderString SHARD_PATH = new XContentBuilderString("shard_path");
static final XContentBuilderString PRIMARY = new XContentBuilderString("primary");
static final XContentBuilderString NODE = new XContentBuilderString("node");
static final XContentBuilderString RELOCATING_NODE = new XContentBuilderString("relocating_node");

View File

@ -189,7 +189,7 @@ public class TransportIndicesStatsAction extends TransportBroadcastAction<Indice
flags.set(CommonStatsFlags.Flag.Recovery);
}
return new ShardStats(indexShard, indexShard.routingEntry(), flags);
return new ShardStats(indexShard, flags);
}
static class IndexShardStatsRequest extends BroadcastShardRequest {

View File

@ -92,6 +92,14 @@ public class TransportDeleteWarmerAction extends TransportMasterNodeAction<Delet
MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData());
boolean globalFoundAtLeastOne = false;
boolean deleteAll = false;
for (int i=0; i<request.names().length; i++){
if (request.names()[i].equals(MetaData.ALL)) {
deleteAll = true;
break;
}
}
for (String index : concreteIndices) {
IndexMetaData indexMetaData = currentState.metaData().index(index);
if (indexMetaData == null) {
@ -103,7 +111,7 @@ public class TransportDeleteWarmerAction extends TransportMasterNodeAction<Delet
for (IndexWarmersMetaData.Entry entry : warmers.entries()) {
boolean keepWarmer = true;
for (String warmer : request.names()) {
if (Regex.simpleMatch(warmer, entry.name()) || warmer.equals("_all")) {
if (Regex.simpleMatch(warmer, entry.name()) || warmer.equals(MetaData.ALL)) {
globalFoundAtLeastOne = true;
keepWarmer = false;
// don't add it...
@ -123,7 +131,7 @@ public class TransportDeleteWarmerAction extends TransportMasterNodeAction<Delet
}
}
if (!globalFoundAtLeastOne) {
if (globalFoundAtLeastOne == false && deleteAll == false) {
throw new IndexWarmerMissingException(request.names());
}
@ -137,11 +145,13 @@ public class TransportDeleteWarmerAction extends TransportMasterNodeAction<Delet
if (warmers != null) {
for (IndexWarmersMetaData.Entry entry : warmers.entries()) {
for (String warmer : request.names()) {
if (Regex.simpleMatch(warmer, entry.name()) || warmer.equals("_all")) {
if (Regex.simpleMatch(warmer, entry.name()) || warmer.equals(MetaData.ALL)) {
logger.info("[{}] delete warmer [{}]", index, entry.name());
}
}
}
} else if(deleteAll){
logger.debug("no warmers to delete on index [{}]", index);
}
}
}

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

@ -78,7 +78,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
private boolean scriptedUpsert = false;
private boolean docAsUpsert = false;
private boolean detectNoop = false;
private boolean detectNoop = true;
@Nullable
private IndexRequest doc;
@ -243,7 +243,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
* The script to execute. Note, make sure not to send different script each
* times and instead use script params if possible with the same
* (automatically compiled) script.
*
*
* @deprecated Use {@link #script(Script)} instead
*/
@Deprecated
@ -256,7 +256,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
* The script to execute. Note, make sure not to send different script each
* times and instead use script params if possible with the same
* (automatically compiled) script.
*
*
* @deprecated Use {@link #script(Script)} instead
*/
@Deprecated
@ -267,7 +267,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
/**
* The language of the script to execute.
*
*
* @deprecated Use {@link #script(Script)} instead
*/
@Deprecated
@ -286,7 +286,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
/**
* Add a script parameter.
*
*
* @deprecated Use {@link #script(Script)} instead
*/
@Deprecated
@ -311,7 +311,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
/**
* Sets the script parameters to use with the script.
*
*
* @deprecated Use {@link #script(Script)} instead
*/
@Deprecated
@ -338,7 +338,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
* The script to execute. Note, make sure not to send different script each
* times and instead use script params if possible with the same
* (automatically compiled) script.
*
*
* @deprecated Use {@link #script(Script)} instead
*/
@Deprecated
@ -360,7 +360,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
* The script type
* @param scriptParams
* The script parameters
*
*
* @deprecated Use {@link #script(Script)} instead
*/
@Deprecated
@ -623,7 +623,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
}
/**
* Should this update attempt to detect if it is a noop?
* Should this update attempt to detect if it is a noop? Defaults to true.
* @return this for chaining
*/
public UpdateRequest detectNoop(boolean detectNoop) {
@ -631,6 +631,9 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
return this;
}
/**
* Should this update attempt to detect if it is a noop? Defaults to true.
*/
public boolean detectNoop() {
return detectNoop;
}
@ -699,16 +702,15 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
this.docAsUpsert = shouldUpsertDoc;
return this;
}
public boolean scriptedUpsert(){
return this.scriptedUpsert;
}
public UpdateRequest scriptedUpsert(boolean scriptedUpsert) {
this.scriptedUpsert = scriptedUpsert;
return this;
}
@Override
public void readFrom(StreamInput in) throws IOException {

View File

@ -308,6 +308,7 @@ public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<U
/**
* Sets whether to perform extra effort to detect noop updates via docAsUpsert.
* Defautls to true.
*/
public UpdateRequestBuilder setDetectNoop(boolean detectNoop) {
request.detectNoop(detectNoop);
@ -322,4 +323,14 @@ public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<U
request.scriptedUpsert(scriptedUpsert);
return this;
}
/**
* Set the new ttl of the document. Note that if detectNoop is true (the default)
* and the source of the document isn't changed then the ttl update won't take
* effect.
*/
public UpdateRequestBuilder setTtl(Long ttl) {
request.doc().ttl(ttl);
return this;
}
}

View File

@ -943,7 +943,8 @@ public class Strings {
boolean changed = false;
for (int i = 0; i < value.length(); i++) {
char c = value.charAt(i);
if (c == '_') {
//e.g. _name stays as-is, _first_name becomes _firstName
if (c == '_' && i > 0) {
if (!changed) {
if (sb != null) {
sb.setLength(0);

View File

@ -198,7 +198,7 @@ public class MultiDataPathUpgrader {
}
}
}
return new ShardPath(target.path, target.path, IndexMetaData.INDEX_UUID_NA_VALUE /* we don't know */, shard);
return new ShardPath(false, target.path, target.path, IndexMetaData.INDEX_UUID_NA_VALUE /* we don't know */, shard);
}
private ShardFileInfo[] getShardFileInfo(ShardId shard, NodeEnvironment.NodePath[] paths) throws IOException {

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() {
@ -286,6 +301,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
Injector shardInjector = null;
try {
lock = nodeEnv.shardLock(shardId, TimeUnit.SECONDS.toMillis(5));
indicesLifecycle.beforeIndexShardCreated(shardId, indexSettings);
ShardPath path;
try {
path = ShardPath.loadShardPath(logger, nodeEnv, shardId, indexSettings);
@ -310,7 +326,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
throw new IndexShardAlreadyExistsException(shardId + " already exists");
}
indicesLifecycle.beforeIndexShardCreated(shardId, indexSettings);
logger.debug("creating shard_id {}", shardId);
// if we are on a shared FS we only own the shard (ie. we can safely delete it) if we are the primary.
final boolean canDeleteShardContent = IndexMetaData.isOnSharedFilesystem(indexSettings) == false ||
@ -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

@ -23,6 +23,7 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.Query;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.xcontent.XContentParser;
@ -38,6 +39,8 @@ public class PrefixQueryParser implements QueryParser {
public static final String NAME = "prefix";
private static final ParseField NAME_FIELD = new ParseField("_name").withAllDeprecated("query name is not supported in short version of prefix query");
@Inject
public PrefixQueryParser() {
}
@ -84,7 +87,7 @@ public class PrefixQueryParser implements QueryParser {
}
}
} else {
if ("_name".equals(currentFieldName)) {
if (parseContext.parseFieldMatcher().match(currentFieldName, NAME_FIELD)) {
queryName = parser.text();
} else {
fieldName = currentFieldName;

View File

@ -40,6 +40,7 @@ public class RangeQueryParser implements QueryParser {
public static final String NAME = "range";
private static final ParseField FIELDDATA_FIELD = new ParseField("fielddata").withAllDeprecated("[no replacement]");
private static final ParseField NAME_FIELD = new ParseField("_name").withAllDeprecated("query name is not supported in short version of range query");
@Inject
public RangeQueryParser() {
@ -109,7 +110,7 @@ public class RangeQueryParser implements QueryParser {
}
}
} else if (token.isValue()) {
if ("_name".equals(currentFieldName)) {
if (parseContext.parseFieldMatcher().match(currentFieldName, NAME_FIELD)) {
queryName = parser.text();
} else if (parseContext.parseFieldMatcher().match(currentFieldName, FIELDDATA_FIELD)) {
// ignore

View File

@ -24,6 +24,7 @@ import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.RegexpQuery;
import org.apache.lucene.util.automaton.Operations;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.xcontent.XContentParser;
@ -41,6 +42,8 @@ public class RegexpQueryParser implements QueryParser {
public static final int DEFAULT_FLAGS_VALUE = RegexpFlag.ALL.value();
private static final ParseField NAME_FIELD = new ParseField("_name").withAllDeprecated("query name is not supported in short version of regexp query");
@Inject
public RegexpQueryParser() {
}
@ -96,7 +99,7 @@ public class RegexpQueryParser implements QueryParser {
}
}
} else {
if ("_name".equals(currentFieldName)) {
if (parseContext.parseFieldMatcher().match(currentFieldName, NAME_FIELD)) {
queryName = parser.text();
} else {
fieldName = currentFieldName;

View File

@ -22,10 +22,10 @@ package org.elasticsearch.index.query;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import java.io.IOException;
@ -37,6 +37,9 @@ public class TermQueryParser implements QueryParser {
public static final String NAME = "term";
private static final ParseField NAME_FIELD = new ParseField("_name").withAllDeprecated("query name is not supported in short version of term query");
private static final ParseField BOOST_FIELD = new ParseField("boost").withAllDeprecated("boost is not supported in short version of term query");
@Inject
public TermQueryParser() {
}
@ -85,9 +88,9 @@ public class TermQueryParser implements QueryParser {
}
}
} else if (token.isValue()) {
if ("_name".equals(currentFieldName)) {
if (parseContext.parseFieldMatcher().match(currentFieldName, NAME_FIELD)) {
queryName = parser.text();
} else if ("boost".equals(currentFieldName)) {
} else if (parseContext.parseFieldMatcher().match(currentFieldName, BOOST_FIELD)) {
boost = parser.floatValue();
} else {
if (fieldName != null) {

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

@ -20,6 +20,7 @@ package org.elasticsearch.index.shard;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
@ -41,10 +42,18 @@ public final class ShardPath {
private final String indexUUID;
private final ShardId shardId;
private final Path shardStatePath;
private final boolean isCustomDataPath;
public ShardPath(Path path, Path shardStatePath, String indexUUID, ShardId shardId) {
this.path = path;
public ShardPath(boolean isCustomDataPath, Path dataPath, Path shardStatePath, String indexUUID, ShardId shardId) {
assert dataPath.getFileName().toString().equals(Integer.toString(shardId.id())) : "dataPath must end with the shard ID but didn't: " + dataPath.toString();
assert shardStatePath.getFileName().toString().equals(Integer.toString(shardId.id())) : "shardStatePath must end with the shard ID but didn't: " + dataPath.toString();
assert dataPath.getParent().getFileName().toString().equals(shardId.getIndex()) : "dataPath must end with index/shardID but didn't: " + dataPath.toString();
assert shardStatePath.getParent().getFileName().toString().equals(shardId.getIndex()) : "shardStatePath must end with index/shardID but didn't: " + dataPath.toString();
if (isCustomDataPath && dataPath.equals(shardStatePath)) {
throw new IllegalArgumentException("shard state path must be different to the data path when using custom data paths");
}
this.isCustomDataPath = isCustomDataPath;
this.path = dataPath;
this.indexUUID = indexUUID;
this.shardId = shardId;
this.shardStatePath = shardStatePath;
@ -78,6 +87,30 @@ public final class ShardPath {
return shardStatePath;
}
/**
* Returns the data-path root for this shard. The root is a parent of {@link #getDataPath()} without the index name
* and the shard ID.
*/
public Path getRootDataPath() {
Path noIndexShardId = getDataPath().getParent().getParent();
return isCustomDataPath ? noIndexShardId : noIndexShardId.getParent(); // also strip the indices folder
}
/**
* Returns the state-path root for this shard. The root is a parent of {@link #getRootStatePath()} ()} without the index name
* and the shard ID.
*/
public Path getRootStatePath() {
return getShardStatePath().getParent().getParent().getParent(); // also strip the indices folder
}
/**
* Returns <code>true</code> iff the data location is a custom data location and therefore outside of the nodes configured data paths.
*/
public boolean isCustomDataPath() {
return isCustomDataPath;
}
/**
* This method walks through the nodes shard paths to find the data and state path for the given shard. If multiple
* directories with a valid shard state exist the one with the highest version will be used.
@ -113,7 +146,7 @@ public final class ShardPath {
dataPath = statePath;
}
logger.debug("{} loaded data path [{}], state path [{}]", shardId, dataPath, statePath);
return new ShardPath(dataPath, statePath, indexUUID, shardId);
return new ShardPath(NodeEnvironment.hasCustomDataPath(indexSettings), dataPath, statePath, indexUUID, shardId);
}
}
@ -202,7 +235,7 @@ public final class ShardPath {
dataPath = statePath;
}
return new ShardPath(dataPath, statePath, indexUUID, shardId);
return new ShardPath(NodeEnvironment.hasCustomDataPath(indexSettings), dataPath, statePath, indexUUID, shardId);
}
@Override

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,14 +228,14 @@ 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) {
continue;
}
IndexShardStats indexShardStats = new IndexShardStats(indexShard.shardId(), new ShardStats[] { new ShardStats(indexShard, indexShard.routingEntry(), flags) });
IndexShardStats indexShardStats = new IndexShardStats(indexShard.shardId(), new ShardStats[] { new ShardStats(indexShard, flags) });
if (!statsByShard.containsKey(indexService.index())) {
statsByShard.put(indexService.index(), Lists.<IndexShardStats>newArrayList(indexShardStats));
} else {
@ -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,6 +29,8 @@ public class StringsTests extends ESTestCase {
assertEquals("fooBar", Strings.toCamelCase("foo_bar"));
assertEquals("fooBarFooBar", Strings.toCamelCase("foo_bar_foo_bar"));
assertEquals("fooBar", Strings.toCamelCase("foo_bar_"));
assertEquals("_foo", Strings.toCamelCase("_foo"));
assertEquals("_fooBar", Strings.toCamelCase("_foo_bar_"));
}
public void testSubstring() {

View File

@ -83,7 +83,7 @@ public class MultiDataPathUpgraderTests extends ESTestCase {
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(metaStateVersion, true, uuid), metaStateVersion, shardDataPaths);
}
final Path path = randomFrom(shardDataPaths);
ShardPath targetPath = new ShardPath(path, path, uuid, new ShardId("foo", 0));
ShardPath targetPath = new ShardPath(false, path, path, uuid, new ShardId("foo", 0));
MultiDataPathUpgrader helper = new MultiDataPathUpgrader(nodeEnvironment);
helper.upgrade(shardId, targetPath);
assertFalse(helper.needsUpgrading(shardId));
@ -177,7 +177,7 @@ public class MultiDataPathUpgraderTests extends ESTestCase {
}
logger.info("--> injecting index [{}] into multiple data paths", indexName);
OldIndexBackwardsCompatibilityIT.copyIndex(logger, src, indexName, multiDataPath);
final ShardPath shardPath = new ShardPath(nodeEnvironment.availableShardPaths(new ShardId(indexName, 0))[0], nodeEnvironment.availableShardPaths(new ShardId(indexName, 0))[0], IndexMetaData.INDEX_UUID_NA_VALUE, new ShardId(indexName, 0));
final ShardPath shardPath = new ShardPath(false, nodeEnvironment.availableShardPaths(new ShardId(indexName, 0))[0], nodeEnvironment.availableShardPaths(new ShardId(indexName, 0))[0], IndexMetaData.INDEX_UUID_NA_VALUE, new ShardId(indexName, 0));
logger.info("{}", FileSystemUtils.files(shardPath.resolveIndex()));

View File

@ -195,7 +195,7 @@ public class BulkIT extends ESIntegTestCase {
bulkResponse = client().prepareBulk()
.add(client().prepareUpdate("test", "type", "e1").setDoc("field", "2").setVersion(10)) // INTERNAL
.add(client().prepareUpdate("test", "type", "e1").setDoc("field", "3").setVersion(20).setVersionType(VersionType.FORCE))
.add(client().prepareUpdate("test", "type", "e1").setDoc("field", "3").setVersion(20).setVersionType(VersionType.INTERNAL)).get();
.add(client().prepareUpdate("test", "type", "e1").setDoc("field", "4").setVersion(20).setVersionType(VersionType.INTERNAL)).get();
assertThat(bulkResponse.getItems()[0].getFailureMessage(), containsString("version conflict"));
assertThat(((UpdateResponse) bulkResponse.getItems()[1].getResponse()).getVersion(), equalTo(20l));

View File

@ -54,9 +54,7 @@ import java.nio.file.Path;
import java.util.Set;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.*;
/**
*
@ -249,7 +247,7 @@ public class AnalysisModuleTests extends ESTestCase {
fail("This should fail with IllegalArgumentException because the analyzers name starts with _");
} catch (ProvisionException e) {
assertTrue(e.getCause() instanceof IllegalArgumentException);
assertThat(e.getCause().getMessage(), equalTo("analyzer name must not start with '_'. got \"_invalid_name\""));
assertThat(e.getCause().getMessage(), either(equalTo("analyzer name must not start with '_'. got \"_invalid_name\"")).or(equalTo("analyzer name must not start with '_'. got \"_invalidName\"")));
}
}

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));
}

View File

@ -20,9 +20,12 @@ package org.elasticsearch.index.shard;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
import org.elasticsearch.action.admin.indices.stats.IndexStats;
import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.*;
@ -31,11 +34,16 @@ import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.env.Environment;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.env.ShardLock;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.query.QueryParsingException;
@ -539,4 +547,36 @@ public class IndexShardTests extends ESSingleNodeTestCase {
assertPathHasBeenCleared(startDir.toAbsolutePath().toString());
assertPathHasBeenCleared(endDir.toAbsolutePath().toString());
}
public void testShardStats() throws IOException {
createIndex("test");
ensureGreen();
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
IndexService test = indicesService.indexService("test");
IndexShard shard = test.shard(0);
ShardStats stats = new ShardStats(shard, new CommonStatsFlags());
assertEquals(shard.shardPath().getRootDataPath().toString(), stats.getDataPath());
assertEquals(shard.shardPath().getRootStatePath().toString(), stats.getStatePath());
assertEquals(shard.shardPath().isCustomDataPath(), stats.isCustomDataPath());
if (randomBoolean() || true) { // try to serialize it to ensure values survive the serialization
BytesStreamOutput out = new BytesStreamOutput();
stats.writeTo(out);
StreamInput in = StreamInput.wrap(out.bytes());
stats = ShardStats.readShardStats(in);
}
XContentBuilder builder = XContentFactory.jsonBuilder();
builder.startObject();
stats.toXContent(builder, EMPTY_PARAMS);
builder.endObject();
String xContent = builder.string();
StringBuilder expectedSubSequence = new StringBuilder("\"shard_path\":{\"state_path\":\"");
expectedSubSequence.append(shard.shardPath().getRootStatePath().toString());
expectedSubSequence.append("\",\"data_path\":\"");
expectedSubSequence.append(shard.shardPath().getRootDataPath().toString());
expectedSubSequence.append("\",\"is_custom_data_path\":").append(shard.shardPath().isCustomDataPath()).append("}");
assumeFalse("Some path weirdness on windows", Constants.WINDOWS);
assertTrue(xContent.contains(expectedSubSequence));
}
}

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.index.shard;
import com.carrotsearch.randomizedtesting.annotations.Repeat;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
@ -26,6 +27,7 @@ import org.junit.Test;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Set;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@ -78,4 +80,71 @@ public class ShardPathTests extends ESTestCase {
}
}
@Test(expected = IllegalArgumentException.class)
public void testIllegalCustomDataPath() {
final Path path = createTempDir().resolve("foo").resolve("0");
new ShardPath(true, path, path, "foo", new ShardId("foo", 0));
}
public void testValidCtor() {
final Path path = createTempDir().resolve("foo").resolve("0");
ShardPath shardPath = new ShardPath(false, path, path, "foo", new ShardId("foo", 0));
assertFalse(shardPath.isCustomDataPath());
assertEquals(shardPath.getDataPath(), path);
assertEquals(shardPath.getShardStatePath(), path);
}
public void testGetRootPaths() throws IOException {
boolean useCustomDataPath = randomBoolean();
final Settings indexSetttings;
final Settings nodeSettings;
Settings.Builder indexSettingsBuilder = settingsBuilder().put(IndexMetaData.SETTING_INDEX_UUID, "0xDEADBEEF");
final Path customPath;
if (useCustomDataPath) {
final Path path = createTempDir();
final boolean includeNodeId = randomBoolean();
indexSetttings = indexSettingsBuilder.put(IndexMetaData.SETTING_DATA_PATH, "custom").build();
nodeSettings = settingsBuilder().put("path.shared_data", path.toAbsolutePath().toAbsolutePath())
.put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH, includeNodeId).build();
if (includeNodeId) {
customPath = path.resolve("custom").resolve("0");
} else {
customPath = path.resolve("custom");
}
} else {
customPath = null;
indexSetttings = indexSettingsBuilder.build();
nodeSettings = Settings.EMPTY;
}
try (final NodeEnvironment env = newNodeEnvironment(nodeSettings)) {
ShardId shardId = new ShardId("foo", 0);
Path[] paths = env.availableShardPaths(shardId);
Path path = randomFrom(paths);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(2, true, "0xDEADBEEF"), 2, path);
ShardPath shardPath = ShardPath.loadShardPath(logger, env, shardId, indexSetttings);
boolean found = false;
for (Path p : env.nodeDataPaths()) {
if (p.equals(shardPath.getRootStatePath())) {
found = true;
break;
}
}
assertTrue("root state paths must be a node path but wasn't: " + shardPath.getRootStatePath(), found);
found = false;
if (useCustomDataPath) {
assertNotEquals(shardPath.getRootDataPath(), shardPath.getRootStatePath());
assertEquals(customPath, shardPath.getRootDataPath());
} else {
assertNull(customPath);
for (Path p : env.nodeDataPaths()) {
if (p.equals(shardPath.getRootDataPath())) {
found = true;
break;
}
}
assertTrue("root state paths must be a node path but wasn't: " + shardPath.getRootDataPath(), found);
}
}
}
}

View File

@ -35,11 +35,11 @@ import java.util.Locale;
public class IndexStoreTests extends ESTestCase {
public void testStoreDirectory() throws IOException {
final Path tempDir = createTempDir();
final Path tempDir = createTempDir().resolve("foo").resolve("0");
final IndexStoreModule.Type[] values = IndexStoreModule.Type.values();
final IndexStoreModule.Type type = RandomPicks.randomFrom(random(), values);
Settings settings = Settings.settingsBuilder().put(IndexStoreModule.STORE_TYPE, type.name().toLowerCase(Locale.ROOT)).build();
FsDirectoryService service = new FsDirectoryService(settings, null, new ShardPath(tempDir, tempDir, "foo", new ShardId("foo", 0)));
FsDirectoryService service = new FsDirectoryService(settings, null, new ShardPath(false, tempDir, tempDir, "foo", new ShardId("foo", 0)));
try (final Directory directory = service.newFSDirectory(tempDir, NoLockFactory.INSTANCE)) {
switch (type) {
case NIOFS:
@ -70,9 +70,9 @@ public class IndexStoreTests extends ESTestCase {
}
public void testStoreDirectoryDefault() throws IOException {
final Path tempDir = createTempDir();
final Path tempDir = createTempDir().resolve("foo").resolve("0");
Settings settings = Settings.EMPTY;
FsDirectoryService service = new FsDirectoryService(settings, null, new ShardPath(tempDir, tempDir, "foo", new ShardId("foo", 0)));
FsDirectoryService service = new FsDirectoryService(settings, null, new ShardPath(false, tempDir, tempDir, "foo", new ShardId("foo", 0)));
try (final Directory directory = service.newFSDirectory(tempDir, NoLockFactory.INSTANCE)) {
if (Constants.WINDOWS) {
assertTrue(directory.toString(), directory instanceof MMapDirectory || directory instanceof SimpleFSDirectory);

View File

@ -173,6 +173,18 @@ public class SimpleIndicesWarmerIT extends ESIntegTestCase {
}
}
@Test // issue 8991
public void deleteAllIndexWarmerDoesNotThrowWhenNoWarmers() {
createIndex("test");
DeleteWarmerResponse deleteWarmerResponse = client().admin().indices().prepareDeleteWarmer()
.setIndices("test").setNames("_all").execute().actionGet();
assertThat(deleteWarmerResponse.isAcknowledged(), equalTo(true));
deleteWarmerResponse = client().admin().indices().prepareDeleteWarmer()
.setIndices("test").setNames("foo", "_all", "bar").execute().actionGet();
assertThat(deleteWarmerResponse.isAcknowledged(), equalTo(true));
}
@Test
public void deleteIndexWarmerTest() {
createIndex("test");

View File

@ -255,9 +255,15 @@ public class ChildrenIT extends ESIntegTestCase {
assertThat(count.getValue(), equalTo(4.));
String idToUpdate = Integer.toString(randomInt(3));
/*
* The whole point of this test is to test these things with deleted
* docs in the index so we turn off detect_noop to make sure that
* the updates cause that.
*/
UpdateResponse updateResponse = client().prepareUpdate(indexName, "child", idToUpdate)
.setParent("1")
.setDoc("count", 1)
.setDetectNoop(false)
.get();
assertThat(updateResponse.getVersion(), greaterThan(1l));
refresh();

View File

@ -20,17 +20,21 @@
package org.elasticsearch.ttl;
import com.google.common.base.Predicate;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.update.UpdateRequestBuilder;
import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.get.GetField;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.junit.Test;
import org.elasticsearch.test.ESIntegTestCase.Scope;
import java.io.IOException;
import java.util.Locale;
@ -39,9 +43,17 @@ import java.util.concurrent.TimeUnit;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.ESIntegTestCase.Scope;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.*;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.hamcrest.Matchers.both;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.hasKey;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.lessThan;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
@ClusterScope(scope= Scope.SUITE, numDataNodes = 1)
public class SimpleTTLIT extends ESIntegTestCase {
@ -63,7 +75,6 @@ public class SimpleTTLIT extends ESIntegTestCase {
.build();
}
@Test
public void testSimpleTTL() throws Exception {
assertAcked(prepareCreate("test")
.addMapping("type1", XContentFactory.jsonBuilder()
@ -200,7 +211,7 @@ public class SimpleTTLIT extends ESIntegTestCase {
assertThat(getResponse.isExists(), equalTo(false));
}
@Test // issue 5053
// issue 5053
public void testThatUpdatingMappingShouldNotRemoveTTLConfiguration() throws Exception {
String index = "foo";
String type = "mytype";
@ -220,6 +231,63 @@ public class SimpleTTLIT extends ESIntegTestCase {
assertTTLMappingEnabled(index, type);
}
/**
* Test that updates with detect_noop set to true (the default) that don't
* change the source don't change the ttl. This is unexpected behavior and
* documented in ttl-field.asciidoc. If this behavior changes it is safe to
* rewrite this test to reflect the new behavior and to change the
* documentation.
*/
public void testNoopUpdate() throws IOException {
assertAcked(prepareCreate("test")
.addMapping("type1", XContentFactory.jsonBuilder()
.startObject()
.startObject("type1")
.startObject("_timestamp").field("enabled", true).endObject()
.startObject("_ttl").field("enabled", true).endObject()
.endObject()
.endObject()));
ensureYellow("test");
long aLongTime = 10000000;
long firstTtl = aLongTime * 3;
long secondTtl = aLongTime * 2;
long thirdTtl = aLongTime * 1;
IndexResponse indexResponse = client().prepareIndex("test", "type1", "1").setSource("field1", "value1")
.setTTL(firstTtl).setRefresh(true).get();
assertTrue(indexResponse.isCreated());
assertThat(getTtl("type1", 1), both(lessThan(firstTtl)).and(greaterThan(secondTtl)));
// Updating with the default detect_noop without a change to the document doesn't change the ttl.
UpdateRequestBuilder update = client().prepareUpdate("test", "type1", "1").setDoc("field1", "value1").setTtl(secondTtl);
assertThat(updateAndGetTtl(update), both(lessThan(firstTtl)).and(greaterThan(secondTtl)));
// Updating with the default detect_noop with a change to the document does change the ttl.
update = client().prepareUpdate("test", "type1", "1").setDoc("field1", "value2").setTtl(secondTtl);
assertThat(updateAndGetTtl(update), both(lessThan(secondTtl)).and(greaterThan(thirdTtl)));
// Updating with detect_noop=true without a change to the document doesn't change the ttl.
update = client().prepareUpdate("test", "type1", "1").setDoc("field1", "value2").setTtl(secondTtl).setDetectNoop(true);
assertThat(updateAndGetTtl(update), both(lessThan(secondTtl)).and(greaterThan(thirdTtl)));
// Updating with detect_noop=false without a change to the document does change the ttl.
update = client().prepareUpdate("test", "type1", "1").setDoc("field1", "value2").setTtl(thirdTtl).setDetectNoop(false);
assertThat(updateAndGetTtl(update), lessThan(thirdTtl));
}
private long updateAndGetTtl(UpdateRequestBuilder update) {
UpdateResponse updateResponse = update.setFields("_ttl").get();
assertThat(updateResponse.getShardInfo().getFailed(), equalTo(0));
// You can't actually fetch _ttl from an update so we use a get.
return getTtl(updateResponse.getType(), updateResponse.getId());
}
private long getTtl(String type, Object id) {
GetResponse getResponse = client().prepareGet("test", type, id.toString()).setFields("_ttl").setRealtime(true).execute()
.actionGet();
return ((Number) getResponse.getField("_ttl").getValue()).longValue();
}
private void assertTTLMappingEnabled(String index, String type) throws IOException {
String errMsg = String.format(Locale.ROOT, "Expected ttl field mapping to be enabled for %s/%s", index, type);

View File

@ -19,6 +19,7 @@
package org.elasticsearch.update;
import org.elasticsearch.action.update.UpdateRequestBuilder;
import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -44,8 +45,10 @@ public class UpdateNoopIT extends ESIntegTestCase {
updateAndCheckSource(4, fields("bar", null));
updateAndCheckSource(4, fields("bar", null));
updateAndCheckSource(5, fields("bar", "foo"));
// detect_noop defaults to true
updateAndCheckSource(5, null, fields("bar", "foo"));
assertEquals(3, totalNoopUpdates());
assertEquals(4, totalNoopUpdates());
}
@Test
@ -210,7 +213,8 @@ public class UpdateNoopIT extends ESIntegTestCase {
}
/**
* Totally empty requests are noop if and only if detect noops is true.
* Totally empty requests are noop if and only if detect noops is true and
* its true by default.
*/
@Test
public void totallyEmpty() throws Exception {
@ -223,6 +227,7 @@ public class UpdateNoopIT extends ESIntegTestCase {
.endObject());
update(true, 1, XContentFactory.jsonBuilder().startObject().endObject());
update(false, 2, XContentFactory.jsonBuilder().startObject().endObject());
update(null, 2, XContentFactory.jsonBuilder().startObject().endObject());
}
private XContentBuilder fields(Object... fields) throws IOException {
@ -237,17 +242,23 @@ public class UpdateNoopIT extends ESIntegTestCase {
}
private void updateAndCheckSource(long expectedVersion, XContentBuilder xContentBuilder) {
UpdateResponse updateResponse = update(true, expectedVersion, xContentBuilder);
updateAndCheckSource(expectedVersion, true, xContentBuilder);
}
private void updateAndCheckSource(long expectedVersion, Boolean detectNoop, XContentBuilder xContentBuilder) {
UpdateResponse updateResponse = update(detectNoop, expectedVersion, xContentBuilder);
assertEquals(updateResponse.getGetResult().sourceRef().toUtf8(), xContentBuilder.bytes().toUtf8());
}
private UpdateResponse update(boolean detectNoop, long expectedVersion, XContentBuilder xContentBuilder) {
UpdateResponse updateResponse = client().prepareUpdate("test", "type1", "1")
private UpdateResponse update(Boolean detectNoop, long expectedVersion, XContentBuilder xContentBuilder) {
UpdateRequestBuilder updateRequest = client().prepareUpdate("test", "type1", "1")
.setDoc(xContentBuilder.bytes().toUtf8())
.setDocAsUpsert(true)
.setDetectNoop(detectNoop)
.setFields("_source")
.execute().actionGet();
.setFields("_source");
if (detectNoop != null) {
updateRequest.setDetectNoop(detectNoop);
}
UpdateResponse updateResponse = updateRequest.get();
assertThat(updateResponse.getGetResult(), notNullValue());
assertEquals(expectedVersion, updateResponse.getVersion());
return updateResponse;

View File

@ -114,25 +114,23 @@ If both `doc` and `script` is specified, then `doc` is ignored. Best is
to put your field pairs of the partial document in the script itself.
[float]
=== Detecting noop
By default if `doc` is specified then the document is always updated even
if the merging process doesn't cause any changes. Specifying `detect_noop`
as `true` will cause Elasticsearch to check if there are changes and, if
there aren't, turn the update request into a noop. For example:
=== Detecting noop updates
If `doc` is specified its value is merged with the existing `_source`. By
default the document is only reindexed if the new `_source` field differs from
the old. Setting `detect_noop` to `false` will cause Elasticsearch to always
update the document even if it hasn't changed. For example:
[source,js]
--------------------------------------------------
curl -XPOST 'localhost:9200/test/type1/1/_update' -d '{
"doc" : {
"name" : "new_name"
},
"detect_noop": true
"detect_noop": false
}'
--------------------------------------------------
If `name` was `new_name` before the request was sent then the entire update
request is ignored.
If `name` was `new_name` before the request was sent then document is still
reindexed.
[[upserts]]
[float]

View File

@ -104,3 +104,7 @@ may still be retrieved before they are purged.
How many deletions are handled by a single <<docs-bulk,`bulk`>> request. The
default value is `10000`.
==== Note on `detect_noop`
If an update tries to update just the `_ttl` without changing the `_source` of
the document it's expiration time won't be updated if `detect_noop` is `true`.
In 2.1 `detect_noop` defaults to `true`.

View File

@ -178,7 +178,7 @@ The parameter `percent_terms_to_match` has been removed in favor of
The `limit` filter is deprecated and becomes a no-op. You can achieve similar
behaviour using the <<search-request-body,terminate_after>> parameter.
==== Jave plugins registering custom queries
==== Java plugins registering custom queries
Java plugins that register custom queries can do so by using the
`IndicesQueriesModule#addQuery(Class<? extends QueryParser>)` method. Other

View File

@ -40,6 +40,24 @@ can install the plugin with:
./bin/plugin install delete-by-query
------------------
See {plugins}/plugins-delete-by-query.html for more information.
==== Multicast Discovery is now a plugin
Support for multicast is very patchy. Linux doesnt allow multicast listening on localhost,
while OS/X sends multicast broadcasts across all interfaces regardless of the configured
bind address. On top of that, some networks have multicast disabled by default.
This feature has been moved to a plugin. The default discovery mechanism now uses
unicast, with a default setup which looks for the first 5 ports on localhost. If you
still need to use multicast discovery, you can install the plugin with:
[source,sh]
------------------
./bin/plugin install discovery-multicast
------------------
See {plugins}/discovery-multicast.html for more information.
==== `_shutdown` API

View File

@ -24,4 +24,14 @@ GET /my_index/_search?scroll=2m
Scroll requests sorted by `_doc` have been optimized to more efficiently resume
from where the previous request stopped, so this will have the same performance
characteristics as the former `scan` search type.
characteristics as the former `scan` search type.
=== Update changes
==== Updates now `detect_noop` by default
We've switched the default value of the `detect_noop` option from `false` to
`true`. This means that Elasticsearch will ignore updates that don't change
source unless you explicitly set `"detect_noop": false`. `detect_noop` was
always computationally cheap compared to the expense of the update which can be
thought of as a delete operation followed by an index operation.

View File

@ -41,7 +41,7 @@
},
"request_cache": {
"type" : "boolean",
"description" : "Specify whether the request to be wamred shoyd use the request cache, defaults to index level setting"
"description" : "Specify whether the request to be warmed should use the request cache, defaults to index level setting"
}
}
},

View File

@ -56,7 +56,7 @@
- lte: { _ttl: 100000}
- gt: { _ttl: 10000}
# duration
# seconds
- do:
update:
@ -66,6 +66,7 @@
body:
doc: { foo: baz }
upsert: { foo: bar }
detect_noop: false
ttl: 20s
- do:
@ -89,4 +90,3 @@
body: { foo: bar }
ttl: 20s
timestamp: 2013-06-23T18:14:40