Merge branch 'master' into feature/query-refactoring

This commit is contained in:
Christoph Büscher 2015-07-21 10:58:58 +02:00
commit 37cdc1344a
97 changed files with 1478 additions and 972 deletions

View File

@ -1033,8 +1033,6 @@
<configuration>
<target>
<ant antfile="${elasticsearch.integ.antfile}" target="stop-external-cluster"/>
<!-- TODO: remove this and the xslt when junit4 is fixed -->
<ant antfile="${elasticsearch.integ.antfile}" target="fixup-failsafe-summary"/>
</target>
</configuration>
</execution>

View File

@ -83,6 +83,7 @@ public class TransportIndicesShardStoresAction extends TransportMasterNodeReadAc
final String[] concreteIndices = indexNameExpressionResolver.concreteIndices(state, request);
final Set<ShardId> shardIdsToFetch = new HashSet<>();
logger.trace("using cluster state version [{}] to determine shards", state.version());
// collect relevant shard ids of the requested indices for fetching store infos
for (String index : concreteIndices) {
IndexRoutingTable indexShardRoutingTables = routingTables.index(index);

View File

@ -457,7 +457,7 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
if (item.request() instanceof IndexRequest) {
IndexRequest indexRequest = (IndexRequest) item.request();
try {
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, indexRequest.source()).type(indexRequest.type()).id(indexRequest.id())
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, indexRequest.source()).index(shardId.getIndex()).type(indexRequest.type()).id(indexRequest.id())
.routing(indexRequest.routing()).parent(indexRequest.parent()).timestamp(indexRequest.timestamp()).ttl(indexRequest.ttl());
final Engine.IndexingOperation operation;

View File

@ -180,7 +180,7 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
protected void shardOperationOnReplica(ShardId shardId, IndexRequest request) {
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardId.id());
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, request.source()).type(request.type()).id(request.id())
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, request.source()).index(shardId.getIndex()).type(request.type()).id(request.id())
.routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl());
final Engine.IndexingOperation operation;

View File

@ -1048,7 +1048,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
/** Utility method to create either an index or a create operation depending
* on the {@link OpType} of the request. */
private final Engine.IndexingOperation prepareIndexOperationOnPrimary(BulkShardRequest shardRequest, IndexRequest request, IndexShard indexShard) {
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.PRIMARY, request.source()).type(request.type()).id(request.id())
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.PRIMARY, request.source()).index(request.index()).type(request.type()).id(request.id())
.routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl());
boolean canHaveDuplicates = request.canHaveDuplicates();
if (shardRequest != null) {

View File

@ -52,9 +52,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
private final Map<String, RoutingNode> nodesToShards = newHashMap();
private final UnassignedShards unassignedShards = new UnassignedShards();
private final List<ShardRouting> ignoredUnassignedShards = newArrayList();
private final UnassignedShards unassignedShards = new UnassignedShards(this);
private final Map<ShardId, List<ShardRouting>> assignedShards = newHashMap();
@ -185,10 +183,6 @@ public class RoutingNodes implements Iterable<RoutingNode> {
return !unassignedShards.isEmpty();
}
public List<ShardRouting> ignoredUnassigned() {
return this.ignoredUnassignedShards;
}
public UnassignedShards unassigned() {
return this.unassignedShards;
}
@ -526,9 +520,11 @@ public class RoutingNodes implements Iterable<RoutingNode> {
}
public final static class UnassignedShards implements Iterable<ShardRouting> {
public static final class UnassignedShards implements Iterable<ShardRouting> {
private final RoutingNodes nodes;
private final List<ShardRouting> unassigned;
private final List<ShardRouting> ignored;
private int primaries = 0;
private long transactionId = 0;
@ -536,14 +532,18 @@ public class RoutingNodes implements Iterable<RoutingNode> {
private final long sourceTransactionId;
public UnassignedShards(UnassignedShards other) {
this.nodes = other.nodes;
source = other;
sourceTransactionId = other.transactionId;
unassigned = new ArrayList<>(other.unassigned);
ignored = new ArrayList<>(other.ignored);
primaries = other.primaries;
}
public UnassignedShards() {
public UnassignedShards(RoutingNodes nodes) {
this.nodes = nodes;
unassigned = new ArrayList<>();
ignored = new ArrayList<>();
source = null;
sourceTransactionId = -1;
}
@ -556,12 +556,6 @@ public class RoutingNodes implements Iterable<RoutingNode> {
transactionId++;
}
public void addAll(Collection<ShardRouting> mutableShardRoutings) {
for (ShardRouting r : mutableShardRoutings) {
add(r);
}
}
public void sort(Comparator<ShardRouting> comparator) {
CollectionUtil.timSort(unassigned, comparator);
}
@ -575,29 +569,87 @@ public class RoutingNodes implements Iterable<RoutingNode> {
}
@Override
public Iterator<ShardRouting> iterator() {
final Iterator<ShardRouting> iterator = unassigned.iterator();
return new Iterator<ShardRouting>() {
private ShardRouting current;
@Override
public boolean hasNext() {
return iterator.hasNext();
}
public UnassignedIterator iterator() {
return new UnassignedIterator();
}
@Override
public ShardRouting next() {
return current = iterator.next();
}
/**
* The list of ignored unassigned shards (read only). The ignored unassigned shards
* are not part of the formal unassigned list, but are kept around and used to build
* back the list of unassigned shards as part of the routing table.
*/
public List<ShardRouting> ignored() {
return Collections.unmodifiableList(ignored);
}
@Override
public void remove() {
iterator.remove();
if (current.primary()) {
primaries--;
}
transactionId++;
/**
* Adds a shard to the ignore unassigned list. Should be used with caution, typically,
* the correct usage is to removeAndIgnore from the iterator.
*/
public void ignoreShard(ShardRouting shard) {
ignored.add(shard);
transactionId++;
}
public class UnassignedIterator implements Iterator<ShardRouting> {
private final Iterator<ShardRouting> iterator;
private ShardRouting current;
public UnassignedIterator() {
this.iterator = unassigned.iterator();
}
@Override
public boolean hasNext() {
return iterator.hasNext();
}
@Override
public ShardRouting next() {
return current = iterator.next();
}
/**
* Initializes the current unassigned shard and moves it from the unassigned list.
*/
public void initialize(String nodeId) {
initialize(nodeId, current.version());
}
/**
* Initializes the current unassigned shard and moves it from the unassigned list.
*/
public void initialize(String nodeId, long version) {
innerRemove();
nodes.initialize(new ShardRouting(current, version), nodeId);
}
/**
* Removes and ignores the unassigned shard (will be ignored for this run, but
* will be added back to unassigned once the metadata is constructed again).
*/
public void removeAndIgnore() {
innerRemove();
ignoreShard(current);
}
/**
* Unsupported operation, just there for the interface. Use {@link #removeAndIgnore()} or
* {@link #initialize(String)}.
*/
@Override
public void remove() {
throw new UnsupportedOperationException("remove is not supported in unassigned iterator, use removeAndIgnore or initialize");
}
private void innerRemove() {
iterator.remove();
if (current.primary()) {
primaries--;
}
};
transactionId++;
}
}
public boolean isEmpty() {
@ -611,16 +663,19 @@ public class RoutingNodes implements Iterable<RoutingNode> {
public void clear() {
transactionId++;
unassigned.clear();
ignored.clear();
primaries = 0;
}
public void transactionEnd(UnassignedShards shards) {
assert shards.source == this && shards.sourceTransactionId == transactionId :
"Expected ID: " + shards.sourceTransactionId + " actual: " + transactionId + " Expected Source: " + shards.source + " actual: " + this;
transactionId++;
this.unassigned.clear();
this.unassigned.addAll(shards.unassigned);
this.primaries = shards.primaries;
assert shards.source == this && shards.sourceTransactionId == transactionId :
"Expected ID: " + shards.sourceTransactionId + " actual: " + transactionId + " Expected Source: " + shards.source + " actual: " + this;
transactionId++;
this.unassigned.clear();
this.unassigned.addAll(shards.unassigned);
this.ignored.clear();
this.ignored.addAll(shards.ignored);
this.primaries = shards.primaries;
}
public UnassignedShards transactionBegin() {

View File

@ -362,7 +362,7 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
indexBuilder.addShard(refData, shardRoutingEntry);
}
}
for (ShardRouting shardRoutingEntry : Iterables.concat(routingNodes.unassigned(), routingNodes.ignoredUnassigned())) {
for (ShardRouting shardRoutingEntry : Iterables.concat(routingNodes.unassigned(), routingNodes.unassigned().ignored())) {
String index = shardRoutingEntry.index();
IndexRoutingTable.Builder indexBuilder = indexRoutingTableBuilders.get(index);
if (indexBuilder == null) {

View File

@ -460,22 +460,6 @@ public class AllocationService extends AbstractComponent {
}
}
// move all the shards matching the failed shard to the end of the unassigned list
// so we give a chance for other allocations and won't create poison failed allocations
// that can keep other shards from being allocated (because of limits applied on how many
// shards we can start per node)
List<ShardRouting> shardsToMove = Lists.newArrayList();
for (Iterator<ShardRouting> unassignedIt = routingNodes.unassigned().iterator(); unassignedIt.hasNext(); ) {
ShardRouting unassignedShardRouting = unassignedIt.next();
if (unassignedShardRouting.shardId().equals(failedShard.shardId())) {
unassignedIt.remove();
shardsToMove.add(unassignedShardRouting);
}
}
if (!shardsToMove.isEmpty()) {
routingNodes.unassigned().addAll(shardsToMove);
}
matchedNode.moveToUnassigned(unassignedInfo);
}
assert matchedNode.isRemoved() : "failedShard " + failedShard + " was matched but wasn't removed";

View File

@ -41,7 +41,6 @@ import org.elasticsearch.node.settings.NodeSettingsService;
import java.util.*;
import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
/**
@ -292,7 +291,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
}
indices.addAll(allocation.routingTable().indicesRouting().keySet());
buildModelFromAssigned(routing.shards(assignedFilter));
return allocateUnassigned(unassigned, routing.ignoredUnassigned());
return allocateUnassigned(unassigned);
}
private static float absDelta(float lower, float higher) {
@ -551,7 +550,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
* Allocates all given shards on the minimal eligable node for the shards index
* with respect to the weight function. All given shards must be unassigned.
*/
private boolean allocateUnassigned(RoutingNodes.UnassignedShards unassigned, List<ShardRouting> ignoredUnassigned) {
private boolean allocateUnassigned(RoutingNodes.UnassignedShards unassigned) {
assert !nodes.isEmpty();
if (logger.isTraceEnabled()) {
logger.trace("Start allocating unassigned shards");
@ -600,9 +599,9 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
if (!shard.primary()) {
boolean drop = deciders.canAllocate(shard, allocation).type() == Type.NO;
if (drop) {
ignoredUnassigned.add(shard);
unassigned.ignoreShard(shard);
while(i < primaryLength-1 && comparator.compare(primary[i], primary[i+1]) == 0) {
ignoredUnassigned.add(primary[++i]);
unassigned.ignoreShard(primary[++i]);
}
continue;
} else {
@ -706,10 +705,10 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
} else if (logger.isTraceEnabled()) {
logger.trace("No Node found to assign shard [{}]", shard);
}
ignoredUnassigned.add(shard);
unassigned.ignoreShard(shard);
if (!shard.primary()) { // we could not allocate it and we are a replica - check if we can ignore the other replicas
while(secondaryLength > 0 && comparator.compare(shard, secondary[secondaryLength-1]) == 0) {
ignoredUnassigned.add(secondary[--secondaryLength]);
unassigned.ignoreShard(secondary[--secondaryLength]);
}
}
}

View File

@ -220,12 +220,11 @@ public class AllocateAllocationCommand implements AllocationCommand {
throw new IllegalArgumentException("[allocate] allocation of " + shardId + " on node " + discoNode + " is not allowed, reason: " + decision);
}
// go over and remove it from the unassigned
for (Iterator<ShardRouting> it = routingNodes.unassigned().iterator(); it.hasNext(); ) {
for (RoutingNodes.UnassignedShards.UnassignedIterator it = routingNodes.unassigned().iterator(); it.hasNext(); ) {
if (it.next() != shardRouting) {
continue;
}
it.remove();
routingNodes.initialize(shardRouting, routingNode.nodeId());
it.initialize(routingNode.nodeId());
if (shardRouting.primary()) {
// we need to clear the post allocation flag, since its an explicit allocation of the primary shard
// and we want to force allocate it (and create a new index for it)

View File

@ -19,41 +19,28 @@
package org.elasticsearch.gateway;
import com.carrotsearch.hppc.ObjectLongHashMap;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.action.support.nodes.BaseNodeResponse;
import org.elasticsearch.action.support.nodes.BaseNodesResponse;
import org.elasticsearch.cluster.*;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.*;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.cluster.routing.RoutingService;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData;
import java.util.*;
import java.util.concurrent.ConcurrentMap;
/**
@ -61,26 +48,19 @@ import java.util.concurrent.ConcurrentMap;
*/
public class GatewayAllocator extends AbstractComponent {
public static final String INDEX_RECOVERY_INITIAL_SHARDS = "index.recovery.initial_shards";
private final String initialShards;
private final TransportNodesListGatewayStartedShards startedAction;
private final TransportNodesListShardStoreMetaData storeAction;
private RoutingService routingService;
private final PrimaryShardAllocator primaryShardAllocator;
private final ReplicaShardAllocator replicaShardAllocator;
private final ConcurrentMap<ShardId, AsyncShardFetch<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards>> asyncFetchStarted = ConcurrentCollections.newConcurrentMap();
private final ConcurrentMap<ShardId, AsyncShardFetch<TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData>> asyncFetchStore = ConcurrentCollections.newConcurrentMap();
@Inject
public GatewayAllocator(Settings settings, TransportNodesListGatewayStartedShards startedAction, TransportNodesListShardStoreMetaData storeAction) {
public GatewayAllocator(Settings settings, final TransportNodesListGatewayStartedShards startedAction, final TransportNodesListShardStoreMetaData storeAction) {
super(settings);
this.startedAction = startedAction;
this.storeAction = storeAction;
this.initialShards = settings.get("gateway.initial_shards", settings.get("gateway.local.initial_shards", "quorum"));
logger.debug("using initial_shards [{}]", initialShards);
this.primaryShardAllocator = new InternalPrimaryShardAllocator(settings, startedAction);
this.replicaShardAllocator = new InternalReplicaShardAllocator(settings, storeAction);
}
public void setReallocation(final ClusterService clusterService, final RoutingService routingService) {
@ -132,416 +112,21 @@ public class GatewayAllocator extends AbstractComponent {
}
}
/**
* Return {@code true} if the index is configured to allow shards to be
* recovered on any node
*/
private boolean recoverOnAnyNode(@IndexSettings Settings idxSettings) {
return IndexMetaData.isOnSharedFilesystem(idxSettings) &&
idxSettings.getAsBoolean(IndexMetaData.SETTING_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE, false);
}
public boolean allocateUnassigned(RoutingAllocation allocation) {
public boolean allocateUnassigned(final RoutingAllocation allocation) {
boolean changed = false;
DiscoveryNodes nodes = allocation.nodes();
RoutingNodes routingNodes = allocation.routingNodes();
// First, handle primaries, they must find a place to be allocated on here
final MetaData metaData = routingNodes.metaData();
RoutingNodes.UnassignedShards unassigned = routingNodes.unassigned();
RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned();
unassigned.sort(new PriorityComparator() {
@Override
protected Settings getIndexSettings(String index) {
IndexMetaData indexMetaData = metaData.index(index);
IndexMetaData indexMetaData = allocation.metaData().index(index);
return indexMetaData.getSettings();
}
}); // sort for priority ordering
Iterator<ShardRouting> unassignedIterator = unassigned.iterator();
while (unassignedIterator.hasNext()) {
ShardRouting shard = unassignedIterator.next();
if (!shard.primary()) {
continue;
}
// this is an API allocation, ignore since we know there is no data...
if (!routingNodes.routingTable().index(shard.index()).shard(shard.id()).primaryAllocatedPostApi()) {
continue;
}
AsyncShardFetch<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> fetch = asyncFetchStarted.get(shard.shardId());
if (fetch == null) {
fetch = new InternalAsyncFetch<>(logger, "shard_started", shard.shardId(), startedAction);
asyncFetchStarted.put(shard.shardId(), fetch);
}
AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> shardState = fetch.fetchData(nodes, metaData, allocation.getIgnoreNodes(shard.shardId()));
if (shardState.hasData() == false) {
logger.trace("{}: ignoring allocation, still fetching shard started state", shard);
unassignedIterator.remove();
routingNodes.ignoredUnassigned().add(shard);
continue;
}
shardState.processAllocation(allocation);
IndexMetaData indexMetaData = metaData.index(shard.getIndex());
/**
* Build a map of DiscoveryNodes to shard state number for the given shard.
* A state of -1 means the shard does not exist on the node, where any
* shard state >= 0 is the state version of the shard on that node's disk.
*
* A shard on shared storage will return at least shard state 0 for all
* nodes, indicating that the shard can be allocated to any node.
*/
ObjectLongHashMap<DiscoveryNode> nodesState = new ObjectLongHashMap<>();
for (TransportNodesListGatewayStartedShards.NodeGatewayStartedShards nodeShardState : shardState.getData().values()) {
long version = nodeShardState.version();
// -1 version means it does not exists, which is what the API returns, and what we expect to
if (nodeShardState.storeException() == null) {
logger.trace("[{}] on node [{}] has version [{}] of shard", shard, nodeShardState.getNode(), version);
nodesState.put(nodeShardState.getNode(), version);
} else {
// when there is an store exception, we disregard the reported version and assign it as -1 (same as shard does not exist)
logger.trace("[{}] on node [{}] has version [{}] but the store can not be opened, treating as version -1", nodeShardState.storeException(), shard, nodeShardState.getNode(), version);
nodesState.put(nodeShardState.getNode(), -1);
}
}
int numberOfAllocationsFound = 0;
long highestVersion = -1;
final Map<DiscoveryNode, Long> nodesWithVersion = Maps.newHashMap();
assert !nodesState.containsKey(null);
final Object[] keys = nodesState.keys;
final long[] values = nodesState.values;
Settings idxSettings = indexMetaData.settings();
for (int i = 0; i < keys.length; i++) {
if (keys[i] == null) {
continue;
}
DiscoveryNode node = (DiscoveryNode) keys[i];
long version = values[i];
// since we don't check in NO allocation, we need to double check here
if (allocation.shouldIgnoreShardForNode(shard.shardId(), node.id())) {
continue;
}
if (recoverOnAnyNode(idxSettings)) {
numberOfAllocationsFound++;
if (version > highestVersion) {
highestVersion = version;
}
// We always put the node without clearing the map
nodesWithVersion.put(node, version);
} else if (version != -1) {
numberOfAllocationsFound++;
// If we've found a new "best" candidate, clear the
// current candidates and add it
if (version > highestVersion) {
highestVersion = version;
nodesWithVersion.clear();
nodesWithVersion.put(node, version);
} else if (version == highestVersion) {
// If the candidate is the same, add it to the
// list, but keep the current candidate
nodesWithVersion.put(node, version);
}
}
}
// Now that we have a map of nodes to versions along with the
// number of allocations found (and not ignored), we need to sort
// it so the node with the highest version is at the beginning
List<DiscoveryNode> nodesWithHighestVersion = Lists.newArrayList();
nodesWithHighestVersion.addAll(nodesWithVersion.keySet());
CollectionUtil.timSort(nodesWithHighestVersion, new Comparator<DiscoveryNode>() {
@Override
public int compare(DiscoveryNode o1, DiscoveryNode o2) {
return Long.compare(nodesWithVersion.get(o2), nodesWithVersion.get(o1));
}
});
if (logger.isDebugEnabled()) {
logger.debug("[{}][{}] found {} allocations of {}, highest version: [{}]",
shard.index(), shard.id(), numberOfAllocationsFound, shard, highestVersion);
}
if (logger.isTraceEnabled()) {
StringBuilder sb = new StringBuilder("[");
for (DiscoveryNode n : nodesWithHighestVersion) {
sb.append("[");
sb.append(n.getName());
sb.append("]");
sb.append(" -> ");
sb.append(nodesWithVersion.get(n));
sb.append(", ");
}
sb.append("]");
logger.trace("{} candidates for allocation: {}", shard, sb.toString());
}
// check if the counts meets the minimum set
int requiredAllocation = 1;
// if we restore from a repository one copy is more then enough
if (shard.restoreSource() == null) {
try {
String initialShards = indexMetaData.settings().get(INDEX_RECOVERY_INITIAL_SHARDS, settings.get(INDEX_RECOVERY_INITIAL_SHARDS, this.initialShards));
if ("quorum".equals(initialShards)) {
if (indexMetaData.numberOfReplicas() > 1) {
requiredAllocation = ((1 + indexMetaData.numberOfReplicas()) / 2) + 1;
}
} else if ("quorum-1".equals(initialShards) || "half".equals(initialShards)) {
if (indexMetaData.numberOfReplicas() > 2) {
requiredAllocation = ((1 + indexMetaData.numberOfReplicas()) / 2);
}
} else if ("one".equals(initialShards)) {
requiredAllocation = 1;
} else if ("full".equals(initialShards) || "all".equals(initialShards)) {
requiredAllocation = indexMetaData.numberOfReplicas() + 1;
} else if ("full-1".equals(initialShards) || "all-1".equals(initialShards)) {
if (indexMetaData.numberOfReplicas() > 1) {
requiredAllocation = indexMetaData.numberOfReplicas();
}
} else {
requiredAllocation = Integer.parseInt(initialShards);
}
} catch (Exception e) {
logger.warn("[{}][{}] failed to derived initial_shards from value {}, ignore allocation for {}", shard.index(), shard.id(), initialShards, shard);
}
}
// not enough found for this shard, continue...
if (numberOfAllocationsFound < requiredAllocation) {
// if we are restoring this shard we still can allocate
if (shard.restoreSource() == null) {
// we can't really allocate, so ignore it and continue
unassignedIterator.remove();
routingNodes.ignoredUnassigned().add(shard);
if (logger.isDebugEnabled()) {
logger.debug("[{}][{}]: not allocating, number_of_allocated_shards_found [{}], required_number [{}]", shard.index(), shard.id(), numberOfAllocationsFound, requiredAllocation);
}
} else if (logger.isDebugEnabled()) {
logger.debug("[{}][{}]: missing local data, will restore from [{}]", shard.index(), shard.id(), shard.restoreSource());
}
continue;
}
Set<DiscoveryNode> throttledNodes = Sets.newHashSet();
Set<DiscoveryNode> noNodes = Sets.newHashSet();
for (DiscoveryNode discoNode : nodesWithHighestVersion) {
RoutingNode node = routingNodes.node(discoNode.id());
if (node == null) {
continue;
}
Decision decision = allocation.deciders().canAllocate(shard, node, allocation);
if (decision.type() == Decision.Type.THROTTLE) {
throttledNodes.add(discoNode);
} else if (decision.type() == Decision.Type.NO) {
noNodes.add(discoNode);
} else {
if (logger.isDebugEnabled()) {
logger.debug("[{}][{}]: allocating [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, discoNode);
}
// we found a match
changed = true;
// make sure we create one with the version from the recovered state
routingNodes.initialize(new ShardRouting(shard, highestVersion), node.nodeId());
unassignedIterator.remove();
// found a node, so no throttling, no "no", and break out of the loop
throttledNodes.clear();
noNodes.clear();
break;
}
}
if (throttledNodes.isEmpty()) {
// if we have a node that we "can't" allocate to, force allocation, since this is our master data!
if (!noNodes.isEmpty()) {
DiscoveryNode discoNode = noNodes.iterator().next();
RoutingNode node = routingNodes.node(discoNode.id());
if (logger.isDebugEnabled()) {
logger.debug("[{}][{}]: forcing allocating [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, discoNode);
}
// we found a match
changed = true;
// make sure we create one with the version from the recovered state
routingNodes.initialize(new ShardRouting(shard, highestVersion), node.nodeId());
unassignedIterator.remove();
}
} else {
if (logger.isDebugEnabled()) {
logger.debug("[{}][{}]: throttling allocation [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, throttledNodes);
}
// we are throttling this, but we have enough to allocate to this node, ignore it for now
unassignedIterator.remove();
routingNodes.ignoredUnassigned().add(shard);
}
}
if (!routingNodes.hasUnassigned()) {
return changed;
}
// Now, handle replicas, try to assign them to nodes that are similar to the one the primary was allocated on
unassignedIterator = unassigned.iterator();
while (unassignedIterator.hasNext()) {
ShardRouting shard = unassignedIterator.next();
if (shard.primary()) {
continue;
}
// pre-check if it can be allocated to any node that currently exists, so we won't list the store for it for nothing
boolean canBeAllocatedToAtLeastOneNode = false;
for (ObjectCursor<DiscoveryNode> cursor : nodes.dataNodes().values()) {
RoutingNode node = routingNodes.node(cursor.value.id());
if (node == null) {
continue;
}
// if we can't allocate it on a node, ignore it, for example, this handles
// cases for only allocating a replica after a primary
Decision decision = allocation.deciders().canAllocate(shard, node, allocation);
if (decision.type() == Decision.Type.YES) {
canBeAllocatedToAtLeastOneNode = true;
break;
}
}
if (!canBeAllocatedToAtLeastOneNode) {
logger.trace("{}: ignoring allocation, can't be allocated on any node", shard);
unassignedIterator.remove();
routingNodes.ignoredUnassigned().add(shard);
continue;
}
AsyncShardFetch<TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData> fetch = asyncFetchStore.get(shard.shardId());
if (fetch == null) {
fetch = new InternalAsyncFetch<>(logger, "shard_store", shard.shardId(), storeAction);
asyncFetchStore.put(shard.shardId(), fetch);
}
AsyncShardFetch.FetchResult<TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData> shardStores = fetch.fetchData(nodes, metaData, allocation.getIgnoreNodes(shard.shardId()));
if (shardStores.hasData() == false) {
logger.trace("{}: ignoring allocation, still fetching shard stores", shard);
unassignedIterator.remove();
routingNodes.ignoredUnassigned().add(shard);
continue; // still fetching
}
shardStores.processAllocation(allocation);
long lastSizeMatched = 0;
DiscoveryNode lastDiscoNodeMatched = null;
RoutingNode lastNodeMatched = null;
boolean hasReplicaData = false;
IndexMetaData indexMetaData = metaData.index(shard.getIndex());
for (Map.Entry<DiscoveryNode, TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData> nodeStoreEntry : shardStores.getData().entrySet()) {
DiscoveryNode discoNode = nodeStoreEntry.getKey();
TransportNodesListShardStoreMetaData.StoreFilesMetaData storeFilesMetaData = nodeStoreEntry.getValue().storeFilesMetaData();
logger.trace("{}: checking node [{}]", shard, discoNode);
if (storeFilesMetaData == null) {
// already allocated on that node...
continue;
}
RoutingNode node = routingNodes.node(discoNode.id());
if (node == null) {
continue;
}
// check if we can allocate on that node...
// we only check for NO, since if this node is THROTTLING and it has enough "same data"
// then we will try and assign it next time
Decision decision = allocation.deciders().canAllocate(shard, node, allocation);
if (decision.type() == Decision.Type.NO) {
continue;
}
// if it is already allocated, we can't assign to it...
if (storeFilesMetaData.allocated()) {
continue;
}
if (!shard.primary()) {
hasReplicaData |= storeFilesMetaData.iterator().hasNext();
ShardRouting primaryShard = routingNodes.activePrimary(shard);
if (primaryShard != null) {
assert primaryShard.active();
DiscoveryNode primaryNode = nodes.get(primaryShard.currentNodeId());
if (primaryNode != null) {
TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData primaryNodeFilesStore = shardStores.getData().get(primaryNode);
if (primaryNodeFilesStore != null) {
TransportNodesListShardStoreMetaData.StoreFilesMetaData primaryNodeStore = primaryNodeFilesStore.storeFilesMetaData();
if (primaryNodeStore != null && primaryNodeStore.allocated()) {
long sizeMatched = 0;
String primarySyncId = primaryNodeStore.syncId();
String replicaSyncId = storeFilesMetaData.syncId();
// see if we have a sync id we can make use of
if (replicaSyncId != null && replicaSyncId.equals(primarySyncId)) {
logger.trace("{}: node [{}] has same sync id {} as primary", shard, discoNode.name(), replicaSyncId);
lastNodeMatched = node;
lastSizeMatched = Long.MAX_VALUE;
lastDiscoNodeMatched = discoNode;
} else {
for (StoreFileMetaData storeFileMetaData : storeFilesMetaData) {
String metaDataFileName = storeFileMetaData.name();
if (primaryNodeStore.fileExists(metaDataFileName) && primaryNodeStore.file(metaDataFileName).isSame(storeFileMetaData)) {
sizeMatched += storeFileMetaData.length();
}
}
logger.trace("{}: node [{}] has [{}/{}] bytes of re-usable data",
shard, discoNode.name(), new ByteSizeValue(sizeMatched), sizeMatched);
if (sizeMatched > lastSizeMatched) {
lastSizeMatched = sizeMatched;
lastDiscoNodeMatched = discoNode;
lastNodeMatched = node;
}
}
}
}
}
}
}
}
if (lastNodeMatched != null) {
// we only check on THROTTLE since we checked before before on NO
Decision decision = allocation.deciders().canAllocate(shard, lastNodeMatched, allocation);
if (decision.type() == Decision.Type.THROTTLE) {
if (logger.isDebugEnabled()) {
logger.debug("[{}][{}]: throttling allocation [{}] to [{}] in order to reuse its unallocated persistent store with total_size [{}]", shard.index(), shard.id(), shard, lastDiscoNodeMatched, new ByteSizeValue(lastSizeMatched));
}
// we are throttling this, but we have enough to allocate to this node, ignore it for now
unassignedIterator.remove();
routingNodes.ignoredUnassigned().add(shard);
} else {
if (logger.isDebugEnabled()) {
logger.debug("[{}][{}]: allocating [{}] to [{}] in order to reuse its unallocated persistent store with total_size [{}]", shard.index(), shard.id(), shard, lastDiscoNodeMatched, new ByteSizeValue(lastSizeMatched));
}
// we found a match
changed = true;
routingNodes.initialize(shard, lastNodeMatched.nodeId());
unassignedIterator.remove();
}
} else if (hasReplicaData == false) {
// if we didn't manage to find *any* data (regardless of matching sizes), check if the allocation
// of the replica shard needs to be delayed, and if so, add it to the ignore unassigned list
// note: we only care about replica in delayed allocation, since if we have an unassigned primary it
// will anyhow wait to find an existing copy of the shard to be allocated
// note: the other side of the equation is scheduling a reroute in a timely manner, which happens in the RoutingService
long delay = shard.unassignedInfo().getDelayAllocationExpirationIn(settings, indexMetaData.getSettings());
if (delay > 0) {
logger.debug("[{}][{}]: delaying allocation of [{}] for [{}]", shard.index(), shard.id(), shard, TimeValue.timeValueMillis(delay));
/**
* mark it as changed, since we want to kick a publishing to schedule future allocation,
* see {@link org.elasticsearch.cluster.routing.RoutingService#clusterChanged(ClusterChangedEvent)}).
*/
changed = true;
unassignedIterator.remove();
routingNodes.ignoredUnassigned().add(shard);
}
}
}
changed |= primaryShardAllocator.allocateUnassigned(allocation);
changed |= replicaShardAllocator.allocateUnassigned(allocation);
return changed;
}
@ -558,4 +143,54 @@ public class GatewayAllocator extends AbstractComponent {
}
}
class InternalPrimaryShardAllocator extends PrimaryShardAllocator {
private final TransportNodesListGatewayStartedShards startedAction;
public InternalPrimaryShardAllocator(Settings settings, TransportNodesListGatewayStartedShards startedAction) {
super(settings);
this.startedAction = startedAction;
}
@Override
protected AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> fetchData(ShardRouting shard, RoutingAllocation allocation) {
AsyncShardFetch<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> fetch = asyncFetchStarted.get(shard.shardId());
if (fetch == null) {
fetch = new InternalAsyncFetch<>(logger, "shard_started", shard.shardId(), startedAction);
asyncFetchStarted.put(shard.shardId(), fetch);
}
AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> shardState =
fetch.fetchData(allocation.nodes(), allocation.metaData(), allocation.getIgnoreNodes(shard.shardId()));
if (shardState.hasData() == true) {
shardState.processAllocation(allocation);
}
return shardState;
}
}
class InternalReplicaShardAllocator extends ReplicaShardAllocator {
private final TransportNodesListShardStoreMetaData storeAction;
public InternalReplicaShardAllocator(Settings settings, TransportNodesListShardStoreMetaData storeAction) {
super(settings);
this.storeAction = storeAction;
}
@Override
protected AsyncShardFetch.FetchResult<TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData> fetchData(ShardRouting shard, RoutingAllocation allocation) {
AsyncShardFetch<TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData> fetch = asyncFetchStore.get(shard.shardId());
if (fetch == null) {
fetch = new InternalAsyncFetch<>(logger, "shard_store", shard.shardId(), storeAction);
asyncFetchStore.put(shard.shardId(), fetch);
}
AsyncShardFetch.FetchResult<TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData> shardStores =
fetch.fetchData(allocation.nodes(), allocation.metaData(), allocation.getIgnoreNodes(shard.shardId()));
if (shardStores.hasData() == true) {
shardStores.processAllocation(allocation);
}
return shardStores;
}
}
}

View File

@ -0,0 +1,293 @@
/*
* 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.gateway;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
import java.util.*;
/**
* The primary shard allocator allocates primary shard that were not created as
* a result of an API to a node that held them last to be recovered.
*/
public abstract class PrimaryShardAllocator extends AbstractComponent {
public static final String INDEX_RECOVERY_INITIAL_SHARDS = "index.recovery.initial_shards";
private final String initialShards;
public PrimaryShardAllocator(Settings settings) {
super(settings);
this.initialShards = settings.get("gateway.initial_shards", settings.get("gateway.local.initial_shards", "quorum"));
logger.debug("using initial_shards [{}]", initialShards);
}
public boolean allocateUnassigned(RoutingAllocation allocation) {
boolean changed = false;
final RoutingNodes routingNodes = allocation.routingNodes();
final MetaData metaData = routingNodes.metaData();
final RoutingNodes.UnassignedShards.UnassignedIterator unassignedIterator = routingNodes.unassigned().iterator();
while (unassignedIterator.hasNext()) {
ShardRouting shard = unassignedIterator.next();
if (needToFindPrimaryCopy(shard, routingNodes.routingTable().index(shard.index()).shard(shard.id())) == false) {
continue;
}
AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> shardState = fetchData(shard, allocation);
if (shardState.hasData() == false) {
logger.trace("{}: ignoring allocation, still fetching shard started state", shard);
unassignedIterator.removeAndIgnore();
continue;
}
IndexMetaData indexMetaData = metaData.index(shard.getIndex());
NodesAndVersions nodesAndVersions = buildNodesAndVersions(shard, recoverOnAnyNode(indexMetaData.settings()), allocation.getIgnoreNodes(shard.shardId()), shardState);
logger.debug("[{}][{}] found {} allocations of {}, highest version: [{}]", shard.index(), shard.id(), nodesAndVersions.allocationsFound, shard, nodesAndVersions.highestVersion);
if (isEnoughAllocationsFound(shard, indexMetaData, nodesAndVersions) == false) {
// if we are restoring this shard we still can allocate
if (shard.restoreSource() == null) {
// we can't really allocate, so ignore it and continue
unassignedIterator.removeAndIgnore();
logger.debug("[{}][{}]: not allocating, number_of_allocated_shards_found [{}]", shard.index(), shard.id(), nodesAndVersions.allocationsFound);
} else {
logger.debug("[{}][{}]: missing local data, will restore from [{}]", shard.index(), shard.id(), shard.restoreSource());
}
continue;
}
NodesToAllocate nodesToAllocate = buildNodesToAllocate(shard, allocation, nodesAndVersions);
if (nodesToAllocate.yesNodes.isEmpty() == false) {
DiscoveryNode node = nodesToAllocate.yesNodes.get(0);
logger.debug("[{}][{}]: allocating [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, node);
changed = true;
unassignedIterator.initialize(node.id(), nodesAndVersions.highestVersion);
} else if (nodesToAllocate.throttleNodes.isEmpty() == true && nodesToAllocate.noNodes.isEmpty() == false) {
DiscoveryNode node = nodesToAllocate.noNodes.get(0);
logger.debug("[{}][{}]: forcing allocating [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, node);
changed = true;
unassignedIterator.initialize(node.id(), nodesAndVersions.highestVersion);
} else {
// we are throttling this, but we have enough to allocate to this node, ignore it for now
logger.debug("[{}][{}]: throttling allocation [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, nodesToAllocate.throttleNodes);
unassignedIterator.removeAndIgnore();
}
}
return changed;
}
/**
* Does the shard need to find a primary copy?
*/
boolean needToFindPrimaryCopy(ShardRouting shard, IndexShardRoutingTable indexShardRoutingTable) {
if (shard.primary() == false) {
return false;
}
// this is an API allocation, ignore since we know there is no data...
if (indexShardRoutingTable.primaryAllocatedPostApi() == false) {
return false;
}
return true;
}
private boolean isEnoughAllocationsFound(ShardRouting shard, IndexMetaData indexMetaData, NodesAndVersions nodesAndVersions) {
// check if the counts meets the minimum set
int requiredAllocation = 1;
// if we restore from a repository one copy is more then enough
if (shard.restoreSource() == null) {
try {
String initialShards = indexMetaData.settings().get(INDEX_RECOVERY_INITIAL_SHARDS, settings.get(INDEX_RECOVERY_INITIAL_SHARDS, this.initialShards));
if ("quorum".equals(initialShards)) {
if (indexMetaData.numberOfReplicas() > 1) {
requiredAllocation = ((1 + indexMetaData.numberOfReplicas()) / 2) + 1;
}
} else if ("quorum-1".equals(initialShards) || "half".equals(initialShards)) {
if (indexMetaData.numberOfReplicas() > 2) {
requiredAllocation = ((1 + indexMetaData.numberOfReplicas()) / 2);
}
} else if ("one".equals(initialShards)) {
requiredAllocation = 1;
} else if ("full".equals(initialShards) || "all".equals(initialShards)) {
requiredAllocation = indexMetaData.numberOfReplicas() + 1;
} else if ("full-1".equals(initialShards) || "all-1".equals(initialShards)) {
if (indexMetaData.numberOfReplicas() > 1) {
requiredAllocation = indexMetaData.numberOfReplicas();
}
} else {
requiredAllocation = Integer.parseInt(initialShards);
}
} catch (Exception e) {
logger.warn("[{}][{}] failed to derived initial_shards from value {}, ignore allocation for {}", shard.index(), shard.id(), initialShards, shard);
}
}
return nodesAndVersions.allocationsFound >= requiredAllocation;
}
/**
* Based on the nodes and versions, build the list of yes/no/throttle nodes that the shard applies to.
*/
private NodesToAllocate buildNodesToAllocate(ShardRouting shard, RoutingAllocation allocation, NodesAndVersions nodesAndVersions) {
List<DiscoveryNode> yesNodes = new ArrayList<>();
List<DiscoveryNode> throttledNodes = new ArrayList<>();
List<DiscoveryNode> noNodes = new ArrayList<>();
for (DiscoveryNode discoNode : nodesAndVersions.nodes) {
RoutingNode node = allocation.routingNodes().node(discoNode.id());
if (node == null) {
continue;
}
Decision decision = allocation.deciders().canAllocate(shard, node, allocation);
if (decision.type() == Decision.Type.THROTTLE) {
throttledNodes.add(discoNode);
} else if (decision.type() == Decision.Type.NO) {
noNodes.add(discoNode);
} else {
yesNodes.add(discoNode);
}
}
return new NodesToAllocate(Collections.unmodifiableList(yesNodes), Collections.unmodifiableList(throttledNodes), Collections.unmodifiableList(noNodes));
}
/**
* Builds a list of nodes and version
*/
private NodesAndVersions buildNodesAndVersions(ShardRouting shard, boolean recoveryOnAnyNode, Set<String> ignoreNodes,
AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> shardState) {
final Map<DiscoveryNode, Long> nodesWithVersion = Maps.newHashMap();
int numberOfAllocationsFound = 0;
long highestVersion = -1;
for (TransportNodesListGatewayStartedShards.NodeGatewayStartedShards nodeShardState : shardState.getData().values()) {
long version = nodeShardState.version();
DiscoveryNode node = nodeShardState.getNode();
if (ignoreNodes.contains(node.id())) {
continue;
}
// -1 version means it does not exists, which is what the API returns, and what we expect to
if (nodeShardState.storeException() == null) {
logger.trace("[{}] on node [{}] has version [{}] of shard", shard, nodeShardState.getNode(), version);
} else {
// when there is an store exception, we disregard the reported version and assign it as -1 (same as shard does not exist)
logger.trace("[{}] on node [{}] has version [{}] but the store can not be opened, treating as version -1", nodeShardState.storeException(), shard, nodeShardState.getNode(), version);
version = -1;
}
if (recoveryOnAnyNode) {
numberOfAllocationsFound++;
if (version > highestVersion) {
highestVersion = version;
}
// We always put the node without clearing the map
nodesWithVersion.put(node, version);
} else if (version != -1) {
numberOfAllocationsFound++;
// If we've found a new "best" candidate, clear the
// current candidates and add it
if (version > highestVersion) {
highestVersion = version;
nodesWithVersion.clear();
nodesWithVersion.put(node, version);
} else if (version == highestVersion) {
// If the candidate is the same, add it to the
// list, but keep the current candidate
nodesWithVersion.put(node, version);
}
}
}
// Now that we have a map of nodes to versions along with the
// number of allocations found (and not ignored), we need to sort
// it so the node with the highest version is at the beginning
List<DiscoveryNode> nodesWithHighestVersion = Lists.newArrayList();
nodesWithHighestVersion.addAll(nodesWithVersion.keySet());
CollectionUtil.timSort(nodesWithHighestVersion, new Comparator<DiscoveryNode>() {
@Override
public int compare(DiscoveryNode o1, DiscoveryNode o2) {
return Long.compare(nodesWithVersion.get(o2), nodesWithVersion.get(o1));
}
});
if (logger.isTraceEnabled()) {
StringBuilder sb = new StringBuilder("[");
for (DiscoveryNode n : nodesWithVersion.keySet()) {
sb.append("[").append(n.getName()).append("]").append(" -> ").append(nodesWithVersion.get(n)).append(", ");
}
sb.append("]");
logger.trace("{} candidates for allocation: {}", shard, sb.toString());
}
return new NodesAndVersions(Collections.unmodifiableList(nodesWithHighestVersion), numberOfAllocationsFound, highestVersion);
}
/**
* Return {@code true} if the index is configured to allow shards to be
* recovered on any node
*/
private boolean recoverOnAnyNode(@IndexSettings Settings idxSettings) {
return IndexMetaData.isOnSharedFilesystem(idxSettings) &&
idxSettings.getAsBoolean(IndexMetaData.SETTING_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE, false);
}
protected abstract AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> fetchData(ShardRouting shard, RoutingAllocation allocation);
static class NodesAndVersions {
public final List<DiscoveryNode> nodes;
public final int allocationsFound;
public final long highestVersion;
public NodesAndVersions(List<DiscoveryNode> nodes, int allocationsFound, long highestVersion) {
this.nodes = nodes;
this.allocationsFound = allocationsFound;
this.highestVersion = highestVersion;
}
}
static class NodesToAllocate {
final List<DiscoveryNode> yesNodes;
final List<DiscoveryNode> throttleNodes;
final List<DiscoveryNode> noNodes;
public NodesToAllocate(List<DiscoveryNode> yesNodes, List<DiscoveryNode> throttleNodes, List<DiscoveryNode> noNodes) {
this.yesNodes = yesNodes;
this.throttleNodes = throttleNodes;
this.noNodes = noNodes;
}
}
}

View File

@ -0,0 +1,207 @@
/*
* 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.gateway;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData;
import java.util.Iterator;
import java.util.Map;
/**
*/
public abstract class ReplicaShardAllocator extends AbstractComponent {
public ReplicaShardAllocator(Settings settings) {
super(settings);
}
public boolean allocateUnassigned(RoutingAllocation allocation) {
boolean changed = false;
final RoutingNodes routingNodes = allocation.routingNodes();
final MetaData metaData = routingNodes.metaData();
final RoutingNodes.UnassignedShards.UnassignedIterator unassignedIterator = routingNodes.unassigned().iterator();
while (unassignedIterator.hasNext()) {
ShardRouting shard = unassignedIterator.next();
if (shard.primary()) {
continue;
}
// pre-check if it can be allocated to any node that currently exists, so we won't list the store for it for nothing
boolean canBeAllocatedToAtLeastOneNode = false;
for (ObjectCursor<DiscoveryNode> cursor : allocation.nodes().dataNodes().values()) {
RoutingNode node = routingNodes.node(cursor.value.id());
if (node == null) {
continue;
}
// if we can't allocate it on a node, ignore it, for example, this handles
// cases for only allocating a replica after a primary
Decision decision = allocation.deciders().canAllocate(shard, node, allocation);
if (decision.type() == Decision.Type.YES) {
canBeAllocatedToAtLeastOneNode = true;
break;
}
}
if (!canBeAllocatedToAtLeastOneNode) {
logger.trace("{}: ignoring allocation, can't be allocated on any node", shard);
unassignedIterator.removeAndIgnore();
continue;
}
AsyncShardFetch.FetchResult<TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData> shardStores = fetchData(shard, allocation);
if (shardStores.hasData() == false) {
logger.trace("{}: ignoring allocation, still fetching shard stores", shard);
unassignedIterator.removeAndIgnore();
continue; // still fetching
}
long lastSizeMatched = 0;
DiscoveryNode lastDiscoNodeMatched = null;
RoutingNode lastNodeMatched = null;
boolean hasReplicaData = false;
IndexMetaData indexMetaData = metaData.index(shard.getIndex());
for (Map.Entry<DiscoveryNode, TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData> nodeStoreEntry : shardStores.getData().entrySet()) {
DiscoveryNode discoNode = nodeStoreEntry.getKey();
TransportNodesListShardStoreMetaData.StoreFilesMetaData storeFilesMetaData = nodeStoreEntry.getValue().storeFilesMetaData();
logger.trace("{}: checking node [{}]", shard, discoNode);
if (storeFilesMetaData == null) {
// already allocated on that node...
continue;
}
RoutingNode node = routingNodes.node(discoNode.id());
if (node == null) {
continue;
}
// check if we can allocate on that node...
// we only check for NO, since if this node is THROTTLING and it has enough "same data"
// then we will try and assign it next time
Decision decision = allocation.deciders().canAllocate(shard, node, allocation);
if (decision.type() == Decision.Type.NO) {
continue;
}
// if it is already allocated, we can't assign to it...
if (storeFilesMetaData.allocated()) {
continue;
}
if (!shard.primary()) {
hasReplicaData |= storeFilesMetaData.iterator().hasNext();
ShardRouting primaryShard = routingNodes.activePrimary(shard);
if (primaryShard != null) {
assert primaryShard.active();
DiscoveryNode primaryNode = allocation.nodes().get(primaryShard.currentNodeId());
if (primaryNode != null) {
TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData primaryNodeFilesStore = shardStores.getData().get(primaryNode);
if (primaryNodeFilesStore != null) {
TransportNodesListShardStoreMetaData.StoreFilesMetaData primaryNodeStore = primaryNodeFilesStore.storeFilesMetaData();
if (primaryNodeStore != null && primaryNodeStore.allocated()) {
long sizeMatched = 0;
String primarySyncId = primaryNodeStore.syncId();
String replicaSyncId = storeFilesMetaData.syncId();
// see if we have a sync id we can make use of
if (replicaSyncId != null && replicaSyncId.equals(primarySyncId)) {
logger.trace("{}: node [{}] has same sync id {} as primary", shard, discoNode.name(), replicaSyncId);
lastNodeMatched = node;
lastSizeMatched = Long.MAX_VALUE;
lastDiscoNodeMatched = discoNode;
} else {
for (StoreFileMetaData storeFileMetaData : storeFilesMetaData) {
String metaDataFileName = storeFileMetaData.name();
if (primaryNodeStore.fileExists(metaDataFileName) && primaryNodeStore.file(metaDataFileName).isSame(storeFileMetaData)) {
sizeMatched += storeFileMetaData.length();
}
}
logger.trace("{}: node [{}] has [{}/{}] bytes of re-usable data",
shard, discoNode.name(), new ByteSizeValue(sizeMatched), sizeMatched);
if (sizeMatched > lastSizeMatched) {
lastSizeMatched = sizeMatched;
lastDiscoNodeMatched = discoNode;
lastNodeMatched = node;
}
}
}
}
}
}
}
}
if (lastNodeMatched != null) {
// we only check on THROTTLE since we checked before before on NO
Decision decision = allocation.deciders().canAllocate(shard, lastNodeMatched, allocation);
if (decision.type() == Decision.Type.THROTTLE) {
if (logger.isDebugEnabled()) {
logger.debug("[{}][{}]: throttling allocation [{}] to [{}] in order to reuse its unallocated persistent store with total_size [{}]", shard.index(), shard.id(), shard, lastDiscoNodeMatched, new ByteSizeValue(lastSizeMatched));
}
// we are throttling this, but we have enough to allocate to this node, ignore it for now
unassignedIterator.removeAndIgnore();
} else {
if (logger.isDebugEnabled()) {
logger.debug("[{}][{}]: allocating [{}] to [{}] in order to reuse its unallocated persistent store with total_size [{}]", shard.index(), shard.id(), shard, lastDiscoNodeMatched, new ByteSizeValue(lastSizeMatched));
}
// we found a match
changed = true;
unassignedIterator.initialize(lastNodeMatched.nodeId());
}
} else if (hasReplicaData == false) {
// if we didn't manage to find *any* data (regardless of matching sizes), check if the allocation
// of the replica shard needs to be delayed, and if so, add it to the ignore unassigned list
// note: we only care about replica in delayed allocation, since if we have an unassigned primary it
// will anyhow wait to find an existing copy of the shard to be allocated
// note: the other side of the equation is scheduling a reroute in a timely manner, which happens in the RoutingService
long delay = shard.unassignedInfo().getDelayAllocationExpirationIn(settings, indexMetaData.getSettings());
if (delay > 0) {
logger.debug("[{}][{}]: delaying allocation of [{}] for [{}]", shard.index(), shard.id(), shard, TimeValue.timeValueMillis(delay));
/**
* mark it as changed, since we want to kick a publishing to schedule future allocation,
* see {@link org.elasticsearch.cluster.routing.RoutingService#clusterChanged(ClusterChangedEvent)}).
*/
changed = true;
unassignedIterator.removeAndIgnore();
}
}
}
return changed;
}
protected abstract AsyncShardFetch.FetchResult<TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData> fetchData(ShardRouting shard, RoutingAllocation allocation);
}

View File

@ -84,8 +84,6 @@ public class DocumentMapper implements ToXContent {
private List<SourceTransform> sourceTransforms = new ArrayList<>(1);
private final String index;
private final Settings indexSettings;
private final RootObjectMapper rootObjectMapper;
@ -94,8 +92,7 @@ public class DocumentMapper implements ToXContent {
private final Mapper.BuilderContext builderContext;
public Builder(String index, Settings indexSettings, RootObjectMapper.Builder builder, MapperService mapperService) {
this.index = index;
public Builder(Settings indexSettings, RootObjectMapper.Builder builder, MapperService mapperService) {
this.indexSettings = indexSettings;
this.builderContext = new Mapper.BuilderContext(indexSettings, new ContentPath(1));
this.rootObjectMapper = builder.build(builderContext);
@ -150,7 +147,7 @@ public class DocumentMapper implements ToXContent {
public DocumentMapper build(MapperService mapperService, DocumentMapperParser docMapperParser) {
Preconditions.checkNotNull(rootObjectMapper, "Mapper builder must have the root object mapper set");
return new DocumentMapper(mapperService, index, indexSettings, docMapperParser, rootObjectMapper, meta, rootMappers, sourceTransforms, mapperService.mappingLock);
return new DocumentMapper(mapperService, indexSettings, docMapperParser, rootObjectMapper, meta, rootMappers, sourceTransforms, mapperService.mappingLock);
}
}
@ -176,7 +173,7 @@ public class DocumentMapper implements ToXContent {
private final ReleasableLock mappingWriteLock;
private final ReentrantReadWriteLock mappingLock;
public DocumentMapper(MapperService mapperService, String index, @Nullable Settings indexSettings, DocumentMapperParser docMapperParser,
public DocumentMapper(MapperService mapperService, @Nullable Settings indexSettings, DocumentMapperParser docMapperParser,
RootObjectMapper rootObjectMapper,
ImmutableMap<String, Object> meta,
Map<Class<? extends MetadataFieldMapper>, MetadataFieldMapper> rootMappers,
@ -191,7 +188,7 @@ public class DocumentMapper implements ToXContent {
rootMappers.values().toArray(new MetadataFieldMapper[rootMappers.values().size()]),
sourceTransforms.toArray(new SourceTransform[sourceTransforms.size()]),
meta);
this.documentParser = new DocumentParser(index, indexSettings, docMapperParser, this, new ReleasableLock(mappingLock.readLock()));
this.documentParser = new DocumentParser(indexSettings, docMapperParser, this, new ReleasableLock(mappingLock.readLock()));
this.typeFilter = typeMapper().fieldType().termQuery(type, null);
this.mappingWriteLock = new ReleasableLock(mappingLock.writeLock());
@ -325,8 +322,8 @@ public class DocumentMapper implements ToXContent {
return this.objectMappers;
}
public ParsedDocument parse(String type, String id, BytesReference source) throws MapperParsingException {
return parse(SourceToParse.source(source).type(type).id(id));
public ParsedDocument parse(String index, String type, String id, BytesReference source) throws MapperParsingException {
return parse(SourceToParse.source(source).index(index).type(type).id(id));
}
public ParsedDocument parse(SourceToParse source) throws MapperParsingException {

View File

@ -56,11 +56,9 @@ import java.util.Map;
import static org.elasticsearch.index.mapper.MapperBuilders.doc;
/**
*
*/
public class DocumentMapperParser extends AbstractIndexComponent {
public class DocumentMapperParser {
private final Settings indexSettings;
final MapperService mapperService;
final AnalysisService analysisService;
private static final ESLogger logger = Loggers.getLogger(DocumentMapperParser.class);
@ -76,9 +74,9 @@ public class DocumentMapperParser extends AbstractIndexComponent {
private volatile ImmutableMap<String, Mapper.TypeParser> typeParsers;
private volatile ImmutableMap<String, Mapper.TypeParser> rootTypeParsers;
public DocumentMapperParser(Index index, @IndexSettings Settings indexSettings, MapperService mapperService, AnalysisService analysisService,
public DocumentMapperParser(@IndexSettings Settings indexSettings, MapperService mapperService, AnalysisService analysisService,
SimilarityLookupService similarityLookupService, ScriptService scriptService) {
super(index, indexSettings);
this.indexSettings = indexSettings;
this.parseFieldMatcher = new ParseFieldMatcher(indexSettings);
this.mapperService = mapperService;
this.analysisService = analysisService;
@ -205,7 +203,7 @@ public class DocumentMapperParser extends AbstractIndexComponent {
Mapper.TypeParser.ParserContext parserContext = parserContext();
// parse RootObjectMapper
DocumentMapper.Builder docBuilder = doc(index.name(), indexSettings, (RootObjectMapper.Builder) rootObjectTypeParser.parse(type, mapping, parserContext), mapperService);
DocumentMapper.Builder docBuilder = doc(indexSettings, (RootObjectMapper.Builder) rootObjectTypeParser.parse(type, mapping, parserContext), mapperService);
Iterator<Map.Entry<String, Object>> iterator = mapping.entrySet().iterator();
// parse DocumentMapper
while(iterator.hasNext()) {

View File

@ -56,18 +56,16 @@ class DocumentParser implements Closeable {
private CloseableThreadLocal<ParseContext.InternalParseContext> cache = new CloseableThreadLocal<ParseContext.InternalParseContext>() {
@Override
protected ParseContext.InternalParseContext initialValue() {
return new ParseContext.InternalParseContext(index, indexSettings, docMapperParser, docMapper, new ContentPath(0));
return new ParseContext.InternalParseContext(indexSettings, docMapperParser, docMapper, new ContentPath(0));
}
};
private final String index;
private final Settings indexSettings;
private final DocumentMapperParser docMapperParser;
private final DocumentMapper docMapper;
private final ReleasableLock parseLock;
public DocumentParser(String index, Settings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, ReleasableLock parseLock) {
this.index = index;
public DocumentParser(Settings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, ReleasableLock parseLock) {
this.indexSettings = indexSettings;
this.docMapperParser = docMapperParser;
this.docMapper = docMapper;

View File

@ -32,8 +32,8 @@ public final class MapperBuilders {
private MapperBuilders() {}
public static DocumentMapper.Builder doc(String index, Settings settings, RootObjectMapper.Builder objectBuilder, MapperService mapperService) {
return new DocumentMapper.Builder(index, settings, objectBuilder, mapperService);
public static DocumentMapper.Builder doc(Settings settings, RootObjectMapper.Builder objectBuilder, MapperService mapperService) {
return new DocumentMapper.Builder(settings, objectBuilder, mapperService);
}
public static RootObjectMapper.Builder rootObject(String name) {

View File

@ -146,7 +146,7 @@ public class MapperService extends AbstractIndexComponent {
this.analysisService = analysisService;
this.fieldDataService = fieldDataService;
this.fieldTypes = new FieldTypeLookup();
this.documentParser = new DocumentMapperParser(index, indexSettings, this, analysisService, similarityLookupService, scriptService);
this.documentParser = new DocumentMapperParser(indexSettings, this, analysisService, similarityLookupService, scriptService);
this.indexAnalyzer = new MapperAnalyzerWrapper(analysisService.defaultIndexAnalyzer(), INDEX_ANALYZER_EXTRACTOR);
this.searchAnalyzer = new MapperAnalyzerWrapper(analysisService.defaultSearchAnalyzer(), SEARCH_ANALYZER_EXTRACTOR);
this.searchQuoteAnalyzer = new MapperAnalyzerWrapper(analysisService.defaultSearchQuoteAnalyzer(), SEARCH_QUOTE_ANALYZER_EXTRACTOR);

View File

@ -380,8 +380,6 @@ public abstract class ParseContext {
private List<Document> documents = Lists.newArrayList();
private final String index;
@Nullable
private final Settings indexSettings;
@ -402,8 +400,7 @@ public abstract class ParseContext {
private Mapper dynamicMappingsUpdate = null;
public InternalParseContext(String index, @Nullable Settings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, ContentPath path) {
this.index = index;
public InternalParseContext(@Nullable Settings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, ContentPath path) {
this.indexSettings = indexSettings;
this.docMapper = docMapper;
this.docMapperParser = docMapperParser;
@ -443,7 +440,7 @@ public abstract class ParseContext {
@Override
public String index() {
return this.index;
return sourceToParse.index();
}
@Override

View File

@ -47,6 +47,8 @@ public class SourceToParse {
private boolean flyweight = false;
private String index;
private String type;
private String id;
@ -59,13 +61,13 @@ public class SourceToParse {
private long ttl;
public SourceToParse(Origin origin, XContentParser parser) {
private SourceToParse(Origin origin, XContentParser parser) {
this.origin = origin;
this.parser = parser;
this.source = null;
}
public SourceToParse(Origin origin, BytesReference source) {
private SourceToParse(Origin origin, BytesReference source) {
this.origin = origin;
// we always convert back to byte array, since we store it and Field only supports bytes..
// so, we might as well do it here, and improve the performance of working with direct byte arrays
@ -85,6 +87,15 @@ public class SourceToParse {
return this.source;
}
public String index() {
return this.index;
}
public SourceToParse index(String index) {
this.index = index;
return this;
}
public String type() {
return this.type;
}

View File

@ -102,10 +102,11 @@ public class IndexFieldMapper extends MetadataFieldMapper {
@Override
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
Builder builder = new Builder(parserContext.mapperService().fullName(NAME));
if (parserContext.indexVersionCreated().before(Version.V_2_0_0_beta1)) {
parseField(builder, builder.name, node, parserContext);
if (parserContext.indexVersionCreated().onOrAfter(Version.V_2_0_0_beta1)) {
return builder;
}
parseField(builder, builder.name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String fieldName = Strings.toUnderscoreCase(entry.getKey());

View File

@ -175,7 +175,7 @@ public final class ShardSearchStats {
public void onFreeScrollContext(SearchContext context) {
totalStats.scrollCurrent.dec();
totalStats.scrollMetric.inc(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - context.nowInMillis()));
totalStats.scrollMetric.inc(System.nanoTime() - context.getOriginNanoTime());
}
public void onRefreshSettings(Settings settings) {

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.settings;
import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.gateway.PrimaryShardAllocator;
import org.elasticsearch.index.shard.MergeSchedulerConfig;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider;
@ -74,7 +75,7 @@ public class IndexDynamicSettingsModule extends AbstractModule {
indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_PRIORITY, Validator.NON_NEGATIVE_INTEGER);
indexDynamicSettings.addDynamicSetting(IndicesTTLService.INDEX_TTL_DISABLE_PURGE);
indexDynamicSettings.addDynamicSetting(IndexShard.INDEX_REFRESH_INTERVAL, Validator.TIME);
indexDynamicSettings.addDynamicSetting(GatewayAllocator.INDEX_RECOVERY_INITIAL_SHARDS);
indexDynamicSettings.addDynamicSetting(PrimaryShardAllocator.INDEX_RECOVERY_INITIAL_SHARDS);
indexDynamicSettings.addDynamicSetting(EngineConfig.INDEX_COMPOUND_ON_FLUSH, Validator.BOOLEAN);
indexDynamicSettings.addDynamicSetting(EngineConfig.INDEX_GC_DELETES_SETTING, Validator.TIME);
indexDynamicSettings.addDynamicSetting(IndexShard.INDEX_FLUSH_ON_CLOSE, Validator.BOOLEAN);

View File

@ -145,7 +145,7 @@ public class TranslogRecoveryPerformer {
case CREATE:
Translog.Create create = (Translog.Create) operation;
Engine.Create engineCreate = IndexShard.prepareCreate(docMapper(create.type()),
source(create.source()).type(create.type()).id(create.id())
source(create.source()).index(shardId.getIndex()).type(create.type()).id(create.id())
.routing(create.routing()).parent(create.parent()).timestamp(create.timestamp()).ttl(create.ttl()),
create.version(), create.versionType().versionTypeForReplicationAndRecovery(), Engine.Operation.Origin.RECOVERY, true, false);
maybeAddMappingUpdate(engineCreate.type(), engineCreate.parsedDoc().dynamicMappingsUpdate(), engineCreate.id(), allowMappingUpdates);

View File

@ -309,7 +309,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
// 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).type(type).flyweight(true));
ParsedDocument parsedDocument = docMapper.v1().parse(source(doc).index(index).type(type).flyweight(true));
if (docMapper.v2() != null) {
parsedDocument.addDynamicMappingsUpdate(docMapper.v2());
}

View File

@ -98,6 +98,7 @@ public class PercolateContext extends SearchContext {
private final ConcurrentMap<BytesRef, Query> percolateQueries;
private final int numberOfShards;
private final Query aliasFilter;
private final long originNanoTime = System.nanoTime();
private final long startTime;
private String[] types;
@ -337,6 +338,11 @@ public class PercolateContext extends SearchContext {
throw new UnsupportedOperationException();
}
@Override
public long getOriginNanoTime() {
return originNanoTime;
}
@Override
protected long nowInMillisImpl() {
return startTime;

View File

@ -182,13 +182,13 @@ public class PercolatorService extends AbstractComponent {
);
SearchContext.setCurrent(context);
try {
ParsedDocument parsedDocument = parseRequest(percolateIndexService, request, context);
ParsedDocument parsedDocument = parseRequest(percolateIndexService, request, context, request.shardId().getIndex());
if (context.percolateQueries().isEmpty()) {
return new PercolateShardResponse(context, request.shardId());
}
if (request.docSource() != null && request.docSource().length() != 0) {
parsedDocument = parseFetchedDoc(context, request.docSource(), percolateIndexService, request.documentType());
parsedDocument = parseFetchedDoc(context, request.docSource(), percolateIndexService, request.shardId().getIndex(), request.documentType());
} else if (parsedDocument == null) {
throw new IllegalArgumentException("Nothing to percolate");
}
@ -242,7 +242,7 @@ public class PercolatorService extends AbstractComponent {
}
}
private ParsedDocument parseRequest(IndexService documentIndexService, PercolateShardRequest request, PercolateContext context) {
private ParsedDocument parseRequest(IndexService documentIndexService, PercolateShardRequest request, PercolateContext context, String index) {
BytesReference source = request.source();
if (source == null || source.length() == 0) {
return null;
@ -276,7 +276,7 @@ public class PercolatorService extends AbstractComponent {
MapperService mapperService = documentIndexService.mapperService();
Tuple<DocumentMapper, Mapping> docMapper = mapperService.documentMapperWithAutoCreate(request.documentType());
doc = docMapper.v1().parse(source(parser).type(request.documentType()).flyweight(true));
doc = docMapper.v1().parse(source(parser).index(index).type(request.documentType()).flyweight(true));
if (docMapper.v2() != null) {
doc.addDynamicMappingsUpdate(docMapper.v2());
}
@ -378,14 +378,14 @@ public class PercolatorService extends AbstractComponent {
}
}
private ParsedDocument parseFetchedDoc(PercolateContext context, BytesReference fetchedDoc, IndexService documentIndexService, String type) {
private ParsedDocument parseFetchedDoc(PercolateContext context, BytesReference fetchedDoc, IndexService documentIndexService, String index, String type) {
ParsedDocument doc = null;
XContentParser parser = null;
try {
parser = XContentFactory.xContent(fetchedDoc).createParser(fetchedDoc);
MapperService mapperService = documentIndexService.mapperService();
Tuple<DocumentMapper, Mapping> docMapper = mapperService.documentMapperWithAutoCreate(type);
doc = docMapper.v1().parse(source(parser).type(type).flyweight(true));
doc = docMapper.v1().parse(source(parser).index(index).type(type).flyweight(true));
if (context.highlight() != null) {
doc.setSource(fetchedDoc);

View File

@ -122,6 +122,7 @@ public class DefaultSearchContext extends SearchContext {
private boolean queryRewritten;
private volatile long keepAlive;
private ScoreDoc lastEmittedDoc;
private final long originNanoTime = System.nanoTime();
private volatile long lastAccessTime = -1;
private InnerHitsContext innerHitsContext;
@ -269,6 +270,11 @@ public class DefaultSearchContext extends SearchContext {
return this;
}
@Override
public long getOriginNanoTime() {
return originNanoTime;
}
@Override
protected long nowInMillisImpl() {
return request.nowInMillis();

View File

@ -139,6 +139,11 @@ public abstract class FilteredSearchContext extends SearchContext {
return in.queryBoost(queryBoost);
}
@Override
public long getOriginNanoTime() {
return in.getOriginNanoTime();
}
@Override
protected long nowInMillisImpl() {
return in.nowInMillisImpl();

View File

@ -142,6 +142,8 @@ public abstract class SearchContext implements Releasable, HasContextAndHeaders
public abstract SearchContext queryBoost(float queryBoost);
public abstract long getOriginNanoTime();
public final long nowInMillis() {
nowInMillisUsed = true;
return nowInMillisImpl();

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.action.admin.indices.segments;
package org.elasticsearch.action.admin.indices.shards;
import com.carrotsearch.hppc.cursors.IntObjectCursor;
@ -37,10 +37,10 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.store.MockFSDirectoryService;
import org.junit.Test;
import java.io.IOException;
import java.util.*;
import java.util.concurrent.ExecutionException;
@ -58,6 +58,7 @@ public class IndicesShardStoreRequestTests extends ElasticsearchIntegrationTest
}
@Test
@TestLogging("action.admin.indices.shards:TRACE,cluster.service:TRACE")
public void testBasic() throws Exception {
String index = "test";
internalCluster().ensureAtLeastNumDataNodes(2);
@ -91,10 +92,13 @@ public class IndicesShardStoreRequestTests extends ElasticsearchIntegrationTest
disableAllocation(index);
logger.info("--> stop random node");
internalCluster().stopRandomNode(new IndexNodePredicate(index));
ensureYellow(index);
ClusterState clusterState = client().admin().cluster().prepareState().get().getState();
List<ShardRouting> unassignedShards = clusterState.routingTable().index(index).shardsWithState(ShardRoutingState.UNASSIGNED);
response = client().admin().indices().shardStores(Requests.indicesShardStoresRequest(index)).get();
assertThat(response.getStoreStatuses().containsKey(index), equalTo(true));
ImmutableOpenIntMap<List<IndicesShardStoresResponse.StoreStatus>> shardStoresStatuses = response.getStoreStatuses().get(index);
assertThat(shardStoresStatuses.size(), greaterThan(0));
assertThat(shardStoresStatuses.size(), equalTo(unassignedShards.size()));
for (IntObjectCursor<List<IndicesShardStoresResponse.StoreStatus>> storesStatus : shardStoresStatuses) {
assertThat("must report for one store", storesStatus.value.size(), equalTo(1));
assertThat("reported store should be primary", storesStatus.value.get(0).getAllocation(), equalTo(IndicesShardStoresResponse.StoreStatus.Allocation.PRIMARY));

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.action.admin.indices.segments;
package org.elasticsearch.action.admin.indices.shards;
import com.google.common.collect.ImmutableList;
import org.apache.lucene.index.CorruptIndexException;

View File

@ -32,6 +32,7 @@ import org.elasticsearch.node.internal.InternalSettingsPreparer;
import org.elasticsearch.plugins.PluginsService;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.InternalTestCluster;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.transport.TransportService;
import org.junit.Test;
@ -45,6 +46,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@ClusterScope(scope = Scope.TEST, numClientNodes = 0)
@TestLogging("discovery.zen:TRACE")
public class TransportClientRetryTests extends ElasticsearchIntegrationTest {
@Test

View File

@ -244,7 +244,7 @@ public class PrioritizedExecutorsTests extends ElasticsearchTestCase {
@Test
public void testTimeoutCleanup() throws Exception {
ThreadPool threadPool = new ThreadPool("test");
ScheduledThreadPoolExecutor timer = (ScheduledThreadPoolExecutor) threadPool.scheduler();
final ScheduledThreadPoolExecutor timer = (ScheduledThreadPoolExecutor) threadPool.scheduler();
final AtomicBoolean timeoutCalled = new AtomicBoolean();
PrioritizedEsThreadPoolExecutor executor = EsExecutors.newSinglePrioritizing(EsExecutors.daemonThreadFactory(getTestName()));
final CountDownLatch invoked = new CountDownLatch(1);
@ -262,7 +262,15 @@ public class PrioritizedExecutorsTests extends ElasticsearchTestCase {
}
);
invoked.await();
assertThat(timer.getQueue().size(), equalTo(0));
// the timeout handler is added post execution (and quickly cancelled). We have allow for this
// and use assert busy
assertBusy(new Runnable() {
@Override
public void run() {
assertThat(timer.getQueue().size(), equalTo(0));
}
}, 5, TimeUnit.SECONDS);
assertThat(timeoutCalled.get(), equalTo(false));
assertTrue(terminate(executor));
assertTrue(terminate(threadPool));

View File

@ -0,0 +1,319 @@
/*
* 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.gateway;
import org.apache.lucene.index.CorruptIndexException;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.*;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ElasticsearchAllocationTestCase;
import org.junit.Before;
import org.junit.Test;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.equalTo;
/**
*/
public class PrimaryShardAllocatorTests extends ElasticsearchAllocationTestCase {
private final ShardId shardId = new ShardId("test", 0);
private final DiscoveryNode node1 = newNode("node1");
private final DiscoveryNode node2 = newNode("node2");
private final DiscoveryNode node3 = newNode("node3");
private TestAllocator testAllocator;
@Before
public void buildTestAllocator() {
this.testAllocator = new TestAllocator();
}
/**
* Verifies that the canProcess method of primary allocation behaves correctly
* and processes only the applicable shard.
*/
@Test
public void testNoProcessReplica() {
ShardRouting shard = TestShardRouting.newShardRouting("test", 0, null, null, null, false, ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null));
assertThat(testAllocator.needToFindPrimaryCopy(shard, null), equalTo(false));
}
/**
* Tests that when async fetch returns that there is no data, the shard will not be allocated.
*/
@Test
public void testNoAsyncFetchData() {
RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId));
}
/**
* Tests when the node returns that no data was found for it (-1), it will be moved to ignore unassigned.
*/
@Test
public void testNoAllocationFound() {
RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders());
testAllocator.addData(node1, -1);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId));
}
/**
* Tests when the node returns that no data was found for it (-1), it will be moved to ignore unassigned.
*/
@Test
public void testStoreException() {
RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders());
testAllocator.addData(node1, 3, new CorruptIndexException("test", "test"));
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId));
}
/**
* Tests that when there is a node to allocate the shard to, it will be allocated to it.
*/
@Test
public void testFoundAllocationAndAllocating() {
RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders());
testAllocator.addData(node1, 10);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node1.id()));
}
/**
* Tests that when there is a node to allocate to, but it is throttling (and it is the only one),
* it will be moved to ignore unassigned until it can be allocated to.
*/
@Test
public void testFoundAllocationButThrottlingDecider() {
RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(throttleAllocationDeciders());
testAllocator.addData(node1, 10);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId));
}
/**
* Tests that when there is a node to be allocated to, but it the decider said "no", we still
* force the allocation to it.
*/
@Test
public void testFoundAllocationButNoDecider() {
RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(noAllocationDeciders());
testAllocator.addData(node1, 10);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node1.id()));
}
/**
* Tests that the highest version node is chosed for allocation.
*/
@Test
public void testAllocateToTheHighestVersion() {
RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders());
testAllocator.addData(node1, 10).addData(node2, 12);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node2.id()));
}
/**
* Tests that when restoring from snapshot, even if we didn't find any node to allocate on, the shard
* will remain in the unassigned list to be allocated later.
*/
@Test
public void testRestoreIgnoresNoNodesToAllocate() {
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder(shardId.getIndex()).settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsRestore(metaData.index(shardId.getIndex()), new RestoreSource(new SnapshotId("test", "test"), Version.CURRENT, shardId.getIndex()))
.build();
ClusterState state = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(routingTable)
.nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build();
RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
testAllocator.addData(node1, -1).addData(node2, -1);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
}
/**
* Tests that only when enough copies of the shard exists we are going to allocate it. This test
* verifies that with same version (1), and quorum allocation.
*/
@Test
public void testEnoughCopiesFoundForAllocation() {
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder(shardId.getIndex()).settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(2))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsRecovery(metaData.index(shardId.getIndex()))
.build();
ClusterState state = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(routingTable)
.nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build();
RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
testAllocator.addData(node1, 1);
allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
testAllocator.addData(node2, 1);
allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), anyOf(equalTo(node2.id()), equalTo(node1.id())));
}
/**
* Tests that only when enough copies of the shard exists we are going to allocate it. This test
* verifies that even with different version, we treat different versions as a copy, and count them.
*/
@Test
public void testEnoughCopiesFoundForAllocationWithDifferentVersion() {
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder(shardId.getIndex()).settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(2))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsRecovery(metaData.index(shardId.getIndex()))
.build();
ClusterState state = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(routingTable)
.nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build();
RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
testAllocator.addData(node1, 1);
allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
testAllocator.addData(node2, 2);
allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node2.id()));
}
private RoutingAllocation routingAllocationWithOnePrimaryNoReplicas(AllocationDeciders deciders) {
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder(shardId.getIndex()).settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsRecovery(metaData.index(shardId.getIndex()))
.build();
ClusterState state = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(routingTable)
.nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build();
return new RoutingAllocation(deciders, state.routingNodes(), state.nodes(), null);
}
class TestAllocator extends PrimaryShardAllocator {
private Map<DiscoveryNode, TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> data;
public TestAllocator() {
super(Settings.EMPTY);
}
public TestAllocator clear() {
data = null;
return this;
}
public TestAllocator addData(DiscoveryNode node, long version) {
return addData(node, version, null);
}
public TestAllocator addData(DiscoveryNode node, long version, @Nullable Throwable storeException) {
if (data == null) {
data = new HashMap<>();
}
data.put(node, new TransportNodesListGatewayStartedShards.NodeGatewayStartedShards(node, version, storeException));
return this;
}
@Override
protected AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> fetchData(ShardRouting shard, RoutingAllocation allocation) {
return new AsyncShardFetch.FetchResult<>(shardId, data, Collections.<String>emptySet(), Collections.<String>emptySet());
}
}
}

View File

@ -30,7 +30,7 @@ import java.util.Map;
public class PriorityComparatorTests extends ElasticsearchTestCase {
public void testPriorityComparatorSort() {
RoutingNodes.UnassignedShards shards = new RoutingNodes.UnassignedShards();
RoutingNodes.UnassignedShards shards = new RoutingNodes.UnassignedShards((RoutingNodes) null);
int numIndices = randomIntBetween(3, 99);
IndexMeta[] indices = new IndexMeta[numIndices];
final Map<String, IndexMeta> map = new HashMap<>();

View File

@ -1924,8 +1924,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
AnalysisService analysisService = new AnalysisService(index, settings);
SimilarityLookupService similarityLookupService = new SimilarityLookupService(index, settings);
MapperService mapperService = new MapperService(index, settings, analysisService, null, similarityLookupService, null);
DocumentMapper.Builder b = new DocumentMapper.Builder(indexName, settings, rootBuilder, mapperService);
DocumentMapperParser parser = new DocumentMapperParser(index, settings, mapperService, analysisService, similarityLookupService, null);
DocumentMapper.Builder b = new DocumentMapper.Builder(settings, rootBuilder, mapperService);
DocumentMapperParser parser = new DocumentMapperParser(settings, mapperService, analysisService, similarityLookupService, null);
this.docMapper = b.build(mapperService, parser);
}

View File

@ -60,16 +60,16 @@ public class BinaryDVFieldDataTests extends AbstractFieldDataTests {
bytesList1.add(randomBytes());
bytesList1.add(randomBytes());
XContentBuilder doc = XContentFactory.jsonBuilder().startObject().startArray("field").value(bytesList1.get(0)).value(bytesList1.get(1)).endArray().endObject();
ParsedDocument d = mapper.parse("test", "1", doc.bytes());
ParsedDocument d = mapper.parse("test", "test", "1", doc.bytes());
writer.addDocument(d.rootDoc());
byte[] bytes1 = randomBytes();
doc = XContentFactory.jsonBuilder().startObject().field("field", bytes1).endObject();
d = mapper.parse("test", "2", doc.bytes());
d = mapper.parse("test", "test", "2", doc.bytes());
writer.addDocument(d.rootDoc());
doc = XContentFactory.jsonBuilder().startObject().endObject();
d = mapper.parse("test", "3", doc.bytes());
d = mapper.parse("test", "test", "3", doc.bytes());
writer.addDocument(d.rootDoc());
// test remove duplicate value
@ -77,7 +77,7 @@ public class BinaryDVFieldDataTests extends AbstractFieldDataTests {
bytesList2.add(randomBytes());
bytesList2.add(randomBytes());
doc = XContentFactory.jsonBuilder().startObject().startArray("field").value(bytesList2.get(0)).value(bytesList2.get(1)).value(bytesList2.get(0)).endArray().endObject();
d = mapper.parse("test", "4", doc.bytes());
d = mapper.parse("test", "test", "4", doc.bytes());
writer.addDocument(d.rootDoc());
LeafReaderContext reader = refreshReader();

View File

@ -77,7 +77,7 @@ public class DuelFieldDataTests extends AbstractFieldDataTests {
doc = doc.endObject();
final ParsedDocument d = mapper.parse("type", Integer.toString(i), doc.bytes());
final ParsedDocument d = mapper.parse("test", "type", Integer.toString(i), doc.bytes());
writer.addDocument(d.rootDoc());
@ -169,7 +169,7 @@ public class DuelFieldDataTests extends AbstractFieldDataTests {
}
doc = doc.endObject();
final ParsedDocument d = mapper.parse("type", Integer.toString(i), doc.bytes());
final ParsedDocument d = mapper.parse("test", "type", Integer.toString(i), doc.bytes());
writer.addDocument(d.rootDoc());
if (random.nextInt(10) == 0) {
@ -255,7 +255,7 @@ public class DuelFieldDataTests extends AbstractFieldDataTests {
}
doc = doc.endArray().endObject();
final ParsedDocument d = mapper.parse("type", Integer.toString(i), doc.bytes());
final ParsedDocument d = mapper.parse("test", "type", Integer.toString(i), doc.bytes());
writer.addDocument(d.rootDoc());
if (random.nextInt(10) == 0) {
@ -423,7 +423,7 @@ public class DuelFieldDataTests extends AbstractFieldDataTests {
}
}
doc = doc.endArray().endObject();
final ParsedDocument d = mapper.parse("type", Integer.toString(i), doc.bytes());
final ParsedDocument d = mapper.parse("test", "type", Integer.toString(i), doc.bytes());
writer.addDocument(d.rootDoc());
if (random.nextInt(10) == 0) {

View File

@ -52,7 +52,7 @@ public class DynamicMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", jsonBuilder()
.startObject()
.field("field1", "value1")
.field("field2", "value2")
@ -72,7 +72,7 @@ public class DynamicMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", jsonBuilder()
.startObject()
.field("field1", "value1")
.field("field2", "value2")
@ -94,7 +94,7 @@ public class DynamicMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
try {
defaultMapper.parse("type", "1", jsonBuilder()
defaultMapper.parse("test", "type", "1", jsonBuilder()
.startObject()
.field("field1", "value1")
.field("field2", "value2")
@ -105,7 +105,7 @@ public class DynamicMappingTests extends ElasticsearchSingleNodeTest {
}
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field1", "value1")
.field("field2", (String) null)
@ -128,7 +128,7 @@ public class DynamicMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", jsonBuilder()
.startObject().startObject("obj1")
.field("field1", "value1")
.field("field2", "value2")
@ -152,7 +152,7 @@ public class DynamicMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
try {
defaultMapper.parse("type", "1", jsonBuilder()
defaultMapper.parse("test", "type", "1", jsonBuilder()
.startObject().startObject("obj1")
.field("field1", "value1")
.field("field2", "value2")
@ -197,7 +197,7 @@ public class DynamicMappingTests extends ElasticsearchSingleNodeTest {
private Mapper parse(DocumentMapper mapper, DocumentMapperParser parser, XContentBuilder builder) throws Exception {
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
ParseContext.InternalParseContext ctx = new ParseContext.InternalParseContext("test", settings, parser, mapper, new ContentPath(0));
ParseContext.InternalParseContext ctx = new ParseContext.InternalParseContext(settings, parser, mapper, new ContentPath(0));
SourceToParse source = SourceToParse.source(builder.bytes());
ctx.reset(XContentHelper.createParser(source.source()), new ParseContext.Document(), source);
assertEquals(XContentParser.Token.START_OBJECT, ctx.parser().nextToken());

View File

@ -74,7 +74,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/all/mapping.json");
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/all/test1.json");
Document doc = docMapper.parse("person", "1", new BytesArray(json)).rootDoc();
Document doc = docMapper.parse("test", "person", "1", new BytesArray(json)).rootDoc();
AllField field = (AllField) doc.getField("_all");
// One field is boosted so we should see AllTokenStream used:
assertThat(field.tokenStream(docMapper.mappers().indexAnalyzer(), null), Matchers.instanceOf(AllTokenStream.class));
@ -93,7 +93,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
IndexService index = createIndex("test");
DocumentMapper docMapper = index.mapperService().documentMapperParser().parse(mapping);
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/all/test1.json");
Document doc = docMapper.parse("person", "1", new BytesArray(json)).rootDoc();
Document doc = docMapper.parse("test", "person", "1", new BytesArray(json)).rootDoc();
AllField field = (AllField) doc.getField("_all");
AllEntries allEntries = field.getAllEntries();
assertThat(allEntries.fields().size(), equalTo(3));
@ -107,7 +107,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/all/mapping_omit_positions_on_all.json");
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/all/test1.json");
Document doc = docMapper.parse("person", "1", new BytesArray(json)).rootDoc();
Document doc = docMapper.parse("test", "person", "1", new BytesArray(json)).rootDoc();
AllField field = (AllField) doc.getField("_all");
AllEntries allEntries = field.getAllEntries();
assertThat(allEntries.fields().size(), equalTo(3));
@ -125,7 +125,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/all/mapping_offsets_on_all.json");
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/all/test1.json");
Document doc = docMapper.parse("person", "1", new BytesArray(json)).rootDoc();
Document doc = docMapper.parse("test", "person", "1", new BytesArray(json)).rootDoc();
AllField field = (AllField) doc.getField("_all");
// _all field indexes positions, and mapping has boosts, so we should see AllTokenStream:
assertThat(field.tokenStream(docMapper.mappers().indexAnalyzer(), null), Matchers.instanceOf(AllTokenStream.class));
@ -144,7 +144,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/all/mapping_boost_omit_positions_on_all.json");
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/all/test1.json");
Document doc = docMapper.parse("person", "1", new BytesArray(json)).rootDoc();
Document doc = docMapper.parse("test", "person", "1", new BytesArray(json)).rootDoc();
AllField field = (AllField) doc.getField("_all");
// _all field omits positions, so we should not get AllTokenStream even though fields are boosted
assertThat(field.tokenStream(docMapper.mappers().indexAnalyzer(), null), Matchers.not(Matchers.instanceOf(AllTokenStream.class)));
@ -155,7 +155,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/all/noboost-mapping.json");
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/all/test1.json");
Document doc = docMapper.parse("person", "1", new BytesArray(json)).rootDoc();
Document doc = docMapper.parse("test", "person", "1", new BytesArray(json)).rootDoc();
AllField field = (AllField) doc.getField("_all");
// no fields have boost, so we should not see AllTokenStream:
assertThat(field.tokenStream(docMapper.mappers().indexAnalyzer(), null), Matchers.not(Matchers.instanceOf(AllTokenStream.class)));
@ -169,7 +169,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
// reparse it
DocumentMapper builtDocMapper = parser.parse(builtMapping);
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/all/test1.json");
Document doc = builtDocMapper.parse("person", "1", new BytesArray(json)).rootDoc();
Document doc = builtDocMapper.parse("test", "person", "1", new BytesArray(json)).rootDoc();
AllField field = (AllField) doc.getField("_all");
AllEntries allEntries = field.getAllEntries();
@ -184,7 +184,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/all/store-mapping.json");
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/all/test1.json");
Document doc = docMapper.parse("person", "1", new BytesArray(json)).rootDoc();
Document doc = docMapper.parse("test", "person", "1", new BytesArray(json)).rootDoc();
AllField field = (AllField) doc.getField("_all");
AllEntries allEntries = field.getAllEntries();
assertThat(allEntries.fields().size(), equalTo(2));
@ -204,7 +204,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
// reparse it
DocumentMapper builtDocMapper = parser.parse(builtMapping);
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/all/test1.json");
Document doc = builtDocMapper.parse("person", "1", new BytesArray(json)).rootDoc();
Document doc = builtDocMapper.parse("test", "person", "1", new BytesArray(json)).rootDoc();
AllField field = (AllField) doc.getField("_all");
AllEntries allEntries = field.getAllEntries();
@ -277,7 +277,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
.field("foo", "bar")
.field("foobar", "foobar")
.endObject().bytes().toBytes();
Document doc = builtDocMapper.parse("test", "1", new BytesArray(json)).rootDoc();
Document doc = builtDocMapper.parse("test", "test", "1", new BytesArray(json)).rootDoc();
AllField field = (AllField) doc.getField("_all");
if (enabled) {
assertThat(field.fieldType().omitNorms(), equalTo(omitNorms));
@ -325,7 +325,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
.endObject()
.endObject();
Document doc = docMapper.parse("test", "1", builder.bytes()).rootDoc();
Document doc = docMapper.parse("test", "test", "1", builder.bytes()).rootDoc();
AllField field = (AllField) doc.getField("_all");
AllEntries allEntries = field.getAllEntries();
assertThat(allEntries.fields(), empty());
@ -343,7 +343,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
.endObject()
.endObject();
Document doc = docMapper.parse("test", "1", builder.bytes()).rootDoc();
Document doc = docMapper.parse("test", "test", "1", builder.bytes()).rootDoc();
AllField field = (AllField) doc.getField("_all");
AllEntries allEntries = field.getAllEntries();
assertThat(allEntries.fields(), hasSize(1));
@ -390,7 +390,6 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
Map<String, String> rootTypes = new HashMap<>();
//just pick some example from DocumentMapperParser.rootTypeParsers
rootTypes.put(SizeFieldMapper.NAME, "{\"enabled\" : true}");
rootTypes.put(IndexFieldMapper.NAME, "{\"enabled\" : true}");
rootTypes.put("include_in_all", "true");
rootTypes.put("dynamic_date_formats", "[\"yyyy-MM-dd\", \"dd-MM-yyyy\"]");
rootTypes.put("numeric_detection", "true");
@ -448,7 +447,7 @@ public class SimpleAllMapperTests extends ElasticsearchSingleNodeTest {
String mapping = jsonBuilder().startObject().startObject("type").endObject().endObject().string();
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build();
DocumentMapper docMapper = createIndex("test", settings).mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject().field("_all", "foo").endObject().bytes());
assertNull(doc.rootDoc().get("_all"));

View File

@ -85,7 +85,7 @@ public class BinaryMappingTests extends ElasticsearchSingleNodeTest {
assertTrue(CompressorFactory.isCompressed(new BytesArray(binaryValue2)));
for (byte[] value : Arrays.asList(binaryValue1, binaryValue2)) {
ParsedDocument doc = mapper.parse("type", "id", XContentFactory.jsonBuilder().startObject().field("field", value).endObject().bytes());
ParsedDocument doc = mapper.parse("test", "type", "id", XContentFactory.jsonBuilder().startObject().field("field", value).endObject().bytes());
BytesRef indexedValue = doc.rootDoc().getBinaryValue("field");
assertEquals(new BytesRef(value), indexedValue);
FieldMapper fieldMapper = mapper.mappers().smartNameFieldMapper("field");
@ -116,7 +116,7 @@ public class BinaryMappingTests extends ElasticsearchSingleNodeTest {
final byte[] binaryValue = out.bytes().toBytes();
assertTrue(CompressorFactory.isCompressed(new BytesArray(binaryValue)));
ParsedDocument doc = mapper.parse("type", "id", XContentFactory.jsonBuilder().startObject().field("field", binaryValue).endObject().bytes());
ParsedDocument doc = mapper.parse("test", "type", "id", XContentFactory.jsonBuilder().startObject().field("field", binaryValue).endObject().bytes());
BytesRef indexedValue = doc.rootDoc().getBinaryValue("field");
assertEquals(new BytesRef(binaryValue), indexedValue);
FieldMapper fieldMapper = mapper.mappers().smartNameFieldMapper("field");

View File

@ -44,7 +44,7 @@ public class CustomBoostMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper mapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = mapper.parse("type", "1", XContentFactory.jsonBuilder().startObject()
ParsedDocument doc = mapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject()
.startObject("s_field").field("value", "s_value").field("boost", 2.0f).endObject()
.startObject("l_field").field("value", 1l).field("boost", 3.0f).endObject()
.startObject("i_field").field("value", 1).field("boost", 4.0f).endObject()

View File

@ -58,7 +58,7 @@ public class FieldLevelBoostTests extends ElasticsearchSingleNodeTest {
.startObject("long_field").field("boost", 8.0).field("value", 50).endObject()
.startObject("short_field").field("boost", 9.0).field("value", 60).endObject()
.bytes();
Document doc = docMapper.parse("person", "1", json).rootDoc();
Document doc = docMapper.parse("test", "person", "1", json).rootDoc();
IndexableField f = doc.getField("str_field");
assertThat((double) f.boost(), closeTo(2.0, 0.001));
@ -100,7 +100,7 @@ public class FieldLevelBoostTests extends ElasticsearchSingleNodeTest {
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
try {
docMapper.parse("person", "1", XContentFactory.jsonBuilder().startObject()
docMapper.parse("test", "person", "1", XContentFactory.jsonBuilder().startObject()
.startObject("str_field").field("foo", "bar")
.endObject().bytes()).rootDoc();
fail();
@ -109,7 +109,7 @@ public class FieldLevelBoostTests extends ElasticsearchSingleNodeTest {
}
try {
docMapper.parse("person", "1", XContentFactory.jsonBuilder().startObject()
docMapper.parse("test", "person", "1", XContentFactory.jsonBuilder().startObject()
.startObject("int_field").field("foo", "bar")
.endObject().bytes()).rootDoc();
fail();
@ -118,7 +118,7 @@ public class FieldLevelBoostTests extends ElasticsearchSingleNodeTest {
}
try {
docMapper.parse("person", "1", XContentFactory.jsonBuilder().startObject()
docMapper.parse("test", "person", "1", XContentFactory.jsonBuilder().startObject()
.startObject("byte_field").field("foo", "bar")
.endObject().bytes()).rootDoc();
fail();
@ -127,7 +127,7 @@ public class FieldLevelBoostTests extends ElasticsearchSingleNodeTest {
}
try {
docMapper.parse("person", "1", XContentFactory.jsonBuilder().startObject()
docMapper.parse("test", "person", "1", XContentFactory.jsonBuilder().startObject()
.startObject("date_field").field("foo", "bar")
.endObject().bytes()).rootDoc();
fail();
@ -136,7 +136,7 @@ public class FieldLevelBoostTests extends ElasticsearchSingleNodeTest {
}
try {
docMapper.parse("person", "1", XContentFactory.jsonBuilder().startObject()
docMapper.parse("test", "person", "1", XContentFactory.jsonBuilder().startObject()
.startObject("double_field").field("foo", "bar")
.endObject().bytes()).rootDoc();
fail();
@ -145,7 +145,7 @@ public class FieldLevelBoostTests extends ElasticsearchSingleNodeTest {
}
try {
docMapper.parse("person", "1", XContentFactory.jsonBuilder().startObject()
docMapper.parse("test", "person", "1", XContentFactory.jsonBuilder().startObject()
.startObject("float_field").field("foo", "bar")
.endObject().bytes()).rootDoc();
fail();
@ -154,7 +154,7 @@ public class FieldLevelBoostTests extends ElasticsearchSingleNodeTest {
}
try {
docMapper.parse("person", "1", XContentFactory.jsonBuilder().startObject()
docMapper.parse("test", "person", "1", XContentFactory.jsonBuilder().startObject()
.startObject("long_field").field("foo", "bar")
.endObject().bytes()).rootDoc();
fail();
@ -163,7 +163,7 @@ public class FieldLevelBoostTests extends ElasticsearchSingleNodeTest {
}
try {
docMapper.parse("person", "1", XContentFactory.jsonBuilder().startObject()
docMapper.parse("test", "person", "1", XContentFactory.jsonBuilder().startObject()
.startObject("short_field").field("foo", "bar")
.endObject().bytes()).rootDoc();
fail();

View File

@ -40,7 +40,7 @@ public class CamelCaseFieldNameTests extends ElasticsearchSingleNodeTest {
client().admin().indices().preparePutMapping("test").setType("type").setSource(mapping).get();
DocumentMapper documentMapper = index.mapperService().documentMapper("type");
ParsedDocument doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject()
ParsedDocument doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject()
.field("thisIsCamelCase", "value1")
.endObject().bytes());

View File

@ -40,7 +40,7 @@ public class CompoundTypesTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field1", "value1")
.field("field2", "value2")
@ -50,7 +50,7 @@ public class CompoundTypesTests extends ElasticsearchSingleNodeTest {
assertThat((double) doc.rootDoc().getField("field1").boost(), closeTo(1.0d, 0.000001d));
assertThat(doc.rootDoc().get("field2"), equalTo("value2"));
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("field1").field("value", "value1").field("boost", 2.0f).endObject()
.field("field2", "value2")
@ -60,7 +60,7 @@ public class CompoundTypesTests extends ElasticsearchSingleNodeTest {
assertThat((double) doc.rootDoc().getField("field1").boost(), closeTo(2.0d, 0.000001d));
assertThat(doc.rootDoc().get("field2"), equalTo("value2"));
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field1", "value1")
.field("field2", "value2")

View File

@ -110,7 +110,7 @@ public class CopyToMapperTests extends ElasticsearchSingleNodeTest {
.field("int_to_str_test", 42)
.endObject().bytes();
ParsedDocument parsedDoc = docMapper.parse("type1", "1", json);
ParsedDocument parsedDoc = docMapper.parse("test", "type1", "1", json);
ParseContext.Document doc = parsedDoc.rootDoc();
assertThat(doc.getFields("copy_test").length, equalTo(2));
assertThat(doc.getFields("copy_test")[0].stringValue(), equalTo("foo"));
@ -165,7 +165,7 @@ public class CopyToMapperTests extends ElasticsearchSingleNodeTest {
.startObject("foo").startObject("bar").field("baz", "zoo").endObject().endObject()
.endObject().bytes();
ParseContext.Document doc = docMapper.parse("type1", "1", json).rootDoc();
ParseContext.Document doc = docMapper.parse("test", "type1", "1", json).rootDoc();
assertThat(doc.getFields("copy_test").length, equalTo(1));
assertThat(doc.getFields("copy_test")[0].stringValue(), equalTo("foo"));
@ -193,7 +193,7 @@ public class CopyToMapperTests extends ElasticsearchSingleNodeTest {
.endObject().bytes();
try {
docMapper.parse("type1", "1", json).rootDoc();
docMapper.parse("test", "type1", "1", json).rootDoc();
fail();
} catch (MapperParsingException ex) {
assertThat(ex.getMessage(), startsWith("attempt to copy value to non-existing object"));
@ -313,7 +313,7 @@ public class CopyToMapperTests extends ElasticsearchSingleNodeTest {
.endArray()
.endObject();
ParsedDocument doc = mapper.parse("type", "1", jsonDoc.bytes());
ParsedDocument doc = mapper.parse("test", "type", "1", jsonDoc.bytes());
assertEquals(6, doc.docs().size());
Document nested = doc.docs().get(0);

View File

@ -59,7 +59,7 @@ public class BooleanFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = parser.parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", true)
.endObject()

View File

@ -67,9 +67,9 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
.startObject("properties").endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = mapper("type", mapping);
DocumentMapper defaultMapper = mapper("test", "type", mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field1", "2011/01/22")
.field("date_field2", "2011/01/22 00:00:00")
@ -79,7 +79,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
.endObject()
.bytes());
assertNotNull(doc.dynamicMappingsUpdate());
client().admin().indices().preparePutMapping("test-0").setType("type").setSource(doc.dynamicMappingsUpdate().toString()).get();
client().admin().indices().preparePutMapping("test").setType("type").setSource(doc.dynamicMappingsUpdate().toString()).get();
FieldMapper fieldMapper = defaultMapper.mappers().smartNameFieldMapper("date_field1");
assertThat(fieldMapper, instanceOf(DateFieldMapper.class));
@ -132,8 +132,8 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
.endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = mapper("type", mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
DocumentMapper defaultMapper = mapper("test", "type", mapping);
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field_en", "Wed, 06 Dec 2000 02:55:00 -0800")
.field("date_field_de", "Mi, 06 Dez 2000 02:55:00 -0800")
@ -151,12 +151,11 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
int i = 0;
private DocumentMapper mapper(String type, String mapping) throws IOException {
return mapper(type, mapping, Version.CURRENT);
private DocumentMapper mapper(String indexName, String type, String mapping) throws IOException {
return mapper(indexName, type, mapping, Version.CURRENT);
}
private DocumentMapper mapper(String type, String mapping, Version version) throws IOException {
final String indexName = "test-" + (i++);
private DocumentMapper mapper(String indexName, String type, String mapping, Version version) throws IOException {
IndexService index;
if (version.equals(Version.CURRENT)) {
index = createIndex(indexName);
@ -194,10 +193,10 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
.startObject("properties").startObject("date_field").field("type", "date").endObject().endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = mapper("type", mapping);
DocumentMapper defaultMapper = mapper("test", "type", mapping);
long value = System.currentTimeMillis();
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", value)
.endObject()
@ -212,9 +211,9 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
.startObject("properties").startObject("date_field").field("type", "date").endObject().endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = mapper("type", mapping);
DocumentMapper defaultMapper = mapper("test", "type", mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "2010-01-01")
.field("date_field_x", "2010-01-01")
@ -231,13 +230,13 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
.startObject("properties").startObject("date_field").field("type", "date").field("format", "HH:mm:ss").endObject().endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = mapper("type", mapping);
DocumentMapper defaultMapper = mapper("test", "type", mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "10:00:00")
.endObject()
.bytes());
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "10:00:00")
.endObject()
.bytes());
assertThat(((LongFieldMapper.CustomLongNumericField) doc.rootDoc().getField("date_field")).numericAsString(), equalTo(Long.toString(new DateTime(TimeValue.timeValueHours(10).millis(), DateTimeZone.UTC).getMillis())));
NumericRangeQuery<Long> rangeQuery;
@ -257,9 +256,9 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
.startObject("properties").startObject("date_field").field("type", "date").field("format", "MMM dd HH:mm:ss").endObject().endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = mapper("type", mapping);
DocumentMapper defaultMapper = mapper("test", "type", mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "Jan 02 10:00:00")
.endObject()
@ -286,9 +285,9 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
.endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = mapper("type", mapping);
DocumentMapper defaultMapper = mapper("test", "type", mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field1", "a")
.field("field2", "2010-01-01")
@ -298,7 +297,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
assertThat(doc.rootDoc().getField("field2"), notNullValue());
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field2", "a")
.endObject()
@ -310,7 +309,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
// Verify that the default is false
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field3", "a")
.endObject()
@ -323,7 +322,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
// Unless the global ignore_malformed option is set to true
Settings indexSettings = settingsBuilder().put("index.mapping.ignore_malformed", true).build();
defaultMapper = createIndex("test2", indexSettings).mapperService().documentMapperParser().parse(mapping);
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field3", "a")
.endObject()
@ -332,7 +331,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
// This should still throw an exception, since field2 is specifically set to ignore_malformed=false
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field2", "a")
.endObject()
@ -361,8 +360,8 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
.endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = mapper("type", initialMapping);
DocumentMapper mergeMapper = mapper("type", updatedMapping);
DocumentMapper defaultMapper = mapper("test1", "type", initialMapping);
DocumentMapper mergeMapper = mapper("test2", "type", updatedMapping);
assertThat(defaultMapper.mappers().getMapper("field"), is(instanceOf(DateFieldMapper.class)));
DateFieldMapper initialDateFieldMapper = (DateFieldMapper) defaultMapper.mappers().getMapper("field");
@ -384,9 +383,9 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
.startObject("properties").startObject("date_field").field("type", "date").endObject().endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = mapper("type", mapping);
DocumentMapper defaultMapper = mapper("test", "type", mapping);
ParsedDocument parsedDoc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument parsedDoc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "2010-01-01")
.endObject()
@ -418,44 +417,47 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
public void testNumericResolutionBackwardsCompat() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("date_field").field("type", "date").field("format", "date_time").field("numeric_resolution", "seconds").endObject().endObject()
.endObject().endObject().string();
.startObject("properties").startObject("date_field").field("type", "date").field("format", "date_time").field("numeric_resolution", "seconds").endObject().endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = mapper("type", mapping, Version.V_0_90_0);
DocumentMapper defaultMapper = mapper("test1", "type", mapping, Version.V_0_90_0);
// provided as an int
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", 42)
.endObject()
.bytes());
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", 42)
.endObject()
.bytes());
assertThat(getDateAsMillis(doc.rootDoc(), "date_field"), equalTo(42000L));
// provided as a string
doc = defaultMapper.parse("type", "2", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "43")
.endObject()
.bytes());
doc = defaultMapper.parse("test", "type", "2", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "43")
.endObject()
.bytes());
assertThat(getDateAsMillis(doc.rootDoc(), "date_field"), equalTo(43000L));
// but formatted dates still parse as milliseconds
doc = defaultMapper.parse("type", "2", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "1970-01-01T00:00:44.000Z")
.endObject()
.bytes());
doc = defaultMapper.parse("test", "type", "2", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "1970-01-01T00:00:44.000Z")
.endObject()
.bytes());
assertThat(getDateAsMillis(doc.rootDoc(), "date_field"), equalTo(44000L));
// expected to fail due to field epoch date formatters not being set
DocumentMapper currentMapper = mapper("type", mapping);
DocumentMapper currentMapper = mapper("test2", "type", mapping);
try {
currentMapper.parse("type", "2", XContentFactory.jsonBuilder()
currentMapper.parse("test", "type", "2", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", randomBoolean() ? "43" : 43)
.endObject()
.bytes());
} catch (MapperParsingException e) {}
fail("expected parse failure");
} catch (MapperParsingException e) {
assertTrue(e.getMessage(), e.getMessage().contains("failed to parse [date_field]"));
}
}
public void testThatEpochCanBeIgnoredWithCustomFormat() throws Exception {
@ -463,19 +465,19 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
.startObject("properties").startObject("date_field").field("type", "date").field("format", "yyyyMMddHH").endObject().endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = mapper("type", mapping);
DocumentMapper defaultMapper = mapper("test1", "type", mapping);
XContentBuilder document = XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "2015060210")
.endObject();
ParsedDocument doc = defaultMapper.parse("type", "1", document.bytes());
ParsedDocument doc = defaultMapper.parse("test", "type", "1", document.bytes());
assertThat(getDateAsMillis(doc.rootDoc(), "date_field"), equalTo(1433239200000L));
IndexResponse indexResponse = client().prepareIndex("test", "test").setSource(document).get();
IndexResponse indexResponse = client().prepareIndex("test2", "test").setSource(document).get();
assertThat(indexResponse.isCreated(), is(true));
// integers should always be parsed as well... cannot be sure it is a unix timestamp only
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", 2015060210)
.endObject()
@ -496,14 +498,14 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
assertDateFormat("epoch_millis||dateOptionalTime");
DocumentMapper defaultMapper = index.mapperService().documentMapper("type");
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "1-1-1T00:00:44.000Z")
.endObject()
.bytes());
// also test normal date
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "2015-06-06T00:00:44.000Z")
.endObject()
@ -521,14 +523,14 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = index.mapperService().documentMapper("type");
// also test normal date
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "2015-06-06T00:00:44.000Z")
.endObject()
.bytes());
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("date_field", "1-1-1T00:00:44.000Z")
.endObject()

View File

@ -46,7 +46,7 @@ public class GenericStoreDynamicTemplateTests extends ElasticsearchSingleNodeTes
client().admin().indices().preparePutMapping("test").setType("person").setSource(mapping).get();
DocumentMapper docMapper = index.mapperService().documentMapper("person");
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/dynamictemplate/genericstore/test-data.json");
ParsedDocument parsedDoc = docMapper.parse("person", "1", new BytesArray(json));
ParsedDocument parsedDoc = docMapper.parse("test", "person", "1", new BytesArray(json));
client().admin().indices().preparePutMapping("test").setType("person").setSource(parsedDoc.dynamicMappingsUpdate().toString()).get();
Document doc = parsedDoc.rootDoc();

View File

@ -47,7 +47,7 @@ public class PathMatchDynamicTemplateTests extends ElasticsearchSingleNodeTest {
client().admin().indices().preparePutMapping("test").setType("person").setSource(mapping).get();
DocumentMapper docMapper = index.mapperService().documentMapper("person");
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/dynamictemplate/pathmatch/test-data.json");
ParsedDocument parsedDoc = docMapper.parse("person", "1", new BytesArray(json));
ParsedDocument parsedDoc = docMapper.parse("test", "person", "1", new BytesArray(json));
client().admin().indices().preparePutMapping("test").setType("person").setSource(parsedDoc.dynamicMappingsUpdate().toString()).get();
Document doc = parsedDoc.rootDoc();

View File

@ -52,7 +52,7 @@ public class SimpleDynamicTemplatesTests extends ElasticsearchSingleNodeTest {
DocumentMapper docMapper = index.mapperService().documentMapper("person");
builder = JsonXContent.contentBuilder();
builder.startObject().field("s", "hello").field("l", 1).endObject();
ParsedDocument parsedDoc = docMapper.parse("person", "1", builder.bytes());
ParsedDocument parsedDoc = docMapper.parse("test", "person", "1", builder.bytes());
client().admin().indices().preparePutMapping("test").setType("person").setSource(parsedDoc.dynamicMappingsUpdate().toString()).get();
DocumentFieldMappers mappers = docMapper.mappers();
@ -74,7 +74,7 @@ public class SimpleDynamicTemplatesTests extends ElasticsearchSingleNodeTest {
client().admin().indices().preparePutMapping("test").setType("person").setSource(mapping).get();
DocumentMapper docMapper = index.mapperService().documentMapper("person");
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/dynamictemplate/simple/test-data.json");
ParsedDocument parsedDoc = docMapper.parse("person", "1", new BytesArray(json));
ParsedDocument parsedDoc = docMapper.parse("test", "person", "1", new BytesArray(json));
client().admin().indices().preparePutMapping("test").setType("person").setSource(parsedDoc.dynamicMappingsUpdate().toString()).get();
Document doc = parsedDoc.rootDoc();
@ -131,7 +131,7 @@ public class SimpleDynamicTemplatesTests extends ElasticsearchSingleNodeTest {
client().admin().indices().preparePutMapping("test").setType("person").setSource(mapping).get();
DocumentMapper docMapper = index.mapperService().documentMapper("person");
byte[] json = copyToBytesFromClasspath("/org/elasticsearch/index/mapper/dynamictemplate/simple/test-data.json");
ParsedDocument parsedDoc = docMapper.parse("person", "1", new BytesArray(json));
ParsedDocument parsedDoc = docMapper.parse("test", "person", "1", new BytesArray(json));
client().admin().indices().preparePutMapping("test").setType("person").setSource(parsedDoc.dynamicMappingsUpdate().toString()).get();
Document doc = parsedDoc.rootDoc();

View File

@ -51,7 +51,7 @@ public class SimpleExternalMappingTests extends ElasticsearchSingleNodeTest {
.endObject().endObject().string()
);
ParsedDocument doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "1234")
.endObject()
@ -99,7 +99,7 @@ public class SimpleExternalMappingTests extends ElasticsearchSingleNodeTest {
.endObject().endObject().endObject()
.string());
ParsedDocument doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "1234")
.endObject()
@ -153,7 +153,7 @@ public class SimpleExternalMappingTests extends ElasticsearchSingleNodeTest {
.endObject().endObject().endObject()
.string());
ParsedDocument doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "1234")
.endObject()

View File

@ -44,7 +44,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 1.2).field("lon", 1.3).endObject()
.endObject()
@ -66,7 +66,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 1.2).field("lon", 1.3).endObject()
.endObject()
@ -85,7 +85,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("point", "1.2,1.3")
.endObject()
@ -104,7 +104,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("point", GeoHashUtils.encode(1.2, 1.3))
.endObject()
@ -123,7 +123,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("point", GeoHashUtils.encode(1.2, 1.3))
.endObject()
@ -143,7 +143,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 91).field("lon", 181).endObject()
.endObject()
@ -151,7 +151,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
assertThat(doc.rootDoc().get("point"), equalTo("89.0,1.0"));
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", -91).field("lon", -181).endObject()
.endObject()
@ -159,7 +159,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
assertThat(doc.rootDoc().get("point"), equalTo("-89.0,-1.0"));
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 181).field("lon", 361).endObject()
.endObject()
@ -177,14 +177,14 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 90).field("lon", 1.3).endObject()
.endObject()
.bytes());
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", -91).field("lon", 1.3).endObject()
.endObject()
@ -195,7 +195,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
}
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 91).field("lon", 1.3).endObject()
.endObject()
@ -206,7 +206,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
}
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 1.2).field("lon", -181).endObject()
.endObject()
@ -217,7 +217,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
}
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 1.2).field("lon", 181).endObject()
.endObject()
@ -237,31 +237,31 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 90).field("lon", 1.3).endObject()
.endObject()
.bytes());
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", -91).field("lon", 1.3).endObject()
.endObject()
.bytes());
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 91).field("lon", 1.3).endObject()
.endObject()
.bytes());
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 1.2).field("lon", -181).endObject()
.endObject()
.bytes());
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 1.2).field("lon", 181).endObject()
.endObject()
@ -276,7 +276,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 1.2).field("lon", 1.3).endObject()
.endObject()
@ -298,7 +298,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startArray("point")
.startObject().field("lat", 1.2).field("lon", 1.3).endObject()
@ -325,7 +325,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("point", "1.2,1.3")
.endObject()
@ -344,7 +344,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("point", "1.2,1.3")
.endObject()
@ -365,7 +365,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startArray("point")
.value("1.2,1.3")
@ -392,7 +392,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startArray("point").value(1.3).value(1.2).endArray()
.endObject()
@ -413,7 +413,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startArray("point").value(1.3).value(1.2).endArray()
.endObject()
@ -432,7 +432,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startArray("point").value(1.3).value(1.2).endArray()
.endObject()
@ -453,7 +453,7 @@ public class GeoPointFieldMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startArray("point")
.startArray().value(1.3).value(1.2).endArray()

View File

@ -43,7 +43,7 @@ public class GeohashMappingGeoPointTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("point").field("lat", 1.2).field("lon", 1.3).endObject()
.endObject()
@ -62,7 +62,7 @@ public class GeohashMappingGeoPointTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("point", "1.2,1.3")
.endObject()
@ -81,7 +81,7 @@ public class GeohashMappingGeoPointTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("point", GeoHashUtils.encode(1.2, 1.3))
.endObject()
@ -125,7 +125,7 @@ public class GeohashMappingGeoPointTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("point", (Object) null)
.endObject()

View File

@ -45,7 +45,7 @@ public class IdMappingTests extends ElasticsearchSingleNodeTest {
.endObject().endObject().string();
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.endObject()
.bytes());
@ -54,7 +54,7 @@ public class IdMappingTests extends ElasticsearchSingleNodeTest {
assertThat(doc.rootDoc().get(IdFieldMapper.NAME), nullValue());
try {
docMapper.parse("type", null, XContentFactory.jsonBuilder()
docMapper.parse("test", "type", null, XContentFactory.jsonBuilder()
.startObject()
.endObject()
.bytes());
@ -71,7 +71,7 @@ public class IdMappingTests extends ElasticsearchSingleNodeTest {
Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build();
DocumentMapper docMapper = createIndex("test", indexSettings).mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.endObject()
.bytes());

View File

@ -1,71 +0,0 @@
/*
* 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.index;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test;
import java.io.IOException;
import java.util.Locale;
import java.util.Map;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.*;
/**
*
*/
public class IndexTypeMapperIntegrationTests extends ElasticsearchIntegrationTest {
@Test // issue 5053
public void testThatUpdatingMappingShouldNotRemoveSizeMappingConfiguration() throws Exception {
String index = "foo";
String type = "mytype";
XContentBuilder builder = jsonBuilder().startObject().startObject("_index").field("enabled", true).endObject().endObject();
assertAcked(client().admin().indices().prepareCreate(index).addMapping(type, builder));
// check mapping again
assertIndexMappingEnabled(index, type);
// update some field in the mapping
XContentBuilder updateMappingBuilder = jsonBuilder().startObject().startObject("properties").startObject("otherField").field("type", "string").endObject().endObject();
PutMappingResponse putMappingResponse = client().admin().indices().preparePutMapping(index).setType(type).setSource(updateMappingBuilder).get();
assertAcked(putMappingResponse);
// make sure timestamp field is still in mapping
assertIndexMappingEnabled(index, type);
}
private void assertIndexMappingEnabled(String index, String type) throws IOException {
String errMsg = String.format(Locale.ROOT, "Expected index field mapping to be enabled for %s/%s", index, type);
GetMappingsResponse getMappingsResponse = client().admin().indices().prepareGetMappings(index).addTypes(type).get();
Map<String, Object> mappingSource = getMappingsResponse.getMappings().get(index).get(type).getSourceAsMap();
assertThat(errMsg, mappingSource, hasKey("_index"));
String ttlAsString = mappingSource.get("_index").toString();
assertThat(ttlAsString, is(notNullValue()));
assertThat(errMsg, ttlAsString, is("{enabled=true}"));
}
}

View File

@ -32,16 +32,17 @@ import org.elasticsearch.test.ElasticsearchSingleNodeTest;
import static org.hamcrest.Matchers.*;
public class IndexTypeMapperTests extends ElasticsearchSingleNodeTest {
private Settings bwcSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build();
public void testSimpleIndexMapper() throws Exception {
public void testSimpleIndexMapperEnabledBackcompat() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("_index").field("enabled", true).endObject()
.endObject().endObject().string();
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
DocumentMapper docMapper = createIndex("test", bwcSettings).mapperService().documentMapperParser().parse(mapping);
IndexFieldMapper indexMapper = docMapper.indexMapper();
assertThat(indexMapper.enabled(), equalTo(true));
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.endObject()
@ -51,15 +52,15 @@ public class IndexTypeMapperTests extends ElasticsearchSingleNodeTest {
assertThat(doc.rootDoc().get("field"), equalTo("value"));
}
public void testExplicitDisabledIndexMapper() throws Exception {
public void testExplicitDisabledIndexMapperBackcompat() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("_index").field("enabled", false).endObject()
.endObject().endObject().string();
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
DocumentMapper docMapper = createIndex("test", bwcSettings).mapperService().documentMapperParser().parse(mapping);
IndexFieldMapper indexMapper = docMapper.rootMapper(IndexFieldMapper.class);
assertThat(indexMapper.enabled(), equalTo(false));
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.endObject()
@ -76,7 +77,7 @@ public class IndexTypeMapperTests extends ElasticsearchSingleNodeTest {
IndexFieldMapper indexMapper = docMapper.rootMapper(IndexFieldMapper.class);
assertThat(indexMapper.enabled(), equalTo(false));
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.endObject()
@ -86,11 +87,11 @@ public class IndexTypeMapperTests extends ElasticsearchSingleNodeTest {
assertThat(doc.rootDoc().get("field"), equalTo("value"));
}
public void testThatMergingFieldMappingAllowsDisabling() throws Exception {
public void testThatMergingFieldMappingAllowsDisablingBackcompat() throws Exception {
String mappingWithIndexEnabled = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("_index").field("enabled", true).endObject()
.endObject().endObject().string();
DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser();
DocumentMapperParser parser = createIndex("test", bwcSettings).mapperService().documentMapperParser();
DocumentMapper mapperEnabled = parser.parse(mappingWithIndexEnabled);
@ -103,11 +104,11 @@ public class IndexTypeMapperTests extends ElasticsearchSingleNodeTest {
assertThat(mapperEnabled.IndexFieldMapper().enabled(), is(false));
}
public void testThatDisablingWorksWhenMerging() throws Exception {
public void testThatDisablingWorksWhenMergingBackcompat() throws Exception {
String enabledMapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("_index").field("enabled", true).endObject()
.endObject().endObject().string();
DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser();
DocumentMapperParser parser = createIndex("test", bwcSettings).mapperService().documentMapperParser();
DocumentMapper enabledMapper = parser.parse(enabledMapping);
String disabledMapping = XContentFactory.jsonBuilder().startObject().startObject("type")
@ -125,13 +126,13 @@ public class IndexTypeMapperTests extends ElasticsearchSingleNodeTest {
.field("enabled", true)
.field("store", "yes").endObject()
.endObject().endObject().string();
Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build();
DocumentMapper docMapper = createIndex("test", indexSettings).mapperService().documentMapperParser().parse(mapping);
DocumentMapper docMapper = createIndex("test", bwcSettings).mapperService().documentMapperParser().parse(mapping);
IndexFieldMapper indexMapper = docMapper.rootMapper(IndexFieldMapper.class);
assertThat(indexMapper.enabled(), equalTo(true));
assertThat(indexMapper.fieldType().stored(), equalTo(true));
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.endObject()

View File

@ -80,7 +80,7 @@ public class FieldNamesFieldMapperTests extends ElasticsearchSingleNodeTest {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type").endObject().endObject().string();
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("a", "100")
.startObject("b")
@ -100,7 +100,7 @@ public class FieldNamesFieldMapperTests extends ElasticsearchSingleNodeTest {
FieldNamesFieldMapper fieldNamesMapper = docMapper.rootMapper(FieldNamesFieldMapper.class);
assertTrue(fieldNamesMapper.fieldType().isEnabled());
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.endObject()
@ -117,7 +117,7 @@ public class FieldNamesFieldMapperTests extends ElasticsearchSingleNodeTest {
FieldNamesFieldMapper fieldNamesMapper = docMapper.rootMapper(FieldNamesFieldMapper.class);
assertFalse(fieldNamesMapper.fieldType().isEnabled());
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.endObject()
@ -145,7 +145,7 @@ public class FieldNamesFieldMapperTests extends ElasticsearchSingleNodeTest {
FieldNamesFieldMapper fieldNamesMapper = docMapper.rootMapper(FieldNamesFieldMapper.class);
assertFalse(fieldNamesMapper.fieldType().isEnabled());
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.endObject()

View File

@ -47,7 +47,7 @@ public class SimpleIpMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("ip", "127.0.0.1")
.endObject()
@ -91,20 +91,20 @@ public class SimpleIpMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1",
ParsedDocument doc = defaultMapper.parse("test", "type", "1",
XContentFactory.jsonBuilder().startObject().field("field1", "").field("field2", "10.20.30.40").endObject().bytes());
assertThat(doc.rootDoc().getField("field1"), nullValue());
assertThat(doc.rootDoc().getField("field2"), notNullValue());
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject().field("field2", "").endObject().bytes());
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject().field("field2", "").endObject().bytes());
} catch (MapperParsingException e) {
assertThat(e.getCause(), instanceOf(IllegalArgumentException.class));
}
// Verify that the default is false
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject().field("field3", "").endObject().bytes());
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject().field("field3", "").endObject().bytes());
} catch (MapperParsingException e) {
assertThat(e.getCause(), instanceOf(IllegalArgumentException.class));
}
@ -112,12 +112,12 @@ public class SimpleIpMappingTests extends ElasticsearchSingleNodeTest {
// Unless the global ignore_malformed option is set to true
Settings indexSettings = settingsBuilder().put("index.mapping.ignore_malformed", true).build();
defaultMapper = createIndex("test2", indexSettings).mapperService().documentMapperParser().parse(mapping);
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject().field("field3", "").endObject().bytes());
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject().field("field3", "").endObject().bytes());
assertThat(doc.rootDoc().getField("field3"), nullValue());
// This should still throw an exception, since field2 is specifically set to ignore_malformed=false
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject().field("field2", "").endObject().bytes());
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject().field("field2", "").endObject().bytes());
} catch (MapperParsingException e) {
assertThat(e.getCause(), instanceOf(IllegalArgumentException.class));
}

View File

@ -53,7 +53,7 @@ public class DoubleIndexingDocTest extends ElasticsearchSingleNodeTest {
client().admin().indices().preparePutMapping("test").setType("type").setSource(mapping).get();
DocumentMapper mapper = index.mapperService().documentMapper("type");
ParsedDocument doc = mapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = mapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field1", "value1")
.field("field2", 1)

View File

@ -65,7 +65,7 @@ public class StoredNumericValuesTest extends ElasticsearchSingleNodeTest {
.string();
DocumentMapper mapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = mapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = mapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field1", 1)
.field("field2", 1.1)

View File

@ -174,7 +174,7 @@ public class TestMergeMapperTests extends ElasticsearchSingleNodeTest {
barrier.await();
for (int i = 0; i < 200 && stopped.get() == false; i++) {
final String fieldName = Integer.toString(i);
ParsedDocument doc = documentMapper.parse("test", fieldName, new BytesArray("{ \"" + fieldName + "\" : \"test\" }"));
ParsedDocument doc = documentMapper.parse("test", "test", fieldName, new BytesArray("{ \"" + fieldName + "\" : \"test\" }"));
Mapping update = doc.dynamicMappingsUpdate();
assert update != null;
lastIntroducedFieldName.set(fieldName);
@ -193,7 +193,7 @@ public class TestMergeMapperTests extends ElasticsearchSingleNodeTest {
while(stopped.get() == false) {
final String fieldName = lastIntroducedFieldName.get();
final BytesReference source = new BytesArray("{ \"" + fieldName + "\" : \"test\" }");
ParsedDocument parsedDoc = documentMapper.parse("test", "random", source);
ParsedDocument parsedDoc = documentMapper.parse("test", "test", "random", source);
if (parsedDoc.dynamicMappingsUpdate() != null) {
// not in the mapping yet, try again
continue;

View File

@ -77,7 +77,7 @@ public class MultiFieldTests extends ElasticsearchSingleNodeTest {
private void testMultiField(String mapping) throws Exception {
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
BytesReference json = new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/mapper/multifield/test-data.json"));
Document doc = docMapper.parse("person", "1", json).rootDoc();
Document doc = docMapper.parse("test", "person", "1", json).rootDoc();
IndexableField f = doc.getField("name");
assertThat(f.name(), equalTo("name"));
@ -151,7 +151,7 @@ public class MultiFieldTests extends ElasticsearchSingleNodeTest {
Settings settings = indexService.settingsService().getSettings();
DocumentMapperParser mapperParser = indexService.mapperService().documentMapperParser();
DocumentMapper builderDocMapper = doc("test", settings, rootObject("person").add(
DocumentMapper builderDocMapper = doc(settings, rootObject("person").add(
stringField("name").store(true)
.addMultiField(stringField("indexed").index(true).tokenized(true))
.addMultiField(stringField("not_indexed").index(false).store(true))
@ -164,7 +164,7 @@ public class MultiFieldTests extends ElasticsearchSingleNodeTest {
BytesReference json = new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/mapper/multifield/test-data.json"));
Document doc = docMapper.parse("person", "1", json).rootDoc();
Document doc = docMapper.parse("test", "person", "1", json).rootDoc();
IndexableField f = doc.getField("name");
assertThat(f.name(), equalTo("name"));
@ -191,7 +191,7 @@ public class MultiFieldTests extends ElasticsearchSingleNodeTest {
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/test-multi-field-type-no-default-field.json");
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
BytesReference json = new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/mapper/multifield/test-data.json"));
Document doc = docMapper.parse("person", "1", json).rootDoc();
Document doc = docMapper.parse("test", "person", "1", json).rootDoc();
assertNull(doc.getField("name"));
IndexableField f = doc.getField("name.indexed");
@ -276,7 +276,7 @@ public class MultiFieldTests extends ElasticsearchSingleNodeTest {
BytesReference json = jsonBuilder().startObject()
.field("a", "-1,-1")
.endObject().bytes();
Document doc = docMapper.parse("type", "1", json).rootDoc();
Document doc = docMapper.parse("test", "type", "1", json).rootDoc();
IndexableField f = doc.getField("a");
assertThat(f, notNullValue());
@ -307,7 +307,7 @@ public class MultiFieldTests extends ElasticsearchSingleNodeTest {
json = jsonBuilder().startObject()
.field("b", "-1,-1")
.endObject().bytes();
doc = docMapper.parse("type", "1", json).rootDoc();
doc = docMapper.parse("test", "type", "1", json).rootDoc();
f = doc.getField("b");
assertThat(f, notNullValue());
@ -326,7 +326,7 @@ public class MultiFieldTests extends ElasticsearchSingleNodeTest {
json = jsonBuilder().startObject()
.startArray("b").startArray().value(-1).value(-1).endArray().startArray().value(-2).value(-2).endArray().endArray()
.endObject().bytes();
doc = docMapper.parse("type", "1", json).rootDoc();
doc = docMapper.parse("test", "type", "1", json).rootDoc();
f = doc.getFields("b")[0];
assertThat(f, notNullValue());
@ -373,7 +373,7 @@ public class MultiFieldTests extends ElasticsearchSingleNodeTest {
BytesReference json = jsonBuilder().startObject()
.field("a", "complete me")
.endObject().bytes();
Document doc = docMapper.parse("type", "1", json).rootDoc();
Document doc = docMapper.parse("test", "type", "1", json).rootDoc();
IndexableField f = doc.getField("a");
assertThat(f, notNullValue());
@ -404,7 +404,7 @@ public class MultiFieldTests extends ElasticsearchSingleNodeTest {
json = jsonBuilder().startObject()
.field("b", "complete me")
.endObject().bytes();
doc = docMapper.parse("type", "1", json).rootDoc();
doc = docMapper.parse("test", "type", "1", json).rootDoc();
f = doc.getField("b");
assertThat(f, notNullValue());

View File

@ -53,7 +53,7 @@ public class JavaMultiFieldMergeTests extends ElasticsearchSingleNodeTest {
assertThat(docMapper.mappers().getMapper("name.indexed"), nullValue());
BytesReference json = XContentFactory.jsonBuilder().startObject().field("name", "some name").endObject().bytes();
Document doc = docMapper.parse("person", "1", json).rootDoc();
Document doc = docMapper.parse("test", "person", "1", json).rootDoc();
IndexableField f = doc.getField("name");
assertThat(f, notNullValue());
f = doc.getField("name.indexed");
@ -75,7 +75,7 @@ public class JavaMultiFieldMergeTests extends ElasticsearchSingleNodeTest {
assertThat(docMapper.mappers().getMapper("name.not_indexed2"), nullValue());
assertThat(docMapper.mappers().getMapper("name.not_indexed3"), nullValue());
doc = docMapper.parse("person", "1", json).rootDoc();
doc = docMapper.parse("test", "person", "1", json).rootDoc();
f = doc.getField("name");
assertThat(f, notNullValue());
f = doc.getField("name.indexed");
@ -125,7 +125,7 @@ public class JavaMultiFieldMergeTests extends ElasticsearchSingleNodeTest {
assertThat(docMapper.mappers().getMapper("name.indexed"), nullValue());
BytesReference json = XContentFactory.jsonBuilder().startObject().field("name", "some name").endObject().bytes();
Document doc = docMapper.parse("person", "1", json).rootDoc();
Document doc = docMapper.parse("test", "person", "1", json).rootDoc();
IndexableField f = doc.getField("name");
assertThat(f, notNullValue());
f = doc.getField("name.indexed");
@ -148,7 +148,7 @@ public class JavaMultiFieldMergeTests extends ElasticsearchSingleNodeTest {
assertThat(docMapper.mappers().getMapper("name.not_indexed2"), nullValue());
assertThat(docMapper.mappers().getMapper("name.not_indexed3"), nullValue());
doc = docMapper.parse("person", "1", json).rootDoc();
doc = docMapper.parse("test", "person", "1", json).rootDoc();
f = doc.getField("name");
assertThat(f, notNullValue());
f = doc.getField("name.indexed");

View File

@ -41,7 +41,7 @@ public class NestedMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.nullField("nested1")
@ -50,7 +50,7 @@ public class NestedMappingTests extends ElasticsearchSingleNodeTest {
assertThat(doc.docs().size(), equalTo(1));
doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.startArray("nested").endArray()
@ -72,7 +72,7 @@ public class NestedMappingTests extends ElasticsearchSingleNodeTest {
ObjectMapper nested1Mapper = docMapper.objectMappers().get("nested1");
assertThat(nested1Mapper.nested().isNested(), equalTo(true));
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.startObject("nested1").field("field1", "1").field("field2", "2").endObject()
@ -87,7 +87,7 @@ public class NestedMappingTests extends ElasticsearchSingleNodeTest {
assertThat(doc.docs().get(1).get("field"), equalTo("value"));
doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.startArray("nested1")
@ -128,7 +128,7 @@ public class NestedMappingTests extends ElasticsearchSingleNodeTest {
assertThat(nested2Mapper.nested().isIncludeInParent(), equalTo(false));
assertThat(nested2Mapper.nested().isIncludeInRoot(), equalTo(false));
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.startArray("nested1")
@ -180,7 +180,7 @@ public class NestedMappingTests extends ElasticsearchSingleNodeTest {
assertThat(nested2Mapper.nested().isIncludeInParent(), equalTo(true));
assertThat(nested2Mapper.nested().isIncludeInRoot(), equalTo(false));
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.startArray("nested1")
@ -232,7 +232,7 @@ public class NestedMappingTests extends ElasticsearchSingleNodeTest {
assertThat(nested2Mapper.nested().isIncludeInParent(), equalTo(true));
assertThat(nested2Mapper.nested().isIncludeInRoot(), equalTo(false));
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.startArray("nested1")
@ -284,7 +284,7 @@ public class NestedMappingTests extends ElasticsearchSingleNodeTest {
assertThat(nested2Mapper.nested().isIncludeInParent(), equalTo(false));
assertThat(nested2Mapper.nested().isIncludeInRoot(), equalTo(true));
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.startArray("nested1")
@ -331,7 +331,7 @@ public class NestedMappingTests extends ElasticsearchSingleNodeTest {
assertThat(nested1Mapper.nested().isNested(), equalTo(true));
assertThat(nested1Mapper.dynamic(), equalTo(Dynamic.STRICT));
ParsedDocument doc = docMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = docMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "value")
.startArray("nested1")

View File

@ -63,7 +63,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
client().admin().indices().preparePutMapping("test").setType("type").setSource(mapping).get();
DocumentMapper defaultMapper = index.mapperService().documentMapper("type");
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("s_long", "100")
.field("s_double", "100.0")
@ -88,7 +88,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
client().admin().indices().preparePutMapping("test").setType("type").setSource(mapping).get();
DocumentMapper defaultMapper = index.mapperService().documentMapper("type");
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("s_long", "100")
.field("s_double", "100.0")
@ -116,7 +116,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field1", "a")
.field("field2", "1")
@ -126,7 +126,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
assertThat(doc.rootDoc().getField("field2"), notNullValue());
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field2", "a")
.endObject()
@ -137,7 +137,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
// Verify that the default is false
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field3", "a")
.endObject()
@ -149,7 +149,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
// Unless the global ignore_malformed option is set to true
Settings indexSettings = settingsBuilder().put("index.mapping.ignore_malformed", true).build();
defaultMapper = createIndex("test2", indexSettings).mapperService().documentMapperParser().parse(mapping);
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field3", "a")
.endObject()
@ -158,7 +158,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
// This should still throw an exception, since field2 is specifically set to ignore_malformed=false
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field2", "a")
.endObject()
@ -190,7 +190,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
//Test numbers passed as strings
String invalidJsonNumberAsString="1";
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("noErrorNoCoerceField", invalidJsonNumberAsString)
.field("noErrorCoerceField", invalidJsonNumberAsString)
@ -204,7 +204,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
//Test valid case of numbers passed as numbers
int validNumber=1;
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("noErrorNoCoerceField", validNumber)
.field("noErrorCoerceField", validNumber)
@ -217,7 +217,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
//Test valid case of negative numbers passed as numbers
int validNegativeNumber=-1;
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("noErrorNoCoerceField", validNegativeNumber)
.field("noErrorCoerceField", validNegativeNumber)
@ -230,7 +230,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("errorNoCoerce", invalidJsonNumberAsString)
.endObject()
@ -243,7 +243,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
//Test questionable case of floats passed to ints
float invalidJsonForInteger=1.9f;
int coercedFloatValue=1; //This is what the JSON parser will do to a float - truncate not round
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("noErrorNoCoerceField", invalidJsonForInteger)
.field("noErrorCoerceField", invalidJsonForInteger)
@ -256,7 +256,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
assertEquals(coercedFloatValue,doc.rootDoc().getField("errorDefaultCoerce").numericValue().intValue());
try {
defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("errorNoCoerce", invalidJsonForInteger)
.endObject()
@ -288,7 +288,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument parsedDoc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument parsedDoc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("int", "1234")
.field("double", "1234")
@ -324,7 +324,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument parsedDoc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument parsedDoc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startArray("nested")
.startObject()
@ -357,7 +357,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
DocumentMapper mapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = mapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = mapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("long", "100")
.field("double", "100.0")
@ -408,7 +408,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
DocumentMapper mapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = mapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = mapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("int", "100")
.field("float", "100.0")
@ -479,7 +479,7 @@ public class SimpleNumericTests extends ElasticsearchSingleNodeTest {
DocumentMapper mapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = mapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = mapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("int", "100")
.field("float", "100.0")

View File

@ -42,7 +42,7 @@ public class NullValueObjectMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("obj1").endObject()
.field("value1", "test1")
@ -51,7 +51,7 @@ public class NullValueObjectMappingTests extends ElasticsearchSingleNodeTest {
assertThat(doc.rootDoc().get("value1"), equalTo("test1"));
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.nullField("obj1")
.field("value1", "test1")
@ -60,7 +60,7 @@ public class NullValueObjectMappingTests extends ElasticsearchSingleNodeTest {
assertThat(doc.rootDoc().get("value1"), equalTo("test1"));
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("obj1").field("field", "value").endObject()
.field("value1", "test1")

View File

@ -38,7 +38,7 @@ public class SimpleObjectMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
try {
defaultMapper.parse("type", "1", new BytesArray(" {\n" +
defaultMapper.parse("test", "type", "1", new BytesArray(" {\n" +
" \"object\": {\n" +
" \"array\":[\n" +
" {\n" +

View File

@ -120,6 +120,6 @@ public class RoutingTypeMapperTests extends ElasticsearchSingleNodeTest {
// _routing in a document never worked, so backcompat is ignoring the field
assertNull(request.routing());
assertNull(docMapper.parse("type", "1", doc.bytes()).rootDoc().get("_routing"));
assertNull(docMapper.parse("test", "type", "1", doc.bytes()).rootDoc().get("_routing"));
}
}

View File

@ -45,18 +45,18 @@ public class SimpleMapperTests extends ElasticsearchSingleNodeTest {
IndexService indexService = createIndex("test");
Settings settings = indexService.settingsService().getSettings();
DocumentMapperParser mapperParser = indexService.mapperService().documentMapperParser();
DocumentMapper docMapper = doc("test", settings,
DocumentMapper docMapper = doc(settings,
rootObject("person")
.add(object("name").add(stringField("first").store(true).index(false))),
indexService.mapperService()).build(indexService.mapperService(), mapperParser);
BytesReference json = new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/mapper/simple/test1.json"));
Document doc = docMapper.parse("person", "1", json).rootDoc();
Document doc = docMapper.parse("test", "person", "1", json).rootDoc();
assertThat(doc.get(docMapper.mappers().getMapper("name.first").fieldType().names().indexName()), equalTo("shay"));
// System.out.println("Document: " + doc);
// System.out.println("Json: " + docMapper.sourceMapper().value(doc));
doc = docMapper.parse("person", "1", json).rootDoc();
doc = docMapper.parse("test", "person", "1", json).rootDoc();
// System.out.println("Document: " + doc);
// System.out.println("Json: " + docMapper.sourceMapper().value(doc));
}
@ -71,7 +71,7 @@ public class SimpleMapperTests extends ElasticsearchSingleNodeTest {
// reparse it
DocumentMapper builtDocMapper = parser.parse(builtMapping);
BytesReference json = new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/mapper/simple/test1.json"));
Document doc = builtDocMapper.parse("person", "1", json).rootDoc();
Document doc = builtDocMapper.parse("test", "person", "1", json).rootDoc();
assertThat(doc.get(docMapper.uidMapper().fieldType().names().indexName()), equalTo(Uid.createUid("person", "1")));
assertThat(doc.get(docMapper.mappers().getMapper("name.first").fieldType().names().indexName()), equalTo("shay"));
// System.out.println("Document: " + doc);
@ -86,7 +86,7 @@ public class SimpleMapperTests extends ElasticsearchSingleNodeTest {
assertThat((String) docMapper.meta().get("param1"), equalTo("value1"));
BytesReference json = new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/mapper/simple/test1.json"));
Document doc = docMapper.parse("person", "1", json).rootDoc();
Document doc = docMapper.parse("test", "person", "1", json).rootDoc();
assertThat(doc.get(docMapper.uidMapper().fieldType().names().indexName()), equalTo(Uid.createUid("person", "1")));
assertThat(doc.get(docMapper.mappers().getMapper("name.first").fieldType().names().indexName()), equalTo("shay"));
// System.out.println("Document: " + doc);
@ -98,7 +98,7 @@ public class SimpleMapperTests extends ElasticsearchSingleNodeTest {
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/simple/test-mapping.json");
DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
BytesReference json = new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/mapper/simple/test1-notype-noid.json"));
Document doc = docMapper.parse("person", "1", json).rootDoc();
Document doc = docMapper.parse("test", "person", "1", json).rootDoc();
assertThat(doc.get(docMapper.uidMapper().fieldType().names().indexName()), equalTo(Uid.createUid("person", "1")));
assertThat(doc.get(docMapper.mappers().getMapper("name.first").fieldType().names().indexName()), equalTo("shay"));
// System.out.println("Document: " + doc);
@ -123,14 +123,14 @@ public class SimpleMapperTests extends ElasticsearchSingleNodeTest {
IndexService indexService = createIndex("test");
Settings settings = indexService.settingsService().getSettings();
DocumentMapperParser mapperParser = indexService.mapperService().documentMapperParser();
DocumentMapper docMapper = doc("test", settings,
DocumentMapper docMapper = doc(settings,
rootObject("person")
.add(object("name").add(stringField("first").store(true).index(false))),
indexService.mapperService()).build(indexService.mapperService(), mapperParser);
BytesReference json = new BytesArray("".getBytes(Charsets.UTF_8));
try {
docMapper.parse("person", "1", json).rootDoc();
docMapper.parse("test", "person", "1", json).rootDoc();
fail("this point is never reached");
} catch (MapperParsingException e) {
assertThat(e.getMessage(), equalTo("failed to parse, document is empty"));

View File

@ -47,7 +47,7 @@ public class CompressSourceMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper documentMapper = createIndex("test", settings).mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject()
ParsedDocument doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject()
.field("field1", "value1")
.field("field2", "value2")
.endObject().bytes());
@ -63,7 +63,7 @@ public class CompressSourceMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper documentMapper = createIndex("test", settings).mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject()
ParsedDocument doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject()
.field("field1", "value1")
.field("field2", "value2")
.endObject().bytes());
@ -80,14 +80,14 @@ public class CompressSourceMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper documentMapper = createIndex("test", settings).mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject()
ParsedDocument doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject()
.field("field1", "value1")
.endObject().bytes());
BytesRef bytes = doc.rootDoc().getBinaryValue("_source");
assertThat(CompressorFactory.isCompressed(new BytesArray(bytes)), equalTo(false));
doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject()
doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject()
.field("field1", "value1")
.field("field2", "value2 xxxxxxxxxxxxxx yyyyyyyyyyyyyyyyyyy zzzzzzzzzzzzzzzzz")
.field("field2", "value2 xxxxxxxxxxxxxx yyyyyyyyyyyyyyyyyyy zzzzzzzzzzzzzzzzz")

View File

@ -49,14 +49,14 @@ public class DefaultSourceMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser();
DocumentMapper documentMapper = parser.parse(mapping);
ParsedDocument doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject()
ParsedDocument doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject()
.field("field", "value")
.endObject().bytes());
assertThat(XContentFactory.xContentType(doc.source()), equalTo(XContentType.JSON));
documentMapper = parser.parse(mapping);
doc = documentMapper.parse("type", "1", XContentFactory.smileBuilder().startObject()
doc = documentMapper.parse("test", "type", "1", XContentFactory.smileBuilder().startObject()
.field("field", "value")
.endObject().bytes());
@ -70,14 +70,14 @@ public class DefaultSourceMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser();
DocumentMapper documentMapper = parser.parse(mapping);
ParsedDocument doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject()
ParsedDocument doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject()
.field("field", "value")
.endObject().bytes());
assertThat(XContentFactory.xContentType(doc.source()), equalTo(XContentType.JSON));
documentMapper = parser.parse(mapping);
doc = documentMapper.parse("type", "1", XContentFactory.smileBuilder().startObject()
doc = documentMapper.parse("test", "type", "1", XContentFactory.smileBuilder().startObject()
.field("field", "value")
.endObject().bytes());
@ -92,7 +92,7 @@ public class DefaultSourceMappingTests extends ElasticsearchSingleNodeTest {
Settings backcompatSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build();
DocumentMapperParser parser = createIndex("test", backcompatSettings).mapperService().documentMapperParser();
DocumentMapper documentMapper = parser.parse(mapping);
ParsedDocument doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject()
ParsedDocument doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject()
.field("field", "value")
.endObject().bytes());
@ -101,7 +101,7 @@ public class DefaultSourceMappingTests extends ElasticsearchSingleNodeTest {
assertThat(XContentFactory.xContentType(uncompressed), equalTo(XContentType.JSON));
documentMapper = parser.parse(mapping);
doc = documentMapper.parse("type", "1", XContentFactory.smileBuilder().startObject()
doc = documentMapper.parse("test", "type", "1", XContentFactory.smileBuilder().startObject()
.field("field", "value")
.endObject().bytes());
@ -117,7 +117,7 @@ public class DefaultSourceMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper documentMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject()
ParsedDocument doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject()
.startObject("path1").field("field1", "value1").endObject()
.startObject("path2").field("field2", "value2").endObject()
.endObject().bytes());
@ -138,7 +138,7 @@ public class DefaultSourceMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper documentMapper = createIndex("test").mapperService().documentMapperParser().parse(mapping);
ParsedDocument doc = documentMapper.parse("type", "1", XContentFactory.jsonBuilder().startObject()
ParsedDocument doc = documentMapper.parse("test", "type", "1", XContentFactory.jsonBuilder().startObject()
.startObject("path1").field("field1", "value1").endObject()
.startObject("path2").field("field2", "value2").endObject()
.endObject().bytes());

View File

@ -83,7 +83,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = parser.parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "1234")
.endObject()
@ -91,7 +91,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
assertThat(doc.rootDoc().getField("field"), notNullValue());
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "12345")
.endObject()
@ -99,7 +99,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
assertThat(doc.rootDoc().getField("field"), notNullValue());
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "123456")
.endObject()
@ -128,7 +128,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
private void assertParseIdemPotent(IndexableFieldType expected, DocumentMapper mapper) throws Exception {
String mapping = mapper.toXContent(XContentFactory.jsonBuilder().startObject(), new ToXContent.MapParams(ImmutableMap.<String, String>of())).endObject().string();
mapper = parser.parse(mapping);
ParsedDocument doc = mapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = mapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "2345")
.endObject()
@ -144,7 +144,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = parser.parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "1234")
.endObject()
@ -163,7 +163,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = parser.parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "1234")
.endObject()
@ -186,7 +186,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
defaultMapper = parser.parse(mapping);
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "1234")
.endObject()
@ -209,7 +209,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
defaultMapper = parser.parse(mapping);
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "1234")
.endObject()
@ -329,7 +329,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = parser.parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field1", "1234")
.field("field2", "1234")
@ -400,7 +400,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = parser.parse(mapping);
ParsedDocument parsedDoc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument parsedDoc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("str1", "1234")
.field("str2", "1234")
@ -454,7 +454,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = parser.parse(mapping);
ParsedDocument parsedDoc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument parsedDoc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("str1", "1234")
.field("str2", "1234")
@ -490,7 +490,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = parser.parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "1234")
.endObject()
@ -505,7 +505,7 @@ public class SimpleStringMappingTests extends ElasticsearchSingleNodeTest {
MergeResult mergeResult = defaultMapper.merge(parser.parse(updatedMapping).mapping(), false, false);
assertFalse(Arrays.toString(mergeResult.buildConflicts()), mergeResult.hasConflicts());
doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", "1234")
.endObject()

View File

@ -767,7 +767,7 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest {
// _timestamp in a document never worked, so backcompat is ignoring the field
assertEquals(MappingMetaData.Timestamp.parseStringTimestamp("1970", Joda.forPattern("YYYY"), Version.V_1_4_2), request.timestamp());
assertNull(docMapper.parse("type", "1", doc.bytes()).rootDoc().get("_timestamp"));
assertNull(docMapper.parse("test", "type", "1", doc.bytes()).rootDoc().get("_timestamp"));
}
public void testThatEpochCanBeIgnoredWithCustomFormat() throws Exception {

View File

@ -308,7 +308,7 @@ public class TTLMappingTests extends ElasticsearchSingleNodeTest {
// _ttl in a document never worked, so backcompat is ignoring the field
assertEquals(-1, request.ttl());
assertNull(docMapper.parse("type", "1", doc.bytes()).rootDoc().get("_ttl"));
assertNull(docMapper.parse("test", "type", "1", doc.bytes()).rootDoc().get("_ttl"));
}
private org.elasticsearch.common.xcontent.XContentBuilder getMappingWithTtlEnabled() throws IOException {

View File

@ -38,7 +38,7 @@ public class ParseDocumentTypeLevelsTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(defaultMapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("test1", "value1")
.field("test2", "value2")
@ -57,7 +57,7 @@ public class ParseDocumentTypeLevelsTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(defaultMapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject().startObject("type")
.field("test1", "value1")
.field("test2", "value2")
@ -76,7 +76,7 @@ public class ParseDocumentTypeLevelsTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(defaultMapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("type", "value_type")
.field("test1", "value1")
@ -97,7 +97,7 @@ public class ParseDocumentTypeLevelsTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(defaultMapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject().startObject("type")
.field("type", "value_type")
.field("test1", "value1")
@ -118,7 +118,7 @@ public class ParseDocumentTypeLevelsTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(defaultMapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.startObject("type").field("type_field", "type_value").endObject()
.field("test1", "value1")
@ -139,7 +139,7 @@ public class ParseDocumentTypeLevelsTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(defaultMapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject().startObject("type")
.startObject("type").field("type_field", "type_value").endObject()
.field("test1", "value1")
@ -160,7 +160,7 @@ public class ParseDocumentTypeLevelsTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(defaultMapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject().startObject("type")
.field("test1", "value1")
.field("test2", "value2")
@ -181,7 +181,7 @@ public class ParseDocumentTypeLevelsTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(defaultMapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject().startObject("type")
.field("test1", "value1")
.field("type", "value_type")
@ -202,7 +202,7 @@ public class ParseDocumentTypeLevelsTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(defaultMapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("test1", "value1")
.startObject("type").field("type_field", "type_value").endObject()
@ -224,7 +224,7 @@ public class ParseDocumentTypeLevelsTests extends ElasticsearchSingleNodeTest {
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse(defaultMapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject().startObject("type")
.field("test1", "value1")
.startObject("type").field("type_field", "type_value").endObject()

View File

@ -33,16 +33,16 @@ public class ParseMappingTypeLevelTests extends ElasticsearchSingleNodeTest {
@Test
public void testTypeLevel() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("_index").field("enabled", true).endObject()
.startObject("_size").field("enabled", true).endObject()
.endObject().endObject().string();
DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser();
DocumentMapper mapper = parser.parse("type", mapping);
assertThat(mapper.type(), equalTo("type"));
assertThat(mapper.indexMapper().enabled(), equalTo(true));
assertThat(mapper.sizeFieldMapper().enabled(), equalTo(true));
mapper = parser.parse(mapping);
assertThat(mapper.type(), equalTo("type"));
assertThat(mapper.indexMapper().enabled(), equalTo(true));
assertThat(mapper.sizeFieldMapper().enabled(), equalTo(true));
}
}

View File

@ -115,8 +115,8 @@ public class UpdateMappingTests extends ElasticsearchSingleNodeTest {
}
@Test
public void testIndexFieldParsing() throws IOException {
IndexService indexService = createIndex("test", Settings.settingsBuilder().build());
public void testIndexFieldParsingBackcompat() throws IOException {
IndexService indexService = createIndex("test", Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build());
XContentBuilder indexMapping = XContentFactory.jsonBuilder();
boolean enabled = randomBoolean();
indexMapping.startObject()

View File

@ -1 +1 @@
{"type":{"_index":{"enabled":false},"_size":{"enabled":false},"_timestamp":{"enabled":false}}}
{"type":{"_size":{"enabled":false},"_timestamp":{"enabled":false}}}

View File

@ -59,7 +59,7 @@ public class IndexQueryParserFilterDateRangeFormatTests extends ElasticsearchSin
MapperService mapperService = indexService.mapperService();
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/query/mapping.json");
mapperService.merge("person", new CompressedXContent(mapping), true, false);
ParsedDocument doc = mapperService.documentMapper("person").parse("person", "1", new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/query/data.json")));
ParsedDocument doc = mapperService.documentMapper("person").parse("test", "person", "1", new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/query/data.json")));
assertNotNull(doc.dynamicMappingsUpdate());
client().admin().indices().preparePutMapping("test").setType("person").setSource(doc.dynamicMappingsUpdate().toString()).get();
queryParser = injector.getInstance(IndexQueryParserService.class);

View File

@ -60,7 +60,7 @@ public class IndexQueryParserFilterDateRangeTimezoneTests extends ElasticsearchS
MapperService mapperService = indexService.mapperService();
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/query/mapping.json");
mapperService.merge("person", new CompressedXContent(mapping), true, false);
ParsedDocument doc = mapperService.documentMapper("person").parse("person", "1", new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/query/data.json")));
ParsedDocument doc = mapperService.documentMapper("person").parse("test", "person", "1", new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/query/data.json")));
assertNotNull(doc.dynamicMappingsUpdate());
client().admin().indices().preparePutMapping("test").setType("person").setSource(doc.dynamicMappingsUpdate().toString()).get();
queryParser = injector.getInstance(IndexQueryParserService.class);

View File

@ -96,7 +96,7 @@ public class SimpleIndexQueryParserTests extends ElasticsearchSingleNodeTest {
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/query/mapping.json");
mapperService.merge("person", new CompressedXContent(mapping), true, false);
ParsedDocument doc = mapperService.documentMapper("person").parse("person", "1", new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/query/data.json")));
ParsedDocument doc = mapperService.documentMapper("person").parse("test", "person", "1", new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/query/data.json")));
assertNotNull(doc.dynamicMappingsUpdate());
client().admin().indices().preparePutMapping("test").setType("person").setSource(doc.dynamicMappingsUpdate().toString()).get();

View File

@ -51,7 +51,7 @@ import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.gateway.GatewayAllocator;
import org.elasticsearch.gateway.PrimaryShardAllocator;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.translog.TranslogService;
@ -514,7 +514,7 @@ public class CorruptedFileTest extends ElasticsearchIntegrationTest {
internalCluster().ensureAtLeastNumDataNodes(2);
assertAcked(prepareCreate("test").setSettings(Settings.builder()
.put(GatewayAllocator.INDEX_RECOVERY_INITIAL_SHARDS, "one")
.put(PrimaryShardAllocator.INDEX_RECOVERY_INITIAL_SHARDS, "one")
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, cluster().numDataNodes() - 1)
.put(MergePolicyConfig.INDEX_MERGE_ENABLED, false)
.put(MockFSDirectoryService.CHECK_INDEX_ON_CLOSE, false) // no checkindex - we corrupt shards on purpose

View File

@ -26,7 +26,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.hamcrest.Matchers;
import org.junit.Test;
import java.io.IOException;
import java.util.Arrays;
@ -40,7 +39,7 @@ import static org.hamcrest.Matchers.*;
public class SimpleGetFieldMappingsTests extends ElasticsearchIntegrationTest {
public void getMappingsWhereThereAreNone() {
public void testGetMappingsWhereThereAreNone() {
createIndex("index");
ensureYellow();
GetFieldMappingsResponse response = client().admin().indices().prepareGetFieldMappings().get();
@ -57,7 +56,7 @@ public class SimpleGetFieldMappingsTests extends ElasticsearchIntegrationTest {
.endObject().endObject().endObject();
}
public void simpleGetFieldMappings() throws Exception {
public void testSimpleGetFieldMappings() throws Exception {
assertAcked(prepareCreate("indexa")
.addMapping("typeA", getMappingForType("typeA"))
@ -128,7 +127,7 @@ public class SimpleGetFieldMappingsTests extends ElasticsearchIntegrationTest {
}
@SuppressWarnings("unchecked")
public void simpleGetFieldMappingsWithDefaults() throws Exception {
public void testSimpleGetFieldMappingsWithDefaults() throws Exception {
assertAcked(prepareCreate("test").addMapping("type", getMappingForType("type")));
client().prepareIndex("test", "type", "1").setSource("num", 1).get();
@ -147,7 +146,7 @@ public class SimpleGetFieldMappingsTests extends ElasticsearchIntegrationTest {
}
//fix #6552
public void simpleGetFieldMappingsWithPretty() throws Exception {
public void testSimpleGetFieldMappingsWithPretty() throws Exception {
assertAcked(prepareCreate("index").addMapping("type", getMappingForType("type")));
Map<String, String> params = Maps.newHashMap();
params.put("pretty", "true");
@ -179,7 +178,6 @@ public class SimpleGetFieldMappingsTests extends ElasticsearchIntegrationTest {
}
@Test
public void testGetFieldMappingsWithBlocks() throws Exception {
assertAcked(prepareCreate("test")
.addMapping("typeA", getMappingForType("typeA"))

View File

@ -47,6 +47,7 @@ import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
@ -204,19 +205,15 @@ public class IndicesStoreIntegrationTests extends ElasticsearchIntegrationTest {
@Test @Slow
public void testShardActiveElseWhere() throws Exception {
boolean node1IsMasterEligible = randomBoolean();
boolean node2IsMasterEligible = !node1IsMasterEligible || randomBoolean();
Future<String> node_1_future = internalCluster().startNodeAsync(Settings.builder().put("node.master", node1IsMasterEligible).build());
Future<String> node_2_future = internalCluster().startNodeAsync(Settings.builder().put("node.master", node2IsMasterEligible).build());
final String node_1 = node_1_future.get();
final String node_2 = node_2_future.get();
final String node_1_id = internalCluster().getInstance(DiscoveryService.class, node_1).localNode().getId();
final String node_2_id = internalCluster().getInstance(DiscoveryService.class, node_2).localNode().getId();
List<String> nodes = internalCluster().startNodesAsync(2).get();
logger.debug("node {} (node_1) is {}master eligible", node_1, node1IsMasterEligible ? "" : "not ");
logger.debug("node {} (node_2) is {}master eligible", node_2, node2IsMasterEligible ? "" : "not ");
logger.debug("node {} became master", internalCluster().getMasterName());
final int numShards = scaledRandomIntBetween(2, 20);
final String masterNode = internalCluster().getMasterName();
final String nonMasterNode = nodes.get(0).equals(masterNode) ? nodes.get(1) : nodes.get(0);
final String masterId = internalCluster().clusterService(masterNode).localNode().id();
final String nonMasterId = internalCluster().clusterService(nonMasterNode).localNode().id();
final int numShards = scaledRandomIntBetween(2, 10);
assertAcked(prepareCreate("test")
.setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, numShards))
);
@ -225,14 +222,14 @@ public class IndicesStoreIntegrationTests extends ElasticsearchIntegrationTest {
waitNoPendingTasksOnAll();
ClusterStateResponse stateResponse = client().admin().cluster().prepareState().get();
RoutingNode routingNode = stateResponse.getState().routingNodes().node(node_2_id);
RoutingNode routingNode = stateResponse.getState().routingNodes().node(nonMasterId);
final int[] node2Shards = new int[routingNode.numberOfOwningShards()];
int i = 0;
for (ShardRouting shardRouting : routingNode) {
node2Shards[i] = shardRouting.shardId().id();
i++;
}
logger.info("Node 2 has shards: {}", Arrays.toString(node2Shards));
logger.info("Node [{}] has shards: {}", nonMasterNode, Arrays.toString(node2Shards));
final long shardVersions[] = new long[numShards];
final int shardIds[] = new int[numShards];
i = 0;
@ -241,17 +238,18 @@ public class IndicesStoreIntegrationTests extends ElasticsearchIntegrationTest {
shardIds[i] = shardRouting.getId();
i++;
}
// disable relocations when we do this, to make sure the shards are not relocated from node2
// due to rebalancing, and delete its content
client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, EnableAllocationDecider.Rebalance.NONE)).get();
internalCluster().getInstance(ClusterService.class, node_2).submitStateUpdateTask("test", Priority.IMMEDIATE, new ClusterStateUpdateTask() {
internalCluster().getInstance(ClusterService.class, nonMasterNode).submitStateUpdateTask("test", Priority.IMMEDIATE, new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder("test");
for (int i = 0; i < numShards; i++) {
indexRoutingTableBuilder.addIndexShard(
new IndexShardRoutingTable.Builder(new ShardId("test", i), false)
.addShard(TestShardRouting.newShardRouting("test", i, node_1_id, true, ShardRoutingState.STARTED, shardVersions[shardIds[i]]))
.addShard(TestShardRouting.newShardRouting("test", i, masterId, true, ShardRoutingState.STARTED, shardVersions[shardIds[i]]))
.build()
);
}
@ -271,7 +269,7 @@ public class IndicesStoreIntegrationTests extends ElasticsearchIntegrationTest {
waitNoPendingTasksOnAll();
logger.info("Checking if shards aren't removed");
for (int shard : node2Shards) {
assertTrue(waitForShardDeletion(node_2, "test", shard));
assertTrue(waitForShardDeletion(nonMasterNode, "test", shard));
}
}

View File

@ -50,7 +50,6 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.range;
import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilders.derivative;
import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilders.movingAvg;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.closeTo;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.core.IsNull.notNullValue;
import static org.hamcrest.core.IsNull.nullValue;
@ -92,7 +91,7 @@ public class MovAvgTests extends ElasticsearchIntegrationTest {
}
enum MetricTarget {
VALUE ("value"), COUNT("count");
VALUE ("value"), COUNT("count"), METRIC("metric");
private final String name;
@ -1326,8 +1325,8 @@ public class MovAvgTests extends ElasticsearchIntegrationTest {
assertThat("[_count] movavg should be NaN, but is ["+countMovAvg.value()+"] instead", countMovAvg.value(), equalTo(Double.NaN));
} else {
assertThat("[_count] movavg is null", countMovAvg, notNullValue());
assertThat("[_count] movavg does not match expected ["+countMovAvg.value()+" vs "+expectedCount+"]",
countMovAvg.value(), closeTo(expectedCount, 0.1));
assertTrue("[_count] movavg does not match expected [" + countMovAvg.value() + " vs " + expectedCount + "]",
nearlyEqual(countMovAvg.value(), expectedCount, 0.1));
}
// This is a gap bucket
@ -1338,8 +1337,34 @@ public class MovAvgTests extends ElasticsearchIntegrationTest {
assertThat("[value] movavg should be NaN, but is ["+valuesMovAvg.value()+"] instead", valuesMovAvg.value(), equalTo(Double.NaN));
} else {
assertThat("[value] movavg is null", valuesMovAvg, notNullValue());
assertThat("[value] movavg does not match expected ["+valuesMovAvg.value()+" vs "+expectedValue+"]",
valuesMovAvg.value(), closeTo(expectedValue, 0.1));
assertTrue("[value] movavg does not match expected [" + valuesMovAvg.value() + " vs " + expectedValue + "]",
nearlyEqual(valuesMovAvg.value(), expectedValue, 0.1));
}
}
/**
* Better floating point comparisons courtesy of https://github.com/brazzy/floating-point-gui.de
*
* Snippet adapted to use doubles instead of floats
*
* @param a
* @param b
* @param epsilon
* @return
*/
private static boolean nearlyEqual(double a, double b, double epsilon) {
final double absA = Math.abs(a);
final double absB = Math.abs(b);
final double diff = Math.abs(a - b);
if (a == b) { // shortcut, handles infinities
return true;
} else if (a == 0 || b == 0 || diff < Double.MIN_NORMAL) {
// a or b is zero or both are extremely close to it
// relative error is less meaningful here
return diff < (epsilon * Double.MIN_NORMAL);
} else { // use relative error
return diff / Math.min((absA + absB), Double.MAX_VALUE) < epsilon;
}
}

View File

@ -23,13 +23,16 @@ import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterInfoService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocators;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecidersModule;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.common.transport.TransportAddress;
@ -121,4 +124,41 @@ public abstract class ElasticsearchAllocationTestCase extends ElasticsearchTestC
RoutingTable routingTable = strategy.applyStartedShards(clusterState, newArrayList(initializingShards.get(randomInt(initializingShards.size() - 1)))).routingTable();
return ClusterState.builder(clusterState).routingTable(routingTable).build();
}
public static AllocationDeciders yesAllocationDeciders() {
return new AllocationDeciders(Settings.EMPTY, new AllocationDecider[] {new TestAllocateDecision(Decision.YES)});
}
public static AllocationDeciders noAllocationDeciders() {
return new AllocationDeciders(Settings.EMPTY, new AllocationDecider[] {new TestAllocateDecision(Decision.NO)});
}
public static AllocationDeciders throttleAllocationDeciders() {
return new AllocationDeciders(Settings.EMPTY, new AllocationDecider[] {new TestAllocateDecision(Decision.THROTTLE)});
}
static class TestAllocateDecision extends AllocationDecider {
private final Decision decision;
public TestAllocateDecision(Decision decision) {
super(Settings.EMPTY);
this.decision = decision;
}
@Override
public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
return decision;
}
@Override
public Decision canAllocate(ShardRouting shardRouting, RoutingAllocation allocation) {
return decision;
}
@Override
public Decision canAllocate(RoutingNode node, RoutingAllocation allocation) {
return decision;
}
}
}

View File

@ -82,6 +82,8 @@ public class TestSearchContext extends SearchContext {
private String[] types;
private SearchContextAggregations aggregations;
private final long originNanoTime = System.nanoTime();
public TestSearchContext(ThreadPool threadPool,PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, IndexService indexService, QueryCache filterCache, IndexFieldDataService indexFieldDataService) {
super(ParseFieldMatcher.STRICT);
this.pageCacheRecycler = pageCacheRecycler;
@ -170,6 +172,11 @@ public class TestSearchContext extends SearchContext {
return null;
}
@Override
public long getOriginNanoTime() {
return originNanoTime;
}
@Override
protected long nowInMillisImpl() {
return 0;

View File

@ -1,17 +0,0 @@
<?xml version="1.0" encoding="UTF-8"?>
<!-- remove this when junit4 summary format is fixed -->
<xsl:stylesheet version="1.0" xmlns:xsl="http://www.w3.org/1999/XSL/Transform">
<xsl:output method="xml" version="1.0" encoding="UTF-8" indent="yes"/>
<xsl:template match="/">
<failsafe-summary>
<xsl:attribute name="timeout">
<xsl:value-of select="failsafe-summary/@timeout"/>
</xsl:attribute>
<completed><xsl:value-of select="failsafe-summary/@completed"/></completed>
<errors><xsl:value-of select="failsafe-summary/@errors"/></errors>
<failures><xsl:value-of select="failsafe-summary/@failures"/></failures>
<skipped><xsl:value-of select="failsafe-summary/@skipped"/></skipped>
<failureMessage><xsl:value-of select="failsafe-summary/@failureMessage"/></failureMessage>
</failsafe-summary>
</xsl:template>
</xsl:stylesheet>

View File

@ -183,11 +183,4 @@
<delete file="${integ.pidfile}"/>
</target>
<!-- TODO: remove this and the xslt when junit4 is fixed -->
<target name="fixup-failsafe-summary" unless="${shouldskip}">
<xslt in="${project.build.directory}/failsafe-reports/failsafe-summary-buggy.xml"
out="${project.build.directory}/failsafe-reports/failsafe-summary.xml"
style="${elasticsearch.tools.directory}/ant/fixup-failsafe-summary.xslt"/>
</target>
</project>

View File

@ -50,9 +50,10 @@ See <<time-units>> for accepted abbreviations.
By default, times are stored as UTC milliseconds since the epoch. Thus, all computation and "bucketing" / "rounding" is
done on UTC. It is possible to provide a time zone value, which will cause all bucket
computations to take place in the specified zone. The time returned for each bucket/entry is milliseconds since the
epoch in UTC. The parameters is called `time_zone`. It accepts either a numeric value for the hours offset, for example:
`"time_zone" : -2`. It also accepts a format of hours and minutes, like `"time_zone" : "-02:30"`.
Another option is to provide a time zone accepted as one of the values listed here.
epoch in UTC. The parameters is called `time_zone`. It accepts either a ISO 8601 UTC offset, or a timezone id.
A UTC offset has the form of a `+` or `-`, followed by two digit hour, followed by `:`, followed by two digit minutes.
For example, `+01:00` represents 1 hour ahead of UTC. A timezone id is the identifier for a TZ database. For example,
Pacific time is represented as `America\Los_Angeles`.
Lets take an example. For `2012-04-01T04:15:30Z` (UTC), with a `time_zone` of `"-08:00"`. For day interval, the actual time by
applying the time zone and rounding falls under `2012-03-31`, so the returned value will be (in millis) of

View File

@ -1,10 +1,10 @@
[[mapping-index-field]]
=== `_index` field
The name of the index that contains the document. This field is not indexed
but can be automatically derived from the index itself.
Its value is accessible in queries, aggregations, scripts, and when sorting:
When performing queries across multiple indexes, it is sometimes desirable
to add query clauses that are associated with documents of only certain
indexes. The `_index` field allows matching on the index a document was
indexed into. Its value is accessible in queries, aggregations, scripts, and when sorting:
[source,js]
--------------------------

View File

@ -290,7 +290,7 @@ to provide special features. They now have limited configuration options.
* `_id` configuration can no longer be changed. If you need to sort, use `_uid` instead.
* `_type` configuration can no longer be changed.
* `_index` configuration is limited to enabling the field.
* `_index` configuration can no longer be changed.
* `_routing` configuration is limited to requiring the field.
* `_boost` has been removed.
* `_field_names` configuration is limited to disabling the field.
@ -546,6 +546,12 @@ query to not compute scores and optionally caches the result.
As a consequence the `query` filter serves no purpose anymore and is deprecated.
=== Timezone for date field
Specifying the `time_zone` parameter on queries or aggregations of `date` type fields
must now be either an ISO 8601 UTC offset, or a timezone id. For example, the value
`+1:00` must now be `+01:00`.
=== Snapshot and Restore
Locations of the shared file system repositories and the URL repositories with `file:` URLs has to be now registered

View File

@ -43,7 +43,7 @@ move them to UTC time based date:
"born" : {
"gte": "2012-01-01",
"lte": "now",
"time_zone": "+1:00"
"time_zone": "+01:00"
}
}
}

View File

@ -321,8 +321,6 @@
<configuration>
<target>
<ant antfile="${elasticsearch.integ.antfile}" target="stop-external-cluster"/>
<!-- TODO: remove this and the xslt when junit4 is fixed -->
<ant antfile="${elasticsearch.integ.antfile}" target="fixup-failsafe-summary"/>
</target>
</configuration>
</execution>

View File

@ -43,7 +43,7 @@
<!-- libraries -->
<lucene.version>5.2.1</lucene.version>
<lucene.maven.version>5.2.1</lucene.maven.version>
<testframework.version>2.1.15</testframework.version>
<testframework.version>2.1.16</testframework.version>
<jackson.version>2.5.3</jackson.version>
<slf4j.version>1.6.2</slf4j.version>
<log4j.version>1.2.17</log4j.version>
@ -718,7 +718,7 @@
<listeners>
<report-ant-xml
mavenExtensions="true"
summaryFile="${project.build.directory}/failsafe-reports/failsafe-summary-buggy.xml"
summaryFile="${project.build.directory}/failsafe-reports/failsafe-summary.xml"
dir="${project.build.directory}/failsafe-reports"/>
</listeners>
<!-- currently only 1 cpu works, because integ tests don't make "unique" test directories? -->