Merge branch 'master' into feature-suggest-refactoring

This commit is contained in:
Christoph Büscher 2016-03-11 10:37:28 +01:00
commit daeffb149c
123 changed files with 3742 additions and 1971 deletions

View File

@ -31,6 +31,7 @@ import org.elasticsearch.discovery.DiscoveryStats;
import org.elasticsearch.http.HttpStats;
import org.elasticsearch.indices.NodeIndicesStats;
import org.elasticsearch.indices.breaker.AllCircuitBreakerStats;
import org.elasticsearch.ingest.IngestStats;
import org.elasticsearch.monitor.fs.FsInfo;
import org.elasticsearch.monitor.jvm.JvmStats;
import org.elasticsearch.monitor.os.OsStats;
@ -81,6 +82,9 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
@Nullable
private DiscoveryStats discoveryStats;
@Nullable
private IngestStats ingestStats;
NodeStats() {
}
@ -89,7 +93,8 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
@Nullable FsInfo fs, @Nullable TransportStats transport, @Nullable HttpStats http,
@Nullable AllCircuitBreakerStats breaker,
@Nullable ScriptStats scriptStats,
@Nullable DiscoveryStats discoveryStats) {
@Nullable DiscoveryStats discoveryStats,
@Nullable IngestStats ingestStats) {
super(node);
this.timestamp = timestamp;
this.indices = indices;
@ -103,6 +108,7 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
this.breaker = breaker;
this.scriptStats = scriptStats;
this.discoveryStats = discoveryStats;
this.ingestStats = ingestStats;
}
public long getTimestamp() {
@ -187,6 +193,11 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
return this.discoveryStats;
}
@Nullable
public IngestStats getIngestStats() {
return ingestStats;
}
public static NodeStats readNodeStats(StreamInput in) throws IOException {
NodeStats nodeInfo = new NodeStats();
nodeInfo.readFrom(in);
@ -224,7 +235,7 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
breaker = AllCircuitBreakerStats.readOptionalAllCircuitBreakerStats(in);
scriptStats = in.readOptionalStreamable(ScriptStats::new);
discoveryStats = in.readOptionalStreamable(() -> new DiscoveryStats(null));
ingestStats = in.readOptionalWritable(IngestStats.PROTO);
}
@Override
@ -282,6 +293,7 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
out.writeOptionalStreamable(breaker);
out.writeOptionalStreamable(scriptStats);
out.writeOptionalStreamable(discoveryStats);
out.writeOptionalWriteable(ingestStats);
}
@Override
@ -337,6 +349,10 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
getDiscoveryStats().toXContent(builder, params);
}
if (getIngestStats() != null) {
getIngestStats().toXContent(builder, params);
}
return builder;
}
}

View File

@ -42,6 +42,7 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
private boolean breaker;
private boolean script;
private boolean discovery;
private boolean ingest;
public NodesStatsRequest() {
}
@ -69,6 +70,7 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
this.breaker = true;
this.script = true;
this.discovery = true;
this.ingest = true;
return this;
}
@ -87,6 +89,7 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
this.breaker = false;
this.script = false;
this.discovery = false;
this.ingest = false;
return this;
}
@ -250,6 +253,17 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
return this;
}
public boolean ingest() {
return ingest;
}
/**
* Should ingest statistics be returned.
*/
public NodesStatsRequest ingest(boolean ingest) {
this.ingest = ingest;
return this;
}
@Override
public void readFrom(StreamInput in) throws IOException {
@ -265,6 +279,7 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
breaker = in.readBoolean();
script = in.readBoolean();
discovery = in.readBoolean();
ingest = in.readBoolean();
}
@Override
@ -281,6 +296,7 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
out.writeBoolean(breaker);
out.writeBoolean(script);
out.writeBoolean(discovery);
out.writeBoolean(ingest);
}
}

View File

@ -137,4 +137,12 @@ public class NodesStatsRequestBuilder extends NodesOperationRequestBuilder<Nodes
request.discovery(discovery);
return this;
}
/**
* Should ingest statistics be returned.
*/
public NodesStatsRequestBuilder ingest(boolean ingest) {
request.ingest(ingest);
return this;
}
}

View File

@ -80,7 +80,8 @@ public class TransportNodesStatsAction extends TransportNodesAction<NodesStatsRe
protected NodeStats nodeOperation(NodeStatsRequest nodeStatsRequest) {
NodesStatsRequest request = nodeStatsRequest.request;
return nodeService.stats(request.indices(), request.os(), request.process(), request.jvm(), request.threadPool(),
request.fs(), request.transport(), request.http(), request.breaker(), request.script(), request.discovery());
request.fs(), request.transport(), request.http(), request.breaker(), request.script(), request.discovery(),
request.ingest());
}
@Override

View File

@ -99,7 +99,7 @@ public class TransportClusterStatsAction extends TransportNodesAction<ClusterSta
@Override
protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeRequest) {
NodeInfo nodeInfo = nodeService.info(false, true, false, true, false, true, false, true, false);
NodeStats nodeStats = nodeService.stats(CommonStatsFlags.NONE, false, true, true, false, true, false, false, false, false, false);
NodeStats nodeStats = nodeService.stats(CommonStatsFlags.NONE, false, true, true, false, true, false, false, false, false, false, false);
List<ShardStats> shardsStats = new ArrayList<>();
for (IndexService indexService : indicesService) {
for (IndexShard indexShard : indexService) {

View File

@ -112,7 +112,7 @@ public final class IngestActionFilter extends AbstractComponent implements Actio
logger.error("failed to execute pipeline for a bulk request", throwable);
listener.onFailure(throwable);
} else {
long ingestTookInMillis = TimeUnit.MILLISECONDS.convert(System.nanoTime() - ingestStartTimeInNanos, TimeUnit.NANOSECONDS);
long ingestTookInMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - ingestStartTimeInNanos);
BulkRequest bulkRequest = bulkRequestModifier.getBulkRequest();
ActionListener<BulkResponse> actionListener = bulkRequestModifier.wrapActionListenerIfNeeded(ingestTookInMillis, listener);
if (bulkRequest.requests().isEmpty()) {

View File

@ -136,6 +136,7 @@ public class ClusterModule extends AbstractModule {
bind(AllocationService.class).asEagerSingleton();
bind(DiscoveryNodeService.class).asEagerSingleton();
bind(ClusterService.class).to(InternalClusterService.class).asEagerSingleton();
bind(NodeConnectionsService.class).asEagerSingleton();
bind(OperationRouting.class).asEagerSingleton();
bind(MetaDataCreateIndexService.class).asEagerSingleton();
bind(MetaDataDeleteIndexService.class).asEagerSingleton();

View File

@ -26,7 +26,6 @@ import org.elasticsearch.cluster.service.PendingClusterTask;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.tasks.TaskManager;
import java.util.List;
@ -154,9 +153,4 @@ public interface ClusterService extends LifecycleComponent<ClusterService> {
* @return A zero time value if the queue is empty, otherwise the time value oldest task waiting in the queue
*/
TimeValue getMaxTaskWaitTime();
/**
* Returns task manager created in the cluster service
*/
TaskManager getTaskManager();
}

View File

@ -0,0 +1,156 @@
/*
* 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.cluster;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.common.util.concurrent.KeyedLock;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ScheduledFuture;
/**
* This component is responsible for connecting to nodes once they are added to the cluster state, and disconnect when they are
* removed. Also, it periodically checks that all connections are still open and if needed restores them.
* Note that this component is *not* responsible for removing nodes from the cluster if they disconnect / do not respond
* to pings. This is done by {@link org.elasticsearch.discovery.zen.fd.NodesFaultDetection}. Master fault detection
* is done by {@link org.elasticsearch.discovery.zen.fd.MasterFaultDetection}.
*/
public class NodeConnectionsService extends AbstractLifecycleComponent<NodeConnectionsService> {
public static final Setting<TimeValue> CLUSTER_NODE_RECONNECT_INTERVAL_SETTING =
Setting.positiveTimeSetting("cluster.nodes.reconnect_interval", TimeValue.timeValueSeconds(10), false, Setting.Scope.CLUSTER);
private final ThreadPool threadPool;
private final TransportService transportService;
// map between current node and the number of failed connection attempts. 0 means successfully connected.
// if a node doesn't appear in this list it shouldn't be monitored
private ConcurrentMap<DiscoveryNode, Integer> nodes = ConcurrentCollections.newConcurrentMap();
final private KeyedLock<DiscoveryNode> nodeLocks = new KeyedLock<>();
private final TimeValue reconnectInterval;
private volatile ScheduledFuture<?> backgroundFuture = null;
@Inject
public NodeConnectionsService(Settings settings, ThreadPool threadPool, TransportService transportService) {
super(settings);
this.threadPool = threadPool;
this.transportService = transportService;
this.reconnectInterval = NodeConnectionsService.CLUSTER_NODE_RECONNECT_INTERVAL_SETTING.get(settings);
}
public void connectToAddedNodes(ClusterChangedEvent event) {
// TODO: do this in parallel (and wait)
for (final DiscoveryNode node : event.nodesDelta().addedNodes()) {
try (Releasable ignored = nodeLocks.acquire(node)) {
Integer current = nodes.put(node, 0);
assert current == null : "node " + node + " was added in event but already in internal nodes";
validateNodeConnected(node);
}
}
}
public void disconnectFromRemovedNodes(ClusterChangedEvent event) {
for (final DiscoveryNode node : event.nodesDelta().removedNodes()) {
try (Releasable ignored = nodeLocks.acquire(node)) {
Integer current = nodes.remove(node);
assert current != null : "node " + node + " was removed in event but not in internal nodes";
try {
transportService.disconnectFromNode(node);
} catch (Throwable e) {
logger.warn("failed to disconnect to node [" + node + "]", e);
}
}
}
}
void validateNodeConnected(DiscoveryNode node) {
assert nodeLocks.isHeldByCurrentThread(node) : "validateNodeConnected must be called under lock";
if (lifecycle.stoppedOrClosed() ||
nodes.containsKey(node) == false) { // we double check existence of node since connectToNode might take time...
// nothing to do
} else {
try {
// connecting to an already connected node is a noop
transportService.connectToNode(node);
nodes.put(node, 0);
} catch (Exception e) {
Integer nodeFailureCount = nodes.get(node);
assert nodeFailureCount != null : node + " didn't have a counter in nodes map";
nodeFailureCount = nodeFailureCount + 1;
// log every 6th failure
if ((nodeFailureCount % 6) == 1) {
logger.warn("failed to connect to node {} (tried [{}] times)", e, node, nodeFailureCount);
}
nodes.put(node, nodeFailureCount);
}
}
}
class ConnectionChecker extends AbstractRunnable {
@Override
public void onFailure(Throwable t) {
logger.warn("unexpected error while checking for node reconnects", t);
}
protected void doRun() {
for (DiscoveryNode node : nodes.keySet()) {
try (Releasable ignored = nodeLocks.acquire(node)) {
validateNodeConnected(node);
}
}
}
@Override
public void onAfter() {
if (lifecycle.started()) {
backgroundFuture = threadPool.schedule(reconnectInterval, ThreadPool.Names.GENERIC, this);
}
}
}
@Override
protected void doStart() {
backgroundFuture = threadPool.schedule(reconnectInterval, ThreadPool.Names.GENERIC, new ConnectionChecker());
}
@Override
protected void doStop() {
FutureUtils.cancel(backgroundFuture);
}
@Override
protected void doClose() {
}
}

View File

@ -69,15 +69,17 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
private final State state;
private final SnapshotId snapshotId;
private final boolean includeGlobalState;
private final boolean partial;
private final ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards;
private final List<String> indices;
private final ImmutableOpenMap<String, List<ShardId>> waitingIndices;
private final long startTime;
public Entry(SnapshotId snapshotId, boolean includeGlobalState, State state, List<String> indices, long startTime, ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards) {
public Entry(SnapshotId snapshotId, boolean includeGlobalState, boolean partial, State state, List<String> indices, long startTime, ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards) {
this.state = state;
this.snapshotId = snapshotId;
this.includeGlobalState = includeGlobalState;
this.partial = partial;
this.indices = indices;
this.startTime = startTime;
if (shards == null) {
@ -90,7 +92,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
}
public Entry(Entry entry, State state, ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards) {
this(entry.snapshotId, entry.includeGlobalState, state, entry.indices, entry.startTime, shards);
this(entry.snapshotId, entry.includeGlobalState, entry.partial, state, entry.indices, entry.startTime, shards);
}
public Entry(Entry entry, ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards) {
@ -121,6 +123,10 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
return includeGlobalState;
}
public boolean partial() {
return partial;
}
public long startTime() {
return startTime;
}
@ -133,6 +139,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
Entry entry = (Entry) o;
if (includeGlobalState != entry.includeGlobalState) return false;
if (partial != entry.partial) return false;
if (startTime != entry.startTime) return false;
if (!indices.equals(entry.indices)) return false;
if (!shards.equals(entry.shards)) return false;
@ -148,6 +155,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
int result = state.hashCode();
result = 31 * result + snapshotId.hashCode();
result = 31 * result + (includeGlobalState ? 1 : 0);
result = 31 * result + (partial ? 1 : 0);
result = 31 * result + shards.hashCode();
result = 31 * result + indices.hashCode();
result = 31 * result + waitingIndices.hashCode();
@ -360,6 +368,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
for (int i = 0; i < entries.length; i++) {
SnapshotId snapshotId = SnapshotId.readSnapshotId(in);
boolean includeGlobalState = in.readBoolean();
boolean partial = in.readBoolean();
State state = State.fromValue(in.readByte());
int indices = in.readVInt();
List<String> indexBuilder = new ArrayList<>();
@ -375,7 +384,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
State shardState = State.fromValue(in.readByte());
builder.put(shardId, new ShardSnapshotStatus(nodeId, shardState));
}
entries[i] = new Entry(snapshotId, includeGlobalState, state, Collections.unmodifiableList(indexBuilder), startTime, builder.build());
entries[i] = new Entry(snapshotId, includeGlobalState, partial, state, Collections.unmodifiableList(indexBuilder), startTime, builder.build());
}
return new SnapshotsInProgress(entries);
}
@ -386,6 +395,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
for (Entry entry : entries) {
entry.snapshotId().writeTo(out);
out.writeBoolean(entry.includeGlobalState());
out.writeBoolean(entry.partial());
out.writeByte(entry.state().value());
out.writeVInt(entry.indices().size());
for (String index : entry.indices()) {
@ -406,6 +416,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
static final XContentBuilderString SNAPSHOTS = new XContentBuilderString("snapshots");
static final XContentBuilderString SNAPSHOT = new XContentBuilderString("snapshot");
static final XContentBuilderString INCLUDE_GLOBAL_STATE = new XContentBuilderString("include_global_state");
static final XContentBuilderString PARTIAL = new XContentBuilderString("partial");
static final XContentBuilderString STATE = new XContentBuilderString("state");
static final XContentBuilderString INDICES = new XContentBuilderString("indices");
static final XContentBuilderString START_TIME_MILLIS = new XContentBuilderString("start_time_millis");
@ -431,6 +442,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
builder.field(Fields.REPOSITORY, entry.snapshotId().getRepository());
builder.field(Fields.SNAPSHOT, entry.snapshotId().getSnapshot());
builder.field(Fields.INCLUDE_GLOBAL_STATE, entry.includeGlobalState());
builder.field(Fields.PARTIAL, entry.partial());
builder.field(Fields.STATE, entry.state());
builder.startArray(Fields.INDICES);
{

View File

@ -34,11 +34,12 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.snapshots.SnapshotsService;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.Arrays;
import java.util.Collection;
import java.util.Set;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@ -67,7 +68,7 @@ public class MetaDataDeleteIndexService extends AbstractComponent {
}
public void deleteIndices(final Request request, final Listener userListener) {
Collection<String> indices = Arrays.asList(request.indices);
Set<String> indices = Sets.newHashSet(request.indices);
final DeleteIndexListener listener = new DeleteIndexListener(userListener);
clusterService.submitStateUpdateTask("delete-index " + indices, new ClusterStateUpdateTask(Priority.URGENT) {
@ -84,6 +85,9 @@ public class MetaDataDeleteIndexService extends AbstractComponent {
@Override
public ClusterState execute(final ClusterState currentState) {
// Check if index deletion conflicts with any running snapshots
SnapshotsService.checkIndexDeletion(currentState, indices);
RoutingTable.Builder routingTableBuilder = RoutingTable.builder(currentState.routingTable());
MetaData.Builder metaDataBuilder = MetaData.builder(currentState.metaData());
ClusterBlocks.Builder clusterBlocksBuilder = ClusterBlocks.builder().blocks(currentState.blocks());

View File

@ -19,14 +19,12 @@
package org.elasticsearch.cluster.metadata;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.close.CloseIndexClusterStateUpdateRequest;
import org.elasticsearch.action.admin.indices.open.OpenIndexClusterStateUpdateRequest;
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.RestoreInProgress;
import org.elasticsearch.cluster.ack.ClusterStateUpdateResponse;
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
@ -39,8 +37,9 @@ import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.snapshots.RestoreService;
import org.elasticsearch.snapshots.SnapshotsService;
import java.util.ArrayList;
import java.util.Arrays;
@ -99,27 +98,10 @@ public class MetaDataIndexStateService extends AbstractComponent {
return currentState;
}
// Check if any of the indices to be closed are currently being restored from a snapshot and fail closing if such an index
// is found as closing an index that is being restored makes the index unusable (it cannot be recovered).
RestoreInProgress restore = currentState.custom(RestoreInProgress.TYPE);
if (restore != null) {
Set<String> indicesToFail = null;
for (RestoreInProgress.Entry entry : restore.entries()) {
for (ObjectObjectCursor<ShardId, RestoreInProgress.ShardRestoreStatus> shard : entry.shards()) {
if (!shard.value.state().completed()) {
if (indicesToClose.contains(shard.key.getIndexName())) {
if (indicesToFail == null) {
indicesToFail = new HashSet<>();
}
indicesToFail.add(shard.key.getIndexName());
}
}
}
}
if (indicesToFail != null) {
throw new IllegalArgumentException("Cannot close indices that are being restored: " + indicesToFail);
}
}
// Check if index closing conflicts with any running restores
RestoreService.checkIndexClosing(currentState, indicesToClose);
// Check if index closing conflicts with any running snapshots
SnapshotsService.checkIndexClosing(currentState, indicesToClose);
logger.info("closing indices [{}]", indicesAsString);

View File

@ -19,24 +19,40 @@
package org.elasticsearch.cluster.node;
import org.elasticsearch.Version;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.CopyOnWriteArrayList;
/**
*/
public class DiscoveryNodeService extends AbstractComponent {
public static final Setting<Long> NODE_ID_SEED_SETTING =
// don't use node.id.seed so it won't be seen as an attribute
Setting.longSetting("node_id.seed", 0L, Long.MIN_VALUE, false, Setting.Scope.CLUSTER);
private final List<CustomAttributesProvider> customAttributesProviders = new CopyOnWriteArrayList<>();
private final Version version;
@Inject
public DiscoveryNodeService(Settings settings) {
public DiscoveryNodeService(Settings settings, Version version) {
super(settings);
this.version = version;
}
public static String generateNodeId(Settings settings) {
Random random = Randomness.get(settings, NODE_ID_SEED_SETTING);
return Strings.randomBase64UUID(random);
}
public DiscoveryNodeService addCustomAttributeProvider(CustomAttributesProvider customAttributesProvider) {
@ -44,7 +60,7 @@ public class DiscoveryNodeService extends AbstractComponent {
return this;
}
public Map<String, String> buildAttributes() {
public DiscoveryNode buildLocalNode(TransportAddress publishAddress) {
Map<String, String> attributes = new HashMap<>(settings.getByPrefix("node.").getAsMap());
attributes.remove("name"); // name is extracted in other places
if (attributes.containsKey("client")) {
@ -76,10 +92,11 @@ public class DiscoveryNodeService extends AbstractComponent {
}
}
return attributes;
final String nodeId = generateNodeId(settings);
return new DiscoveryNode(settings.get("node.name"), nodeId, publishAddress, attributes, version);
}
public static interface CustomAttributesProvider {
public interface CustomAttributesProvider {
Map<String, String> buildAttributes();
}

View File

@ -597,6 +597,13 @@ public class RoutingNodes implements Iterable<RoutingNode> {
}
/**
* Returns the number of routing nodes
*/
public int size() {
return nodesToShards.size();
}
public static final class UnassignedShards implements Iterable<ShardRouting> {
private final RoutingNodes nodes;

View File

@ -20,7 +20,6 @@
package org.elasticsearch.cluster.routing.allocation;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.apache.lucene.util.ArrayUtil;
import org.elasticsearch.cluster.ClusterInfoService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
@ -36,13 +35,13 @@ import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocators;
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocator;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.gateway.GatewayAllocator;
import java.util.ArrayList;
import java.util.Collections;
@ -63,14 +62,17 @@ import java.util.stream.Collectors;
public class AllocationService extends AbstractComponent {
private final AllocationDeciders allocationDeciders;
private final GatewayAllocator gatewayAllocator;
private final ShardsAllocator shardsAllocator;
private final ClusterInfoService clusterInfoService;
private final ShardsAllocators shardsAllocators;
@Inject
public AllocationService(Settings settings, AllocationDeciders allocationDeciders, ShardsAllocators shardsAllocators, ClusterInfoService clusterInfoService) {
public AllocationService(Settings settings, AllocationDeciders allocationDeciders, GatewayAllocator gatewayAllocator,
ShardsAllocator shardsAllocator, ClusterInfoService clusterInfoService) {
super(settings);
this.allocationDeciders = allocationDeciders;
this.shardsAllocators = shardsAllocators;
this.gatewayAllocator = gatewayAllocator;
this.shardsAllocator = shardsAllocator;
this.clusterInfoService = clusterInfoService;
}
@ -92,7 +94,7 @@ public class AllocationService extends AbstractComponent {
if (!changed) {
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
}
shardsAllocators.applyStartedShards(allocation);
gatewayAllocator.applyStartedShards(allocation);
if (withReroute) {
reroute(allocation);
}
@ -192,7 +194,7 @@ public class AllocationService extends AbstractComponent {
if (!changed) {
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
}
shardsAllocators.applyFailedShards(allocation);
gatewayAllocator.applyFailedShards(allocation);
reroute(allocation);
final RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes);
String failedShardsAsString = firstListElementsToCommaDelimitedString(failedShards, s -> s.shard.shardId().toString());
@ -306,14 +308,10 @@ public class AllocationService extends AbstractComponent {
if (allocation.routingNodes().unassigned().size() > 0) {
updateLeftDelayOfUnassignedShards(allocation, settings);
changed |= shardsAllocators.allocateUnassigned(allocation);
changed |= gatewayAllocator.allocateUnassigned(allocation);
}
// move shards that no longer can be allocated
changed |= shardsAllocators.moveShards(allocation);
// rebalance
changed |= shardsAllocators.rebalance(allocation);
changed |= shardsAllocator.allocate(allocation);
assert RoutingNodes.assertShardStats(allocation.routingNodes());
return changed;
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.cluster.routing.allocation.allocator;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.IntroSorter;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -28,9 +27,7 @@ import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
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.AllocationDeciders;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type;
@ -42,18 +39,14 @@ import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.gateway.PriorityComparator;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Predicate;
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
@ -103,27 +96,16 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
}
@Override
public void applyStartedShards(StartedRerouteAllocation allocation) { /* ONLY FOR GATEWAYS */ }
@Override
public void applyFailedShards(FailedRerouteAllocation allocation) { /* ONLY FOR GATEWAYS */ }
@Override
public boolean allocateUnassigned(RoutingAllocation allocation) {
public boolean allocate(RoutingAllocation allocation) {
if (allocation.routingNodes().size() == 0) {
/* with no nodes this is pointless */
return false;
}
final Balancer balancer = new Balancer(logger, allocation, weightFunction, threshold);
return balancer.allocateUnassigned();
}
@Override
public boolean rebalance(RoutingAllocation allocation) {
final Balancer balancer = new Balancer(logger, allocation, weightFunction, threshold);
return balancer.balance();
}
@Override
public boolean moveShards(RoutingAllocation allocation) {
final Balancer balancer = new Balancer(logger, allocation, weightFunction, threshold);
return balancer.moveShards();
boolean changed = balancer.allocateUnassigned();
changed |= balancer.moveShards();
changed |= balancer.balance();
return changed;
}
/**
@ -203,8 +185,8 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
}
private float weight(Balancer balancer, ModelNode node, String index, int numAdditionalShards) {
final float weightShard = (node.numShards() + numAdditionalShards - balancer.avgShardsPerNode());
final float weightIndex = (node.numShards(index) + numAdditionalShards - balancer.avgShardsPerNode(index));
final float weightShard = node.numShards() + numAdditionalShards - balancer.avgShardsPerNode();
final float weightIndex = node.numShards(index) + numAdditionalShards - balancer.avgShardsPerNode(index);
return theta0 * weightShard + theta1 * weightIndex;
}
@ -216,7 +198,6 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
public static class Balancer {
private final ESLogger logger;
private final Map<String, ModelNode> nodes = new HashMap<>();
private final HashSet<String> indices = new HashSet<>();
private final RoutingAllocation allocation;
private final RoutingNodes routingNodes;
private final WeightFunction weight;
@ -225,19 +206,15 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
private final MetaData metaData;
private final float avgShardsPerNode;
private final Predicate<ShardRouting> assignedFilter = shard -> shard.assignedToNode();
public Balancer(ESLogger logger, RoutingAllocation allocation, WeightFunction weight, float threshold) {
this.logger = logger;
this.allocation = allocation;
this.weight = weight;
this.threshold = threshold;
this.routingNodes = allocation.routingNodes();
for (RoutingNode node : routingNodes) {
nodes.put(node.nodeId(), new ModelNode(node.nodeId()));
}
metaData = routingNodes.metaData();
avgShardsPerNode = ((float) metaData.totalNumberOfShards()) / nodes.size();
avgShardsPerNode = ((float) metaData.totalNumberOfShards()) / routingNodes.size();
buildModelFromAssigned();
}
/**
@ -271,17 +248,6 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
return new NodeSorter(nodesArray(), weight, this);
}
private boolean initialize(RoutingNodes routing, RoutingNodes.UnassignedShards unassigned) {
if (logger.isTraceEnabled()) {
logger.trace("Start distributing Shards");
}
for (ObjectCursor<String> index : allocation.routingTable().indicesRouting().keys()) {
indices.add(index.value);
}
buildModelFromAssigned(routing.shards(assignedFilter));
return allocateUnassigned(unassigned);
}
private static float absDelta(float lower, float higher) {
assert higher >= lower : higher + " lt " + lower +" but was expected to be gte";
return Math.abs(higher - lower);
@ -295,12 +261,36 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
}
/**
* Allocates all possible unassigned shards
* Balances the nodes on the cluster model according to the weight function.
* The actual balancing is delegated to {@link #balanceByWeights()}
*
* @return <code>true</code> if the current configuration has been
* changed, otherwise <code>false</code>
*/
final boolean allocateUnassigned() {
return balance(true);
private boolean balance() {
if (logger.isTraceEnabled()) {
logger.trace("Start balancing cluster");
}
if (allocation.hasPendingAsyncFetch()) {
/*
* see https://github.com/elastic/elasticsearch/issues/14387
* if we allow rebalance operations while we are still fetching shard store data
* we might end up with unnecessary rebalance operations which can be super confusion/frustrating
* since once the fetches come back we might just move all the shards back again.
* Therefore we only do a rebalance if we have fetched all information.
*/
logger.debug("skipping rebalance due to in-flight shard/store fetches");
return false;
}
if (allocation.deciders().canRebalance(allocation).type() != Type.YES) {
logger.trace("skipping rebalance as it is disabled");
return false;
}
if (nodes.size() < 2) { /* skip if we only have one node */
logger.trace("skipping rebalance as single node only");
return false;
}
return balanceByWeights();
}
/**
@ -317,120 +307,100 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
* @return <code>true</code> if the current configuration has been
* changed, otherwise <code>false</code>
*/
public boolean balance() {
return balance(false);
}
private boolean balanceByWeights() {
boolean changed = false;
final NodeSorter sorter = newNodeSorter();
final AllocationDeciders deciders = allocation.deciders();
final ModelNode[] modelNodes = sorter.modelNodes;
final float[] weights = sorter.weights;
for (String index : buildWeightOrderedIndices(sorter)) {
IndexMetaData indexMetaData = metaData.index(index);
private boolean balance(boolean onlyAssign) {
if (this.nodes.isEmpty()) {
/* with no nodes this is pointless */
return false;
}
if (logger.isTraceEnabled()) {
if (onlyAssign) {
logger.trace("Start balancing cluster");
} else {
logger.trace("Start assigning unassigned shards");
// find nodes that have a shard of this index or where shards of this index are allowed to stay
// move these nodes to the front of modelNodes so that we can only balance based on these nodes
int relevantNodes = 0;
for (int i = 0; i < modelNodes.length; i++) {
ModelNode modelNode = modelNodes[i];
if (modelNode.getIndex(index) != null
|| deciders.canAllocate(indexMetaData, modelNode.getRoutingNode(), allocation).type() != Type.NO) {
// swap nodes at position i and relevantNodes
modelNodes[i] = modelNodes[relevantNodes];
modelNodes[relevantNodes] = modelNode;
relevantNodes++;
}
}
}
final RoutingNodes.UnassignedShards unassigned = routingNodes.unassigned();
boolean changed = initialize(routingNodes, unassigned);
if (onlyAssign == false && changed == false && allocation.deciders().canRebalance(allocation).type() == Type.YES) {
NodeSorter sorter = newNodeSorter();
if (nodes.size() > 1) { /* skip if we only have one node */
AllocationDeciders deciders = allocation.deciders();
final ModelNode[] modelNodes = sorter.modelNodes;
final float[] weights = sorter.weights;
for (String index : buildWeightOrderedIndices(sorter)) {
IndexMetaData indexMetaData = metaData.index(index);
// find nodes that have a shard of this index or where shards of this index are allowed to stay
// move these nodes to the front of modelNodes so that we can only balance based on these nodes
int relevantNodes = 0;
for (int i = 0; i < modelNodes.length; i++) {
ModelNode modelNode = modelNodes[i];
if (modelNode.getIndex(index) != null
|| deciders.canAllocate(indexMetaData, modelNode.getRoutingNode(routingNodes), allocation).type() != Type.NO) {
// swap nodes at position i and relevantNodes
modelNodes[i] = modelNodes[relevantNodes];
modelNodes[relevantNodes] = modelNode;
relevantNodes++;
if (relevantNodes < 2) {
continue;
}
sorter.reset(index, 0, relevantNodes);
int lowIdx = 0;
int highIdx = relevantNodes - 1;
while (true) {
final ModelNode minNode = modelNodes[lowIdx];
final ModelNode maxNode = modelNodes[highIdx];
advance_range:
if (maxNode.numShards(index) > 0) {
final float delta = absDelta(weights[lowIdx], weights[highIdx]);
if (lessThan(delta, threshold)) {
if (lowIdx > 0 && highIdx-1 > 0 // is there a chance for a higher delta?
&& (absDelta(weights[0], weights[highIdx-1]) > threshold) // check if we need to break at all
) {
/* This is a special case if allocations from the "heaviest" to the "lighter" nodes is not possible
* due to some allocation decider restrictions like zone awareness. if one zone has for instance
* less nodes than another zone. so one zone is horribly overloaded from a balanced perspective but we
* can't move to the "lighter" shards since otherwise the zone would go over capacity.
*
* This break jumps straight to the condition below were we start moving from the high index towards
* the low index to shrink the window we are considering for balance from the other direction.
* (check shrinking the window from MAX to MIN)
* See #3580
*/
break advance_range;
}
if (logger.isTraceEnabled()) {
logger.trace("Stop balancing index [{}] min_node [{}] weight: [{}] max_node [{}] weight: [{}] delta: [{}]",
index, maxNode.getNodeId(), weights[highIdx], minNode.getNodeId(), weights[lowIdx], delta);
}
break;
}
if (relevantNodes < 2) {
if (logger.isTraceEnabled()) {
logger.trace("Balancing from node [{}] weight: [{}] to node [{}] weight: [{}] delta: [{}]",
maxNode.getNodeId(), weights[highIdx], minNode.getNodeId(), weights[lowIdx], delta);
}
/* pass the delta to the replication function to prevent relocations that only swap the weights of the two nodes.
* a relocation must bring us closer to the balance if we only achieve the same delta the relocation is useless */
if (tryRelocateShard(minNode, maxNode, index, delta)) {
/*
* TODO we could be a bit smarter here, we don't need to fully sort necessarily
* we could just find the place to insert linearly but the win might be minor
* compared to the added complexity
*/
weights[lowIdx] = sorter.weight(modelNodes[lowIdx]);
weights[highIdx] = sorter.weight(modelNodes[highIdx]);
sorter.sort(0, relevantNodes);
lowIdx = 0;
highIdx = relevantNodes - 1;
changed = true;
continue;
}
sorter.reset(index, 0, relevantNodes);
int lowIdx = 0;
int highIdx = relevantNodes - 1;
while (true) {
final ModelNode minNode = modelNodes[lowIdx];
final ModelNode maxNode = modelNodes[highIdx];
advance_range:
if (maxNode.numShards(index) > 0) {
final float delta = absDelta(weights[lowIdx], weights[highIdx]);
if (lessThan(delta, threshold)) {
if (lowIdx > 0 && highIdx-1 > 0 // is there a chance for a higher delta?
&& (absDelta(weights[0], weights[highIdx-1]) > threshold) // check if we need to break at all
) {
/* This is a special case if allocations from the "heaviest" to the "lighter" nodes is not possible
* due to some allocation decider restrictions like zone awareness. if one zone has for instance
* less nodes than another zone. so one zone is horribly overloaded from a balanced perspective but we
* can't move to the "lighter" shards since otherwise the zone would go over capacity.
*
* This break jumps straight to the condition below were we start moving from the high index towards
* the low index to shrink the window we are considering for balance from the other direction.
* (check shrinking the window from MAX to MIN)
* See #3580
*/
break advance_range;
}
if (logger.isTraceEnabled()) {
logger.trace("Stop balancing index [{}] min_node [{}] weight: [{}] max_node [{}] weight: [{}] delta: [{}]",
index, maxNode.getNodeId(), weights[highIdx], minNode.getNodeId(), weights[lowIdx], delta);
}
break;
}
if (logger.isTraceEnabled()) {
logger.trace("Balancing from node [{}] weight: [{}] to node [{}] weight: [{}] delta: [{}]",
maxNode.getNodeId(), weights[highIdx], minNode.getNodeId(), weights[lowIdx], delta);
}
/* pass the delta to the replication function to prevent relocations that only swap the weights of the two nodes.
* a relocation must bring us closer to the balance if we only achieve the same delta the relocation is useless */
if (tryRelocateShard(minNode, maxNode, index, delta)) {
/*
* TODO we could be a bit smarter here, we don't need to fully sort necessarily
* we could just find the place to insert linearly but the win might be minor
* compared to the added complexity
*/
weights[lowIdx] = sorter.weight(modelNodes[lowIdx]);
weights[highIdx] = sorter.weight(modelNodes[highIdx]);
sorter.sort(0, relevantNodes);
lowIdx = 0;
highIdx = relevantNodes - 1;
changed = true;
continue;
}
}
if (lowIdx < highIdx - 1) {
/* Shrinking the window from MIN to MAX
* we can't move from any shard from the min node lets move on to the next node
* and see if the threshold still holds. We either don't have any shard of this
* index on this node of allocation deciders prevent any relocation.*/
lowIdx++;
} else if (lowIdx > 0) {
/* Shrinking the window from MAX to MIN
* now we go max to min since obviously we can't move anything to the max node
* lets pick the next highest */
lowIdx = 0;
highIdx--;
} else {
/* we are done here, we either can't relocate anymore or we are balanced */
break;
}
}
}
if (lowIdx < highIdx - 1) {
/* Shrinking the window from MIN to MAX
* we can't move from any shard from the min node lets move on to the next node
* and see if the threshold still holds. We either don't have any shard of this
* index on this node of allocation deciders prevent any relocation.*/
lowIdx++;
} else if (lowIdx > 0) {
/* Shrinking the window from MAX to MIN
* now we go max to min since obviously we can't move anything to the max node
* lets pick the next highest */
lowIdx = 0;
highIdx--;
} else {
/* we are done here, we either can't relocate anymore or we are balanced */
break;
}
}
}
@ -451,7 +421,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
* to the nodes we relocated them from.
*/
private String[] buildWeightOrderedIndices(NodeSorter sorter) {
final String[] indices = this.indices.toArray(new String[this.indices.size()]);
final String[] indices = allocation.routingTable().indicesRouting().keys().toArray(String.class);
final float[] deltas = new float[indices.length];
for (int i = 0; i < deltas.length; i++) {
sorter.reset(indices[i]);
@ -503,20 +473,16 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
* @return <code>true</code> if the allocation has changed, otherwise <code>false</code>
*/
public boolean moveShards() {
if (nodes.isEmpty()) {
/* with no nodes this is pointless */
return false;
}
// Create a copy of the started shards interleaving between nodes, and check if they can remain. In the presence of throttling
// Iterate over the started shards interleaving between nodes, and check if they can remain. In the presence of throttling
// shard movements, the goal of this iteration order is to achieve a fairer movement of shards from the nodes that are
// offloading the shards.
List<ShardRouting> shards = new ArrayList<>();
boolean changed = false;
int index = 0;
boolean found = true;
final NodeSorter sorter = newNodeSorter();
while (found) {
found = false;
for (RoutingNode routingNode : routingNodes) {
for (RoutingNode routingNode : allocation.routingNodes()) {
if (index >= routingNode.size()) {
continue;
}
@ -524,64 +490,52 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
ShardRouting shardRouting = routingNode.get(index);
// we can only move started shards...
if (shardRouting.started()) {
shards.add(shardRouting);
final ModelNode sourceNode = nodes.get(shardRouting.currentNodeId());
assert sourceNode != null && sourceNode.containsShard(shardRouting);
Decision decision = allocation.deciders().canRemain(shardRouting, routingNode, allocation);
if (decision.type() == Decision.Type.NO) {
changed |= moveShard(sorter, shardRouting, sourceNode, routingNode);
}
}
}
index++;
}
if (shards.isEmpty()) {
return false;
}
final RoutingNodes.UnassignedShards unassigned = routingNodes.unassigned();
boolean changed = initialize(routingNodes, unassigned);
if (changed == false) {
final NodeSorter sorter = newNodeSorter();
final ModelNode[] modelNodes = sorter.modelNodes;
for (ShardRouting shardRouting : shards) {
final ModelNode sourceNode = nodes.get(shardRouting.currentNodeId());
assert sourceNode != null && sourceNode.containsShard(shardRouting);
final RoutingNode routingNode = sourceNode.getRoutingNode(routingNodes);
Decision decision = allocation.deciders().canRemain(shardRouting, routingNode, allocation);
if (decision.type() == Decision.Type.NO) {
logger.debug("[{}][{}] allocated on [{}], but can no longer be allocated on it, moving...", shardRouting.index(), shardRouting.id(), routingNode.node());
sorter.reset(shardRouting.getIndexName());
/*
* the sorter holds the minimum weight node first for the shards index.
* We now walk through the nodes until we find a node to allocate the shard.
* This is not guaranteed to be balanced after this operation we still try best effort to
* allocate on the minimal eligible node.
*/
boolean moved = false;
for (ModelNode currentNode : modelNodes) {
if (currentNode == sourceNode) {
continue;
}
RoutingNode target = currentNode.getRoutingNode(routingNodes);
Decision allocationDecision = allocation.deciders().canAllocate(shardRouting, target, allocation);
Decision rebalanceDecision = allocation.deciders().canRebalance(shardRouting, allocation);
if (allocationDecision.type() == Type.YES && rebalanceDecision.type() == Type.YES) { // TODO maybe we can respect throttling here too?
Decision sourceDecision = sourceNode.removeShard(shardRouting);
ShardRouting targetRelocatingShard = routingNodes.relocate(shardRouting, target.nodeId(), allocation.clusterInfo().getShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
// re-add (now relocating shard) to source node
sourceNode.addShard(shardRouting, sourceDecision);
Decision targetDecision = new Decision.Multi().add(allocationDecision).add(rebalanceDecision);
currentNode.addShard(targetRelocatingShard, targetDecision);
if (logger.isTraceEnabled()) {
logger.trace("Moved shard [{}] to node [{}]", shardRouting, routingNode.node());
}
moved = true;
changed = true;
break;
}
}
if (moved == false) {
logger.debug("[{}][{}] can't move", shardRouting.index(), shardRouting.id());
return changed;
}
/**
* Move started shard to the minimal eligible node with respect to the weight function
*
* @return <code>true</code> if the shard was moved successfully, otherwise <code>false</code>
*/
private boolean moveShard(NodeSorter sorter, ShardRouting shardRouting, ModelNode sourceNode, RoutingNode routingNode) {
logger.debug("[{}][{}] allocated on [{}], but can no longer be allocated on it, moving...", shardRouting.index(), shardRouting.id(), routingNode.node());
sorter.reset(shardRouting.getIndexName());
/*
* the sorter holds the minimum weight node first for the shards index.
* We now walk through the nodes until we find a node to allocate the shard.
* This is not guaranteed to be balanced after this operation we still try best effort to
* allocate on the minimal eligible node.
*/
for (ModelNode currentNode : sorter.modelNodes) {
if (currentNode != sourceNode) {
RoutingNode target = currentNode.getRoutingNode();
Decision allocationDecision = allocation.deciders().canAllocate(shardRouting, target, allocation);
Decision rebalanceDecision = allocation.deciders().canRebalance(shardRouting, allocation);
if (allocationDecision.type() == Type.YES && rebalanceDecision.type() == Type.YES) { // TODO maybe we can respect throttling here too?
sourceNode.removeShard(shardRouting);
ShardRouting targetRelocatingShard = routingNodes.relocate(shardRouting, target.nodeId(), allocation.clusterInfo().getShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
currentNode.addShard(targetRelocatingShard);
if (logger.isTraceEnabled()) {
logger.trace("Moved shard [{}] to node [{}]", shardRouting, routingNode.node());
}
return true;
}
}
}
return changed;
logger.debug("[{}][{}] can't move", shardRouting.index(), shardRouting.id());
return false;
}
/**
@ -593,18 +547,19 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
* on the target node which we respect during the allocation / balancing
* process. In short, this method recreates the status-quo in the cluster.
*/
private void buildModelFromAssigned(Iterable<ShardRouting> shards) {
for (ShardRouting shard : shards) {
assert shard.assignedToNode();
/* we skip relocating shards here since we expect an initializing shard with the same id coming in */
if (shard.state() == RELOCATING) {
continue;
}
ModelNode node = nodes.get(shard.currentNodeId());
assert node != null;
node.addShard(shard, Decision.single(Type.YES, "Already allocated on node", node.getNodeId()));
if (logger.isTraceEnabled()) {
logger.trace("Assigned shard [{}] to node [{}]", shard, node.getNodeId());
private void buildModelFromAssigned() {
for (RoutingNode rn : routingNodes) {
ModelNode node = new ModelNode(rn);
nodes.put(rn.nodeId(), node);
for (ShardRouting shard : rn) {
assert rn.nodeId().equals(shard.currentNodeId());
/* we skip relocating shards here since we expect an initializing shard with the same id coming in */
if (shard.state() != RELOCATING) {
node.addShard(shard);
if (logger.isTraceEnabled()) {
logger.trace("Assigned shard [{}] to node [{}]", shard, node.getNodeId());
}
}
}
}
}
@ -612,8 +567,11 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
/**
* Allocates all given shards on the minimal eligible node for the shards index
* with respect to the weight function. All given shards must be unassigned.
* @return <code>true</code> if the current configuration has been
* changed, otherwise <code>false</code>
*/
private boolean allocateUnassigned(RoutingNodes.UnassignedShards unassigned) {
private boolean allocateUnassigned() {
RoutingNodes.UnassignedShards unassigned = routingNodes.unassigned();
assert !nodes.isEmpty();
if (logger.isTraceEnabled()) {
logger.trace("Start allocating unassigned shards");
@ -657,7 +615,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
int secondaryLength = 0;
int primaryLength = primary.length;
ArrayUtil.timSort(primary, comparator);
final Set<ModelNode> throttledNodes = Collections.newSetFromMap(new IdentityHashMap<ModelNode, Boolean>());
final Set<ModelNode> throttledNodes = Collections.newSetFromMap(new IdentityHashMap<>());
do {
for (int i = 0; i < primaryLength; i++) {
ShardRouting shard = primary[i];
@ -695,7 +653,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
* don't check deciders
*/
if (currentWeight <= minWeight) {
Decision currentDecision = deciders.canAllocate(shard, node.getRoutingNode(routingNodes), allocation);
Decision currentDecision = deciders.canAllocate(shard, node.getRoutingNode(), allocation);
NOUPDATE:
if (currentDecision.type() == Type.YES || currentDecision.type() == Type.THROTTLE) {
if (currentWeight == minWeight) {
@ -736,7 +694,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
}
assert decision != null && minNode != null || decision == null && minNode == null;
if (minNode != null) {
minNode.addShard(shard, decision);
minNode.addShard(shard);
if (decision.type() == Type.YES) {
if (logger.isTraceEnabled()) {
logger.trace("Assigned shard [{}] to [{}]", shard, minNode.getNodeId());
@ -745,7 +703,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
changed = true;
continue; // don't add to ignoreUnassigned
} else {
final RoutingNode node = minNode.getRoutingNode(routingNodes);
final RoutingNode node = minNode.getRoutingNode();
if (deciders.canAllocate(node, allocation).type() != Type.YES) {
if (logger.isTraceEnabled()) {
logger.trace("Can not allocate on node [{}] remove from round decision [{}]", node, decision.type());
@ -791,10 +749,10 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
}
ShardRouting candidate = null;
final AllocationDeciders deciders = allocation.deciders();
for (ShardRouting shard : index.getAllShards()) {
for (ShardRouting shard : index) {
if (shard.started()) {
// skip initializing, unassigned and relocating shards we can't relocate them anyway
Decision allocationDecision = deciders.canAllocate(shard, minNode.getRoutingNode(routingNodes), allocation);
Decision allocationDecision = deciders.canAllocate(shard, minNode.getRoutingNode(), allocation);
Decision rebalanceDecision = deciders.canRebalance(shard, allocation);
if (((allocationDecision.type() == Type.YES) || (allocationDecision.type() == Type.THROTTLE))
&& ((rebalanceDecision.type() == Type.YES) || (rebalanceDecision.type() == Type.THROTTLE))) {
@ -815,24 +773,17 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
}
if (candidate != null) {
/* allocate on the model even if not throttled */
maxNode.removeShard(candidate);
minNode.addShard(candidate, decision);
minNode.addShard(candidate);
if (decision.type() == Type.YES) { /* only allocate on the cluster if we are not throttled */
if (logger.isTraceEnabled()) {
logger.trace("Relocate shard [{}] from node [{}] to node [{}]", candidate, maxNode.getNodeId(),
minNode.getNodeId());
}
/* now allocate on the cluster - if we are started we need to relocate the shard */
if (candidate.started()) {
routingNodes.relocate(candidate, minNode.getNodeId(), allocation.clusterInfo().getShardSize(candidate, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
} else {
routingNodes.initialize(candidate, minNode.getNodeId(), null, allocation.clusterInfo().getShardSize(candidate, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
}
/* now allocate on the cluster */
routingNodes.relocate(candidate, minNode.getNodeId(), allocation.clusterInfo().getShardSize(candidate, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
return true;
}
}
}
@ -846,14 +797,12 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
}
static class ModelNode implements Iterable<ModelIndex> {
private final String id;
private final Map<String, ModelIndex> indices = new HashMap<>();
private int numShards = 0;
// lazily calculated
private RoutingNode routingNode;
private final RoutingNode routingNode;
public ModelNode(String id) {
this.id = id;
public ModelNode(RoutingNode routingNode) {
this.routingNode = routingNode;
}
public ModelIndex getIndex(String indexId) {
@ -861,13 +810,10 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
}
public String getNodeId() {
return id;
return routingNode.nodeId();
}
public RoutingNode getRoutingNode(RoutingNodes routingNodes) {
if (routingNode == null) {
routingNode = routingNodes.node(id);
}
public RoutingNode getRoutingNode() {
return routingNode;
}
@ -888,33 +834,31 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
return -1;
}
public void addShard(ShardRouting shard, Decision decision) {
public void addShard(ShardRouting shard) {
ModelIndex index = indices.get(shard.getIndexName());
if (index == null) {
index = new ModelIndex(shard.getIndexName());
indices.put(index.getIndexId(), index);
}
index.addShard(shard, decision);
index.addShard(shard);
numShards++;
}
public Decision removeShard(ShardRouting shard) {
public void removeShard(ShardRouting shard) {
ModelIndex index = indices.get(shard.getIndexName());
Decision removed = null;
if (index != null) {
removed = index.removeShard(shard);
if (removed != null && index.numShards() == 0) {
index.removeShard(shard);
if (index.numShards() == 0) {
indices.remove(shard.getIndexName());
}
}
numShards--;
return removed;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append("Node(").append(id).append(")");
sb.append("Node(").append(routingNode.nodeId()).append(")");
return sb.toString();
}
@ -930,9 +874,9 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
}
static final class ModelIndex {
static final class ModelIndex implements Iterable<ShardRouting> {
private final String id;
private final Map<ShardRouting, Decision> shards = new HashMap<>();
private final Set<ShardRouting> shards = new HashSet<>(4); // expect few shards of same index to be allocated on same node
private int highestPrimary = -1;
public ModelIndex(String id) {
@ -942,7 +886,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
public int highestPrimary() {
if (highestPrimary == -1) {
int maxId = -1;
for (ShardRouting shard : shards.keySet()) {
for (ShardRouting shard : shards) {
if (shard.primary()) {
maxId = Math.max(maxId, shard.id());
}
@ -960,24 +904,25 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
return shards.size();
}
public Collection<ShardRouting> getAllShards() {
return shards.keySet();
@Override
public Iterator<ShardRouting> iterator() {
return shards.iterator();
}
public Decision removeShard(ShardRouting shard) {
public void removeShard(ShardRouting shard) {
highestPrimary = -1;
return shards.remove(shard);
assert shards.contains(shard) : "Shard not allocated on current node: " + shard;
shards.remove(shard);
}
public void addShard(ShardRouting shard, Decision decision) {
public void addShard(ShardRouting shard) {
highestPrimary = -1;
assert decision != null;
assert !shards.containsKey(shard) : "Shard already allocated on current node: " + shards.get(shard) + " " + shard;
shards.put(shard, decision);
assert !shards.contains(shard) : "Shard already allocated on current node: " + shard;
shards.add(shard);
}
public boolean containsShard(ShardRouting shard) {
return shards.containsKey(shard);
return shards.contains(shard);
}
}

View File

@ -19,56 +19,25 @@
package org.elasticsearch.cluster.routing.allocation.allocator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation;
/**
* <p>
* A {@link ShardsAllocator} is the main entry point for shard allocation on nodes in the cluster.
* The allocator makes basic decision where a shard instance will be allocated, if already allocated instances
* need relocate to other nodes due to node failures or due to rebalancing decisions.
* need to relocate to other nodes due to node failures or due to rebalancing decisions.
* </p>
*/
public interface ShardsAllocator {
/**
* Applies changes on started nodes based on the implemented algorithm. For example if a
* shard has changed to {@link ShardRoutingState#STARTED} from {@link ShardRoutingState#RELOCATING}
* this allocator might apply some cleanups on the node that used to hold the shard.
* @param allocation all started {@link ShardRouting shards}
*/
void applyStartedShards(StartedRerouteAllocation allocation);
/**
* Applies changes on failed nodes based on the implemented algorithm.
* @param allocation all failed {@link ShardRouting shards}
*/
void applyFailedShards(FailedRerouteAllocation allocation);
/**
* Assign all unassigned shards to nodes
* Allocates shards to nodes in the cluster. An implementation of this method should:
* - assign unassigned shards
* - relocate shards that cannot stay on a node anymore
* - relocate shards to find a good shard balance in the cluster
*
* @param allocation current node allocation
* @return <code>true</code> if the allocation has changed, otherwise <code>false</code>
*/
boolean allocateUnassigned(RoutingAllocation allocation);
/**
* Rebalancing number of shards on all nodes
*
* @param allocation current node allocation
* @return <code>true</code> if the allocation has changed, otherwise <code>false</code>
*/
boolean rebalance(RoutingAllocation allocation);
/**
* Move started shards that can not be allocated to a node anymore
*
* @param allocation current node allocation
* @return <code>true</code> if the allocation has changed, otherwise <code>false</code>
*/
boolean moveShards(RoutingAllocation allocation);
boolean allocate(RoutingAllocation allocation);
}

View File

@ -1,100 +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.cluster.routing.allocation.allocator;
import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.gateway.GatewayAllocator;
/**
* The {@link ShardsAllocator} class offers methods for allocating shard within a cluster.
* These methods include moving shards and re-balancing the cluster. It also allows management
* of shards by their state.
*/
public class ShardsAllocators extends AbstractComponent implements ShardsAllocator {
private final GatewayAllocator gatewayAllocator;
private final ShardsAllocator allocator;
public ShardsAllocators(GatewayAllocator allocator) {
this(Settings.Builder.EMPTY_SETTINGS, allocator);
}
public ShardsAllocators(Settings settings, GatewayAllocator allocator) {
this(settings, allocator, new BalancedShardsAllocator(settings));
}
@Inject
public ShardsAllocators(Settings settings, GatewayAllocator gatewayAllocator, ShardsAllocator allocator) {
super(settings);
this.gatewayAllocator = gatewayAllocator;
this.allocator = allocator;
}
@Override
public void applyStartedShards(StartedRerouteAllocation allocation) {
gatewayAllocator.applyStartedShards(allocation);
allocator.applyStartedShards(allocation);
}
@Override
public void applyFailedShards(FailedRerouteAllocation allocation) {
gatewayAllocator.applyFailedShards(allocation);
allocator.applyFailedShards(allocation);
}
@Override
public boolean allocateUnassigned(RoutingAllocation allocation) {
boolean changed = false;
changed |= gatewayAllocator.allocateUnassigned(allocation);
changed |= allocator.allocateUnassigned(allocation);
return changed;
}
protected long nanoTime() {
return System.nanoTime();
}
@Override
public boolean rebalance(RoutingAllocation allocation) {
if (allocation.hasPendingAsyncFetch() == false) {
/*
* see https://github.com/elastic/elasticsearch/issues/14387
* if we allow rebalance operations while we are still fetching shard store data
* we might end up with unnecessary rebalance operations which can be super confusion/frustrating
* since once the fetches come back we might just move all the shards back again.
* Therefore we only do a rebalance if we have fetched all information.
*/
return allocator.rebalance(allocation);
} else {
logger.debug("skipping rebalance due to in-flight shard/store fetches");
return false;
}
}
@Override
public boolean moveShards(RoutingAllocation allocation) {
return allocator.moveShards(allocation);
}
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.cluster.service;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.AckedClusterStateTaskListener;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterName;
@ -32,19 +31,18 @@ import org.elasticsearch.cluster.ClusterStateTaskExecutor;
import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.LocalNodeMasterListener;
import org.elasticsearch.cluster.NodeConnectionsService;
import org.elasticsearch.cluster.TimeoutClusterStateListener;
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodeService;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.OperationRouting;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
@ -54,7 +52,6 @@ import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.CountDown;
@ -65,9 +62,7 @@ import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor;
import org.elasticsearch.common.util.concurrent.PrioritizedRunnable;
import org.elasticsearch.common.util.iterable.Iterables;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.tasks.TaskManager;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.util.ArrayList;
import java.util.Collection;
@ -78,8 +73,6 @@ import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Queue;
import java.util.Random;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.Executor;
import java.util.concurrent.Future;
@ -97,25 +90,15 @@ import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadF
public class InternalClusterService extends AbstractLifecycleComponent<ClusterService> implements ClusterService {
public static final Setting<TimeValue> CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING = Setting.positiveTimeSetting("cluster.service.slow_task_logging_threshold", TimeValue.timeValueSeconds(30), true, Setting.Scope.CLUSTER);
public static final Setting<TimeValue> CLUSTER_SERVICE_RECONNECT_INTERVAL_SETTING = Setting.positiveTimeSetting("cluster.service.reconnect_interval", TimeValue.timeValueSeconds(10), false, Setting.Scope.CLUSTER);
public static final String UPDATE_THREAD_NAME = "clusterService#updateTask";
public static final Setting<Long> NODE_ID_SEED_SETTING =
// don't use node.id.seed so it won't be seen as an attribute
Setting.longSetting("node_id.seed", 0L, Long.MIN_VALUE, false, Setting.Scope.CLUSTER);
private final ThreadPool threadPool;
private BiConsumer<ClusterChangedEvent, Discovery.AckListener> clusterStatePublisher;
private final OperationRouting operationRouting;
private final TransportService transportService;
private final ClusterSettings clusterSettings;
private final DiscoveryNodeService discoveryNodeService;
private final Version version;
private final TimeValue reconnectInterval;
private TimeValue slowTaskLoggingThreshold;
@ -140,47 +123,49 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
private final ClusterBlocks.Builder initialBlocks;
private final TaskManager taskManager;
private volatile ScheduledFuture reconnectToNodes;
private NodeConnectionsService nodeConnectionsService;
@Inject
public InternalClusterService(Settings settings, OperationRouting operationRouting, TransportService transportService,
ClusterSettings clusterSettings, ThreadPool threadPool, ClusterName clusterName, DiscoveryNodeService discoveryNodeService, Version version) {
public InternalClusterService(Settings settings, OperationRouting operationRouting,
ClusterSettings clusterSettings, ThreadPool threadPool, ClusterName clusterName) {
super(settings);
this.operationRouting = operationRouting;
this.transportService = transportService;
this.threadPool = threadPool;
this.clusterSettings = clusterSettings;
this.discoveryNodeService = discoveryNodeService;
this.version = version;
// will be replaced on doStart.
this.clusterState = ClusterState.builder(clusterName).build();
this.clusterSettings.addSettingsUpdateConsumer(CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, this::setSlowTaskLoggingThreshold);
this.reconnectInterval = CLUSTER_SERVICE_RECONNECT_INTERVAL_SETTING.get(settings);
this.slowTaskLoggingThreshold = CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get(settings);
localNodeMasterListeners = new LocalNodeMasterListeners(threadPool);
initialBlocks = ClusterBlocks.builder();
taskManager = transportService.getTaskManager();
}
private void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) {
this.slowTaskLoggingThreshold = slowTaskLoggingThreshold;
}
public void setClusterStatePublisher(BiConsumer<ClusterChangedEvent, Discovery.AckListener> publisher) {
synchronized public void setClusterStatePublisher(BiConsumer<ClusterChangedEvent, Discovery.AckListener> publisher) {
clusterStatePublisher = publisher;
}
synchronized public void setLocalNode(DiscoveryNode localNode) {
assert clusterState.nodes().localNodeId() == null : "local node is already set";
DiscoveryNodes.Builder nodeBuilder = DiscoveryNodes.builder(clusterState.nodes()).put(localNode).localNodeId(localNode.id());
this.clusterState = ClusterState.builder(clusterState).nodes(nodeBuilder).build();
}
synchronized public void setNodeConnectionsService(NodeConnectionsService nodeConnectionsService) {
assert this.nodeConnectionsService == null : "nodeConnectionsService is already set";
this.nodeConnectionsService = nodeConnectionsService;
}
@Override
public void addInitialStateBlock(ClusterBlock block) throws IllegalStateException {
synchronized public void addInitialStateBlock(ClusterBlock block) throws IllegalStateException {
if (lifecycle.started()) {
throw new IllegalStateException("can't set initial block when started");
}
@ -188,12 +173,12 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
}
@Override
public void removeInitialStateBlock(ClusterBlock block) throws IllegalStateException {
synchronized public void removeInitialStateBlock(ClusterBlock block) throws IllegalStateException {
removeInitialStateBlock(block.id());
}
@Override
public void removeInitialStateBlock(int blockId) throws IllegalStateException {
synchronized public void removeInitialStateBlock(int blockId) throws IllegalStateException {
if (lifecycle.started()) {
throw new IllegalStateException("can't set initial block when started");
}
@ -201,26 +186,18 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
}
@Override
protected void doStart() {
synchronized protected void doStart() {
Objects.requireNonNull(clusterStatePublisher, "please set a cluster state publisher before starting");
Objects.requireNonNull(clusterState.nodes().localNode(), "please set the local node before starting");
Objects.requireNonNull(nodeConnectionsService, "please set the node connection service before starting");
add(localNodeMasterListeners);
add(taskManager);
this.clusterState = ClusterState.builder(clusterState).blocks(initialBlocks).build();
this.updateTasksExecutor = EsExecutors.newSinglePrioritizing(UPDATE_THREAD_NAME, daemonThreadFactory(settings, UPDATE_THREAD_NAME), threadPool.getThreadContext());
this.reconnectToNodes = threadPool.schedule(reconnectInterval, ThreadPool.Names.GENERIC, new ReconnectToNodes());
Map<String, String> nodeAttributes = discoveryNodeService.buildAttributes();
// note, we rely on the fact that its a new id each time we start, see FD and "kill -9" handling
final String nodeId = generateNodeId(settings);
final TransportAddress publishAddress = transportService.boundAddress().publishAddress();
DiscoveryNode localNode = new DiscoveryNode(settings.get("node.name"), nodeId, publishAddress, nodeAttributes, version);
DiscoveryNodes.Builder nodeBuilder = DiscoveryNodes.builder().put(localNode).localNodeId(localNode.id());
this.clusterState = ClusterState.builder(clusterState).nodes(nodeBuilder).blocks(initialBlocks).build();
this.transportService.setLocalNode(localNode);
this.clusterState = ClusterState.builder(clusterState).blocks(initialBlocks).build();
}
@Override
protected void doStop() {
FutureUtils.cancel(this.reconnectToNodes);
synchronized protected void doStop() {
for (NotifyTimeout onGoingTimeout : onGoingTimeouts) {
onGoingTimeout.cancel();
onGoingTimeout.listener.onClose();
@ -230,7 +207,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
}
@Override
protected void doClose() {
synchronized protected void doClose() {
}
@Override
@ -400,11 +377,6 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
return updateTasksExecutor.getMaxTaskWaitTime();
}
@Override
public TaskManager getTaskManager() {
return taskManager;
}
/** asserts that the current thread is the cluster state update thread */
public boolean assertClusterStateThread() {
assert Thread.currentThread().getName().contains(InternalClusterService.UPDATE_THREAD_NAME) : "not called from the cluster state update thread";
@ -457,14 +429,14 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
return;
}
ClusterStateTaskExecutor.BatchResult<T> batchResult;
long startTimeNS = System.nanoTime();
long startTimeNS = currentTimeInNanos();
try {
List<T> inputs = toExecute.stream().map(tUpdateTask -> tUpdateTask.task).collect(Collectors.toList());
batchResult = executor.execute(previousClusterState, inputs);
} catch (Throwable e) {
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(System.nanoTime() - startTimeNS)));
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
if (logger.isTraceEnabled()) {
StringBuilder sb = new StringBuilder("failed to execute cluster state update in ").append(executionTime).append(", state:\nversion [").append(previousClusterState.version()).append("], source [").append(source).append("]\n");
StringBuilder sb = new StringBuilder("failed to execute cluster state update in [").append(executionTime).append("], state:\nversion [").append(previousClusterState.version()).append("], source [").append(source).append("]\n");
sb.append(previousClusterState.nodes().prettyPrint());
sb.append(previousClusterState.routingTable().prettyPrint());
sb.append(previousClusterState.getRoutingNodes().prettyPrint());
@ -509,8 +481,8 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
}
task.listener.clusterStateProcessed(task.source, previousClusterState, newClusterState);
}
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(System.nanoTime() - startTimeNS)));
logger.debug("processing [{}]: took {} no change in cluster_state", source, executionTime);
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
logger.debug("processing [{}]: took [{}] no change in cluster_state", source, executionTime);
warnAboutSlowTaskIfNeeded(executionTime, source);
return;
}
@ -568,15 +540,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
}
}
// TODO, do this in parallel (and wait)
for (DiscoveryNode node : nodesDelta.addedNodes()) {
try {
transportService.connectToNode(node);
} catch (Throwable e) {
// the fault detection will detect it as failed as well
logger.warn("failed to connect to node [" + node + "]", e);
}
}
nodeConnectionsService.connectToAddedNodes(clusterChangedEvent);
// if we are the master, publish the new state to all nodes
// we publish here before we send a notification to all the listeners, since if it fails
@ -612,13 +576,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
}
}
for (DiscoveryNode node : nodesDelta.removedNodes()) {
try {
transportService.disconnectFromNode(node);
} catch (Throwable e) {
logger.warn("failed to disconnect to node [" + node + "]", e);
}
}
nodeConnectionsService.disconnectFromRemovedNodes(clusterChangedEvent);
newClusterState.status(ClusterState.ClusterStateStatus.APPLIED);
@ -649,11 +607,11 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
logger.error("exception thrown while notifying executor of new cluster state publication [{}]", e, source);
}
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(System.nanoTime() - startTimeNS)));
logger.debug("processing [{}]: took {} done applying updated cluster_state (version: {}, uuid: {})", source, executionTime, newClusterState.version(), newClusterState.stateUUID());
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
logger.debug("processing [{}]: took [{}] done applying updated cluster_state (version: {}, uuid: {})", source, executionTime, newClusterState.version(), newClusterState.stateUUID());
warnAboutSlowTaskIfNeeded(executionTime, source);
} catch (Throwable t) {
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(System.nanoTime() - startTimeNS)));
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
StringBuilder sb = new StringBuilder("failed to apply updated cluster state in ").append(executionTime).append(":\nversion [").append(newClusterState.version()).append("], uuid [").append(newClusterState.stateUUID()).append("], source [").append(source).append("]\n");
sb.append(newClusterState.nodes().prettyPrint());
sb.append(newClusterState.routingTable().prettyPrint());
@ -664,6 +622,9 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
}
// this one is overridden in tests so we can control time
protected long currentTimeInNanos() {return System.nanoTime();}
private static SafeClusterStateTaskListener safe(ClusterStateTaskListener listener, ESLogger logger) {
if (listener instanceof AckedClusterStateTaskListener) {
return new SafeAckedClusterStateTaskListener((AckedClusterStateTaskListener) listener, logger);
@ -777,7 +738,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
private void warnAboutSlowTaskIfNeeded(TimeValue executionTime, String source) {
if (executionTime.getMillis() > slowTaskLoggingThreshold.getMillis()) {
logger.warn("cluster state update task [{}] took {} above the warn threshold of {}", source, executionTime, slowTaskLoggingThreshold);
logger.warn("cluster state update task [{}] took [{}] above the warn threshold of {}", source, executionTime, slowTaskLoggingThreshold);
}
}
@ -809,64 +770,6 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
}
}
private class ReconnectToNodes implements Runnable {
private ConcurrentMap<DiscoveryNode, Integer> failureCount = ConcurrentCollections.newConcurrentMap();
@Override
public void run() {
// master node will check against all nodes if its alive with certain discoveries implementations,
// but we can't rely on that, so we check on it as well
for (DiscoveryNode node : clusterState.nodes()) {
if (lifecycle.stoppedOrClosed()) {
return;
}
if (clusterState.nodes().nodeExists(node.id())) { // we double check existence of node since connectToNode might take time...
if (!transportService.nodeConnected(node)) {
try {
transportService.connectToNode(node);
} catch (Exception e) {
if (lifecycle.stoppedOrClosed()) {
return;
}
if (clusterState.nodes().nodeExists(node.id())) { // double check here as well, maybe its gone?
Integer nodeFailureCount = failureCount.get(node);
if (nodeFailureCount == null) {
nodeFailureCount = 1;
} else {
nodeFailureCount = nodeFailureCount + 1;
}
// log every 6th failure
if ((nodeFailureCount % 6) == 0) {
// reset the failure count...
nodeFailureCount = 0;
logger.warn("failed to reconnect to node {}", e, node);
}
failureCount.put(node, nodeFailureCount);
}
}
}
}
}
// go over and remove failed nodes that have been removed
DiscoveryNodes nodes = clusterState.nodes();
for (Iterator<DiscoveryNode> failedNodesIt = failureCount.keySet().iterator(); failedNodesIt.hasNext(); ) {
DiscoveryNode failedNode = failedNodesIt.next();
if (!nodes.nodeExists(failedNode.id())) {
failedNodesIt.remove();
}
}
if (lifecycle.started()) {
reconnectToNodes = threadPool.schedule(reconnectInterval, ThreadPool.Names.GENERIC, this);
}
}
}
public static String generateNodeId(Settings settings) {
Random random = Randomness.get(settings, NODE_ID_SEED_SETTING);
return Strings.randomBase64UUID(random);
}
private static class LocalNodeMasterListeners implements ClusterStateListener {
private final List<LocalNodeMasterListener> listeners = new CopyOnWriteArrayList<>();

View File

@ -37,6 +37,7 @@ import org.elasticsearch.common.geo.builders.ShapeBuilder;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
import org.elasticsearch.ingest.IngestStats;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.search.suggest.SuggestionBuilder;
import org.elasticsearch.search.suggest.completion.context.QueryContext;
@ -563,6 +564,14 @@ public abstract class StreamInput extends InputStream {
}
}
public <T extends Writeable> T readOptionalWritable(T prototype) throws IOException {
if (readBoolean()) {
return (T) prototype.readFrom(this);
} else {
return null;
}
}
public <T extends Throwable> T readThrowable() throws IOException {
if (readBoolean()) {
int key = readVInt();

View File

@ -532,6 +532,15 @@ public abstract class StreamOutput extends OutputStream {
}
}
public void writeOptionalWriteable(@Nullable Writeable writeable) throws IOException {
if (writeable != null) {
writeBoolean(true);
writeable.writeTo(this);
} else {
writeBoolean(false);
}
}
public void writeThrowable(Throwable throwable) throws IOException {
if (throwable == null) {
writeBoolean(false);

View File

@ -29,8 +29,10 @@ import org.elasticsearch.client.transport.TransportClientNodesService;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.InternalClusterInfoService;
import org.elasticsearch.cluster.NodeConnectionsService;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNodeService;
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
import org.elasticsearch.cluster.routing.allocation.decider.AwarenessAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
@ -259,7 +261,7 @@ public final class ClusterSettings extends AbstractScopedSettings {
TransportService.TRACE_LOG_INCLUDE_SETTING,
TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING,
ShardsLimitAllocationDecider.CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING,
InternalClusterService.CLUSTER_SERVICE_RECONNECT_INTERVAL_SETTING,
NodeConnectionsService.CLUSTER_NODE_RECONNECT_INTERVAL_SETTING,
HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING,
HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_TYPE_SETTING,
Transport.TRANSPORT_TCP_COMPRESS,
@ -326,7 +328,7 @@ public final class ClusterSettings extends AbstractScopedSettings {
Environment.PATH_SCRIPTS_SETTING,
Environment.PATH_SHARED_DATA_SETTING,
Environment.PIDFILE_SETTING,
InternalClusterService.NODE_ID_SEED_SETTING,
DiscoveryNodeService.NODE_ID_SEED_SETTING,
DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING,
DiscoveryModule.DISCOVERY_TYPE_SETTING,
DiscoveryModule.ZEN_MASTER_SERVICE_TYPE_SETTING,

View File

@ -20,7 +20,10 @@
package org.elasticsearch.common.util.concurrent;
import org.elasticsearch.common.lease.Releasable;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantLock;
@ -29,9 +32,8 @@ import java.util.concurrent.locks.ReentrantLock;
* created the first time they are acquired and removed if no thread hold the
* lock. The latter is important to assure that the list of locks does not grow
* infinitely.
*
* A Thread can acquire a lock only once.
*
*
*
* */
public class KeyedLock<T> {
@ -50,48 +52,38 @@ public class KeyedLock<T> {
private final ConcurrentMap<T, KeyLock> map = ConcurrentCollections.newConcurrentMap();
protected final ThreadLocal<KeyLock> threadLocal = new ThreadLocal<>();
public void acquire(T key) {
public Releasable acquire(T key) {
assert isHeldByCurrentThread(key) == false : "lock for " + key + " is already heald by this thread";
while (true) {
if (threadLocal.get() != null) {
// if we are here, the thread already has the lock
throw new IllegalStateException("Lock already acquired in Thread" + Thread.currentThread().getId()
+ " for key " + key);
}
KeyLock perNodeLock = map.get(key);
if (perNodeLock == null) {
KeyLock newLock = new KeyLock(fair);
perNodeLock = map.putIfAbsent(key, newLock);
if (perNodeLock == null) {
newLock.lock();
threadLocal.set(newLock);
return;
return new ReleasableLock(key, newLock);
}
}
assert perNodeLock != null;
int i = perNodeLock.count.get();
if (i > 0 && perNodeLock.count.compareAndSet(i, i + 1)) {
perNodeLock.lock();
threadLocal.set(perNodeLock);
return;
return new ReleasableLock(key, perNodeLock);
}
}
}
public void release(T key) {
KeyLock lock = threadLocal.get();
public boolean isHeldByCurrentThread(T key) {
KeyLock lock = map.get(key);
if (lock == null) {
throw new IllegalStateException("Lock not acquired");
return false;
}
release(key, lock);
return lock.isHeldByCurrentThread();
}
void release(T key, KeyLock lock) {
assert lock.isHeldByCurrentThread();
assert lock == map.get(key);
lock.unlock();
threadLocal.set(null);
int decrementAndGet = lock.count.decrementAndGet();
if (decrementAndGet == 0) {
map.remove(key, lock);
@ -99,6 +91,24 @@ public class KeyedLock<T> {
}
private final class ReleasableLock implements Releasable {
final T key;
final KeyLock lock;
final AtomicBoolean closed = new AtomicBoolean();
private ReleasableLock(T key, KeyLock lock) {
this.key = key;
this.lock = lock;
}
@Override
public void close() {
if (closed.compareAndSet(false, true)) {
release(key, lock);
}
}
}
@SuppressWarnings("serial")
private final static class KeyLock extends ReentrantLock {
KeyLock(boolean fair) {

View File

@ -19,8 +19,14 @@
package org.elasticsearch.index.mapper;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.util.CloseableThreadLocal;
import org.elasticsearch.common.Strings;
@ -48,15 +54,8 @@ import org.elasticsearch.index.mapper.object.ArrayValueMapperParser;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.mapper.object.RootObjectMapper;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
/** A parser for documents, given mappings from a DocumentMapper */
class DocumentParser implements Closeable {
final class DocumentParser implements Closeable {
private CloseableThreadLocal<ParseContext.InternalParseContext> cache = new CloseableThreadLocal<ParseContext.InternalParseContext>() {
@Override
@ -99,7 +98,7 @@ class DocumentParser implements Closeable {
reverseOrder(context);
ParsedDocument doc = parsedDocument(source, context, update(context, mapping));
ParsedDocument doc = parsedDocument(source, context, createDynamicUpdate(mapping, docMapper, context.getDynamicMappers()));
// reset the context to free up memory
context.reset(null, null, null);
return doc;
@ -116,10 +115,7 @@ class DocumentParser implements Closeable {
// entire type is disabled
parser.skipChildren();
} else if (emptyDoc == false) {
Mapper update = parseObject(context, mapping.root, true);
if (update != null) {
context.addDynamicMappingsUpdate(update);
}
parseObjectOrNested(context, mapping.root, true);
}
for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) {
@ -201,11 +197,6 @@ class DocumentParser implements Closeable {
}
private static Mapping update(ParseContext.InternalParseContext context, Mapping mapping) {
Mapper rootDynamicUpdate = context.dynamicMappingsUpdate();
return rootDynamicUpdate != null ? mapping.mappingUpdate(rootDynamicUpdate) : null;
}
private static MapperParsingException wrapInMapperParsingException(SourceToParse source, Throwable e) {
// if its already a mapper parsing exception, no need to wrap it...
if (e instanceof MapperParsingException) {
@ -220,10 +211,156 @@ class DocumentParser implements Closeable {
return new MapperParsingException("failed to parse", e);
}
static ObjectMapper parseObject(ParseContext context, ObjectMapper mapper, boolean atRoot) throws IOException {
/** Creates a Mapping containing any dynamically added fields, or returns null if there were no dynamic mappings. */
static Mapping createDynamicUpdate(Mapping mapping, DocumentMapper docMapper, List<Mapper> dynamicMappers) {
if (dynamicMappers.isEmpty()) {
return null;
}
// We build a mapping by first sorting the mappers, so that all mappers containing a common prefix
// will be processed in a contiguous block. When the prefix is no longer seen, we pop the extra elements
// off the stack, merging them upwards into the existing mappers.
Collections.sort(dynamicMappers, (Mapper o1, Mapper o2) -> o1.name().compareTo(o2.name()));
Iterator<Mapper> dynamicMapperItr = dynamicMappers.iterator();
List<ObjectMapper> parentMappers = new ArrayList<>();
Mapper firstUpdate = dynamicMapperItr.next();
parentMappers.add(createUpdate(mapping.root(), firstUpdate.name().split("\\."), 0, firstUpdate));
Mapper previousMapper = null;
while (dynamicMapperItr.hasNext()) {
Mapper newMapper = dynamicMapperItr.next();
if (previousMapper != null && newMapper.name().equals(previousMapper.name())) {
// We can see the same mapper more than once, for example, if we had foo.bar and foo.baz, where
// foo did not yet exist. This will create 2 copies in dynamic mappings, which should be identical.
// Here we just skip over the duplicates, but we merge them to ensure there are no conflicts.
newMapper.merge(previousMapper, false);
continue;
}
previousMapper = newMapper;
String[] nameParts = newMapper.name().split("\\.");
// We first need the stack to only contain mappers in common with the previously processed mapper
// For example, if the first mapper processed was a.b.c, and we now have a.d, the stack will contain
// a.b, and we want to merge b back into the stack so it just contains a
int i = removeUncommonMappers(parentMappers, nameParts);
// Then we need to add back mappers that may already exist within the stack, but are not on it.
// For example, if we processed a.b, followed by an object mapper a.c.d, and now are adding a.c.d.e
// then the stack will only have a on it because we will have already merged a.c.d into the stack.
// So we need to pull a.c, followed by a.c.d, onto the stack so e can be added to the end.
i = expandCommonMappers(parentMappers, nameParts, i);
// If there are still parents of the new mapper which are not on the stack, we need to pull them
// from the existing mappings. In order to maintain the invariant that the stack only contains
// fields which are updated, we cannot simply add the existing mappers to the stack, since they
// may have other subfields which will not be updated. Instead, we pull the mapper from the existing
// mappings, and build an update with only the new mapper and its parents. This then becomes our
// "new mapper", and can be added to the stack.
if (i < nameParts.length - 1) {
newMapper = createExistingMapperUpdate(parentMappers, nameParts, i, docMapper, newMapper);
}
if (newMapper instanceof ObjectMapper) {
parentMappers.add((ObjectMapper)newMapper);
} else {
addToLastMapper(parentMappers, newMapper, true);
}
}
popMappers(parentMappers, 1, true);
assert parentMappers.size() == 1;
return mapping.mappingUpdate(parentMappers.get(0));
}
private static void popMappers(List<ObjectMapper> parentMappers, int keepBefore, boolean merge) {
assert keepBefore >= 1; // never remove the root mapper
// pop off parent mappers not needed by the current mapper,
// merging them backwards since they are immutable
for (int i = parentMappers.size() - 1; i >= keepBefore; --i) {
addToLastMapper(parentMappers, parentMappers.remove(i), merge);
}
}
/**
* Adds a mapper as an update into the last mapper. If merge is true, the new mapper
* will be merged in with other child mappers of the last parent, otherwise it will be a new update.
*/
private static void addToLastMapper(List<ObjectMapper> parentMappers, Mapper mapper, boolean merge) {
assert parentMappers.size() >= 1;
int lastIndex = parentMappers.size() - 1;
ObjectMapper withNewMapper = parentMappers.get(lastIndex).mappingUpdate(mapper);
if (merge) {
withNewMapper = parentMappers.get(lastIndex).merge(withNewMapper, false);
}
parentMappers.set(lastIndex, withNewMapper);
}
/**
* Removes mappers that exist on the stack, but are not part of the path of the current nameParts,
* Returns the next unprocessed index from nameParts.
*/
private static int removeUncommonMappers(List<ObjectMapper> parentMappers, String[] nameParts) {
int keepBefore = 1;
while (keepBefore < parentMappers.size() &&
parentMappers.get(keepBefore).simpleName().equals(nameParts[keepBefore - 1])) {
++keepBefore;
}
popMappers(parentMappers, keepBefore, true);
return keepBefore - 1;
}
/**
* Adds mappers from the end of the stack that exist as updates within those mappers.
* Returns the next unprocessed index from nameParts.
*/
private static int expandCommonMappers(List<ObjectMapper> parentMappers, String[] nameParts, int i) {
ObjectMapper last = parentMappers.get(parentMappers.size() - 1);
while (i < nameParts.length - 1 && last.getMapper(nameParts[i]) != null) {
Mapper newLast = last.getMapper(nameParts[i]);
assert newLast instanceof ObjectMapper;
last = (ObjectMapper) newLast;
parentMappers.add(last);
++i;
}
return i;
}
/** Creates an update for intermediate object mappers that are not on the stack, but parents of newMapper. */
private static ObjectMapper createExistingMapperUpdate(List<ObjectMapper> parentMappers, String[] nameParts, int i,
DocumentMapper docMapper, Mapper newMapper) {
String updateParentName = nameParts[i];
final ObjectMapper lastParent = parentMappers.get(parentMappers.size() - 1);
if (parentMappers.size() > 1) {
// only prefix with parent mapper if the parent mapper isn't the root (which has a fake name)
updateParentName = lastParent.name() + '.' + nameParts[i];
}
ObjectMapper updateParent = docMapper.objectMappers().get(updateParentName);
assert updateParent != null : updateParentName + " doesn't exist";
return createUpdate(updateParent, nameParts, i + 1, newMapper);
}
/** Build an update for the parent which will contain the given mapper and any intermediate fields. */
private static ObjectMapper createUpdate(ObjectMapper parent, String[] nameParts, int i, Mapper mapper) {
List<ObjectMapper> parentMappers = new ArrayList<>();
ObjectMapper previousIntermediate = parent;
for (; i < nameParts.length - 1; ++i) {
Mapper intermediate = previousIntermediate.getMapper(nameParts[i]);
assert intermediate != null : "Field " + previousIntermediate.name() + " does not have a subfield " + nameParts[i];
assert intermediate instanceof ObjectMapper;
parentMappers.add((ObjectMapper)intermediate);
previousIntermediate = (ObjectMapper)intermediate;
}
if (parentMappers.isEmpty() == false) {
// add the new mapper to the stack, and pop down to the original parent level
addToLastMapper(parentMappers, mapper, false);
popMappers(parentMappers, 1, false);
mapper = parentMappers.get(0);
}
return parent.mappingUpdate(mapper);
}
static void parseObjectOrNested(ParseContext context, ObjectMapper mapper, boolean atRoot) throws IOException {
if (mapper.isEnabled() == false) {
context.parser().skipChildren();
return null;
return;
}
XContentParser parser = context.parser();
@ -234,7 +371,7 @@ class DocumentParser implements Closeable {
XContentParser.Token token = parser.currentToken();
if (token == XContentParser.Token.VALUE_NULL) {
// the object is null ("obj1" : null), simply bail
return null;
return;
}
if (token.isValue()) {
@ -256,21 +393,19 @@ class DocumentParser implements Closeable {
}
ObjectMapper update = null;
update = innerParseObject(context, mapper, parser, currentFieldName, token, update);
innerParseObject(context, mapper, parser, currentFieldName, token);
// restore the enable path flag
if (nested.isNested()) {
nested(context, nested);
}
return update;
}
private static ObjectMapper innerParseObject(ParseContext context, ObjectMapper mapper, XContentParser parser, String currentFieldName, XContentParser.Token token, ObjectMapper update) throws IOException {
private static void innerParseObject(ParseContext context, ObjectMapper mapper, XContentParser parser, String currentFieldName, XContentParser.Token token) throws IOException {
while (token != XContentParser.Token.END_OBJECT) {
ObjectMapper newUpdate = null;
if (token == XContentParser.Token.START_OBJECT) {
newUpdate = parseObject(context, mapper, currentFieldName);
parseObject(context, mapper, currentFieldName);
} else if (token == XContentParser.Token.START_ARRAY) {
newUpdate = parseArray(context, mapper, currentFieldName);
parseArray(context, mapper, currentFieldName);
} else if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NULL) {
@ -278,18 +413,10 @@ class DocumentParser implements Closeable {
} else if (token == null) {
throw new MapperParsingException("object mapping for [" + mapper.name() + "] tried to parse field [" + currentFieldName + "] as object, but got EOF, has a concrete value been provided to it?");
} else if (token.isValue()) {
newUpdate = parseValue(context, mapper, currentFieldName, token);
parseValue(context, mapper, currentFieldName, token);
}
token = parser.nextToken();
if (newUpdate != null) {
if (update == null) {
update = newUpdate;
} else {
update = update.merge(newUpdate, false);
}
}
}
return update;
}
private static void nested(ParseContext context, ObjectMapper.Nested nested) {
@ -335,33 +462,29 @@ class DocumentParser implements Closeable {
return context;
}
private static Mapper parseObjectOrField(ParseContext context, Mapper mapper) throws IOException {
private static void parseObjectOrField(ParseContext context, Mapper mapper) throws IOException {
if (mapper instanceof ObjectMapper) {
return parseObject(context, (ObjectMapper) mapper, false);
parseObjectOrNested(context, (ObjectMapper) mapper, false);
} else {
FieldMapper fieldMapper = (FieldMapper)mapper;
Mapper update = fieldMapper.parse(context);
if (update != null) {
context.addDynamicMapper(update);
}
if (fieldMapper.copyTo() != null) {
parseCopyFields(context, fieldMapper, fieldMapper.copyTo().copyToFields());
}
return update;
}
}
private static ObjectMapper parseObject(final ParseContext context, ObjectMapper mapper, String currentFieldName) throws IOException {
if (currentFieldName == null) {
throw new MapperParsingException("object mapping [" + mapper.name() + "] trying to serialize an object with no field associated with it, current value [" + context.parser().textOrNull() + "]");
}
assert currentFieldName != null;
context.path().add(currentFieldName);
ObjectMapper update = null;
Mapper objectMapper = mapper.getMapper(currentFieldName);
if (objectMapper != null) {
final Mapper subUpdate = parseObjectOrField(context, objectMapper);
if (subUpdate != null) {
// propagate mapping update
update = mapper.mappingUpdate(subUpdate);
}
parseObjectOrField(context, objectMapper);
} else {
ObjectMapper.Dynamic dynamic = mapper.dynamic();
if (dynamic == null) {
@ -382,8 +505,9 @@ class DocumentParser implements Closeable {
}
Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
objectMapper = builder.build(builderContext);
context.addDynamicMapper(objectMapper);
context.path().add(currentFieldName);
update = mapper.mappingUpdate(parseAndMergeUpdate(objectMapper, context));
parseObjectOrField(context, objectMapper);
} else {
// not dynamic, read everything up to end object
context.parser().skipChildren();
@ -394,7 +518,7 @@ class DocumentParser implements Closeable {
return update;
}
private static ObjectMapper parseArray(ParseContext context, ObjectMapper parentMapper, String lastFieldName) throws IOException {
private static void parseArray(ParseContext context, ObjectMapper parentMapper, String lastFieldName) throws IOException {
String arrayFieldName = lastFieldName;
Mapper mapper = parentMapper.getMapper(lastFieldName);
if (mapper != null) {
@ -402,15 +526,9 @@ class DocumentParser implements Closeable {
// expects an array, if so we pass the context straight to the mapper and if not
// we serialize the array components
if (mapper instanceof ArrayValueMapperParser) {
final Mapper subUpdate = parseObjectOrField(context, mapper);
if (subUpdate != null) {
// propagate the mapping update
return parentMapper.mappingUpdate(subUpdate);
} else {
return null;
}
parseObjectOrField(context, mapper);
} else {
return parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
}
} else {
@ -423,31 +541,34 @@ class DocumentParser implements Closeable {
} else if (dynamic == ObjectMapper.Dynamic.TRUE) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, arrayFieldName, "object");
if (builder == null) {
return parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
return;
}
Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
mapper = builder.build(builderContext);
if (mapper != null && mapper instanceof ArrayValueMapperParser) {
assert mapper != null;
if (mapper instanceof ArrayValueMapperParser) {
context.addDynamicMapper(mapper);
context.path().add(arrayFieldName);
mapper = parseAndMergeUpdate(mapper, context);
return parentMapper.mappingUpdate(mapper);
parseObjectOrField(context, mapper);
} else {
return parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
}
} else {
return parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
// TODO: shouldn't this skip, not parse?
parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
}
}
}
private static ObjectMapper parseNonDynamicArray(ParseContext context, ObjectMapper mapper, String lastFieldName, String arrayFieldName) throws IOException {
private static void parseNonDynamicArray(ParseContext context, ObjectMapper mapper, String lastFieldName, String arrayFieldName) throws IOException {
XContentParser parser = context.parser();
XContentParser.Token token;
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (token == XContentParser.Token.START_OBJECT) {
return parseObject(context, mapper, lastFieldName);
parseObject(context, mapper, lastFieldName);
} else if (token == XContentParser.Token.START_ARRAY) {
return parseArray(context, mapper, lastFieldName);
parseArray(context, mapper, lastFieldName);
} else if (token == XContentParser.Token.FIELD_NAME) {
lastFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NULL) {
@ -455,25 +576,20 @@ class DocumentParser implements Closeable {
} else if (token == null) {
throw new MapperParsingException("object mapping for [" + mapper.name() + "] with array for [" + arrayFieldName + "] tried to parse as array, but got EOF, is there a mismatch in types for the same field?");
} else {
return parseValue(context, mapper, lastFieldName, token);
parseValue(context, mapper, lastFieldName, token);
}
}
return null;
}
private static ObjectMapper parseValue(final ParseContext context, ObjectMapper parentMapper, String currentFieldName, XContentParser.Token token) throws IOException {
private static void parseValue(final ParseContext context, ObjectMapper parentMapper, String currentFieldName, XContentParser.Token token) throws IOException {
if (currentFieldName == null) {
throw new MapperParsingException("object mapping [" + parentMapper.name() + "] trying to serialize a value with no field associated with it, current value [" + context.parser().textOrNull() + "]");
}
Mapper mapper = parentMapper.getMapper(currentFieldName);
if (mapper != null) {
Mapper subUpdate = parseObjectOrField(context, mapper);
if (subUpdate == null) {
return null;
}
return parentMapper.mappingUpdate(subUpdate);
parseObjectOrField(context, mapper);
} else {
return parseDynamicValue(context, parentMapper, currentFieldName, token);
parseDynamicValue(context, parentMapper, currentFieldName, token);
}
}
@ -641,7 +757,7 @@ class DocumentParser implements Closeable {
throw new IllegalStateException("Can't handle serializing a dynamic type with content token [" + token + "] and field name [" + currentFieldName + "]");
}
private static ObjectMapper parseDynamicValue(final ParseContext context, ObjectMapper parentMapper, String currentFieldName, XContentParser.Token token) throws IOException {
private static void parseDynamicValue(final ParseContext context, ObjectMapper parentMapper, String currentFieldName, XContentParser.Token token) throws IOException {
ObjectMapper.Dynamic dynamic = parentMapper.dynamic();
if (dynamic == null) {
dynamic = dynamicOrDefault(context.root().dynamic());
@ -650,7 +766,7 @@ class DocumentParser implements Closeable {
throw new StrictDynamicMappingException(parentMapper.fullPath(), currentFieldName);
}
if (dynamic == ObjectMapper.Dynamic.FALSE) {
return null;
return;
}
final String path = context.path().pathAsText(currentFieldName);
final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
@ -668,14 +784,9 @@ class DocumentParser implements Closeable {
// try to not introduce a conflict
mapper = mapper.updateFieldType(Collections.singletonMap(path, existingFieldType));
}
context.addDynamicMapper(mapper);
mapper = parseAndMergeUpdate(mapper, context);
ObjectMapper update = null;
if (mapper != null) {
update = parentMapper.mappingUpdate(mapper);
}
return update;
parseObjectOrField(context, mapper);
}
/** Creates instances of the fields that the current field should be copied to */
@ -713,8 +824,9 @@ class DocumentParser implements Closeable {
// The path of the dest field might be completely different from the current one so we need to reset it
context = context.overridePath(new ContentPath(0));
String[] paths = Strings.splitStringToArray(field, '.');
String fieldName = paths[paths.length-1];
// TODO: why Strings.splitStringToArray instead of String.split?
final String[] paths = Strings.splitStringToArray(field, '.');
final String fieldName = paths[paths.length-1];
ObjectMapper mapper = context.root();
ObjectMapper[] mappers = new ObjectMapper[paths.length-1];
if (paths.length > 1) {
@ -745,6 +857,7 @@ class DocumentParser implements Closeable {
if (mapper.nested() != ObjectMapper.Nested.NO) {
throw new MapperParsingException("It is forbidden to create dynamic nested objects ([" + context.path().pathAsText(paths[i]) + "]) through `copy_to`");
}
context.addDynamicMapper(mapper);
break;
case FALSE:
// Maybe we should log something to tell the user that the copy_to is ignored in this case.
@ -759,36 +872,10 @@ class DocumentParser implements Closeable {
parent = mapper;
}
}
ObjectMapper update = parseDynamicValue(context, mapper, fieldName, context.parser().currentToken());
assert update != null; // we are parsing a dynamic value so we necessarily created a new mapping
if (paths.length > 1) {
for (int i = paths.length - 2; i >= 0; i--) {
ObjectMapper parent = context.root();
if (i > 0) {
parent = mappers[i-1];
}
assert parent != null;
update = parent.mappingUpdate(update);
}
}
context.addDynamicMappingsUpdate(update);
parseDynamicValue(context, mapper, fieldName, context.parser().currentToken());
}
}
/**
* Parse the given {@code context} with the given {@code mapper} and apply
* the potential mapping update in-place. This method is useful when
* composing mapping updates.
*/
private static <M extends Mapper> M parseAndMergeUpdate(M mapper, ParseContext context) throws IOException {
final Mapper update = parseObjectOrField(context, mapper);
if (update != null) {
mapper = (M) mapper.merge(update, false);
}
return mapper;
}
private static ObjectMapper.Dynamic dynamicOrDefault(ObjectMapper.Dynamic dynamic) {
return dynamic == null ? ObjectMapper.Dynamic.TRUE : dynamic;
}

View File

@ -374,7 +374,8 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
// this can happen if this mapper represents a mapping update
return this;
} else if (fieldType.getClass() != newFieldType.getClass()) {
throw new IllegalStateException("Mixing up field types: " + fieldType.getClass() + " != " + newFieldType.getClass());
throw new IllegalStateException("Mixing up field types: " +
fieldType.getClass() + " != " + newFieldType.getClass() + " on field " + fieldType.name());
}
MultiFields updatedMultiFields = multiFields.updateFieldType(fullNameToFieldType);
if (fieldType == newFieldType && multiFields == updatedMultiFields) {

View File

@ -76,6 +76,7 @@ public abstract class Mapper implements ToXContent, Iterable<Mapper> {
return this.name;
}
/** Returns a newly built mapper. */
public abstract Y build(BuilderContext context);
}

View File

@ -331,13 +331,13 @@ public abstract class ParseContext {
}
@Override
public void addDynamicMappingsUpdate(Mapper update) {
in.addDynamicMappingsUpdate(update);
public void addDynamicMapper(Mapper update) {
in.addDynamicMapper(update);
}
@Override
public Mapper dynamicMappingsUpdate() {
return in.dynamicMappingsUpdate();
public List<Mapper> getDynamicMappers() {
return in.getDynamicMappers();
}
}
@ -369,7 +369,7 @@ public abstract class ParseContext {
private AllEntries allEntries = new AllEntries();
private Mapper dynamicMappingsUpdate = null;
private List<Mapper> dynamicMappers = new ArrayList<>();
public InternalParseContext(@Nullable Settings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, ContentPath path) {
this.indexSettings = indexSettings;
@ -394,7 +394,7 @@ public abstract class ParseContext {
this.source = source == null ? null : sourceToParse.source();
this.path.reset();
this.allEntries = new AllEntries();
this.dynamicMappingsUpdate = null;
this.dynamicMappers = new ArrayList<>();
}
@Override
@ -536,18 +536,13 @@ public abstract class ParseContext {
}
@Override
public void addDynamicMappingsUpdate(Mapper mapper) {
assert mapper instanceof RootObjectMapper : mapper;
if (dynamicMappingsUpdate == null) {
dynamicMappingsUpdate = mapper;
} else {
dynamicMappingsUpdate = dynamicMappingsUpdate.merge(mapper, false);
}
public void addDynamicMapper(Mapper mapper) {
dynamicMappers.add(mapper);
}
@Override
public Mapper dynamicMappingsUpdate() {
return dynamicMappingsUpdate;
public List<Mapper> getDynamicMappers() {
return dynamicMappers;
}
}
@ -747,12 +742,12 @@ public abstract class ParseContext {
public abstract StringBuilder stringBuilder();
/**
* Add a dynamic update to the root object mapper.
* Add a new mapper dynamically created while parsing.
*/
public abstract void addDynamicMappingsUpdate(Mapper update);
public abstract void addDynamicMapper(Mapper update);
/**
* Get dynamic updates to the root object mapper.
* Get dynamic mappers created while parsing.
*/
public abstract Mapper dynamicMappingsUpdate();
public abstract List<Mapper> getDynamicMappers();
}

View File

@ -26,6 +26,9 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.Version;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
@ -39,9 +42,12 @@ import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.internal.AllFieldMapper;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static org.apache.lucene.index.IndexOptions.NONE;
import static org.elasticsearch.index.mapper.core.TypeParsers.parseMultiField;
@ -52,6 +58,11 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
public static final String CONTENT_TYPE = "string";
private static final int POSITION_INCREMENT_GAP_USE_ANALYZER = -1;
private static final Set<String> SUPPORTED_PARAMETERS_FOR_AUTO_UPGRADE = new HashSet<>(Arrays.asList(
"type",
// most common parameters, for which the upgrade is straightforward
"index", "store", "doc_values", "omit_norms", "norms", "fields", "copy_to"));
public static class Defaults {
public static final MappedFieldType FIELD_TYPE = new StringFieldType();
@ -130,13 +141,33 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
}
public static class TypeParser implements Mapper.TypeParser {
private final DeprecationLogger deprecationLogger;
public TypeParser() {
ESLogger logger = Loggers.getLogger(getClass());
this.deprecationLogger = new DeprecationLogger(logger);
}
@Override
public Mapper.Builder parse(String fieldName, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
// TODO: temporarily disabled to give Kibana time to upgrade to text/keyword mappings
/*if (parserContext.indexVersionCreated().onOrAfter(Version.V_5_0_0)) {
if (parserContext.indexVersionCreated().onOrAfter(Version.V_5_0_0)) {
// Automatically upgrade simple mappings for ease of upgrade, otherwise fail
if (SUPPORTED_PARAMETERS_FOR_AUTO_UPGRADE.containsAll(node.keySet())) {
deprecationLogger.deprecated("The [string] field is deprecated, please use [text] or [keyword] instead on [{}]",
fieldName);
final Object index = node.remove("index");
final boolean keyword = index != null && "analyzed".equals(index) == false;
// upgrade the index setting
node.put("index", "no".equals(index) == false);
if (keyword) {
return new KeywordFieldMapper.TypeParser().parse(fieldName, node, parserContext);
} else {
return new TextFieldMapper.TypeParser().parse(fieldName, node, parserContext);
}
}
throw new IllegalArgumentException("The [string] type is removed in 5.0. You should now use either a [text] "
+ "or [keyword] field instead for field [" + fieldName + "]");
}*/
}
StringFieldMapper.Builder builder = new StringFieldMapper.Builder(fieldName);
// hack for the fact that string can't just accept true/false for
// the index property and still accepts no/not_analyzed/analyzed
@ -241,11 +272,10 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
int positionIncrementGap, int ignoreAbove,
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo);
// TODO: temporarily disabled to give Kibana time to upgrade to text/keyword mappings
/*if (Version.indexCreated(indexSettings).onOrAfter(Version.V_5_0_0)) {
if (Version.indexCreated(indexSettings).onOrAfter(Version.V_5_0_0)) {
throw new IllegalArgumentException("The [string] type is removed in 5.0. You should now use either a [text] "
+ "or [keyword] field instead for field [" + fieldType.name() + "]");
}*/
}
if (fieldType.tokenized() && fieldType.indexOptions() != NONE && fieldType().hasDocValues()) {
throw new MapperParsingException("Field [" + fieldType.name() + "] cannot be analyzed and have doc values");
}

View File

@ -25,7 +25,9 @@ import org.elasticsearch.Version;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
import org.elasticsearch.common.joda.Joda;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.loader.SettingsLoader;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
@ -39,11 +41,14 @@ import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.similarity.SimilarityProvider;
import org.elasticsearch.index.similarity.SimilarityService;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.isArray;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.lenientNodeBooleanValue;
@ -63,10 +68,18 @@ public class TypeParsers {
public static final String INDEX_OPTIONS_POSITIONS = "positions";
public static final String INDEX_OPTIONS_OFFSETS = "offsets";
private static boolean nodeBooleanValue(Object node, Mapper.TypeParser.ParserContext parserContext) {
if (parserContext.indexVersionCreated().onOrAfter(Version.V_5_0_0)) {
private static final DeprecationLogger DEPRECATION_LOGGER = new DeprecationLogger(Loggers.getLogger(TypeParsers.class));
private static final Set<String> BOOLEAN_STRINGS = new HashSet<>(Arrays.asList("true", "false"));
private static boolean nodeBooleanValue(String name, Object node, Mapper.TypeParser.ParserContext parserContext) {
// Hook onto ParseFieldMatcher so that parsing becomes strict when setting index.query.parse.strict
if (parserContext.parseFieldMatcher().isStrict()) {
return XContentMapValues.nodeBooleanValue(node);
} else {
// TODO: remove this leniency in 6.0
if (BOOLEAN_STRINGS.contains(node.toString()) == false) {
DEPRECATION_LOGGER.deprecated("Expected a boolean for property [{}] but got [{}]", name, node);
}
return XContentMapValues.lenientNodeBooleanValue(node);
}
}
@ -81,13 +94,13 @@ public class TypeParsers {
builder.precisionStep(nodeIntegerValue(propNode));
iterator.remove();
} else if (propName.equals("ignore_malformed")) {
builder.ignoreMalformed(nodeBooleanValue(propNode, parserContext));
builder.ignoreMalformed(nodeBooleanValue("ignore_malformed", propNode, parserContext));
iterator.remove();
} else if (propName.equals("coerce")) {
builder.coerce(nodeBooleanValue(propNode, parserContext));
builder.coerce(nodeBooleanValue("coerce", propNode, parserContext));
iterator.remove();
} else if (propName.equals("omit_norms")) {
builder.omitNorms(nodeBooleanValue(propNode, parserContext));
builder.omitNorms(nodeBooleanValue("omit_norms", propNode, parserContext));
iterator.remove();
} else if (propName.equals("similarity")) {
SimilarityProvider similarityProvider = resolveSimilarity(parserContext, name, propNode.toString());
@ -112,16 +125,16 @@ public class TypeParsers {
parseTermVector(name, propNode.toString(), builder);
iterator.remove();
} else if (propName.equals("store_term_vectors")) {
builder.storeTermVectors(nodeBooleanValue(propNode, parserContext));
builder.storeTermVectors(nodeBooleanValue("store_term_vectors", propNode, parserContext));
iterator.remove();
} else if (propName.equals("store_term_vector_offsets")) {
builder.storeTermVectorOffsets(nodeBooleanValue(propNode, parserContext));
builder.storeTermVectorOffsets(nodeBooleanValue("store_term_vector_offsets", propNode, parserContext));
iterator.remove();
} else if (propName.equals("store_term_vector_positions")) {
builder.storeTermVectorPositions(nodeBooleanValue(propNode, parserContext));
builder.storeTermVectorPositions(nodeBooleanValue("store_term_vector_positions", propNode, parserContext));
iterator.remove();
} else if (propName.equals("store_term_vector_payloads")) {
builder.storeTermVectorPayloads(nodeBooleanValue(propNode, parserContext));
builder.storeTermVectorPayloads(nodeBooleanValue("store_term_vector_payloads", propNode, parserContext));
iterator.remove();
} else if (propName.equals("analyzer")) {
NamedAnalyzer analyzer = parserContext.analysisService().analyzer(propNode.toString());
@ -199,13 +212,13 @@ public class TypeParsers {
builder.index(parseIndex(name, propNode.toString(), parserContext));
iterator.remove();
} else if (propName.equals(DOC_VALUES)) {
builder.docValues(nodeBooleanValue(propNode, parserContext));
builder.docValues(nodeBooleanValue(DOC_VALUES, propNode, parserContext));
iterator.remove();
} else if (propName.equals("boost")) {
builder.boost(nodeFloatValue(propNode));
iterator.remove();
} else if (propName.equals("omit_norms")) {
builder.omitNorms(nodeBooleanValue(propNode, parserContext));
builder.omitNorms(nodeBooleanValue("omit_norms", propNode, parserContext));
iterator.remove();
} else if (propName.equals("norms")) {
final Map<String, Object> properties = nodeMapValue(propNode, "norms");
@ -227,7 +240,7 @@ public class TypeParsers {
builder.indexOptions(nodeIndexOptionValue(propNode));
iterator.remove();
} else if (propName.equals("include_in_all")) {
builder.includeInAll(nodeBooleanValue(propNode, parserContext));
builder.includeInAll(nodeBooleanValue("include_in_all", propNode, parserContext));
iterator.remove();
} else if (propName.equals("similarity")) {
SimilarityProvider similarityProvider = resolveSimilarity(parserContext, name, propNode.toString());
@ -353,35 +366,32 @@ public class TypeParsers {
}
public static boolean parseIndex(String fieldName, String index, Mapper.TypeParser.ParserContext parserContext) throws MapperParsingException {
if (parserContext.indexVersionCreated().onOrAfter(Version.V_5_0_0)) {
switch (index) {
case "true":
return true;
case "false":
return false;
default:
switch (index) {
case "true":
return true;
case "false":
return false;
case "not_analyzed":
case "analyzed":
case "no":
if (parserContext.parseFieldMatcher().isStrict() == false) {
DEPRECATION_LOGGER.deprecated("Expected a boolean for property [index] but got [{}]", index);
return "no".equals(index) == false;
} else {
throw new IllegalArgumentException("Can't parse [index] value [" + index + "] for field [" + fieldName + "], expected [true] or [false]");
}
} else {
final String normalizedIndex = Strings.toUnderscoreCase(index);
switch (normalizedIndex) {
case "true":
case "not_analyzed":
case "analyzed":
return true;
case "false":
case "no":
return false;
default:
throw new IllegalArgumentException("Can't parse [index] value [" + index + "] for field [" + fieldName + "], expected [true], [false], [no], [not_analyzed] or [analyzed]");
}
default:
throw new IllegalArgumentException("Can't parse [index] value [" + index + "] for field [" + fieldName + "], expected [true] or [false]");
}
}
public static boolean parseStore(String fieldName, String store, Mapper.TypeParser.ParserContext parserContext) throws MapperParsingException {
if (parserContext.indexVersionCreated().onOrAfter(Version.V_5_0_0)) {
if (parserContext.parseFieldMatcher().isStrict()) {
return XContentMapValues.nodeBooleanValue(store);
} else {
if (BOOLEAN_STRINGS.contains(store) == false) {
DEPRECATION_LOGGER.deprecated("Expected a boolean for property [store] but got [{}]", store);
}
if ("no".equals(store)) {
return false;
} else if ("yes".equals(store)) {

View File

@ -0,0 +1,171 @@
/*
* 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.ingest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
public class IngestStats implements Writeable<IngestStats>, ToXContent {
public final static IngestStats PROTO = new IngestStats(null, null);
private final Stats totalStats;
private final Map<String, Stats> statsPerPipeline;
public IngestStats(Stats totalStats, Map<String, Stats> statsPerPipeline) {
this.totalStats = totalStats;
this.statsPerPipeline = statsPerPipeline;
}
/**
* @return The accumulated stats for all pipelines
*/
public Stats getTotalStats() {
return totalStats;
}
/**
* @return The stats on a per pipeline basis
*/
public Map<String, Stats> getStatsPerPipeline() {
return statsPerPipeline;
}
@Override
public IngestStats readFrom(StreamInput in) throws IOException {
Stats totalStats = Stats.PROTO.readFrom(in);
totalStats.readFrom(in);
int size = in.readVInt();
Map<String, Stats> statsPerPipeline = new HashMap<>(size);
for (int i = 0; i < size; i++) {
Stats stats = Stats.PROTO.readFrom(in);
statsPerPipeline.put(in.readString(), stats);
stats.readFrom(in);
}
return new IngestStats(totalStats, statsPerPipeline);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
totalStats.writeTo(out);
out.writeVLong(statsPerPipeline.size());
for (Map.Entry<String, Stats> entry : statsPerPipeline.entrySet()) {
out.writeString(entry.getKey());
entry.getValue().writeTo(out);
}
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject("ingest");
builder.startObject("total");
totalStats.toXContent(builder, params);
builder.endObject();
builder.startObject("pipelines");
for (Map.Entry<String, Stats> entry : statsPerPipeline.entrySet()) {
builder.startObject(entry.getKey());
entry.getValue().toXContent(builder, params);
builder.endObject();
}
builder.endObject();
builder.endObject();
return builder;
}
public static class Stats implements Writeable<Stats>, ToXContent {
private final static Stats PROTO = new Stats(0, 0, 0, 0);
private final long ingestCount;
private final long ingestTimeInMillis;
private final long ingestCurrent;
private final long ingestFailedCount;
public Stats(long ingestCount, long ingestTimeInMillis, long ingestCurrent, long ingestFailedCount) {
this.ingestCount = ingestCount;
this.ingestTimeInMillis = ingestTimeInMillis;
this.ingestCurrent = ingestCurrent;
this.ingestFailedCount = ingestFailedCount;
}
/**
* @return The total number of executed ingest preprocessing operations.
*/
public long getIngestCount() {
return ingestCount;
}
/**
*
* @return The total time spent of ingest preprocessing in millis.
*/
public long getIngestTimeInMillis() {
return ingestTimeInMillis;
}
/**
* @return The total number of ingest preprocessing operations currently executing.
*/
public long getIngestCurrent() {
return ingestCurrent;
}
/**
* @return The total number of ingest preprocessing operations that have failed.
*/
public long getIngestFailedCount() {
return ingestFailedCount;
}
@Override
public Stats readFrom(StreamInput in) throws IOException {
long ingestCount = in.readVLong();
long ingestTimeInMillis = in.readVLong();
long ingestCurrent = in.readVLong();
long ingestFailedCount = in.readVLong();
return new Stats(ingestCount, ingestTimeInMillis, ingestCurrent, ingestFailedCount);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVLong(ingestCount);
out.writeVLong(ingestTimeInMillis);
out.writeVLong(ingestCurrent);
out.writeVLong(ingestFailedCount);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.field("count", ingestCount);
builder.timeValueField("time_in_millis", "time", ingestTimeInMillis, TimeUnit.MILLISECONDS);
builder.field("current", ingestCurrent);
builder.field("failed", ingestFailedCount);
return builder;
}
}
}

View File

@ -19,23 +19,36 @@
package org.elasticsearch.ingest;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.metrics.MeanMetric;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.ingest.core.IngestDocument;
import org.elasticsearch.ingest.core.Pipeline;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
public class PipelineExecutionService {
public class PipelineExecutionService implements ClusterStateListener {
private final PipelineStore store;
private final ThreadPool threadPool;
private final StatsHolder totalStats = new StatsHolder();
private volatile Map<String, StatsHolder> statsHolderPerPipeline = Collections.emptyMap();
public PipelineExecutionService(PipelineStore store, ThreadPool threadPool) {
this.store = store;
this.threadPool = threadPool;
@ -89,29 +102,85 @@ public class PipelineExecutionService {
});
}
private void innerExecute(IndexRequest indexRequest, Pipeline pipeline) throws Exception {
String index = indexRequest.index();
String type = indexRequest.type();
String id = indexRequest.id();
String routing = indexRequest.routing();
String parent = indexRequest.parent();
String timestamp = indexRequest.timestamp();
String ttl = indexRequest.ttl() == null ? null : indexRequest.ttl().toString();
Map<String, Object> sourceAsMap = indexRequest.sourceAsMap();
IngestDocument ingestDocument = new IngestDocument(index, type, id, routing, parent, timestamp, ttl, sourceAsMap);
pipeline.execute(ingestDocument);
public IngestStats stats() {
Map<String, StatsHolder> statsHolderPerPipeline = this.statsHolderPerPipeline;
Map<IngestDocument.MetaData, String> metadataMap = ingestDocument.extractMetadata();
//it's fine to set all metadata fields all the time, as ingest document holds their starting values
//before ingestion, which might also get modified during ingestion.
indexRequest.index(metadataMap.get(IngestDocument.MetaData.INDEX));
indexRequest.type(metadataMap.get(IngestDocument.MetaData.TYPE));
indexRequest.id(metadataMap.get(IngestDocument.MetaData.ID));
indexRequest.routing(metadataMap.get(IngestDocument.MetaData.ROUTING));
indexRequest.parent(metadataMap.get(IngestDocument.MetaData.PARENT));
indexRequest.timestamp(metadataMap.get(IngestDocument.MetaData.TIMESTAMP));
indexRequest.ttl(metadataMap.get(IngestDocument.MetaData.TTL));
indexRequest.source(ingestDocument.getSourceAndMetadata());
Map<String, IngestStats.Stats> statsPerPipeline = new HashMap<>(statsHolderPerPipeline.size());
for (Map.Entry<String, StatsHolder> entry : statsHolderPerPipeline.entrySet()) {
statsPerPipeline.put(entry.getKey(), entry.getValue().createStats());
}
return new IngestStats(totalStats.createStats(), statsPerPipeline);
}
@Override
public void clusterChanged(ClusterChangedEvent event) {
IngestMetadata ingestMetadata = event.state().getMetaData().custom(IngestMetadata.TYPE);
if (ingestMetadata != null) {
updatePipelineStats(ingestMetadata);
}
}
void updatePipelineStats(IngestMetadata ingestMetadata) {
boolean changed = false;
Map<String, StatsHolder> newStatsPerPipeline = new HashMap<>(statsHolderPerPipeline);
for (String pipeline : newStatsPerPipeline.keySet()) {
if (ingestMetadata.getPipelines().containsKey(pipeline) == false) {
newStatsPerPipeline.remove(pipeline);
changed = true;
}
}
for (String pipeline : ingestMetadata.getPipelines().keySet()) {
if (newStatsPerPipeline.containsKey(pipeline) == false) {
newStatsPerPipeline.put(pipeline, new StatsHolder());
changed = true;
}
}
if (changed) {
statsHolderPerPipeline = Collections.unmodifiableMap(newStatsPerPipeline);
}
}
private void innerExecute(IndexRequest indexRequest, Pipeline pipeline) throws Exception {
long startTimeInNanos = System.nanoTime();
// the pipeline specific stat holder may not exist and that is fine:
// (e.g. the pipeline may have been removed while we're ingesting a document
Optional<StatsHolder> pipelineStats = Optional.ofNullable(statsHolderPerPipeline.get(pipeline.getId()));
try {
totalStats.preIngest();
pipelineStats.ifPresent(StatsHolder::preIngest);
String index = indexRequest.index();
String type = indexRequest.type();
String id = indexRequest.id();
String routing = indexRequest.routing();
String parent = indexRequest.parent();
String timestamp = indexRequest.timestamp();
String ttl = indexRequest.ttl() == null ? null : indexRequest.ttl().toString();
Map<String, Object> sourceAsMap = indexRequest.sourceAsMap();
IngestDocument ingestDocument = new IngestDocument(index, type, id, routing, parent, timestamp, ttl, sourceAsMap);
pipeline.execute(ingestDocument);
Map<IngestDocument.MetaData, String> metadataMap = ingestDocument.extractMetadata();
//it's fine to set all metadata fields all the time, as ingest document holds their starting values
//before ingestion, which might also get modified during ingestion.
indexRequest.index(metadataMap.get(IngestDocument.MetaData.INDEX));
indexRequest.type(metadataMap.get(IngestDocument.MetaData.TYPE));
indexRequest.id(metadataMap.get(IngestDocument.MetaData.ID));
indexRequest.routing(metadataMap.get(IngestDocument.MetaData.ROUTING));
indexRequest.parent(metadataMap.get(IngestDocument.MetaData.PARENT));
indexRequest.timestamp(metadataMap.get(IngestDocument.MetaData.TIMESTAMP));
indexRequest.ttl(metadataMap.get(IngestDocument.MetaData.TTL));
indexRequest.source(ingestDocument.getSourceAndMetadata());
} catch (Exception e) {
totalStats.ingestFailed();
pipelineStats.ifPresent(StatsHolder::ingestFailed);
throw e;
} finally {
long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeInNanos);
totalStats.postIngest(ingestTimeInMillis);
pipelineStats.ifPresent(statsHolder -> statsHolder.postIngest(ingestTimeInMillis));
}
}
private Pipeline getPipeline(String pipelineId) {
@ -121,4 +190,30 @@ public class PipelineExecutionService {
}
return pipeline;
}
static class StatsHolder {
private final MeanMetric ingestMetric = new MeanMetric();
private final CounterMetric ingestCurrent = new CounterMetric();
private final CounterMetric ingestFailed = new CounterMetric();
void preIngest() {
ingestCurrent.inc();
}
void postIngest(long ingestTimeInMillis) {
ingestCurrent.dec();
ingestMetric.inc(ingestTimeInMillis);
}
void ingestFailed() {
ingestFailed.inc();
}
IngestStats.Stats createStats() {
return new IngestStats.Stats(ingestMetric.count(), ingestMetric.sum(), ingestCurrent.count(), ingestFailed.count());
}
}
}

View File

@ -34,8 +34,10 @@ import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.MasterNodeChangePredicate;
import org.elasticsearch.cluster.NodeConnectionsService;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodeService;
import org.elasticsearch.cluster.routing.RoutingService;
import org.elasticsearch.cluster.service.InternalClusterService;
import org.elasticsearch.common.StopWatch;
@ -294,6 +296,10 @@ public class Node implements Closeable {
"node cluster service implementation must inherit from InternalClusterService";
final InternalClusterService clusterService = (InternalClusterService) injector.getInstance(ClusterService.class);
final NodeConnectionsService nodeConnectionsService = injector.getInstance(NodeConnectionsService.class);
nodeConnectionsService.start();
clusterService.setNodeConnectionsService(nodeConnectionsService);
// TODO hack around circular dependencies problems
injector.getInstance(GatewayAllocator.class).setReallocation(clusterService, injector.getInstance(RoutingService.class));
@ -311,6 +317,15 @@ public class Node implements Closeable {
// Start the transport service now so the publish address will be added to the local disco node in ClusterService
TransportService transportService = injector.getInstance(TransportService.class);
transportService.start();
DiscoveryNode localNode = injector.getInstance(DiscoveryNodeService.class)
.buildLocalNode(transportService.boundAddress().publishAddress());
// TODO: need to find a cleaner way to start/construct a service with some initial parameters,
// playing nice with the life cycle interfaces
clusterService.setLocalNode(localNode);
transportService.setLocalNode(localNode);
clusterService.add(transportService.getTaskManager());
clusterService.start();
// start after cluster service so the local disco is known
@ -392,6 +407,7 @@ public class Node implements Closeable {
injector.getInstance(RoutingService.class).stop();
injector.getInstance(ClusterService.class).stop();
injector.getInstance(Discovery.class).stop();
injector.getInstance(NodeConnectionsService.class).stop();
injector.getInstance(MonitorService.class).stop();
injector.getInstance(GatewayService.class).stop();
injector.getInstance(SearchService.class).stop();
@ -449,6 +465,8 @@ public class Node implements Closeable {
toClose.add(injector.getInstance(RoutingService.class));
toClose.add(() -> stopWatch.stop().start("cluster"));
toClose.add(injector.getInstance(ClusterService.class));
toClose.add(() -> stopWatch.stop().start("node_connections_service"));
toClose.add(injector.getInstance(NodeConnectionsService.class));
toClose.add(() -> stopWatch.stop().start("discovery"));
toClose.add(injector.getInstance(Discovery.class));
toClose.add(() -> stopWatch.stop().start("monitor"));

View File

@ -90,6 +90,7 @@ public class NodeService extends AbstractComponent implements Closeable {
this.ingestService = new IngestService(settings, threadPool, processorsRegistryBuilder);
this.settingsFilter = settingsFilter;
clusterService.add(ingestService.getPipelineStore());
clusterService.add(ingestService.getPipelineExecutionService());
}
// can not use constructor injection or there will be a circular dependency
@ -165,13 +166,14 @@ public class NodeService extends AbstractComponent implements Closeable {
httpServer == null ? null : httpServer.stats(),
circuitBreakerService.stats(),
scriptService.stats(),
discovery.stats()
discovery.stats(),
ingestService.getPipelineExecutionService().stats()
);
}
public NodeStats stats(CommonStatsFlags indices, boolean os, boolean process, boolean jvm, boolean threadPool,
boolean fs, boolean transport, boolean http, boolean circuitBreaker,
boolean script, boolean discoveryStats) {
boolean script, boolean discoveryStats, boolean ingest) {
// for indices stats we want to include previous allocated shards stats as well (it will
// only be applied to the sensible ones to use, like refresh/merge/flush/indexing stats)
return new NodeStats(discovery.localNode(), System.currentTimeMillis(),
@ -185,7 +187,8 @@ public class NodeService extends AbstractComponent implements Closeable {
http ? (httpServer == null ? null : httpServer.stats()) : null,
circuitBreaker ? circuitBreakerService.stats() : null,
script ? scriptService.stats() : null,
discoveryStats ? discovery.stats() : null
discoveryStats ? discovery.stats() : null,
ingest ? ingestService.getPipelineExecutionService().stats() : null
);
}

View File

@ -81,6 +81,7 @@ public class RestNodesStatsAction extends BaseRestHandler {
nodesStatsRequest.breaker(metrics.contains("breaker"));
nodesStatsRequest.script(metrics.contains("script"));
nodesStatsRequest.discovery(metrics.contains("discovery"));
nodesStatsRequest.ingest(metrics.contains("ingest"));
// check for index specific metrics
if (metrics.contains("indices")) {
@ -113,6 +114,6 @@ public class RestNodesStatsAction extends BaseRestHandler {
nodesStatsRequest.indices().includeSegmentFileSizes(true);
}
client.admin().cluster().nodesStats(nodesStatsRequest, new RestToXContentListener<NodesStatsResponse>(channel));
client.admin().cluster().nodesStats(nodesStatsRequest, new RestToXContentListener<>(channel));
}
}

View File

@ -92,7 +92,7 @@ public class DerivativePipelineAggregator extends PipelineAggregator {
for (InternalHistogram.Bucket bucket : buckets) {
Long thisBucketKey = resolveBucketKeyAsLong(bucket);
Double thisBucketValue = resolveBucketValue(histo, bucket, bucketsPaths()[0], gapPolicy);
if (lastBucketValue != null) {
if (lastBucketValue != null && thisBucketValue != null) {
double gradient = thisBucketValue - lastBucketValue;
double xDiff = -1;
if (xAxisUnits != null) {

View File

@ -774,6 +774,32 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
return false;
}
/**
* Check if any of the indices to be closed are currently being restored from a snapshot and fail closing if such an index
* is found as closing an index that is being restored makes the index unusable (it cannot be recovered).
*/
public static void checkIndexClosing(ClusterState currentState, Set<String> indices) {
RestoreInProgress restore = currentState.custom(RestoreInProgress.TYPE);
if (restore != null) {
Set<String> indicesToFail = null;
for (RestoreInProgress.Entry entry : restore.entries()) {
for (ObjectObjectCursor<ShardId, RestoreInProgress.ShardRestoreStatus> shard : entry.shards()) {
if (!shard.value.state().completed()) {
if (indices.contains(shard.key.getIndexName())) {
if (indicesToFail == null) {
indicesToFail = new HashSet<>();
}
indicesToFail.add(shard.key.getIndexName());
}
}
}
}
if (indicesToFail != null) {
throw new IllegalArgumentException("Cannot close indices that are being restored: " + indicesToFail);
}
}
}
/**
* Adds restore completion listener
* <p>

View File

@ -206,7 +206,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
// Store newSnapshot here to be processed in clusterStateProcessed
List<String> indices = Arrays.asList(indexNameExpressionResolver.concreteIndices(currentState, request.indicesOptions(), request.indices()));
logger.trace("[{}][{}] creating snapshot for indices [{}]", request.repository(), request.name(), indices);
newSnapshot = new SnapshotsInProgress.Entry(snapshotId, request.includeGlobalState(), State.INIT, indices, System.currentTimeMillis(), null);
newSnapshot = new SnapshotsInProgress.Entry(snapshotId, request.includeGlobalState(), request.partial(), State.INIT, indices, System.currentTimeMillis(), null);
snapshots = new SnapshotsInProgress(newSnapshot);
} else {
// TODO: What should we do if a snapshot is already running?
@ -228,7 +228,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new Runnable() {
@Override
public void run() {
beginSnapshot(newState, newSnapshot, request.partial, listener);
beginSnapshot(newState, newSnapshot, request.partial(), listener);
}
});
}
@ -1061,6 +1061,63 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
return builder.build();
}
/**
* Check if any of the indices to be deleted are currently being snapshotted. Fail as deleting an index that is being
* snapshotted (with partial == false) makes the snapshot fail.
*/
public static void checkIndexDeletion(ClusterState currentState, Set<String> indices) {
Set<String> indicesToFail = indicesToFailForCloseOrDeletion(currentState, indices);
if (indicesToFail != null) {
throw new IllegalArgumentException("Cannot delete indices that are being snapshotted: " + indicesToFail +
". Try again after snapshot finishes or cancel the currently running snapshot.");
}
}
/**
* Check if any of the indices to be closed are currently being snapshotted. Fail as closing an index that is being
* snapshotted (with partial == false) makes the snapshot fail.
*/
public static void checkIndexClosing(ClusterState currentState, Set<String> indices) {
Set<String> indicesToFail = indicesToFailForCloseOrDeletion(currentState, indices);
if (indicesToFail != null) {
throw new IllegalArgumentException("Cannot close indices that are being snapshotted: " + indicesToFail +
". Try again after snapshot finishes or cancel the currently running snapshot.");
}
}
private static Set<String> indicesToFailForCloseOrDeletion(ClusterState currentState, Set<String> indices) {
SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE);
Set<String> indicesToFail = null;
if (snapshots != null) {
for (final SnapshotsInProgress.Entry entry : snapshots.entries()) {
if (entry.partial() == false) {
if (entry.state() == State.INIT) {
for (String index : entry.indices()) {
if (indices.contains(index)) {
if (indicesToFail == null) {
indicesToFail = new HashSet<>();
}
indicesToFail.add(index);
}
}
} else {
for (ObjectObjectCursor<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shard : entry.shards()) {
if (!shard.value.state().completed()) {
if (indices.contains(shard.key.getIndexName())) {
if (indicesToFail == null) {
indicesToFail = new HashSet<>();
}
indicesToFail.add(shard.key.getIndexName());
}
}
}
}
}
}
}
return indicesToFail;
}
/**
* Adds snapshot completion listener
*
@ -1302,6 +1359,15 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
return includeGlobalState;
}
/**
* Returns true if partial snapshot should be allowed
*
* @return true if partial snapshot should be allowed
*/
public boolean partial() {
return partial;
}
/**
* Returns master node timeout
*

View File

@ -33,6 +33,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.math.MathUtils;
import org.elasticsearch.common.metrics.CounterMetric;
@ -943,8 +944,8 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
}
globalLock.readLock().lock();
try {
connectionLock.acquire(node.id());
try {
try (Releasable ignored = connectionLock.acquire(node.id())) {
if (!lifecycle.started()) {
throw new IllegalStateException("can't add nodes to a stopped transport");
}
@ -979,8 +980,6 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
} catch (Exception e) {
throw new ConnectTransportException(node, "general node connection failure", e);
}
} finally {
connectionLock.release(node.id());
}
} finally {
globalLock.readLock().unlock();
@ -1103,8 +1102,8 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
@Override
public void disconnectFromNode(DiscoveryNode node) {
connectionLock.acquire(node.id());
try {
try (Releasable ignored = connectionLock.acquire(node.id())) {
NodeChannels nodeChannels = connectedNodes.remove(node);
if (nodeChannels != null) {
try {
@ -1115,8 +1114,6 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
transportServiceAdapter.raiseNodeDisconnected(node);
}
}
} finally {
connectionLock.release(node.id());
}
}
@ -1128,8 +1125,7 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
// check outside of the lock
NodeChannels nodeChannels = connectedNodes.get(node);
if (nodeChannels != null && nodeChannels.hasChannel(channel)) {
connectionLock.acquire(node.id());
try {
try (Releasable ignored = connectionLock.acquire(node.id())) {
nodeChannels = connectedNodes.get(node);
// check again within the connection lock, if its still applicable to remove it
if (nodeChannels != null && nodeChannels.hasChannel(channel)) {
@ -1143,8 +1139,6 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
}
return true;
}
} finally {
connectionLock.release(node.id());
}
}
return false;

View File

@ -194,7 +194,7 @@ public abstract class TaskManagerTestCase extends ESTestCase {
}
};
transportService.start();
clusterService = new TestClusterService(threadPool, transportService);
clusterService = new TestClusterService(threadPool);
clusterService.add(transportService.getTaskManager());
discoveryNode = new DiscoveryNode(name, transportService.boundAddress().publishAddress(), Version.CURRENT);
IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(settings);
@ -238,7 +238,7 @@ public abstract class TaskManagerTestCase extends ESTestCase {
RecordingTaskManagerListener[] listeners = new RecordingTaskManagerListener[nodes.length];
for (int i = 0; i < nodes.length; i++) {
listeners[i] = new RecordingTaskManagerListener(nodes[i].discoveryNode, actionMasks);
((MockTaskManager) (nodes[i].clusterService.getTaskManager())).addListener(listeners[i]);
((MockTaskManager) (nodes[i].transportService.getTaskManager())).addListener(listeners[i]);
}
return listeners;
}

View File

@ -43,6 +43,7 @@ import org.elasticsearch.test.tasks.MockTaskManager;
import org.elasticsearch.test.tasks.MockTaskManagerListener;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.transport.ReceiveTimeoutTransportException;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.ArrayList;
@ -263,8 +264,8 @@ public class TasksIT extends ESIntegTestCase {
ReentrantLock taskFinishLock = new ReentrantLock();
taskFinishLock.lock();
CountDownLatch taskRegistered = new CountDownLatch(1);
for (ClusterService clusterService : internalCluster().getInstances(ClusterService.class)) {
((MockTaskManager)clusterService.getTaskManager()).addListener(new MockTaskManagerListener() {
for (TransportService transportService : internalCluster().getInstances(TransportService.class)) {
((MockTaskManager) transportService.getTaskManager()).addListener(new MockTaskManagerListener() {
@Override
public void onTaskRegistered(Task task) {
if (task.getAction().startsWith(IndexAction.NAME)) {
@ -408,7 +409,7 @@ public class TasksIT extends ESIntegTestCase {
@Override
public void tearDown() throws Exception {
for (Map.Entry<Tuple<String, String>, RecordingTaskManagerListener> entry : listeners.entrySet()) {
((MockTaskManager)internalCluster().getInstance(ClusterService.class, entry.getKey().v1()).getTaskManager()).removeListener(entry.getValue());
((MockTaskManager) internalCluster().getInstance(TransportService.class, entry.getKey().v1()).getTaskManager()).removeListener(entry.getValue());
}
listeners.clear();
super.tearDown();
@ -418,10 +419,10 @@ public class TasksIT extends ESIntegTestCase {
* Registers recording task event listeners with the given action mask on all nodes
*/
private void registerTaskManageListeners(String actionMasks) {
for (ClusterService clusterService : internalCluster().getInstances(ClusterService.class)) {
DiscoveryNode node = clusterService.localNode();
for (String nodeName : internalCluster().getNodeNames()) {
DiscoveryNode node = internalCluster().getInstance(ClusterService.class, nodeName).localNode();
RecordingTaskManagerListener listener = new RecordingTaskManagerListener(node, Strings.splitStringToArray(actionMasks, ','));
((MockTaskManager)clusterService.getTaskManager()).addListener(listener);
((MockTaskManager) internalCluster().getInstance(TransportService.class, nodeName).getTaskManager()).addListener(listener);
RecordingTaskManagerListener oldListener = listeners.put(new Tuple<>(node.name(), actionMasks), listener);
assertNull(oldListener);
}

View File

@ -48,19 +48,7 @@ public class ClusterModuleTests extends ModuleTestCase {
static class FakeShardsAllocator implements ShardsAllocator {
@Override
public void applyStartedShards(StartedRerouteAllocation allocation) {}
@Override
public void applyFailedShards(FailedRerouteAllocation allocation) {}
@Override
public boolean allocateUnassigned(RoutingAllocation allocation) {
return false;
}
@Override
public boolean rebalance(RoutingAllocation allocation) {
return false;
}
@Override
public boolean moveShards(RoutingAllocation allocation) {
public boolean allocate(RoutingAllocation allocation) {
return false;
}
}

View File

@ -18,16 +18,12 @@
*/
package org.elasticsearch.cluster;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.tasks.PendingClusterTasksResponse;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.service.InternalClusterService;
import org.elasticsearch.cluster.service.PendingClusterTask;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.common.inject.Inject;
@ -35,38 +31,24 @@ import org.elasticsearch.common.inject.Singleton;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.node.Node;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.ESIntegTestCase.Scope;
import org.elasticsearch.test.InternalTestCluster;
import org.elasticsearch.test.MockLogAppender;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@ -85,74 +67,6 @@ public class ClusterServiceIT extends ESIntegTestCase {
return pluginList(TestPlugin.class);
}
public void testTimeoutUpdateTask() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "local")
.build();
internalCluster().startNode(settings);
ClusterService clusterService1 = internalCluster().getInstance(ClusterService.class);
final CountDownLatch block = new CountDownLatch(1);
clusterService1.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
try {
block.await();
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
return currentState;
}
@Override
public void onFailure(String source, Throwable t) {
throw new RuntimeException(t);
}
});
final CountDownLatch timedOut = new CountDownLatch(1);
final AtomicBoolean executeCalled = new AtomicBoolean();
clusterService1.submitStateUpdateTask("test2", new ClusterStateUpdateTask() {
@Override
public TimeValue timeout() {
return TimeValue.timeValueMillis(2);
}
@Override
public void onFailure(String source, Throwable t) {
timedOut.countDown();
}
@Override
public ClusterState execute(ClusterState currentState) {
executeCalled.set(true);
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
}
});
timedOut.await();
block.countDown();
final CountDownLatch allProcessed = new CountDownLatch(1);
clusterService1.submitStateUpdateTask("test3", new ClusterStateUpdateTask() {
@Override
public void onFailure(String source, Throwable t) {
throw new RuntimeException(t);
}
@Override
public ClusterState execute(ClusterState currentState) {
allProcessed.countDown();
return currentState;
}
});
allProcessed.await(); // executed another task to double check that execute on the timed out update task is not called...
assertThat(executeCalled.get(), equalTo(false));
}
public void testAckedUpdateTask() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "local")
@ -299,63 +213,6 @@ public class ClusterServiceIT extends ESIntegTestCase {
assertThat(processedLatch.await(1, TimeUnit.SECONDS), equalTo(true));
}
public void testMasterAwareExecution() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "local")
.build();
InternalTestCluster.Async<String> master = internalCluster().startNodeAsync(settings);
InternalTestCluster.Async<String> nonMaster = internalCluster().startNodeAsync(settingsBuilder().put(settings).put(Node.NODE_MASTER_SETTING.getKey(), false).build());
master.get();
ensureGreen(); // make sure we have a cluster
ClusterService clusterService = internalCluster().getInstance(ClusterService.class, nonMaster.get());
final boolean[] taskFailed = {false};
final CountDownLatch latch1 = new CountDownLatch(1);
clusterService.submitStateUpdateTask("test", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
latch1.countDown();
return currentState;
}
@Override
public void onFailure(String source, Throwable t) {
taskFailed[0] = true;
latch1.countDown();
}
});
latch1.await();
assertTrue("cluster state update task was executed on a non-master", taskFailed[0]);
taskFailed[0] = true;
final CountDownLatch latch2 = new CountDownLatch(1);
clusterService.submitStateUpdateTask("test", new ClusterStateUpdateTask() {
@Override
public boolean runOnlyOnMaster() {
return false;
}
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
taskFailed[0] = false;
latch2.countDown();
return currentState;
}
@Override
public void onFailure(String source, Throwable t) {
taskFailed[0] = true;
latch2.countDown();
}
});
latch2.await();
assertFalse("non-master cluster state update task was not executed", taskFailed[0]);
}
public void testAckedUpdateTaskNoAckExpected() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "local")
@ -715,571 +572,6 @@ public class ClusterServiceIT extends ESIntegTestCase {
}
}
/**
* Note, this test can only work as long as we have a single thread executor executing the state update tasks!
*/
public void testPrioritizedTasks() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "local")
.build();
internalCluster().startNode(settings);
ClusterService clusterService = internalCluster().getInstance(ClusterService.class);
BlockingTask block = new BlockingTask(Priority.IMMEDIATE);
clusterService.submitStateUpdateTask("test", block);
int taskCount = randomIntBetween(5, 20);
Priority[] priorities = Priority.values();
// will hold all the tasks in the order in which they were executed
List<PrioritizedTask> tasks = new ArrayList<>(taskCount);
CountDownLatch latch = new CountDownLatch(taskCount);
for (int i = 0; i < taskCount; i++) {
Priority priority = priorities[randomIntBetween(0, priorities.length - 1)];
clusterService.submitStateUpdateTask("test", new PrioritizedTask(priority, latch, tasks));
}
block.release();
latch.await();
Priority prevPriority = null;
for (PrioritizedTask task : tasks) {
if (prevPriority == null) {
prevPriority = task.priority();
} else {
assertThat(task.priority().sameOrAfter(prevPriority), is(true));
}
}
}
/*
* test that a listener throwing an exception while handling a
* notification does not prevent publication notification to the
* executor
*/
public void testClusterStateTaskListenerThrowingExceptionIsOkay() throws InterruptedException {
Settings settings = settingsBuilder()
.put("discovery.type", "local")
.build();
internalCluster().startNode(settings);
ClusterService clusterService = internalCluster().getInstance(ClusterService.class);
final CountDownLatch latch = new CountDownLatch(1);
AtomicBoolean published = new AtomicBoolean();
clusterService.submitStateUpdateTask(
"testClusterStateTaskListenerThrowingExceptionIsOkay",
new Object(),
ClusterStateTaskConfig.build(Priority.NORMAL),
new ClusterStateTaskExecutor<Object>() {
@Override
public boolean runOnlyOnMaster() {
return false;
}
@Override
public BatchResult<Object> execute(ClusterState currentState, List<Object> tasks) throws Exception {
ClusterState newClusterState = ClusterState.builder(currentState).build();
return BatchResult.builder().successes(tasks).build(newClusterState);
}
@Override
public void clusterStatePublished(ClusterState newClusterState) {
published.set(true);
latch.countDown();
}
},
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
throw new IllegalStateException(source);
}
@Override
public void onFailure(String source, Throwable t) {
}
}
);
latch.await();
assertTrue(published.get());
}
// test that for a single thread, tasks are executed in the order
// that they are submitted
public void testClusterStateUpdateTasksAreExecutedInOrder() throws BrokenBarrierException, InterruptedException {
Settings settings = settingsBuilder()
.put("discovery.type", "local")
.build();
internalCluster().startNode(settings);
ClusterService clusterService = internalCluster().getInstance(ClusterService.class);
class TaskExecutor implements ClusterStateTaskExecutor<Integer> {
List<Integer> tasks = new ArrayList<>();
@Override
public BatchResult<Integer> execute(ClusterState currentState, List<Integer> tasks) throws Exception {
this.tasks.addAll(tasks);
return BatchResult.<Integer>builder().successes(tasks).build(ClusterState.builder(currentState).build());
}
@Override
public boolean runOnlyOnMaster() {
return false;
}
}
int numberOfThreads = randomIntBetween(2, 8);
TaskExecutor[] executors = new TaskExecutor[numberOfThreads];
for (int i = 0; i < numberOfThreads; i++) {
executors[i] = new TaskExecutor();
}
int tasksSubmittedPerThread = randomIntBetween(2, 1024);
CopyOnWriteArrayList<Tuple<String, Throwable>> failures = new CopyOnWriteArrayList<>();
CountDownLatch updateLatch = new CountDownLatch(numberOfThreads * tasksSubmittedPerThread);
ClusterStateTaskListener listener = new ClusterStateTaskListener() {
@Override
public void onFailure(String source, Throwable t) {
logger.error("unexpected failure: [{}]", t, source);
failures.add(new Tuple<>(source, t));
updateLatch.countDown();
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
updateLatch.countDown();
}
};
CyclicBarrier barrier = new CyclicBarrier(1 + numberOfThreads);
for (int i = 0; i < numberOfThreads; i++) {
final int index = i;
Thread thread = new Thread(() -> {
try {
barrier.await();
for (int j = 0; j < tasksSubmittedPerThread; j++) {
clusterService.submitStateUpdateTask("[" + index + "][" + j + "]", j, ClusterStateTaskConfig.build(randomFrom(Priority.values())), executors[index], listener);
}
barrier.await();
} catch (InterruptedException | BrokenBarrierException e) {
throw new AssertionError(e);
}
});
thread.start();
}
// wait for all threads to be ready
barrier.await();
// wait for all threads to finish
barrier.await();
updateLatch.await();
assertThat(failures, empty());
for (int i = 0; i < numberOfThreads; i++) {
assertEquals(tasksSubmittedPerThread, executors[i].tasks.size());
for (int j = 0; j < tasksSubmittedPerThread; j++) {
assertNotNull(executors[i].tasks.get(j));
assertEquals("cluster state update task executed out of order", j, (int)executors[i].tasks.get(j));
}
}
}
public void testClusterStateBatchedUpdates() throws BrokenBarrierException, InterruptedException {
Settings settings = settingsBuilder()
.put("discovery.type", "local")
.build();
internalCluster().startNode(settings);
ClusterService clusterService = internalCluster().getInstance(ClusterService.class);
AtomicInteger counter = new AtomicInteger();
class Task {
private AtomicBoolean state = new AtomicBoolean();
public void execute() {
if (!state.compareAndSet(false, true)) {
throw new IllegalStateException();
} else {
counter.incrementAndGet();
}
}
}
int numberOfThreads = randomIntBetween(2, 8);
int tasksSubmittedPerThread = randomIntBetween(1, 1024);
int numberOfExecutors = Math.max(1, numberOfThreads / 4);
final Semaphore semaphore = new Semaphore(numberOfExecutors);
class TaskExecutor implements ClusterStateTaskExecutor<Task> {
private AtomicInteger counter = new AtomicInteger();
private AtomicInteger batches = new AtomicInteger();
private AtomicInteger published = new AtomicInteger();
@Override
public BatchResult<Task> execute(ClusterState currentState, List<Task> tasks) throws Exception {
tasks.forEach(task -> task.execute());
counter.addAndGet(tasks.size());
ClusterState maybeUpdatedClusterState = currentState;
if (randomBoolean()) {
maybeUpdatedClusterState = ClusterState.builder(currentState).build();
batches.incrementAndGet();
semaphore.acquire();
}
return BatchResult.<Task>builder().successes(tasks).build(maybeUpdatedClusterState);
}
@Override
public boolean runOnlyOnMaster() {
return false;
}
@Override
public void clusterStatePublished(ClusterState newClusterState) {
published.incrementAndGet();
semaphore.release();
}
}
ConcurrentMap<String, AtomicInteger> counters = new ConcurrentHashMap<>();
CountDownLatch updateLatch = new CountDownLatch(numberOfThreads * tasksSubmittedPerThread);
ClusterStateTaskListener listener = new ClusterStateTaskListener() {
@Override
public void onFailure(String source, Throwable t) {
assert false;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
counters.computeIfAbsent(source, key -> new AtomicInteger()).incrementAndGet();
updateLatch.countDown();
}
};
List<TaskExecutor> executors = new ArrayList<>();
for (int i = 0; i < numberOfExecutors; i++) {
executors.add(new TaskExecutor());
}
// randomly assign tasks to executors
List<TaskExecutor> assignments = new ArrayList<>();
for (int i = 0; i < numberOfThreads; i++) {
for (int j = 0; j < tasksSubmittedPerThread; j++) {
assignments.add(randomFrom(executors));
}
}
Map<TaskExecutor, Integer> counts = new HashMap<>();
for (TaskExecutor executor : assignments) {
counts.merge(executor, 1, (previous, one) -> previous + one);
}
CyclicBarrier barrier = new CyclicBarrier(1 + numberOfThreads);
for (int i = 0; i < numberOfThreads; i++) {
final int index = i;
Thread thread = new Thread(() -> {
try {
barrier.await();
for (int j = 0; j < tasksSubmittedPerThread; j++) {
ClusterStateTaskExecutor<Task> executor = assignments.get(index * tasksSubmittedPerThread + j);
clusterService.submitStateUpdateTask(
Thread.currentThread().getName(),
new Task(),
ClusterStateTaskConfig.build(randomFrom(Priority.values())),
executor,
listener);
}
barrier.await();
} catch (BrokenBarrierException | InterruptedException e) {
throw new AssertionError(e);
}
});
thread.start();
}
// wait for all threads to be ready
barrier.await();
// wait for all threads to finish
barrier.await();
// wait until all the cluster state updates have been processed
updateLatch.await();
// and until all of the publication callbacks have completed
semaphore.acquire(numberOfExecutors);
// assert the number of executed tasks is correct
assertEquals(numberOfThreads * tasksSubmittedPerThread, counter.get());
// assert each executor executed the correct number of tasks
for (TaskExecutor executor : executors) {
if (counts.containsKey(executor)) {
assertEquals((int) counts.get(executor), executor.counter.get());
assertEquals(executor.batches.get(), executor.published.get());
}
}
// assert the correct number of clusterStateProcessed events were triggered
for (Map.Entry<String, AtomicInteger> entry : counters.entrySet()) {
assertEquals(entry.getValue().get(), tasksSubmittedPerThread);
}
}
@TestLogging("cluster:TRACE") // To ensure that we log cluster state events on TRACE level
public void testClusterStateUpdateLogging() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "local")
.build();
internalCluster().startNode(settings);
ClusterService clusterService1 = internalCluster().getInstance(ClusterService.class);
MockLogAppender mockAppender = new MockLogAppender();
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation("test1", "cluster.service", Level.DEBUG, "*processing [test1]: took * no change in cluster_state"));
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation("test2", "cluster.service", Level.TRACE, "*failed to execute cluster state update in *"));
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation("test3", "cluster.service", Level.DEBUG, "*processing [test3]: took * done applying updated cluster_state (version: *, uuid: *)"));
Logger rootLogger = Logger.getRootLogger();
rootLogger.addAppender(mockAppender);
try {
final CountDownLatch latch = new CountDownLatch(4);
clusterService1.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
clusterService1.submitStateUpdateTask("test2", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task");
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
fail();
}
@Override
public void onFailure(String source, Throwable t) {
latch.countDown();
}
});
clusterService1.submitStateUpdateTask("test3", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
return ClusterState.builder(currentState).incrementVersion().build();
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
// Additional update task to make sure all previous logging made it to the logger
// We don't check logging for this on since there is no guarantee that it will occur before our check
clusterService1.submitStateUpdateTask("test4", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
assertThat(latch.await(1, TimeUnit.SECONDS), equalTo(true));
} finally {
rootLogger.removeAppender(mockAppender);
}
mockAppender.assertAllExpectationsMatched();
}
@TestLogging("cluster:WARN") // To ensure that we log cluster state events on WARN level
public void testLongClusterStateUpdateLogging() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "local")
.put(InternalClusterService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.getKey(), "10s")
.build();
internalCluster().startNode(settings);
ClusterService clusterService1 = internalCluster().getInstance(ClusterService.class);
MockLogAppender mockAppender = new MockLogAppender();
mockAppender.addExpectation(new MockLogAppender.UnseenEventExpectation("test1 shouldn't see because setting is too low", "cluster.service", Level.WARN, "*cluster state update task [test1] took * above the warn threshold of *"));
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation("test2", "cluster.service", Level.WARN, "*cluster state update task [test2] took * above the warn threshold of 10ms"));
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation("test3", "cluster.service", Level.WARN, "*cluster state update task [test3] took * above the warn threshold of 10ms"));
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation("test4", "cluster.service", Level.WARN, "*cluster state update task [test4] took * above the warn threshold of 10ms"));
Logger rootLogger = Logger.getRootLogger();
rootLogger.addAppender(mockAppender);
try {
final CountDownLatch latch = new CountDownLatch(5);
final CountDownLatch processedFirstTask = new CountDownLatch(1);
clusterService1.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
Thread.sleep(100);
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
processedFirstTask.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
processedFirstTask.await(1, TimeUnit.SECONDS);
assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder()
.put(InternalClusterService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.getKey(), "10ms")));
clusterService1.submitStateUpdateTask("test2", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
Thread.sleep(100);
throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task");
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
fail();
}
@Override
public void onFailure(String source, Throwable t) {
latch.countDown();
}
});
clusterService1.submitStateUpdateTask("test3", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
Thread.sleep(100);
return ClusterState.builder(currentState).incrementVersion().build();
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
clusterService1.submitStateUpdateTask("test4", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
Thread.sleep(100);
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
// Additional update task to make sure all previous logging made it to the logger
// We don't check logging for this on since there is no guarantee that it will occur before our check
clusterService1.submitStateUpdateTask("test5", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
assertThat(latch.await(5, TimeUnit.SECONDS), equalTo(true));
} finally {
rootLogger.removeAppender(mockAppender);
}
mockAppender.assertAllExpectationsMatched();
}
private static class BlockingTask extends ClusterStateUpdateTask {
private final CountDownLatch latch = new CountDownLatch(1);
public BlockingTask(Priority priority) {
super(priority);
}
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
latch.await();
return currentState;
}
@Override
public void onFailure(String source, Throwable t) {
}
public void release() {
latch.countDown();
}
}
private static class PrioritizedTask extends ClusterStateUpdateTask {
private final CountDownLatch latch;
private final List<PrioritizedTask> tasks;
private PrioritizedTask(Priority priority, CountDownLatch latch, List<PrioritizedTask> tasks) {
super(priority);
this.latch = latch;
this.tasks = tasks;
}
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
tasks.add(this);
latch.countDown();
return currentState;
}
@Override
public void onFailure(String source, Throwable t) {
latch.countDown();
}
}
public static class TestPlugin extends Plugin {
@Override

View File

@ -639,6 +639,7 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
return new SnapshotsInProgress(new SnapshotsInProgress.Entry(
new SnapshotId(randomName("repo"), randomName("snap")),
randomBoolean(),
randomBoolean(),
SnapshotsInProgress.State.fromValue((byte) randomIntBetween(0, 6)),
Collections.<String>emptyList(),
Math.abs(randomLong()),

View File

@ -143,11 +143,11 @@ public class DiskUsageTests extends ESTestCase {
};
NodeStats[] nodeStats = new NodeStats[] {
new NodeStats(new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, Version.CURRENT), 0,
null,null,null,null,null,new FsInfo(0, node1FSInfo), null,null,null,null,null),
null,null,null,null,null,new FsInfo(0, node1FSInfo), null,null,null,null,null, null),
new NodeStats(new DiscoveryNode("node_2", DummyTransportAddress.INSTANCE, Version.CURRENT), 0,
null,null,null,null,null, new FsInfo(0, node2FSInfo), null,null,null,null,null),
null,null,null,null,null, new FsInfo(0, node2FSInfo), null,null,null,null,null, null),
new NodeStats(new DiscoveryNode("node_3", DummyTransportAddress.INSTANCE, Version.CURRENT), 0,
null,null,null,null,null, new FsInfo(0, node3FSInfo), null,null,null,null,null)
null,null,null,null,null, new FsInfo(0, node3FSInfo), null,null,null,null,null, null)
};
InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvaiableUsages, newMostAvaiableUsages);
DiskUsage leastNode_1 = newLeastAvaiableUsages.get("node_1");
@ -184,11 +184,11 @@ public class DiskUsageTests extends ESTestCase {
};
NodeStats[] nodeStats = new NodeStats[] {
new NodeStats(new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, Version.CURRENT), 0,
null,null,null,null,null,new FsInfo(0, node1FSInfo), null,null,null,null,null),
null,null,null,null,null,new FsInfo(0, node1FSInfo), null,null,null,null,null, null),
new NodeStats(new DiscoveryNode("node_2", DummyTransportAddress.INSTANCE, Version.CURRENT), 0,
null,null,null,null,null, new FsInfo(0, node2FSInfo), null,null,null,null,null),
null,null,null,null,null, new FsInfo(0, node2FSInfo), null,null,null,null,null, null),
new NodeStats(new DiscoveryNode("node_3", DummyTransportAddress.INSTANCE, Version.CURRENT), 0,
null,null,null,null,null, new FsInfo(0, node3FSInfo), null,null,null,null,null)
null,null,null,null,null, new FsInfo(0, node3FSInfo), null,null,null,null,null, null)
};
InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvailableUsages, newMostAvailableUsages);
DiskUsage leastNode_1 = newLeastAvailableUsages.get("node_1");

View File

@ -0,0 +1,275 @@
/*
* 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.cluster;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.component.Lifecycle;
import org.elasticsearch.common.component.LifecycleListener;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.BoundTransportAddress;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.Transport;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.TransportServiceAdapter;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import static org.hamcrest.Matchers.equalTo;
public class NodeConnectionsServiceTests extends ESTestCase {
private static ThreadPool THREAD_POOL;
private MockTransport transport;
private TransportService transportService;
private List<DiscoveryNode> generateNodes() {
List<DiscoveryNode> nodes = new ArrayList<>();
for (int i = randomIntBetween(20, 50); i > 0; i--) {
final HashMap<String, String> attributes = new HashMap<>();
if (rarely()) {
attributes.put("client", "true");
} else {
attributes.put("master", "" + randomBoolean());
attributes.put("data", "" + randomBoolean());
attributes.put("ingest", "" + randomBoolean());
}
nodes.add(new DiscoveryNode("node_" + i, "" + i, DummyTransportAddress.INSTANCE, attributes, Version.CURRENT));
}
return nodes;
}
private ClusterState clusterStateFromNodes(List<DiscoveryNode> nodes) {
final DiscoveryNodes.Builder builder = DiscoveryNodes.builder();
for (DiscoveryNode node : nodes) {
builder.put(node);
}
return ClusterState.builder(new ClusterName("test")).nodes(builder).build();
}
public void testConnectAndDisconnect() {
List<DiscoveryNode> nodes = generateNodes();
NodeConnectionsService service = new NodeConnectionsService(Settings.EMPTY, THREAD_POOL, transportService);
ClusterState current = clusterStateFromNodes(Collections.emptyList());
ClusterChangedEvent event = new ClusterChangedEvent("test", clusterStateFromNodes(randomSubsetOf(nodes)), current);
service.connectToAddedNodes(event);
assertConnected(event.nodesDelta().addedNodes());
service.disconnectFromRemovedNodes(event);
assertConnectedExactlyToNodes(event.state());
current = event.state();
event = new ClusterChangedEvent("test", clusterStateFromNodes(randomSubsetOf(nodes)), current);
service.connectToAddedNodes(event);
assertConnected(event.nodesDelta().addedNodes());
service.disconnectFromRemovedNodes(event);
assertConnectedExactlyToNodes(event.state());
}
public void testReconnect() {
List<DiscoveryNode> nodes = generateNodes();
NodeConnectionsService service = new NodeConnectionsService(Settings.EMPTY, THREAD_POOL, transportService);
ClusterState current = clusterStateFromNodes(Collections.emptyList());
ClusterChangedEvent event = new ClusterChangedEvent("test", clusterStateFromNodes(randomSubsetOf(nodes)), current);
transport.randomConnectionExceptions = true;
service.connectToAddedNodes(event);
for (int i = 0; i < 3; i++) {
// simulate disconnects
for (DiscoveryNode node : randomSubsetOf(nodes)) {
transport.disconnectFromNode(node);
}
service.new ConnectionChecker().run();
}
// disable exceptions so things can be restored
transport.randomConnectionExceptions = false;
service.new ConnectionChecker().run();
assertConnectedExactlyToNodes(event.state());
}
private void assertConnectedExactlyToNodes(ClusterState state) {
assertConnected(state.nodes());
assertThat(transport.connectedNodes.size(), equalTo(state.nodes().size()));
}
private void assertConnected(Iterable<DiscoveryNode> nodes) {
for (DiscoveryNode node : nodes) {
assertTrue("not connected to " + node, transport.connectedNodes.contains(node));
}
}
private void assertNotConnected(Iterable<DiscoveryNode> nodes) {
for (DiscoveryNode node : nodes) {
assertFalse("still connected to " + node, transport.connectedNodes.contains(node));
}
}
@Override
@Before
public void setUp() throws Exception {
super.setUp();
this.transport = new MockTransport();
transportService = new TransportService(transport, THREAD_POOL);
transportService.start();
transportService.acceptIncomingRequests();
}
@Override
@After
public void tearDown() throws Exception {
transportService.stop();
super.tearDown();
}
@AfterClass
public static void stopThreadPool() {
ThreadPool.terminate(THREAD_POOL, 30, TimeUnit.SECONDS);
THREAD_POOL = null;
}
final class MockTransport implements Transport {
Set<DiscoveryNode> connectedNodes = ConcurrentCollections.newConcurrentSet();
volatile boolean randomConnectionExceptions = false;
@Override
public void transportServiceAdapter(TransportServiceAdapter service) {
}
@Override
public BoundTransportAddress boundAddress() {
return null;
}
@Override
public Map<String, BoundTransportAddress> profileBoundAddresses() {
return null;
}
@Override
public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws Exception {
return new TransportAddress[0];
}
@Override
public boolean addressSupported(Class<? extends TransportAddress> address) {
return false;
}
@Override
public boolean nodeConnected(DiscoveryNode node) {
return connectedNodes.contains(node);
}
@Override
public void connectToNode(DiscoveryNode node) throws ConnectTransportException {
if (connectedNodes.contains(node) == false && randomConnectionExceptions && randomBoolean()) {
throw new ConnectTransportException(node, "simulated");
}
connectedNodes.add(node);
}
@Override
public void connectToNodeLight(DiscoveryNode node) throws ConnectTransportException {
}
@Override
public void disconnectFromNode(DiscoveryNode node) {
connectedNodes.remove(node);
}
@Override
public void sendRequest(DiscoveryNode node, long requestId, String action, TransportRequest request,
TransportRequestOptions options) throws IOException, TransportException {
}
@Override
public long serverOpen() {
return 0;
}
@Override
public List<String> getLocalAddresses() {
return null;
}
@Override
public Lifecycle.State lifecycleState() {
return null;
}
@Override
public void addLifecycleListener(LifecycleListener listener) {
}
@Override
public void removeLifecycleListener(LifecycleListener listener) {
}
@Override
public Transport start() {
return null;
}
@Override
public Transport stop() {
return null;
}
@Override
public void close() {
}
}
}

View File

@ -27,6 +27,7 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateTaskExecutor;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNodeService;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
@ -40,7 +41,6 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
import org.elasticsearch.cluster.service.InternalClusterService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.test.ESAllocationTestCase;
@ -305,7 +305,7 @@ public class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
return randomSubsetOf(1, shards.toArray(new ShardRouting[0])).get(0);
} else {
return
TestShardRouting.newShardRouting(shardRouting.index(), shardRouting.id(), InternalClusterService.generateNodeId(Settings.EMPTY), randomBoolean(), randomFrom(ShardRoutingState.values()));
TestShardRouting.newShardRouting(shardRouting.index(), shardRouting.id(), DiscoveryNodeService.generateNodeId(Settings.EMPTY), randomBoolean(), randomFrom(ShardRoutingState.values()));
}
}

View File

@ -35,7 +35,6 @@ import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocator;
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocators;
import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
@ -311,29 +310,9 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
public void testNoRebalanceOnPrimaryOverload() {
Settings.Builder settings = settingsBuilder();
AllocationService strategy = new AllocationService(settings.build(), randomAllocationDeciders(settings.build(),
new ClusterSettings(Settings.Builder.EMPTY_SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), getRandom()), new ShardsAllocators(settings.build(),
new ClusterSettings(Settings.Builder.EMPTY_SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), getRandom()),
NoopGatewayAllocator.INSTANCE, new ShardsAllocator() {
@Override
public boolean rebalance(RoutingAllocation allocation) {
return false;
}
@Override
public boolean moveShards(RoutingAllocation allocation) {
return false;
}
@Override
public void applyStartedShards(StartedRerouteAllocation allocation) {
}
@Override
public void applyFailedShards(FailedRerouteAllocation allocation) {
}
/*
* // this allocator tries to rebuild this scenario where a rebalance is
* // triggered solely by the primary overload on node [1] where a shard
@ -354,9 +333,8 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
--------[test][2], node[3], [P], s[STARTED]
--------[test][3], node[3], [P], s[STARTED]
---- unassigned
*/
@Override
public boolean allocateUnassigned(RoutingAllocation allocation) {
*/
public boolean allocate(RoutingAllocation allocation) {
RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned();
boolean changed = !unassigned.isEmpty();
ShardRouting[] drain = unassigned.drain();
@ -403,7 +381,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
}
return changed;
}
}), EmptyClusterInfoService.INSTANCE);
}, EmptyClusterInfoService.INSTANCE);
MetaData.Builder metaDataBuilder = MetaData.builder();
RoutingTable.Builder routingTableBuilder = RoutingTable.builder();
IndexMetaData.Builder indexMeta = IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1);

View File

@ -36,7 +36,7 @@ import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocators;
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
@ -333,7 +333,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
AllocationDeciders allocationDeciders = new AllocationDeciders(Settings.EMPTY, new AllocationDecider[] {new NodeVersionAllocationDecider(Settings.EMPTY)});
AllocationService strategy = new MockAllocationService(Settings.EMPTY,
allocationDeciders,
new ShardsAllocators(Settings.EMPTY, NoopGatewayAllocator.INSTANCE), EmptyClusterInfoService.INSTANCE);
NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE);
RoutingAllocation.Result result = strategy.reroute(state, new AllocationCommands(), true);
// the two indices must stay as is, the replicas cannot move to oldNode2 because versions don't match
state = ClusterState.builder(state).routingResult(result).build();
@ -363,7 +363,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
new NodeVersionAllocationDecider(Settings.EMPTY)});
AllocationService strategy = new MockAllocationService(Settings.EMPTY,
allocationDeciders,
new ShardsAllocators(Settings.EMPTY, NoopGatewayAllocator.INSTANCE), EmptyClusterInfoService.INSTANCE);
NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE);
RoutingAllocation.Result result = strategy.reroute(state, new AllocationCommands(), true);
// Make sure that primary shards are only allocated on the new node

View File

@ -30,7 +30,7 @@ import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocators;
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
@ -59,7 +59,7 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase {
RandomAllocationDecider randomAllocationDecider = new RandomAllocationDecider(getRandom());
AllocationService strategy = new AllocationService(settingsBuilder().build(), new AllocationDeciders(Settings.EMPTY,
new HashSet<>(Arrays.asList(new SameShardAllocationDecider(Settings.EMPTY), new ReplicaAfterPrimaryActiveAllocationDecider(Settings.EMPTY),
randomAllocationDecider))), new ShardsAllocators(NoopGatewayAllocator.INSTANCE), EmptyClusterInfoService.INSTANCE);
randomAllocationDecider))), NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE);
int indices = scaledRandomIntBetween(1, 20);
Builder metaBuilder = MetaData.builder();
int maxNumReplicas = 1;

View File

@ -39,7 +39,7 @@ import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting;
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.allocator.BalancedShardsAllocator;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommand;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
@ -65,10 +65,6 @@ import static org.hamcrest.Matchers.nullValue;
public class DiskThresholdDeciderTests extends ESAllocationTestCase {
private static ShardsAllocators makeShardsAllocators() {
return new ShardsAllocators(NoopGatewayAllocator.INSTANCE);
}
public void testDiskThreshold() {
Settings diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
@ -109,7 +105,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
@ -194,7 +190,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -225,7 +221,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -305,7 +301,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(2))
@ -362,7 +358,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -429,7 +425,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -460,7 +456,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -569,7 +565,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0))
@ -637,7 +633,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0))
@ -740,7 +736,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
@ -902,7 +898,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
// Ensure that the reroute call doesn't alter the routing table, since the first primary is relocating away
// and therefor we will have sufficient disk space on node1.
RoutingAllocation.Result result = strategy.reroute(clusterState, "reroute");
@ -1003,7 +999,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
RoutingAllocation.Result result = strategy.reroute(clusterState, "reroute");
assertThat(result.routingTable().index("test").getShards().get(0).primaryShard().state(), equalTo(STARTED));

View File

@ -0,0 +1,824 @@
/*
* 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.cluster.service;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateTaskConfig;
import org.elasticsearch.cluster.ClusterStateTaskExecutor;
import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.NodeConnectionsService;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.OperationRouting;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.MockLogAppender;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.threadpool.ThreadPool;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.Semaphore;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
public class ClusterServiceTests extends ESTestCase {
static ThreadPool threadPool;
TimedClusterService clusterService;
@BeforeClass
public static void createThreadPool() {
threadPool = new ThreadPool(ClusterServiceTests.class.getName());
}
@AfterClass
public static void stopThreadPool() {
if (threadPool != null) {
threadPool.shutdownNow();
threadPool = null;
}
}
@Before
public void setUp() throws Exception {
super.setUp();
clusterService = createClusterService(true);
}
@After
public void tearDown() throws Exception {
clusterService.close();
super.tearDown();
}
TimedClusterService createClusterService(boolean makeMaster) throws InterruptedException {
TimedClusterService test = new TimedClusterService(Settings.EMPTY, null,
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
threadPool, new ClusterName("ClusterServiceTests"));
test.setLocalNode(new DiscoveryNode("node1", DummyTransportAddress.INSTANCE, Version.CURRENT));
test.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) {
@Override
public void connectToAddedNodes(ClusterChangedEvent event) {
// skip
}
@Override
public void disconnectFromRemovedNodes(ClusterChangedEvent event) {
// skip
}
});
test.setClusterStatePublisher((event, ackListener) -> {
});
test.start();
CountDownLatch latch = new CountDownLatch(1);
test.submitStateUpdateTask("making a master", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
final DiscoveryNodes nodes = currentState.nodes();
final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(nodes)
.masterNodeId(makeMaster ? nodes.localNodeId() : null);
return ClusterState.builder(currentState).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).nodes(nodesBuilder).build();
}
@Override
public boolean runOnlyOnMaster() {
return false;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
logger.warn("unexpected exception", t);
fail("unexpected exception" + t);
}
});
latch.await();
return test;
}
public void testTimeoutUpdateTask() throws Exception {
final CountDownLatch block = new CountDownLatch(1);
clusterService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
try {
block.await();
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
return currentState;
}
@Override
public void onFailure(String source, Throwable t) {
throw new RuntimeException(t);
}
});
final CountDownLatch timedOut = new CountDownLatch(1);
final AtomicBoolean executeCalled = new AtomicBoolean();
clusterService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() {
@Override
public TimeValue timeout() {
return TimeValue.timeValueMillis(2);
}
@Override
public void onFailure(String source, Throwable t) {
timedOut.countDown();
}
@Override
public ClusterState execute(ClusterState currentState) {
executeCalled.set(true);
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
}
});
timedOut.await();
block.countDown();
final CountDownLatch allProcessed = new CountDownLatch(1);
clusterService.submitStateUpdateTask("test3", new ClusterStateUpdateTask() {
@Override
public void onFailure(String source, Throwable t) {
throw new RuntimeException(t);
}
@Override
public ClusterState execute(ClusterState currentState) {
allProcessed.countDown();
return currentState;
}
});
allProcessed.await(); // executed another task to double check that execute on the timed out update task is not called...
assertThat(executeCalled.get(), equalTo(false));
}
public void testMasterAwareExecution() throws Exception {
ClusterService nonMaster = createClusterService(false);
final boolean[] taskFailed = {false};
final CountDownLatch latch1 = new CountDownLatch(1);
nonMaster.submitStateUpdateTask("test", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
latch1.countDown();
return currentState;
}
@Override
public void onFailure(String source, Throwable t) {
taskFailed[0] = true;
latch1.countDown();
}
});
latch1.await();
assertTrue("cluster state update task was executed on a non-master", taskFailed[0]);
taskFailed[0] = true;
final CountDownLatch latch2 = new CountDownLatch(1);
nonMaster.submitStateUpdateTask("test", new ClusterStateUpdateTask() {
@Override
public boolean runOnlyOnMaster() {
return false;
}
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
taskFailed[0] = false;
latch2.countDown();
return currentState;
}
@Override
public void onFailure(String source, Throwable t) {
taskFailed[0] = true;
latch2.countDown();
}
});
latch2.await();
assertFalse("non-master cluster state update task was not executed", taskFailed[0]);
nonMaster.close();
}
/*
* test that a listener throwing an exception while handling a
* notification does not prevent publication notification to the
* executor
*/
public void testClusterStateTaskListenerThrowingExceptionIsOkay() throws InterruptedException {
final CountDownLatch latch = new CountDownLatch(1);
AtomicBoolean published = new AtomicBoolean();
clusterService.submitStateUpdateTask(
"testClusterStateTaskListenerThrowingExceptionIsOkay",
new Object(),
ClusterStateTaskConfig.build(Priority.NORMAL),
new ClusterStateTaskExecutor<Object>() {
@Override
public boolean runOnlyOnMaster() {
return false;
}
@Override
public BatchResult<Object> execute(ClusterState currentState, List<Object> tasks) throws Exception {
ClusterState newClusterState = ClusterState.builder(currentState).build();
return BatchResult.builder().successes(tasks).build(newClusterState);
}
@Override
public void clusterStatePublished(ClusterState newClusterState) {
published.set(true);
latch.countDown();
}
},
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
throw new IllegalStateException(source);
}
@Override
public void onFailure(String source, Throwable t) {
}
}
);
latch.await();
assertTrue(published.get());
}
// test that for a single thread, tasks are executed in the order
// that they are submitted
public void testClusterStateUpdateTasksAreExecutedInOrder() throws BrokenBarrierException, InterruptedException {
class TaskExecutor implements ClusterStateTaskExecutor<Integer> {
List<Integer> tasks = new ArrayList<>();
@Override
public BatchResult<Integer> execute(ClusterState currentState, List<Integer> tasks) throws Exception {
this.tasks.addAll(tasks);
return BatchResult.<Integer>builder().successes(tasks).build(ClusterState.builder(currentState).build());
}
@Override
public boolean runOnlyOnMaster() {
return false;
}
}
int numberOfThreads = randomIntBetween(2, 8);
TaskExecutor[] executors = new TaskExecutor[numberOfThreads];
for (int i = 0; i < numberOfThreads; i++) {
executors[i] = new TaskExecutor();
}
int tasksSubmittedPerThread = randomIntBetween(2, 1024);
CopyOnWriteArrayList<Tuple<String, Throwable>> failures = new CopyOnWriteArrayList<>();
CountDownLatch updateLatch = new CountDownLatch(numberOfThreads * tasksSubmittedPerThread);
ClusterStateTaskListener listener = new ClusterStateTaskListener() {
@Override
public void onFailure(String source, Throwable t) {
logger.error("unexpected failure: [{}]", t, source);
failures.add(new Tuple<>(source, t));
updateLatch.countDown();
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
updateLatch.countDown();
}
};
CyclicBarrier barrier = new CyclicBarrier(1 + numberOfThreads);
for (int i = 0; i < numberOfThreads; i++) {
final int index = i;
Thread thread = new Thread(() -> {
try {
barrier.await();
for (int j = 0; j < tasksSubmittedPerThread; j++) {
clusterService.submitStateUpdateTask("[" + index + "][" + j + "]", j,
ClusterStateTaskConfig.build(randomFrom(Priority.values())), executors[index], listener);
}
barrier.await();
} catch (InterruptedException | BrokenBarrierException e) {
throw new AssertionError(e);
}
});
thread.start();
}
// wait for all threads to be ready
barrier.await();
// wait for all threads to finish
barrier.await();
updateLatch.await();
assertThat(failures, empty());
for (int i = 0; i < numberOfThreads; i++) {
assertEquals(tasksSubmittedPerThread, executors[i].tasks.size());
for (int j = 0; j < tasksSubmittedPerThread; j++) {
assertNotNull(executors[i].tasks.get(j));
assertEquals("cluster state update task executed out of order", j, (int) executors[i].tasks.get(j));
}
}
}
public void testClusterStateBatchedUpdates() throws BrokenBarrierException, InterruptedException {
AtomicInteger counter = new AtomicInteger();
class Task {
private AtomicBoolean state = new AtomicBoolean();
public void execute() {
if (!state.compareAndSet(false, true)) {
throw new IllegalStateException();
} else {
counter.incrementAndGet();
}
}
}
int numberOfThreads = randomIntBetween(2, 8);
int tasksSubmittedPerThread = randomIntBetween(1, 1024);
int numberOfExecutors = Math.max(1, numberOfThreads / 4);
final Semaphore semaphore = new Semaphore(numberOfExecutors);
class TaskExecutor implements ClusterStateTaskExecutor<Task> {
private AtomicInteger counter = new AtomicInteger();
private AtomicInteger batches = new AtomicInteger();
private AtomicInteger published = new AtomicInteger();
@Override
public BatchResult<Task> execute(ClusterState currentState, List<Task> tasks) throws Exception {
tasks.forEach(task -> task.execute());
counter.addAndGet(tasks.size());
ClusterState maybeUpdatedClusterState = currentState;
if (randomBoolean()) {
maybeUpdatedClusterState = ClusterState.builder(currentState).build();
batches.incrementAndGet();
semaphore.acquire();
}
return BatchResult.<Task>builder().successes(tasks).build(maybeUpdatedClusterState);
}
@Override
public boolean runOnlyOnMaster() {
return false;
}
@Override
public void clusterStatePublished(ClusterState newClusterState) {
published.incrementAndGet();
semaphore.release();
}
}
ConcurrentMap<String, AtomicInteger> counters = new ConcurrentHashMap<>();
CountDownLatch updateLatch = new CountDownLatch(numberOfThreads * tasksSubmittedPerThread);
ClusterStateTaskListener listener = new ClusterStateTaskListener() {
@Override
public void onFailure(String source, Throwable t) {
assert false;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
counters.computeIfAbsent(source, key -> new AtomicInteger()).incrementAndGet();
updateLatch.countDown();
}
};
List<TaskExecutor> executors = new ArrayList<>();
for (int i = 0; i < numberOfExecutors; i++) {
executors.add(new TaskExecutor());
}
// randomly assign tasks to executors
List<TaskExecutor> assignments = new ArrayList<>();
for (int i = 0; i < numberOfThreads; i++) {
for (int j = 0; j < tasksSubmittedPerThread; j++) {
assignments.add(randomFrom(executors));
}
}
Map<TaskExecutor, Integer> counts = new HashMap<>();
for (TaskExecutor executor : assignments) {
counts.merge(executor, 1, (previous, one) -> previous + one);
}
CyclicBarrier barrier = new CyclicBarrier(1 + numberOfThreads);
for (int i = 0; i < numberOfThreads; i++) {
final int index = i;
Thread thread = new Thread(() -> {
try {
barrier.await();
for (int j = 0; j < tasksSubmittedPerThread; j++) {
ClusterStateTaskExecutor<Task> executor = assignments.get(index * tasksSubmittedPerThread + j);
clusterService.submitStateUpdateTask(
Thread.currentThread().getName(),
new Task(),
ClusterStateTaskConfig.build(randomFrom(Priority.values())),
executor,
listener);
}
barrier.await();
} catch (BrokenBarrierException | InterruptedException e) {
throw new AssertionError(e);
}
});
thread.start();
}
// wait for all threads to be ready
barrier.await();
// wait for all threads to finish
barrier.await();
// wait until all the cluster state updates have been processed
updateLatch.await();
// and until all of the publication callbacks have completed
semaphore.acquire(numberOfExecutors);
// assert the number of executed tasks is correct
assertEquals(numberOfThreads * tasksSubmittedPerThread, counter.get());
// assert each executor executed the correct number of tasks
for (TaskExecutor executor : executors) {
if (counts.containsKey(executor)) {
assertEquals((int) counts.get(executor), executor.counter.get());
assertEquals(executor.batches.get(), executor.published.get());
}
}
// assert the correct number of clusterStateProcessed events were triggered
for (Map.Entry<String, AtomicInteger> entry : counters.entrySet()) {
assertEquals(entry.getValue().get(), tasksSubmittedPerThread);
}
}
/**
* Note, this test can only work as long as we have a single thread executor executing the state update tasks!
*/
public void testPrioritizedTasks() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "local")
.build();
BlockingTask block = new BlockingTask(Priority.IMMEDIATE);
clusterService.submitStateUpdateTask("test", block);
int taskCount = randomIntBetween(5, 20);
Priority[] priorities = Priority.values();
// will hold all the tasks in the order in which they were executed
List<PrioritizedTask> tasks = new ArrayList<>(taskCount);
CountDownLatch latch = new CountDownLatch(taskCount);
for (int i = 0; i < taskCount; i++) {
Priority priority = priorities[randomIntBetween(0, priorities.length - 1)];
clusterService.submitStateUpdateTask("test", new PrioritizedTask(priority, latch, tasks));
}
block.release();
latch.await();
Priority prevPriority = null;
for (PrioritizedTask task : tasks) {
if (prevPriority == null) {
prevPriority = task.priority();
} else {
assertThat(task.priority().sameOrAfter(prevPriority), is(true));
}
}
}
@TestLogging("cluster:TRACE") // To ensure that we log cluster state events on TRACE level
public void testClusterStateUpdateLogging() throws Exception {
MockLogAppender mockAppender = new MockLogAppender();
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation("test1", "cluster.service", Level.DEBUG,
"*processing [test1]: took [1s] no change in cluster_state"));
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation("test2", "cluster.service", Level.TRACE,
"*failed to execute cluster state update in [2s]*"));
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation("test3", "cluster.service", Level.DEBUG,
"*processing [test3]: took [3s] done applying updated cluster_state (version: *, uuid: *)"));
Logger rootLogger = Logger.getRootLogger();
rootLogger.addAppender(mockAppender);
try {
final CountDownLatch latch = new CountDownLatch(4);
clusterService.currentTimeOverride = System.nanoTime();
clusterService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos();
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
clusterService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(2).nanos();
throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task");
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
fail();
}
@Override
public void onFailure(String source, Throwable t) {
latch.countDown();
}
});
clusterService.submitStateUpdateTask("test3", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(3).nanos();
return ClusterState.builder(currentState).incrementVersion().build();
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
// Additional update task to make sure all previous logging made it to the logger
// We don't check logging for this on since there is no guarantee that it will occur before our check
clusterService.submitStateUpdateTask("test4", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
latch.await();
} finally {
rootLogger.removeAppender(mockAppender);
}
mockAppender.assertAllExpectationsMatched();
}
@TestLogging("cluster:WARN") // To ensure that we log cluster state events on WARN level
public void testLongClusterStateUpdateLogging() throws Exception {
MockLogAppender mockAppender = new MockLogAppender();
mockAppender.addExpectation(new MockLogAppender.UnseenEventExpectation("test1 shouldn't see because setting is too low",
"cluster.service", Level.WARN, "*cluster state update task [test1] took [*] above the warn threshold of *"));
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation("test2", "cluster.service", Level.WARN,
"*cluster state update task [test2] took [32s] above the warn threshold of *"));
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation("test3", "cluster.service", Level.WARN,
"*cluster state update task [test3] took [33s] above the warn threshold of *"));
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation("test4", "cluster.service", Level.WARN,
"*cluster state update task [test4] took [34s] above the warn threshold of *"));
Logger rootLogger = Logger.getRootLogger();
rootLogger.addAppender(mockAppender);
try {
final CountDownLatch latch = new CountDownLatch(5);
final CountDownLatch processedFirstTask = new CountDownLatch(1);
clusterService.currentTimeOverride = System.nanoTime();
clusterService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos();
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
processedFirstTask.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
processedFirstTask.await();
clusterService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(32).nanos();
throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task");
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
fail();
}
@Override
public void onFailure(String source, Throwable t) {
latch.countDown();
}
});
clusterService.submitStateUpdateTask("test3", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(33).nanos();
return ClusterState.builder(currentState).incrementVersion().build();
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
clusterService.submitStateUpdateTask("test4", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(34).nanos();
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
// Additional update task to make sure all previous logging made it to the logger
// We don't check logging for this on since there is no guarantee that it will occur before our check
clusterService.submitStateUpdateTask("test5", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Throwable t) {
fail();
}
});
latch.await();
} finally {
rootLogger.removeAppender(mockAppender);
}
mockAppender.assertAllExpectationsMatched();
}
private static class BlockingTask extends ClusterStateUpdateTask {
private final CountDownLatch latch = new CountDownLatch(1);
public BlockingTask(Priority priority) {
super(priority);
}
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
latch.await();
return currentState;
}
@Override
public void onFailure(String source, Throwable t) {
}
public void release() {
latch.countDown();
}
}
private static class PrioritizedTask extends ClusterStateUpdateTask {
private final CountDownLatch latch;
private final List<PrioritizedTask> tasks;
private PrioritizedTask(Priority priority, CountDownLatch latch, List<PrioritizedTask> tasks) {
super(priority);
this.latch = latch;
this.tasks = tasks;
}
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
tasks.add(this);
latch.countDown();
return currentState;
}
@Override
public void onFailure(String source, Throwable t) {
latch.countDown();
}
}
static class TimedClusterService extends InternalClusterService {
public volatile Long currentTimeOverride = null;
public TimedClusterService(Settings settings, OperationRouting operationRouting, ClusterSettings clusterSettings,
ThreadPool threadPool, ClusterName clusterName) {
super(settings, operationRouting, clusterSettings, threadPool, clusterName);
}
@Override
protected long currentTimeInNanos() {
if (currentTimeOverride != null) {
return currentTimeOverride;
}
return super.currentTimeInNanos();
}
}
}

View File

@ -492,7 +492,7 @@ public class NodeJoinControllerTests extends ESTestCase {
static class NoopAllocationService extends AllocationService {
public NoopAllocationService(Settings settings) {
super(settings, null, null, null);
super(settings, null, null, null, null);
}
@Override

View File

@ -19,12 +19,20 @@
package org.elasticsearch.index.mapper;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.test.ESSingleNodeTestCase;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
// TODO: make this a real unit test
public class DocumentParserTests extends ESSingleNodeTestCase {
@ -61,4 +69,113 @@ public class DocumentParserTests extends ESSingleNodeTestCase {
assertNotNull(doc.rootDoc().getField("bar"));
assertNotNull(doc.rootDoc().getField(UidFieldMapper.NAME));
}
DocumentMapper createDummyMapping(MapperService mapperService) throws Exception {
String mapping = jsonBuilder().startObject().startObject("type").startObject("properties")
.startObject("y").field("type", "object").endObject()
.startObject("x").startObject("properties")
.startObject("subx").field("type", "object").startObject("properties")
.startObject("subsubx").field("type", "object")
.endObject().endObject().endObject().endObject().endObject().endObject().endObject().endObject().string();
DocumentMapper defaultMapper = mapperService.documentMapperParser().parse("type", new CompressedXContent(mapping));
return defaultMapper;
}
// creates an object mapper, which is about 100x harder than it should be....
ObjectMapper createObjectMapper(MapperService mapperService, String name) throws Exception {
String[] nameParts = name.split("\\.");
ContentPath path = new ContentPath();
for (int i = 0; i < nameParts.length - 1; ++i) {
path.add(nameParts[i]);
}
ParseContext context = new ParseContext.InternalParseContext(Settings.EMPTY,
mapperService.documentMapperParser(), mapperService.documentMapper("type"), path);
Mapper.Builder builder = new ObjectMapper.Builder(nameParts[nameParts.length - 1]).enabled(true);
Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
return (ObjectMapper)builder.build(builderContext);
}
public void testEmptyMappingUpdate() throws Exception {
DocumentMapper docMapper = createDummyMapping(createIndex("test").mapperService());
assertNull(DocumentParser.createDynamicUpdate(docMapper.mapping(), docMapper, Collections.emptyList()));
}
public void testSingleMappingUpdate() throws Exception {
DocumentMapper docMapper = createDummyMapping(createIndex("test").mapperService());
List<Mapper> updates = Collections.singletonList(new MockFieldMapper("foo"));
Mapping mapping = DocumentParser.createDynamicUpdate(docMapper.mapping(), docMapper, updates);
assertNotNull(mapping.root().getMapper("foo"));
}
public void testSubfieldMappingUpdate() throws Exception {
DocumentMapper docMapper = createDummyMapping(createIndex("test").mapperService());
List<Mapper> updates = Collections.singletonList(new MockFieldMapper("x.foo"));
Mapping mapping = DocumentParser.createDynamicUpdate(docMapper.mapping(), docMapper, updates);
Mapper xMapper = mapping.root().getMapper("x");
assertNotNull(xMapper);
assertTrue(xMapper instanceof ObjectMapper);
assertNotNull(((ObjectMapper)xMapper).getMapper("foo"));
assertNull(((ObjectMapper)xMapper).getMapper("subx"));
}
public void testMultipleSubfieldMappingUpdate() throws Exception {
DocumentMapper docMapper = createDummyMapping(createIndex("test").mapperService());
List<Mapper> updates = new ArrayList<>();
updates.add(new MockFieldMapper("x.foo"));
updates.add(new MockFieldMapper("x.bar"));
Mapping mapping = DocumentParser.createDynamicUpdate(docMapper.mapping(), docMapper, updates);
Mapper xMapper = mapping.root().getMapper("x");
assertNotNull(xMapper);
assertTrue(xMapper instanceof ObjectMapper);
assertNotNull(((ObjectMapper)xMapper).getMapper("foo"));
assertNotNull(((ObjectMapper)xMapper).getMapper("bar"));
assertNull(((ObjectMapper)xMapper).getMapper("subx"));
}
public void testDeepSubfieldMappingUpdate() throws Exception {
DocumentMapper docMapper = createDummyMapping(createIndex("test").mapperService());
List<Mapper> updates = Collections.singletonList(new MockFieldMapper("x.subx.foo"));
Mapping mapping = DocumentParser.createDynamicUpdate(docMapper.mapping(), docMapper, updates);
Mapper xMapper = mapping.root().getMapper("x");
assertNotNull(xMapper);
assertTrue(xMapper instanceof ObjectMapper);
Mapper subxMapper = ((ObjectMapper)xMapper).getMapper("subx");
assertTrue(subxMapper instanceof ObjectMapper);
assertNotNull(((ObjectMapper)subxMapper).getMapper("foo"));
assertNull(((ObjectMapper)subxMapper).getMapper("subsubx"));
}
public void testDeepSubfieldAfterSubfieldMappingUpdate() throws Exception {
DocumentMapper docMapper = createDummyMapping(createIndex("test").mapperService());
List<Mapper> updates = new ArrayList<>();
updates.add(new MockFieldMapper("x.a"));
updates.add(new MockFieldMapper("x.subx.b"));
Mapping mapping = DocumentParser.createDynamicUpdate(docMapper.mapping(), docMapper, updates);
Mapper xMapper = mapping.root().getMapper("x");
assertNotNull(xMapper);
assertTrue(xMapper instanceof ObjectMapper);
assertNotNull(((ObjectMapper)xMapper).getMapper("a"));
Mapper subxMapper = ((ObjectMapper)xMapper).getMapper("subx");
assertTrue(subxMapper instanceof ObjectMapper);
assertNotNull(((ObjectMapper)subxMapper).getMapper("b"));
}
public void testObjectMappingUpdate() throws Exception {
MapperService mapperService = createIndex("test").mapperService();
DocumentMapper docMapper = createDummyMapping(mapperService);
List<Mapper> updates = new ArrayList<>();
updates.add(createObjectMapper(mapperService, "foo"));
updates.add(createObjectMapper(mapperService, "foo.bar"));
updates.add(new MockFieldMapper("foo.bar.baz"));
updates.add(new MockFieldMapper("foo.field"));
Mapping mapping = DocumentParser.createDynamicUpdate(docMapper.mapping(), docMapper, updates);
Mapper fooMapper = mapping.root().getMapper("foo");
assertNotNull(fooMapper);
assertTrue(fooMapper instanceof ObjectMapper);
assertNotNull(((ObjectMapper)fooMapper).getMapper("field"));
Mapper barMapper = ((ObjectMapper)fooMapper).getMapper("bar");
assertTrue(barMapper instanceof ObjectMapper);
assertNotNull(((ObjectMapper)barMapper).getMapper("baz"));
}
}

View File

@ -42,6 +42,7 @@ import org.elasticsearch.index.mapper.core.TextFieldMapper;
import org.elasticsearch.test.ESSingleNodeTestCase;
import java.io.IOException;
import java.util.List;
import static java.util.Collections.emptyMap;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
@ -211,7 +212,9 @@ public class DynamicMappingTests extends ESSingleNodeTestCase {
ctx.reset(XContentHelper.createParser(source.source()), new ParseContext.Document(), source);
assertEquals(XContentParser.Token.START_OBJECT, ctx.parser().nextToken());
ctx.parser().nextToken();
return DocumentParser.parseObject(ctx, mapper.root(), true);
DocumentParser.parseObjectOrNested(ctx, mapper.root(), true);
Mapping mapping = DocumentParser.createDynamicUpdate(mapper.mapping(), mapper, ctx.getDynamicMappers());
return mapping == null ? null : mapping.root();
}
public void testDynamicMappingsNotNeeded() throws Exception {

View File

@ -19,12 +19,8 @@
package org.elasticsearch.index.mapper;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
@ -59,7 +55,7 @@ public class FieldTypeLookupTests extends ESTestCase {
public void testAddNewField() {
FieldTypeLookup lookup = new FieldTypeLookup();
FakeFieldMapper f = new FakeFieldMapper("foo");
MockFieldMapper f = new MockFieldMapper("foo");
FieldTypeLookup lookup2 = lookup.copyAndAddAll("type", newList(f), randomBoolean());
assertNull(lookup.get("foo"));
assertNull(lookup.get("bar"));
@ -73,8 +69,8 @@ public class FieldTypeLookupTests extends ESTestCase {
}
public void testAddExistingField() {
FakeFieldMapper f = new FakeFieldMapper("foo");
FakeFieldMapper f2 = new FakeFieldMapper("foo");
MockFieldMapper f = new MockFieldMapper("foo");
MockFieldMapper f2 = new MockFieldMapper("foo");
FieldTypeLookup lookup = new FieldTypeLookup();
lookup = lookup.copyAndAddAll("type1", newList(f), randomBoolean());
FieldTypeLookup lookup2 = lookup.copyAndAddAll("type2", newList(f2), randomBoolean());
@ -84,8 +80,8 @@ public class FieldTypeLookupTests extends ESTestCase {
}
public void testAddExistingIndexName() {
FakeFieldMapper f = new FakeFieldMapper("foo");
FakeFieldMapper f2 = new FakeFieldMapper("bar");
MockFieldMapper f = new MockFieldMapper("foo");
MockFieldMapper f2 = new MockFieldMapper("bar");
FieldTypeLookup lookup = new FieldTypeLookup();
lookup = lookup.copyAndAddAll("type1", newList(f), randomBoolean());
FieldTypeLookup lookup2 = lookup.copyAndAddAll("type2", newList(f2), randomBoolean());
@ -96,8 +92,8 @@ public class FieldTypeLookupTests extends ESTestCase {
}
public void testAddExistingFullName() {
FakeFieldMapper f = new FakeFieldMapper("foo");
FakeFieldMapper f2 = new FakeFieldMapper("foo");
MockFieldMapper f = new MockFieldMapper("foo");
MockFieldMapper f2 = new MockFieldMapper("foo");
FieldTypeLookup lookup = new FieldTypeLookup();
try {
lookup.copyAndAddAll("type2", newList(f2), randomBoolean());
@ -107,12 +103,13 @@ public class FieldTypeLookupTests extends ESTestCase {
}
public void testCheckCompatibilityMismatchedTypes() {
FieldMapper f1 = new FakeFieldMapper("foo");
FieldMapper f1 = new MockFieldMapper("foo");
FieldTypeLookup lookup = new FieldTypeLookup();
lookup = lookup.copyAndAddAll("type", newList(f1), randomBoolean());
MappedFieldType ft2 = FakeFieldMapper.makeOtherFieldType("foo");
FieldMapper f2 = new FakeFieldMapper("foo", ft2);
OtherFakeFieldType ft2 = new OtherFakeFieldType();
ft2.setName("foo");
FieldMapper f2 = new MockFieldMapper("foo", ft2);
try {
lookup.copyAndAddAll("type2", newList(f2), false);
fail("expected type mismatch");
@ -129,13 +126,14 @@ public class FieldTypeLookupTests extends ESTestCase {
}
public void testCheckCompatibilityConflict() {
FieldMapper f1 = new FakeFieldMapper("foo");
FieldMapper f1 = new MockFieldMapper("foo");
FieldTypeLookup lookup = new FieldTypeLookup();
lookup = lookup.copyAndAddAll("type", newList(f1), randomBoolean());
MappedFieldType ft2 = FakeFieldMapper.makeFieldType("foo");
MappedFieldType ft2 = new MockFieldMapper.FakeFieldType();
ft2.setName("foo");
ft2.setBoost(2.0f);
FieldMapper f2 = new FakeFieldMapper("foo", ft2);
FieldMapper f2 = new MockFieldMapper("foo", ft2);
try {
// different type
lookup.copyAndAddAll("type2", newList(f2), false);
@ -146,9 +144,10 @@ public class FieldTypeLookupTests extends ESTestCase {
lookup.copyAndAddAll("type", newList(f2), false); // boost is updateable, so ok since we are implicitly updating all types
lookup.copyAndAddAll("type2", newList(f2), true); // boost is updateable, so ok if forcing
// now with a non changeable setting
MappedFieldType ft3 = FakeFieldMapper.makeFieldType("foo");
MappedFieldType ft3 = new MockFieldMapper.FakeFieldType();
ft3.setName("foo");
ft3.setStored(true);
FieldMapper f3 = new FakeFieldMapper("foo", ft3);
FieldMapper f3 = new MockFieldMapper("foo", ft3);
try {
lookup.copyAndAddAll("type2", newList(f3), false);
fail("expected conflict");
@ -165,8 +164,8 @@ public class FieldTypeLookupTests extends ESTestCase {
}
public void testSimpleMatchFullNames() {
FakeFieldMapper f1 = new FakeFieldMapper("foo");
FakeFieldMapper f2 = new FakeFieldMapper("bar");
MockFieldMapper f1 = new MockFieldMapper("foo");
MockFieldMapper f2 = new MockFieldMapper("bar");
FieldTypeLookup lookup = new FieldTypeLookup();
lookup = lookup.copyAndAddAll("type", newList(f1, f2), randomBoolean());
Collection<String> names = lookup.simpleMatchToFullName("b*");
@ -175,7 +174,7 @@ public class FieldTypeLookupTests extends ESTestCase {
}
public void testIteratorImmutable() {
FakeFieldMapper f1 = new FakeFieldMapper("foo");
MockFieldMapper f1 = new MockFieldMapper("foo");
FieldTypeLookup lookup = new FieldTypeLookup();
lookup = lookup.copyAndAddAll("type", newList(f1), randomBoolean());
@ -194,59 +193,6 @@ public class FieldTypeLookupTests extends ESTestCase {
return Arrays.asList(mapper);
}
// this sucks how much must be overridden just do get a dummy field mapper...
static class FakeFieldMapper extends FieldMapper {
static Settings dummySettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT.id).build();
public FakeFieldMapper(String fullName) {
super(fullName, makeFieldType(fullName), makeFieldType(fullName), dummySettings, null, null);
}
public FakeFieldMapper(String fullName, MappedFieldType fieldType) {
super(fullName, fieldType, fieldType, dummySettings, null, null);
}
static MappedFieldType makeFieldType(String fullName) {
FakeFieldType fieldType = new FakeFieldType();
fieldType.setName(fullName);
return fieldType;
}
static MappedFieldType makeOtherFieldType(String fullName) {
OtherFakeFieldType fieldType = new OtherFakeFieldType();
fieldType.setName(fullName);
return fieldType;
}
static class FakeFieldType extends MappedFieldType {
public FakeFieldType() {}
protected FakeFieldType(FakeFieldType ref) {
super(ref);
}
@Override
public MappedFieldType clone() {
return new FakeFieldType(this);
}
@Override
public String typeName() {
return "faketype";
}
}
static class OtherFakeFieldType extends MappedFieldType {
public OtherFakeFieldType() {}
protected OtherFakeFieldType(OtherFakeFieldType ref) {
super(ref);
}
@Override
public MappedFieldType clone() {
return new OtherFakeFieldType(this);
}
@Override
public String typeName() {
return "otherfaketype";
}
}
@Override
protected String contentType() { return null; }
@Override
protected void parseCreateField(ParseContext context, List list) throws IOException {}
}
private int size(Iterator<MappedFieldType> iterator) {
if (iterator == null) {
throw new NullPointerException("iterator");
@ -258,4 +204,23 @@ public class FieldTypeLookupTests extends ESTestCase {
}
return count;
}
static class OtherFakeFieldType extends MappedFieldType {
public OtherFakeFieldType() {
}
protected OtherFakeFieldType(OtherFakeFieldType ref) {
super(ref);
}
@Override
public MappedFieldType clone() {
return new OtherFakeFieldType(this);
}
@Override
public String typeName() {
return "otherfaketype";
}
}
}

View File

@ -0,0 +1,177 @@
/*
* 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.core;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.apache.lucene.index.IndexOptions;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.test.InternalSettingsPlugin;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.instanceOf;
public class StringMappingUpgradeTests extends ESSingleNodeTestCase {
@Override
protected Collection<Class<? extends Plugin>> getPlugins() {
return pluginList(InternalSettingsPlugin.class);
}
public void testUpgradeDefaults() throws IOException {
IndexService indexService = createIndex("test");
DocumentMapperParser parser = indexService.mapperService().documentMapperParser();
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field").field("type", "string").endObject().endObject()
.endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
FieldMapper field = mapper.mappers().getMapper("field");
assertThat(field, instanceOf(TextFieldMapper.class));
}
public void testUpgradeAnalyzedString() throws IOException {
IndexService indexService = createIndex("test");
DocumentMapperParser parser = indexService.mapperService().documentMapperParser();
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field").field("type", "string").field("index", "analyzed").endObject().endObject()
.endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
FieldMapper field = mapper.mappers().getMapper("field");
assertThat(field, instanceOf(TextFieldMapper.class));
}
public void testUpgradeNotAnalyzedString() throws IOException {
IndexService indexService = createIndex("test");
DocumentMapperParser parser = indexService.mapperService().documentMapperParser();
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field").field("type", "string")
.field("index", "not_analyzed").endObject().endObject()
.endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
FieldMapper field = mapper.mappers().getMapper("field");
assertThat(field, instanceOf(KeywordFieldMapper.class));
}
public void testUpgradeNotIndexedString() throws IOException {
IndexService indexService = createIndex("test");
DocumentMapperParser parser = indexService.mapperService().documentMapperParser();
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field").field("type", "string").field("index", "no").endObject().endObject()
.endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
FieldMapper field = mapper.mappers().getMapper("field");
assertThat(field, instanceOf(KeywordFieldMapper.class));
assertEquals(IndexOptions.NONE, field.fieldType().indexOptions());
}
public void testNotSupportedUpgrade() throws IOException {
IndexService indexService = createIndex("test");
DocumentMapperParser parser = indexService.mapperService().documentMapperParser();
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field").field("type", "string").field("analyzer", "keyword").endObject().endObject()
.endObject().endObject().string();
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
() -> parser.parse("type", new CompressedXContent(mapping)));
assertThat(e.getMessage(), containsString("The [string] type is removed in 5.0"));
}
public void testUpgradeRandomMapping() throws IOException {
final int iters = 20;
for (int i = 0; i < iters; ++i) {
doTestUpgradeRandomMapping(i);
}
}
private void doTestUpgradeRandomMapping(int iter) throws IOException {
IndexService indexService;
boolean oldIndex = randomBoolean();
String indexName = "test" + iter;
if (oldIndex) {
Settings settings = Settings.builder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_2_3_0)
.build();
indexService = createIndex(indexName, settings);
} else {
indexService = createIndex(indexName);
}
DocumentMapperParser parser = indexService.mapperService().documentMapperParser();
XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field").field("type", "string");
boolean keyword = randomBoolean();
boolean shouldUpgrade = true;
if (keyword) {
mapping.field("index", randomBoolean() ? "not_analyzed" : "no");
} else if (randomBoolean()) {
mapping.field("index", "analyzed");
}
if (randomBoolean()) {
mapping.field("store", RandomPicks.randomFrom(random(), Arrays.asList("yes", "no", true, false)));
}
if (keyword && randomBoolean()) {
mapping.field("doc_values", randomBoolean());
}
if (randomBoolean()) {
mapping.field("omit_norms", randomBoolean());
}
if (randomBoolean()) {
mapping.startObject("fields").startObject("raw").field("type", "keyword").endObject().endObject();
}
if (randomBoolean()) {
mapping.field("copy_to", "bar");
}
if (randomBoolean()) {
// this option is not upgraded automatically
mapping.field("index_options", "docs");
shouldUpgrade = false;
}
mapping.endObject().endObject().endObject().endObject();
if (oldIndex == false && shouldUpgrade == false) {
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
() -> parser.parse("type", new CompressedXContent(mapping.string())));
assertThat(e.getMessage(), containsString("The [string] type is removed in 5.0"));
} else {
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping.string()));
FieldMapper field = mapper.mappers().getMapper("field");
if (oldIndex) {
assertThat(field, instanceOf(StringFieldMapper.class));
} else if (keyword) {
assertThat(field, instanceOf(KeywordFieldMapper.class));
} else {
assertThat(field, instanceOf(TextFieldMapper.class));
}
}
}
}

View File

@ -28,32 +28,28 @@ import org.elasticsearch.test.ESSingleNodeTestCase;
import static org.hamcrest.Matchers.containsString;
/**
*/
public class SimpleObjectMappingTests extends ESSingleNodeTestCase {
public void testDifferentInnerObjectTokenFailure() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.endObject().endObject().string();
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse("type", new CompressedXContent(mapping));
try {
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> {
defaultMapper.parse("test", "type", "1", new BytesArray(" {\n" +
" \"object\": {\n" +
" \"array\":[\n" +
" {\n" +
" \"object\": { \"value\": \"value\" }\n" +
" },\n" +
" {\n" +
" \"object\":\"value\"\n" +
" }\n" +
" ]\n" +
" },\n" +
" \"value\":\"value\"\n" +
" }"));
fail();
} catch (MapperParsingException e) {
// all is well
}
" \"object\": {\n" +
" \"array\":[\n" +
" {\n" +
" \"object\": { \"value\": \"value\" }\n" +
" },\n" +
" {\n" +
" \"object\":\"value\"\n" +
" }\n" +
" ]\n" +
" },\n" +
" \"value\":\"value\"\n" +
" }"));
});
assertTrue(e.getMessage(), e.getMessage().contains("different type"));
}
public void testEmptyArrayProperties() throws Exception {

View File

@ -25,6 +25,7 @@ import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.ingest.core.CompoundProcessor;
import org.elasticsearch.ingest.core.IngestDocument;
@ -38,15 +39,16 @@ import org.mockito.ArgumentMatcher;
import org.mockito.invocation.InvocationOnMock;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import static org.hamcrest.Matchers.equalTo;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.eq;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.anyString;
import static org.mockito.Matchers.argThat;
import static org.mockito.Mockito.doAnswer;
@ -341,6 +343,43 @@ public class PipelineExecutionServiceTests extends ESTestCase {
verify(completionHandler, times(1)).accept(null);
}
public void testStats() throws Exception {
IngestStats ingestStats = executionService.stats();
assertThat(ingestStats.getStatsPerPipeline().size(), equalTo(0));
assertThat(ingestStats.getTotalStats().getIngestCount(), equalTo(0L));
assertThat(ingestStats.getTotalStats().getIngestCurrent(), equalTo(0L));
assertThat(ingestStats.getTotalStats().getIngestFailedCount(), equalTo(0L));
assertThat(ingestStats.getTotalStats().getIngestTimeInMillis(), equalTo(0L));
when(store.get("_id1")).thenReturn(new Pipeline("_id1", null, new CompoundProcessor()));
when(store.get("_id2")).thenReturn(new Pipeline("_id2", null, new CompoundProcessor()));
Map<String, PipelineConfiguration> configurationMap = new HashMap<>();
configurationMap.put("_id1", new PipelineConfiguration("_id1", new BytesArray("{}")));
configurationMap.put("_id2", new PipelineConfiguration("_id2", new BytesArray("{}")));
executionService.updatePipelineStats(new IngestMetadata(configurationMap));
Consumer<Throwable> failureHandler = mock(Consumer.class);
Consumer<Boolean> completionHandler = mock(Consumer.class);
IndexRequest indexRequest = new IndexRequest("_index");
indexRequest.setPipeline("_id1");
executionService.executeIndexRequest(indexRequest, failureHandler, completionHandler);
ingestStats = executionService.stats();
assertThat(ingestStats.getStatsPerPipeline().size(), equalTo(2));
assertThat(ingestStats.getStatsPerPipeline().get("_id1").getIngestCount(), equalTo(1L));
assertThat(ingestStats.getStatsPerPipeline().get("_id2").getIngestCount(), equalTo(0L));
assertThat(ingestStats.getTotalStats().getIngestCount(), equalTo(1L));
indexRequest.setPipeline("_id2");
executionService.executeIndexRequest(indexRequest, failureHandler, completionHandler);
ingestStats = executionService.stats();
assertThat(ingestStats.getStatsPerPipeline().size(), equalTo(2));
assertThat(ingestStats.getStatsPerPipeline().get("_id1").getIngestCount(), equalTo(1L));
assertThat(ingestStats.getStatsPerPipeline().get("_id2").getIngestCount(), equalTo(1L));
assertThat(ingestStats.getTotalStats().getIngestCount(), equalTo(2L));
}
private IngestDocument eqID(String index, String type, String id, Map<String, Object> source) {
return argThat(new IngestDocumentMatcher(index, type, id, source));
}

View File

@ -33,6 +33,7 @@ import org.elasticsearch.search.aggregations.metrics.stats.Stats;
import org.elasticsearch.search.aggregations.metrics.sum.Sum;
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
import org.elasticsearch.search.aggregations.pipeline.derivative.Derivative;
import org.elasticsearch.search.aggregations.pipeline.movavg.models.SimpleModel;
import org.elasticsearch.search.aggregations.support.AggregationPath;
import org.elasticsearch.test.ESIntegTestCase;
import org.hamcrest.Matchers;
@ -47,7 +48,9 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.filters;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.stats;
import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
import static org.elasticsearch.search.aggregations.AggregationBuilders.avg;
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.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.closeTo;
@ -614,6 +617,37 @@ public class DerivativeIT extends ESIntegTestCase {
}
}
public void testAvgMovavgDerivNPE() throws Exception {
createIndex("movavg_npe");
ensureYellow("movavg_npe");
for (int i = 0; i < 10; i++) {
Integer value = i;
if (i == 1 || i == 3) {
value = null;
}
XContentBuilder doc = jsonBuilder()
.startObject()
.field("tick", i)
.field("value", value)
.endObject();
client().prepareIndex("movavg_npe", "type").setSource(doc).get();
}
refresh();
SearchResponse response = client()
.prepareSearch("movavg_npe")
.addAggregation(
histogram("histo").field("tick").interval(1)
.subAggregation(avg("avg").field("value"))
.subAggregation(movingAvg("movavg", "avg").modelBuilder(new SimpleModel.SimpleModelBuilder()).window(3))
.subAggregation(derivative("deriv", "movavg"))).execute().actionGet();
assertSearchResponse(response);
}
private void checkBucketKeyAndDocCount(final String msg, final Histogram.Bucket bucket, final long expectedKey,
final long expectedDocCount) {
assertThat(msg, bucket, notNullValue());

View File

@ -1813,19 +1813,31 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
}
}
public void testDeleteIndexDuringSnapshot() throws Exception {
public void testCloseOrDeleteIndexDuringSnapshot() throws Exception {
Client client = client();
boolean allowPartial = randomBoolean();
logger.info("--> creating repository");
assertAcked(client.admin().cluster().preparePutRepository("test-repo")
// only block on repo init if we have partial snapshot or we run into deadlock when acquiring shard locks for index deletion/closing
boolean initBlocking = allowPartial || randomBoolean();
if (initBlocking) {
assertAcked(client.admin().cluster().preparePutRepository("test-repo")
.setType("mock").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath())
.put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
.put("block_on_init", true)
.put("location", randomRepoPath())
.put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
.put("block_on_init", true)
));
} else {
assertAcked(client.admin().cluster().preparePutRepository("test-repo")
.setType("mock").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath())
.put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
.put("block_on_data", true)
));
}
createIndex("test-idx-1", "test-idx-2", "test-idx-3");
ensureGreen();
@ -1843,25 +1855,61 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
logger.info("--> snapshot allow partial {}", allowPartial);
ListenableActionFuture<CreateSnapshotResponse> future = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap")
.setIndices("test-idx-*").setWaitForCompletion(true).setPartial(allowPartial).execute();
.setIndices("test-idx-*").setWaitForCompletion(true).setPartial(allowPartial).execute();
logger.info("--> wait for block to kick in");
waitForBlock(internalCluster().getMasterName(), "test-repo", TimeValue.timeValueMinutes(1));
logger.info("--> delete some indices while snapshot is running");
client.admin().indices().prepareDelete("test-idx-1", "test-idx-2").get();
logger.info("--> unblock running master node");
unblockNode(internalCluster().getMasterName());
if (initBlocking) {
waitForBlock(internalCluster().getMasterName(), "test-repo", TimeValue.timeValueMinutes(1));
} else {
waitForBlockOnAnyDataNode("test-repo", TimeValue.timeValueMinutes(1));
}
if (allowPartial) {
// partial snapshots allow close / delete operations
if (randomBoolean()) {
logger.info("--> delete index while partial snapshot is running");
client.admin().indices().prepareDelete("test-idx-1").get();
} else {
logger.info("--> close index while partial snapshot is running");
client.admin().indices().prepareClose("test-idx-1").get();
}
} else {
// non-partial snapshots do not allow close / delete operations on indices where snapshot has not been completed
if (randomBoolean()) {
try {
logger.info("--> delete index while non-partial snapshot is running");
client.admin().indices().prepareDelete("test-idx-1").get();
fail("Expected deleting index to fail during snapshot");
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("Cannot delete indices that are being snapshotted: [test-idx-1]"));
}
} else {
try {
logger.info("--> close index while non-partial snapshot is running");
client.admin().indices().prepareClose("test-idx-1").get();
fail("Expected closing index to fail during snapshot");
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("Cannot close indices that are being snapshotted: [test-idx-1]"));
}
}
}
if (initBlocking) {
logger.info("--> unblock running master node");
unblockNode(internalCluster().getMasterName());
} else {
logger.info("--> unblock all data nodes");
unblockAllDataNodes("test-repo");
}
logger.info("--> waiting for snapshot to finish");
CreateSnapshotResponse createSnapshotResponse = future.get();
if (allowPartial) {
logger.info("Deleted index during snapshot, but allow partial");
logger.info("Deleted/Closed index during snapshot, but allow partial");
assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo((SnapshotState.PARTIAL)));
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
assertThat(createSnapshotResponse.getSnapshotInfo().failedShards(), greaterThan(0));
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), lessThan(createSnapshotResponse.getSnapshotInfo().totalShards()));
} else {
logger.info("Deleted index during snapshot and doesn't allow partial");
assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo((SnapshotState.FAILED)));
logger.info("Snapshot successfully completed");
assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo((SnapshotState.SUCCESS)));
}
}
@ -1960,7 +2008,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
shards.put(new ShardId("test-idx", "_na_", 1), new ShardSnapshotStatus("unknown-node", State.ABORTED));
shards.put(new ShardId("test-idx", "_na_", 2), new ShardSnapshotStatus("unknown-node", State.ABORTED));
List<Entry> entries = new ArrayList<>();
entries.add(new Entry(new SnapshotId("test-repo", "test-snap"), true, State.ABORTED, Collections.singletonList("test-idx"), System.currentTimeMillis(), shards.build()));
entries.add(new Entry(new SnapshotId("test-repo", "test-snap"), true, false, State.ABORTED, Collections.singletonList("test-idx"), System.currentTimeMillis(), shards.build()));
return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(Collections.unmodifiableList(entries))).build();
}

View File

@ -80,7 +80,7 @@ public class MockLogAppender extends AppenderSkeleton {
protected final String logger;
protected final Level level;
protected final String message;
protected boolean saw;
volatile boolean saw;
public AbstractEventExpectation(String name, String logger, Level level, String message) {
this.name = name;

View File

@ -19,6 +19,7 @@
package org.elasticsearch.transport.netty;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.util.concurrent.KeyedLock;
import org.elasticsearch.test.ESTestCase;
import org.hamcrest.Matchers;
@ -29,9 +30,7 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.not;
public class KeyedLockTests extends ESTestCase {
@ -68,28 +67,6 @@ public class KeyedLockTests extends ESTestCase {
}
}
public void testCannotAcquireTwoLocks() throws InterruptedException {
KeyedLock<String> connectionLock = new KeyedLock<String>();
String name = randomRealisticUnicodeOfLength(scaledRandomIntBetween(10, 50));
connectionLock.acquire(name);
try {
connectionLock.acquire(name);
fail("Expected IllegalStateException");
} catch (IllegalStateException e) {
assertThat(e.getMessage(), containsString("Lock already acquired"));
}
}
public void testCannotReleaseUnacquiredLock() throws InterruptedException {
KeyedLock<String> connectionLock = new KeyedLock<String>();
String name = randomRealisticUnicodeOfLength(scaledRandomIntBetween(10, 50));
try {
connectionLock.release(name);
fail("Expected IllegalStateException");
} catch (IllegalStateException e) {
assertThat(e.getMessage(), is("Lock not acquired"));
}
}
public static class AcquireAndReleaseThread extends Thread {
private CountDownLatch startLatch;
@ -117,16 +94,16 @@ public class KeyedLockTests extends ESTestCase {
int numRuns = scaledRandomIntBetween(5000, 50000);
for (int i = 0; i < numRuns; i++) {
String curName = names[randomInt(names.length - 1)];
connectionLock.acquire(curName);
try {
assert connectionLock.isHeldByCurrentThread(curName) == false;
try (Releasable ignored = connectionLock.acquire(curName)) {
assert connectionLock.isHeldByCurrentThread(curName);
assert connectionLock.isHeldByCurrentThread(curName + "bla") == false;
Integer integer = counter.get(curName);
if (integer == null) {
counter.put(curName, 1);
} else {
counter.put(curName, integer.intValue() + 1);
}
} finally {
connectionLock.release(curName);
}
AtomicInteger atomicInteger = new AtomicInteger(0);
AtomicInteger value = safeCounter.putIfAbsent(curName, atomicInteger);

View File

@ -29,7 +29,7 @@ my $Issue_URL = "http://github.com/${User_Repo}issues/";
my @Groups = qw(
breaking deprecation feature
enhancement bug regression upgrade build doc test
enhancement bug regression upgrade non-issue build docs test
);
my %Group_Labels = (
breaking => 'Breaking changes',
@ -42,6 +42,7 @@ my %Group_Labels = (
regression => 'Regressions',
test => 'Tests',
upgrade => 'Upgrades',
"non-issue" => 'Non-issue',
other => 'NOT CLASSIFIED',
);
@ -157,6 +158,8 @@ sub fetch_issues {
ISSUE:
for my $issue (@issues) {
next if $seen{ $issue->{number} } && !$issue->{pull_request};
# uncomment for including/excluding PRs already issued in other versions
# next if grep {$_->{name}=~/^v2/} @{$issue->{labels}};
my %labels = map { $_->{name} => 1 } @{ $issue->{labels} };
my ($header) = map { substr( $_, 1 ) } grep {/^:/} keys %labels;
$header ||= 'NOT CLASSIFIED';

View File

@ -107,6 +107,7 @@ export ES_DIRECT_SIZE
export ES_JAVA_OPTS
export ES_GC_LOG_FILE
export JAVA_HOME
export ES_INCLUDE
# Check DAEMON exists
test -x $DAEMON || exit 0

View File

@ -66,6 +66,7 @@ export ES_JAVA_OPTS
export ES_GC_LOG_FILE
export ES_STARTUP_SLEEP_TIME
export JAVA_HOME
export ES_INCLUDE
lockfile=/var/lock/subsys/$prog

View File

@ -85,6 +85,10 @@ REM JAVA_OPTS=%JAVA_OPTS% -XX:HeapDumpPath=$ES_HOME/logs/heapdump.hprof
REM Disables explicit GC
set JAVA_OPTS=%JAVA_OPTS% -XX:+DisableExplicitGC
REM Enable pre-touching of memory pages used by the JVM during hotspot
REM initialization
set JAVA_OPTS=%JAVA_OPTS% -XX:+AlwaysPreTouch
REM Ensure UTF-8 encoding by default (e.g. filenames)
set JAVA_OPTS=%JAVA_OPTS% -Dfile.encoding=UTF-8

View File

@ -81,6 +81,10 @@ JAVA_OPTS="$JAVA_OPTS -XX:+HeapDumpOnOutOfMemoryError"
# Disables explicit GC
JAVA_OPTS="$JAVA_OPTS -XX:+DisableExplicitGC"
# Enable pre-touching of memory pages used by the JVM during hotspot
# initialization
JAVA_OPTS="$JAVA_OPTS -XX:+AlwaysPreTouch"
# Ensure UTF-8 encoding by default (e.g. filenames)
JAVA_OPTS="$JAVA_OPTS -Dfile.encoding=UTF-8"

View File

@ -12,12 +12,16 @@ Obtaining an elasticsearch `Client` is simple. The most common way to
get a client is by creating a <<transport-client,`TransportClient`>>
that connects to a cluster.
*Important:*
______________________________________________________________________________________________________________________________________________________________
Please note that you are encouraged to use the same version on client
and cluster sides. You may hit some incompatibility issues when mixing
major versions.
______________________________________________________________________________________________________________________________________________________________
[IMPORTANT]
==============================
The client must have the same major version (e.g. `2.x`, or `5.x`) as the
nodes in the cluster. Clients may connect to clusters which have a different
minor version (e.g. `2.3.x`) but it is possible that new funcionality may not
be supported. Ideally, the client should have the same version as the
cluster.
==============================
[[transport-client]]
@ -53,11 +57,23 @@ Client client = TransportClient.builder().settings(settings).build();
//Add transport addresses and do something with the client...
--------------------------------------------------
The client allows sniffing the rest of the cluster, which adds data nodes
into its list of machines to use. In this case, note that the IP addresses
used will be the ones that the other nodes were started with (the
"publish" address). In order to enable it, set the
`client.transport.sniff` to `true`:
The Transport client comes with a cluster sniffing feature which
allows it to dynamically add new hosts and remove old ones.
When sniffing is enabled the the transport client will connect to the nodes in its
internal node list, which is built via calls to addTransportAddress.
After this, the client will call the internal cluster state API on those nodes
to discover available data nodes. The internal node list of the client will
be replaced with those data nodes only. This list is refreshed every five seconds by default.
Note that the IP addresses the sniffer connects to are the ones declared as the 'publish'
address in those node's elasticsearch config.
Keep in mind that list might possibly not include the original node it connected to
if that node is not a data node. If, for instance, you initially connect to a
master node, after sniffing no further requests will go to that master node,
but rather to any data nodes instead. The reason the transport excludes non-data
nodes is to avoid sending search traffic to master only nodes.
In order to enable sniffing, set `client.transport.sniff` to `true`:
[source,java]
--------------------------------------------------

View File

@ -176,7 +176,7 @@ need to specify the `type` (like `string` or `date`) since it is already known.
[[mapper-attachments-copy-to]]
==== Copy To feature
If you want to use http://www.elasticsearch.org/guide/en/elasticsearch/reference/current/mapping-core-types.html#copy-to[copy_to]
If you want to use https://www.elastic.co/guide/en/elasticsearch/reference/current/copy-to.html[copy_to]
feature, you need to define it on each sub-field you want to copy to another field:
[source,js]

View File

@ -60,6 +60,9 @@ of `indices`, `os`, `process`, `jvm`, `transport`, `http`,
`discovery`::
Statistics about the discovery
`ingest`::
Statistics about ingest preprocessing
[source,js]
--------------------------------------------------
# return indices and os
@ -227,3 +230,23 @@ curl -XGET 'http://localhost:9200/_nodes/stats?pretty&groups=_all'
# Some groups from just the indices stats
curl -XGET 'http://localhost:9200/_nodes/stats/indices?pretty&groups=foo,bar'
--------------------------------------------------
[float]
[[ingest-stats]]
=== Ingest statistics
The `ingest` flag can be set to retrieve statistics that concern ingest:
`ingest.total.count`::
The total number of document ingested during the lifetime of this node
`ingest.total.time_in_millis`::
The total time spent on ingest preprocessing documents during the lifetime of this node
`ingest.total.current`::
The total number of documents currently being ingested.
`ingest.total.failed`::
The total number ingest preprocessing operations failed during the lifetime of this node
On top of these overall ingest statistics, these statistics are also provided on a per pipeline basis.

View File

@ -83,6 +83,16 @@ Cluster wide settings can be returned using:
curl -XGET localhost:9200/_cluster/settings
--------------------------------------------------
[float]
=== Precedence of settings
Transient cluster settings take precedence over persistent cluster settings,
which take precedence over settings configured in the `elasticsearch.yml`
config file.
For this reason it is preferrable to use the `elasticsearch.yml` file only
for local configurations, and set all cluster-wider settings with the
`settings` API.
A list of dynamically updatable settings can be found in the
<<modules,Modules>> documentation.

View File

@ -251,5 +251,15 @@ sure the document doesn't change during the update. You can use the `version`
parameter to specify that the document should only be updated if its version
matches the one specified. By setting version type to `force` you can force
the new version of the document after update (use with care! with `force`
there is no guarantee the document didn't change).Version types `external` &
`external_gte` are not supported.
there is no guarantee the document didn't change).
[NOTE]
.The update API does not support external versioning
=====================================================
External versioning (version types `external` & `external_gte`) is not
supported by the update API as it would result in Elasticsearch version
numbers being out of sync with the external system. Use the
<<docs-index_,`index` API>> instead.
=====================================================

View File

@ -634,6 +634,9 @@ plugin.mandatory: ingest-attachment,ingest-geoip
A node will not start if either of these plugins are not available.
The <<ingest-stats,node stats API>> can be used to fetch ingest usage statistics, globally and on a per
pipeline basis. Useful to find out which pipelines are used the most or spent the most time on preprocessing.
[[append-procesesor]]
=== Append Processor
Appends one or more values to an existing array if the field already exists and it is an array.

View File

@ -1,7 +1,7 @@
[[mapping-store]]
=== `store`
By default, field values <<mapping-index,indexed>> to make them searchable,
By default, field values are <<mapping-index,indexed>> to make them searchable,
but they are not _stored_. This means that the field can be queried, but the
original field value cannot be retrieved.

View File

@ -10,7 +10,7 @@ your application to Elasticsearch 2.2.
==== Geo Point Type
The `geo_point` format has been changed to reduce index size and the time required to both index and query
geo point data. To make these performance improvements possible both `doc_values` are `coerce` are required
geo point data. To make these performance improvements possible both `doc_values` and `coerce` are required
and therefore cannot be changed. For this reason the `doc_values` and `coerce` parameters have been removed
from the <<geo-point, geo_point>> field mapping.
@ -43,6 +43,21 @@ changed to now route standard output to the journal and standard error
to inherit this setting (these are the defaults for systemd). These
settings can be modified by editing the `elasticsearch.service` file.
[float]
=== Java Client
Previously it was possible to iterate over `ClusterHealthResponse` to get information about `ClusterIndexHealth`.
While this is still possible, it requires now iterating over the values returned from `getIndices()`:
[source,java]
---------------
ClusterHealthResponse clusterHealthResponse = client.admin().cluster().prepareHealth().get();
for (Map.Entry<String, ClusterIndexHealth> index : clusterHealthResponse.getIndices().entrySet()) {
String indexName = index.getKey();
ClusterIndexHealth health = index.getValue();
}
---------------
[float]
=== Cloud AWS Plugin

View File

@ -21,6 +21,7 @@ your application to Elasticsearch 5.0.
* <<breaking_50_scripting>>
* <<breaking_50_term_vectors>>
* <<breaking_50_security>>
* <<breaking_50_snapshot_restore>>
[[breaking_50_search_changes]]
=== Warmers
@ -819,6 +820,15 @@ changed to now route standard output to the journal and standard error
to inherit this setting (these are the defaults for systemd). These
settings can be modified by editing the elasticsearch.service file.
==== Longer startup times
In Elasticsearch 5.0.0 the `-XX:+AlwaysPreTouch` flag has been added to the JVM
startup options. This option touches all memory pages used by the JVM heap
during initialization of the HotSpot VM to reduce the chance of having to commit
a memory page during GC time. This will increase the startup time of
Elasticsearch as well as increasing the initial resident memory usage of the
Java process.
[[breaking_50_scripting]]
=== Scripting
@ -855,3 +865,12 @@ distributed document frequencies anymore.
The option to disable the security manager `--security.manager.enabled` has been removed. In order to grant special
permissions to elasticsearch users must tweak the local Java Security Policy.
[[breaking_50_snapshot_restore]]
=== Snapshot/Restore
==== Closing / deleting indices while running snapshot
In previous versions of Elasticsearch, closing or deleting an index during a full snapshot would make the snapshot fail. This is now changed
by failing the close/delete index request instead. The behavior for partial snapshots remains unchanged: Closing or deleting an index during
a partial snapshot is still possible. The snapshot result is then marked as partial.

View File

@ -215,6 +215,22 @@ There are a number of options for the `field_value_factor` function:
`log1p`, `log2p`, `ln`, `ln1p`, `ln2p`, `square`, `sqrt`, or `reciprocal`.
Defaults to `none`.
[cols="<,<",options="header",]
|=======================================================================
| Modifier | Meaning
| `none` | Do not apply any multiplier to the field value
| `log` | Take the https://en.wikipedia.org/wiki/Logarithm[logarithm] of the field value
| `log1p` | Add 1 to the field value and take the logarithm
| `log2p` | Add 2 to the field value and take the logarithm
| `ln` | Take the https://en.wikipedia.org/wiki/Natural_logarithm[natural logarithm] of the field value
| `ln1p` | Add 1 to the field value and take the natural logarithm
| `ln2p` | Add 2 to the field value and take the natural logarithm
| `square` | Square the field value (multiply it by itself)
| `sqrt` | Take the https://en.wikipedia.org/wiki/Square_root[square root] of the field value
| `reciprocal` | https://en.wikipedia.org/wiki/Multiplicative_inverse[Reciprocate] the field value, same as `1/x` where `x` is the field's value
|=======================================================================
`missing`::
Value used if the document doesn't have that field. The modifier

View File

@ -185,10 +185,10 @@ values separately.
"filter" : {
"geo_bounding_box" : {
"pin.location" : {
"top" : -74.1,
"left" : 40.73,
"bottom" : -71.12,
"right" : 40.01
"top" : 40.73,
"left" : -74.1,
"bottom" : 40.01,
"right" : -71.12
}
}
}

View File

@ -35,7 +35,7 @@ inside the `has_child` query:
{
"has_child" : {
"type" : "blog_tag",
"score_mode" : "sum",
"score_mode" : "min",
"query" : {
"term" : {
"tag" : "something"
@ -57,7 +57,7 @@ a match:
{
"has_child" : {
"type" : "blog_tag",
"score_mode" : "sum",
"score_mode" : "min",
"min_children": 2, <1>
"max_children": 10, <1>
"query" : {

View File

@ -73,7 +73,6 @@ present in the index, the syntax is similar to <<docs-termvectors-artificial-doc
},
"tweet": "You got no idea what I'd... what I'd give to be invisible."
}
}
},
{
"_index" : "marvel",

View File

@ -47,6 +47,7 @@ import org.elasticsearch.threadpool.ThreadPool;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
@ -103,7 +104,8 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
protected abstract BulkRequest buildBulk(Iterable<SearchHit> docs);
protected abstract Response buildResponse(TimeValue took, List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures);
protected abstract Response buildResponse(TimeValue took, List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures,
boolean timedOut);
public void start() {
initialSearch();
@ -161,8 +163,13 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
return;
}
setScroll(searchResponse.getScrollId());
if (searchResponse.getShardFailures() != null && searchResponse.getShardFailures().length > 0) {
startNormalTermination(emptyList(), unmodifiableList(Arrays.asList(searchResponse.getShardFailures())));
if ( // If any of the shards failed that should abort the request.
(searchResponse.getShardFailures() != null && searchResponse.getShardFailures().length > 0)
// Timeouts aren't shard failures but we still need to pass them back to the user.
|| searchResponse.isTimedOut()
) {
startNormalTermination(emptyList(), unmodifiableList(Arrays.asList(searchResponse.getShardFailures())),
searchResponse.isTimedOut());
return;
}
long total = searchResponse.getHits().totalHits();
@ -176,7 +183,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
SearchHit[] docs = searchResponse.getHits().getHits();
logger.debug("scroll returned [{}] documents with a scroll id of [{}]", docs.length, searchResponse.getScrollId());
if (docs.length == 0) {
startNormalTermination(emptyList(), emptyList());
startNormalTermination(emptyList(), emptyList(), false);
return;
}
task.countBatch();
@ -261,18 +268,18 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
throw new IllegalArgumentException("Unknown op type: " + item.getOpType());
}
// Track the indexes we've seen so we can refresh them if requested
destinationIndices.add(item.getIndex());
destinationIndicesThisBatch.add(item.getIndex());
}
destinationIndices.addAll(destinationIndicesThisBatch);
addDestinationIndices(destinationIndicesThisBatch);
if (false == failures.isEmpty()) {
startNormalTermination(unmodifiableList(failures), emptyList());
startNormalTermination(unmodifiableList(failures), emptyList(), false);
return;
}
if (mainRequest.getSize() != SIZE_ALL_MATCHES && task.getSuccessfullyProcessed() >= mainRequest.getSize()) {
// We've processed all the requested docs.
startNormalTermination(emptyList(), emptyList());
startNormalTermination(emptyList(), emptyList(), false);
return;
}
startNextScroll();
@ -311,9 +318,9 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
failures.add(failure);
}
void startNormalTermination(List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures) {
if (false == mainRequest.isRefresh()) {
finishHim(null, indexingFailures, searchFailures);
void startNormalTermination(List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures, boolean timedOut) {
if (task.isCancelled() || false == mainRequest.isRefresh()) {
finishHim(null, indexingFailures, searchFailures, timedOut);
return;
}
RefreshRequest refresh = new RefreshRequest();
@ -321,7 +328,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
client.admin().indices().refresh(refresh, new ActionListener<RefreshResponse>() {
@Override
public void onResponse(RefreshResponse response) {
finishHim(null, indexingFailures, searchFailures);
finishHim(null, indexingFailures, searchFailures, timedOut);
}
@Override
@ -337,7 +344,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
* @param failure if non null then the request failed catastrophically with this exception
*/
void finishHim(Throwable failure) {
finishHim(failure, emptyList(), emptyList());
finishHim(failure, emptyList(), emptyList(), false);
}
/**
@ -346,8 +353,9 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
* @param failure if non null then the request failed catastrophically with this exception
* @param indexingFailures any indexing failures accumulated during the request
* @param searchFailures any search failures accumulated during the request
* @param timedOut have any of the sub-requests timed out?
*/
void finishHim(Throwable failure, List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures) {
void finishHim(Throwable failure, List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures, boolean timedOut) {
String scrollId = scroll.get();
if (Strings.hasLength(scrollId)) {
/*
@ -369,7 +377,8 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
});
}
if (failure == null) {
listener.onResponse(buildResponse(timeValueNanos(System.nanoTime() - startTime.get()), indexingFailures, searchFailures));
listener.onResponse(
buildResponse(timeValueNanos(System.nanoTime() - startTime.get()), indexingFailures, searchFailures, timedOut));
} else {
listener.onFailure(failure);
}
@ -382,6 +391,14 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
return exponentialBackoff(mainRequest.getRetryBackoffInitialTime(), mainRequest.getMaxRetries());
}
/**
* Add to the list of indices that were modified by this request. This is the list of indices refreshed at the end of the request if the
* request asks for a refresh.
*/
void addDestinationIndices(Collection<String> indices) {
destinationIndices.addAll(indices);
}
/**
* Wraps a backoffPolicy in another policy that counts the number of backoffs acquired.
*/

View File

@ -45,16 +45,18 @@ public class BulkIndexByScrollResponse extends ActionResponse implements ToXCont
private BulkByScrollTask.Status status;
private List<Failure> indexingFailures;
private List<ShardSearchFailure> searchFailures;
private boolean timedOut;
public BulkIndexByScrollResponse() {
}
public BulkIndexByScrollResponse(TimeValue took, BulkByScrollTask.Status status, List<Failure> indexingFailures,
List<ShardSearchFailure> searchFailures) {
List<ShardSearchFailure> searchFailures, boolean timedOut) {
this.took = took;
this.status = requireNonNull(status, "Null status not supported");
this.indexingFailures = indexingFailures;
this.searchFailures = searchFailures;
this.timedOut = timedOut;
}
public TimeValue getTook() {
@ -103,6 +105,13 @@ public class BulkIndexByScrollResponse extends ActionResponse implements ToXCont
return searchFailures;
}
/**
* Did any of the sub-requests that were part of this request timeout?
*/
public boolean isTimedOut() {
return timedOut;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
@ -116,6 +125,7 @@ public class BulkIndexByScrollResponse extends ActionResponse implements ToXCont
for (ShardSearchFailure failure: searchFailures) {
failure.writeTo(out);
}
out.writeBoolean(timedOut);
}
@Override
@ -135,11 +145,13 @@ public class BulkIndexByScrollResponse extends ActionResponse implements ToXCont
searchFailures.add(readShardSearchFailure(in));
}
this.searchFailures = unmodifiableList(searchFailures);
this.timedOut = in.readBoolean();
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.field("took", took.millis());
builder.field("timed_out", timedOut);
status.innerXContent(builder, params, false, false);
builder.startArray("failures");
for (Failure failure: indexingFailures) {

View File

@ -19,7 +19,9 @@
package org.elasticsearch.index.reindex;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.rest.RestChannel;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.rest.action.support.RestToXContentListener;
@ -35,12 +37,25 @@ public class BulkIndexByScrollResponseContentListener<R extends BulkIndexByScrol
@Override
protected RestStatus getStatus(R response) {
/*
* Return the highest numbered rest status under the assumption that higher numbered statuses are "more error" and thus more
* interesting to the user.
*/
RestStatus status = RestStatus.OK;
if (response.isTimedOut()) {
status = RestStatus.REQUEST_TIMEOUT;
}
for (Failure failure : response.getIndexingFailures()) {
if (failure.getStatus().getStatus() > status.getStatus()) {
status = failure.getStatus();
}
}
for (ShardSearchFailure failure: response.getSearchFailures()) {
RestStatus failureStatus = ExceptionsHelper.status(failure.getCause());
if (failureStatus.getStatus() > status.getStatus()) {
status = failureStatus;
}
}
return status;
}
}

View File

@ -35,8 +35,9 @@ public class ReindexResponse extends BulkIndexByScrollResponse {
public ReindexResponse() {
}
public ReindexResponse(TimeValue took, Status status, List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures) {
super(took, status, indexingFailures, searchFailures);
public ReindexResponse(TimeValue took, Status status, List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures,
boolean timedOut) {
super(took, status, indexingFailures, searchFailures, timedOut);
}
public long getCreated() {
@ -46,6 +47,7 @@ public class ReindexResponse extends BulkIndexByScrollResponse {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.field("took", getTook());
builder.field("timed_out", isTimedOut());
getStatus().innerXContent(builder, params, true, false);
builder.startArray("failures");
for (Failure failure: getIndexingFailures()) {

View File

@ -107,7 +107,10 @@ public class RestUpdateByQueryAction extends
internalRequest.setSize(internalRequest.getSearchRequest().source().size());
internalRequest.setPipeline(request.param("pipeline"));
internalRequest.getSearchRequest().source().size(request.paramAsInt("scroll_size", scrollSize));
// Let the requester set search timeout. It is probably only going to be useful for testing but who knows.
if (request.hasParam("search_timeout")) {
internalRequest.getSearchRequest().source().timeout(request.paramAsTime("search_timeout", null));
}
execute(request, internalRequest, channel);
}

View File

@ -191,8 +191,9 @@ public class TransportReindexAction extends HandledTransportAction<ReindexReques
}
@Override
protected ReindexResponse buildResponse(TimeValue took, List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures) {
return new ReindexResponse(took, task.getStatus(), indexingFailures, searchFailures);
protected ReindexResponse buildResponse(TimeValue took, List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures,
boolean timedOut) {
return new ReindexResponse(took, task.getStatus(), indexingFailures, searchFailures, timedOut);
}
/*

View File

@ -96,8 +96,8 @@ public class TransportUpdateByQueryAction extends HandledTransportAction<UpdateB
@Override
protected BulkIndexByScrollResponse buildResponse(TimeValue took, List<Failure> indexingFailures,
List<ShardSearchFailure> searchFailures) {
return new BulkIndexByScrollResponse(took, task.getStatus(), indexingFailures, searchFailures);
List<ShardSearchFailure> searchFailures, boolean timedOut) {
return new BulkIndexByScrollResponse(took, task.getStatus(), indexingFailures, searchFailures, timedOut);
}
@Override

View File

@ -25,6 +25,7 @@ import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.bulk.BackoffPolicy;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
@ -74,10 +75,12 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptyMap;
import static java.util.Collections.singleton;
import static org.apache.lucene.util.TestUtil.randomSimpleString;
import static org.elasticsearch.action.bulk.BackoffPolicy.constantBackoff;
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
@ -248,15 +251,33 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
*/
public void testShardFailuresAbortRequest() throws Exception {
ShardSearchFailure shardFailure = new ShardSearchFailure(new RuntimeException("test"));
new DummyAbstractAsyncBulkByScrollAction()
.onScrollResponse(new SearchResponse(null, scrollId(), 5, 4, randomLong(), new ShardSearchFailure[] { shardFailure }));
InternalSearchResponse internalResponse = new InternalSearchResponse(null, null, null, null, false, null);
new DummyAbstractAsyncBulkByScrollAction().onScrollResponse(
new SearchResponse(internalResponse, scrollId(), 5, 4, randomLong(), new ShardSearchFailure[] { shardFailure }));
BulkIndexByScrollResponse response = listener.get();
assertThat(response.getIndexingFailures(), emptyCollectionOf(Failure.class));
assertThat(response.getSearchFailures(), contains(shardFailure));
assertFalse(response.isTimedOut());
assertNull(response.getReasonCancelled());
assertThat(client.scrollsCleared, contains(scrollId));
}
/**
* Mimicks search timeouts.
*/
public void testSearchTimeoutsAbortRequest() throws Exception {
InternalSearchResponse internalResponse = new InternalSearchResponse(null, null, null, null, true, null);
new DummyAbstractAsyncBulkByScrollAction()
.onScrollResponse(new SearchResponse(internalResponse, scrollId(), 5, 4, randomLong(), new ShardSearchFailure[0]));
BulkIndexByScrollResponse response = listener.get();
assertThat(response.getIndexingFailures(), emptyCollectionOf(Failure.class));
assertThat(response.getSearchFailures(), emptyCollectionOf(ShardSearchFailure.class));
assertTrue(response.isTimedOut());
assertNull(response.getReasonCancelled());
assertThat(client.scrollsCleared, contains(scrollId));
}
/**
* Mimicks bulk indexing failures.
*/
@ -370,6 +391,32 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
assertEquals(defaultBackoffBeforeFailing, millis);
}
public void testRefreshIsFalseByDefault() throws Exception {
refreshTestCase(null, false);
}
public void testRefreshFalseDoesntMakeVisible() throws Exception {
refreshTestCase(false, false);
}
public void testRefreshTrueMakesVisible() throws Exception {
refreshTestCase(true, true);
}
private void refreshTestCase(Boolean refresh, boolean shouldRefresh) {
if (refresh != null) {
mainRequest.setRefresh(refresh);
}
DummyAbstractAsyncBulkByScrollAction action = new DummyAbstractAsyncBulkByScrollAction();
action.addDestinationIndices(singleton("foo"));
action.startNormalTermination(emptyList(), emptyList(), false);
if (shouldRefresh) {
assertArrayEquals(new String[] {"foo"}, client.lastRefreshRequest.get().indices());
} else {
assertNull("No refresh was attempted", client.lastRefreshRequest.get());
}
}
public void testCancelBeforeInitialSearch() throws Exception {
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.initialSearch());
}
@ -396,8 +443,8 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
public void testCancelBeforeStartNormalTermination() throws Exception {
// Refresh or not doesn't matter - we don't try to refresh.
mainRequest.setRefresh(usually());
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.startNormalTermination(emptyList(), emptyList()));
// This wouldn't return if we called refresh - the action would hang waiting for the refresh that we haven't mocked.
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.startNormalTermination(emptyList(), emptyList(), false));
assertNull("No refresh was attempted", client.lastRefreshRequest.get());
}
private void cancelTaskCase(Consumer<DummyAbstractAsyncBulkByScrollAction> testMe) throws Exception {
@ -430,8 +477,8 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
@Override
protected BulkIndexByScrollResponse buildResponse(TimeValue took, List<Failure> indexingFailures,
List<ShardSearchFailure> searchFailures) {
return new BulkIndexByScrollResponse(took, task.getStatus(), indexingFailures, searchFailures);
List<ShardSearchFailure> searchFailures, boolean timedOut) {
return new BulkIndexByScrollResponse(took, task.getStatus(), indexingFailures, searchFailures, timedOut);
}
}
@ -445,6 +492,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
private static class MyMockClient extends FilterClient {
private final List<String> scrollsCleared = new ArrayList<>();
private final AtomicInteger bulksAttempts = new AtomicInteger();
private final AtomicReference<RefreshRequest> lastRefreshRequest = new AtomicReference<>();
private int bulksToReject = 0;
@ -457,6 +505,11 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
protected <Request extends ActionRequest<Request>, Response extends ActionResponse,
RequestBuilder extends ActionRequestBuilder<Request, Response, RequestBuilder>> void doExecute(
Action<Request, Response, RequestBuilder> action, Request request, ActionListener<Response> listener) {
if (request instanceof RefreshRequest) {
lastRefreshRequest.set((RefreshRequest) request);
listener.onResponse(null);
return;
}
if (request instanceof ClearScrollRequest) {
ClearScrollRequest clearScroll = (ClearScrollRequest) request;
scrollsCleared.addAll(clearScroll.getScrollIds());

View File

@ -19,14 +19,12 @@
package org.elasticsearch.index.reindex;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.index.IndexRequestBuilder;
import java.util.ArrayList;
import java.util.List;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
public class ReindexBasicTests extends ReindexTestCase {
@ -84,40 +82,4 @@ public class ReindexBasicTests extends ReindexTestCase {
assertThat(copy.get(), responseMatcher().created(half).batches(half, 5));
assertHitCount(client().prepareSearch("dest").setTypes("half").setSize(0).get(), half);
}
public void testRefreshIsFalseByDefault() throws Exception {
refreshTestCase(null, false);
}
public void testRefreshFalseDoesntMakeVisible() throws Exception {
refreshTestCase(false, false);
}
public void testRefreshTrueMakesVisible() throws Exception {
refreshTestCase(true, true);
}
/**
* Executes a reindex into an index with -1 refresh_interval and checks that
* the documents are visible properly.
*/
private void refreshTestCase(Boolean refresh, boolean visible) throws Exception {
CreateIndexRequestBuilder create = client().admin().indices().prepareCreate("dest").setSettings("refresh_interval", -1);
assertAcked(create);
ensureYellow();
indexRandom(true, client().prepareIndex("source", "test", "1").setSource("foo", "a"),
client().prepareIndex("source", "test", "2").setSource("foo", "a"),
client().prepareIndex("source", "test", "3").setSource("foo", "b"),
client().prepareIndex("source", "test", "4").setSource("foo", "c"));
assertHitCount(client().prepareSearch("source").setSize(0).get(), 4);
// Copy all the docs
ReindexRequestBuilder copy = reindex().source("source").destination("dest", "all");
if (refresh != null) {
copy.refresh(refresh);
}
assertThat(copy.get(), responseMatcher().created(4));
assertHitCount(client().prepareSearch("dest").setTypes("all").setSize(0).get(), visible ? 4 : 0);
}
}

View File

@ -102,7 +102,7 @@ public class RoundTripTests extends ESTestCase {
public void testReindexResponse() throws IOException {
ReindexResponse response = new ReindexResponse(timeValueMillis(randomPositiveLong()), randomStatus(), randomIndexingFailures(),
randomSearchFailures());
randomSearchFailures(), randomBoolean());
ReindexResponse tripped = new ReindexResponse();
roundTrip(response, tripped);
assertResponseEquals(response, tripped);
@ -110,7 +110,7 @@ public class RoundTripTests extends ESTestCase {
public void testBulkIndexByScrollResponse() throws IOException {
BulkIndexByScrollResponse response = new BulkIndexByScrollResponse(timeValueMillis(randomPositiveLong()), randomStatus(),
randomIndexingFailures(), randomSearchFailures());
randomIndexingFailures(), randomSearchFailures(), randomBoolean());
BulkIndexByScrollResponse tripped = new BulkIndexByScrollResponse();
roundTrip(response, tripped);
assertResponseEquals(response, tripped);

View File

@ -19,12 +19,9 @@
package org.elasticsearch.index.reindex;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.search.sort.SortOrder;
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
public class UpdateByQueryBasicTests extends UpdateByQueryTestCase {
@ -64,44 +61,4 @@ public class UpdateByQueryBasicTests extends UpdateByQueryTestCase {
assertEquals(3, client().prepareGet("test", "test", "3").get().getVersion());
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
}
public void testRefreshIsFalseByDefault() throws Exception {
refreshTestCase(null, false);
}
public void testRefreshFalseDoesntMakeVisible() throws Exception {
refreshTestCase(false, false);
}
public void testRefreshTrueMakesVisible() throws Exception {
refreshTestCase(true, true);
}
/**
* Executes an update_by_query on an index with -1 refresh_interval and
* checks that the documents are visible properly.
*/
private void refreshTestCase(Boolean refresh, boolean visible) throws Exception {
CreateIndexRequestBuilder create = client().admin().indices().prepareCreate("test").setSettings("refresh_interval", -1);
create.addMapping("test", "{\"dynamic\": \"false\"}");
assertAcked(create);
ensureYellow();
indexRandom(true, client().prepareIndex("test", "test", "1").setSource("foo", "a"),
client().prepareIndex("test", "test", "2").setSource("foo", "a"),
client().prepareIndex("test", "test", "3").setSource("foo", "b"),
client().prepareIndex("test", "test", "4").setSource("foo", "c"));
assertHitCount(client().prepareSearch("test").setQuery(matchQuery("foo", "a")).setSize(0).get(), 0);
// Now make foo searchable
assertAcked(client().admin().indices().preparePutMapping("test").setType("test")
.setSource("{\"test\": {\"properties\":{\"foo\": {\"type\": \"text\"}}}}"));
UpdateByQueryRequestBuilder update = request().source("test");
if (refresh != null) {
update.refresh(refresh);
}
assertThat(update.get(), responseMatcher().updated(4));
assertHitCount(client().prepareSearch("test").setQuery(matchQuery("foo", "a")).setSize(0).get(), visible ? 2 : 0);
}
}

View File

@ -75,6 +75,7 @@
index: source
dest:
index: dest
- is_false: timed_out
- match: {task: '/.+:\d+/'}
- set: {task: task}
- is_false: updated
@ -125,7 +126,8 @@
- match: {failures.0.id: "1"}
- match: {failures.0.status: 409}
- match: {failures.0.cause.type: version_conflict_engine_exception}
- match: {failures.0.cause.reason: "[foo][1]: version conflict, document already exists (current version [1])"}
# Use a regex so we don't mind if the version isn't always 1. Sometimes it comes out 2.
- match: {failures.0.cause.reason: "/\\[foo\\]\\[1\\]:.version.conflict,.document.already.exists.\\(current.version.\\[\\d+\\]\\)/"}
- match: {failures.0.cause.shard: /\d+/}
- match: {failures.0.cause.index: dest}
- is_true: took

View File

@ -12,6 +12,7 @@
- do:
update-by-query:
index: test
- is_false: timed_out
- match: {updated: 1}
- match: {version_conflicts: 0}
- match: {batches: 1}
@ -86,7 +87,8 @@
- match: {failures.0.id: "1"}
- match: {failures.0.status: 409}
- match: {failures.0.cause.type: version_conflict_engine_exception}
- match: {failures.0.cause.reason: "[foo][1]: version conflict, current version [2] is different than the one provided [1]"}
# Use a regex so we don't mind if the current version isn't always 1. Sometimes it comes out 2.
- match: {failures.0.cause.reason: "/\\[foo\\]\\[1\\]:.version.conflict,.current.version.\\[\\d+\\].is.different.than.the.one.provided.\\[\\d+\\]/"}
- match: {failures.0.cause.shard: /\d+/}
- match: {failures.0.cause.index: test}
- is_true: took

View File

@ -47,7 +47,7 @@ public class GceDiscoveryPlugin extends Plugin {
static {
/*
* GCE's http client changes access levels because its silly and we
* can't allow that on any old stack stack so we pull it here, up front,
* can't allow that on any old stack so we pull it here, up front,
* so we can cleanly check the permissions for it. Without this changing
* the permission can fail if any part of core is on the stack because
* our plugin permissions don't allow core to "reach through" plugins to

View File

@ -137,6 +137,8 @@ final class TikaImpl {
perms.add(new SecurityPermission("putProviderProperty.BC"));
perms.add(new SecurityPermission("insertProvider"));
perms.add(new ReflectPermission("suppressAccessChecks"));
// xmlbeans, use by POI, needs to get the context classloader
perms.add(new RuntimePermission("getClassLoader"));
perms.setReadOnly();
return perms;
}

View File

@ -27,4 +27,6 @@ grant {
permission java.security.SecurityPermission "insertProvider";
// TODO: fix POI XWPF to not do this: https://bz.apache.org/bugzilla/show_bug.cgi?id=58597
permission java.lang.reflect.ReflectPermission "suppressAccessChecks";
// needed by xmlbeans, as part of POI for MS xml docs
permission java.lang.RuntimePermission "getClassLoader";
};

File diff suppressed because one or more lines are too long

View File

@ -137,6 +137,8 @@ final class TikaImpl {
perms.add(new SecurityPermission("putProviderProperty.BC"));
perms.add(new SecurityPermission("insertProvider"));
perms.add(new ReflectPermission("suppressAccessChecks"));
// xmlbeans, use by POI, needs to get the context classloader
perms.add(new RuntimePermission("getClassLoader"));
perms.setReadOnly();
return perms;
}

View File

@ -27,4 +27,6 @@ grant {
permission java.security.SecurityPermission "insertProvider";
// TODO: fix POI XWPF to not do this: https://bz.apache.org/bugzilla/show_bug.cgi?id=58597
permission java.lang.reflect.ReflectPermission "suppressAccessChecks";
// needed by xmlbeans, as part of POI for MS xml docs
permission java.lang.RuntimePermission "getClassLoader";
};

File diff suppressed because one or more lines are too long

Some files were not shown because too many files have changed in this diff Show More