Merge branch 'master' into feature-suggest-refactoring
This commit is contained in:
commit
daeffb149c
|
@ -31,6 +31,7 @@ import org.elasticsearch.discovery.DiscoveryStats;
|
||||||
import org.elasticsearch.http.HttpStats;
|
import org.elasticsearch.http.HttpStats;
|
||||||
import org.elasticsearch.indices.NodeIndicesStats;
|
import org.elasticsearch.indices.NodeIndicesStats;
|
||||||
import org.elasticsearch.indices.breaker.AllCircuitBreakerStats;
|
import org.elasticsearch.indices.breaker.AllCircuitBreakerStats;
|
||||||
|
import org.elasticsearch.ingest.IngestStats;
|
||||||
import org.elasticsearch.monitor.fs.FsInfo;
|
import org.elasticsearch.monitor.fs.FsInfo;
|
||||||
import org.elasticsearch.monitor.jvm.JvmStats;
|
import org.elasticsearch.monitor.jvm.JvmStats;
|
||||||
import org.elasticsearch.monitor.os.OsStats;
|
import org.elasticsearch.monitor.os.OsStats;
|
||||||
|
@ -81,6 +82,9 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
|
||||||
@Nullable
|
@Nullable
|
||||||
private DiscoveryStats discoveryStats;
|
private DiscoveryStats discoveryStats;
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
private IngestStats ingestStats;
|
||||||
|
|
||||||
NodeStats() {
|
NodeStats() {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -89,7 +93,8 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
|
||||||
@Nullable FsInfo fs, @Nullable TransportStats transport, @Nullable HttpStats http,
|
@Nullable FsInfo fs, @Nullable TransportStats transport, @Nullable HttpStats http,
|
||||||
@Nullable AllCircuitBreakerStats breaker,
|
@Nullable AllCircuitBreakerStats breaker,
|
||||||
@Nullable ScriptStats scriptStats,
|
@Nullable ScriptStats scriptStats,
|
||||||
@Nullable DiscoveryStats discoveryStats) {
|
@Nullable DiscoveryStats discoveryStats,
|
||||||
|
@Nullable IngestStats ingestStats) {
|
||||||
super(node);
|
super(node);
|
||||||
this.timestamp = timestamp;
|
this.timestamp = timestamp;
|
||||||
this.indices = indices;
|
this.indices = indices;
|
||||||
|
@ -103,6 +108,7 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
|
||||||
this.breaker = breaker;
|
this.breaker = breaker;
|
||||||
this.scriptStats = scriptStats;
|
this.scriptStats = scriptStats;
|
||||||
this.discoveryStats = discoveryStats;
|
this.discoveryStats = discoveryStats;
|
||||||
|
this.ingestStats = ingestStats;
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getTimestamp() {
|
public long getTimestamp() {
|
||||||
|
@ -187,6 +193,11 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
|
||||||
return this.discoveryStats;
|
return this.discoveryStats;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
public IngestStats getIngestStats() {
|
||||||
|
return ingestStats;
|
||||||
|
}
|
||||||
|
|
||||||
public static NodeStats readNodeStats(StreamInput in) throws IOException {
|
public static NodeStats readNodeStats(StreamInput in) throws IOException {
|
||||||
NodeStats nodeInfo = new NodeStats();
|
NodeStats nodeInfo = new NodeStats();
|
||||||
nodeInfo.readFrom(in);
|
nodeInfo.readFrom(in);
|
||||||
|
@ -224,7 +235,7 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
|
||||||
breaker = AllCircuitBreakerStats.readOptionalAllCircuitBreakerStats(in);
|
breaker = AllCircuitBreakerStats.readOptionalAllCircuitBreakerStats(in);
|
||||||
scriptStats = in.readOptionalStreamable(ScriptStats::new);
|
scriptStats = in.readOptionalStreamable(ScriptStats::new);
|
||||||
discoveryStats = in.readOptionalStreamable(() -> new DiscoveryStats(null));
|
discoveryStats = in.readOptionalStreamable(() -> new DiscoveryStats(null));
|
||||||
|
ingestStats = in.readOptionalWritable(IngestStats.PROTO);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -282,6 +293,7 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
|
||||||
out.writeOptionalStreamable(breaker);
|
out.writeOptionalStreamable(breaker);
|
||||||
out.writeOptionalStreamable(scriptStats);
|
out.writeOptionalStreamable(scriptStats);
|
||||||
out.writeOptionalStreamable(discoveryStats);
|
out.writeOptionalStreamable(discoveryStats);
|
||||||
|
out.writeOptionalWriteable(ingestStats);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -337,6 +349,10 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
|
||||||
getDiscoveryStats().toXContent(builder, params);
|
getDiscoveryStats().toXContent(builder, params);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (getIngestStats() != null) {
|
||||||
|
getIngestStats().toXContent(builder, params);
|
||||||
|
}
|
||||||
|
|
||||||
return builder;
|
return builder;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -42,6 +42,7 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
|
||||||
private boolean breaker;
|
private boolean breaker;
|
||||||
private boolean script;
|
private boolean script;
|
||||||
private boolean discovery;
|
private boolean discovery;
|
||||||
|
private boolean ingest;
|
||||||
|
|
||||||
public NodesStatsRequest() {
|
public NodesStatsRequest() {
|
||||||
}
|
}
|
||||||
|
@ -69,6 +70,7 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
|
||||||
this.breaker = true;
|
this.breaker = true;
|
||||||
this.script = true;
|
this.script = true;
|
||||||
this.discovery = true;
|
this.discovery = true;
|
||||||
|
this.ingest = true;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -87,6 +89,7 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
|
||||||
this.breaker = false;
|
this.breaker = false;
|
||||||
this.script = false;
|
this.script = false;
|
||||||
this.discovery = false;
|
this.discovery = false;
|
||||||
|
this.ingest = false;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -250,6 +253,17 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean ingest() {
|
||||||
|
return ingest;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Should ingest statistics be returned.
|
||||||
|
*/
|
||||||
|
public NodesStatsRequest ingest(boolean ingest) {
|
||||||
|
this.ingest = ingest;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void readFrom(StreamInput in) throws IOException {
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
@ -265,6 +279,7 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
|
||||||
breaker = in.readBoolean();
|
breaker = in.readBoolean();
|
||||||
script = in.readBoolean();
|
script = in.readBoolean();
|
||||||
discovery = in.readBoolean();
|
discovery = in.readBoolean();
|
||||||
|
ingest = in.readBoolean();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -281,6 +296,7 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
|
||||||
out.writeBoolean(breaker);
|
out.writeBoolean(breaker);
|
||||||
out.writeBoolean(script);
|
out.writeBoolean(script);
|
||||||
out.writeBoolean(discovery);
|
out.writeBoolean(discovery);
|
||||||
|
out.writeBoolean(ingest);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -137,4 +137,12 @@ public class NodesStatsRequestBuilder extends NodesOperationRequestBuilder<Nodes
|
||||||
request.discovery(discovery);
|
request.discovery(discovery);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Should ingest statistics be returned.
|
||||||
|
*/
|
||||||
|
public NodesStatsRequestBuilder ingest(boolean ingest) {
|
||||||
|
request.ingest(ingest);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -80,7 +80,8 @@ public class TransportNodesStatsAction extends TransportNodesAction<NodesStatsRe
|
||||||
protected NodeStats nodeOperation(NodeStatsRequest nodeStatsRequest) {
|
protected NodeStats nodeOperation(NodeStatsRequest nodeStatsRequest) {
|
||||||
NodesStatsRequest request = nodeStatsRequest.request;
|
NodesStatsRequest request = nodeStatsRequest.request;
|
||||||
return nodeService.stats(request.indices(), request.os(), request.process(), request.jvm(), request.threadPool(),
|
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
|
@Override
|
||||||
|
|
|
@ -99,7 +99,7 @@ public class TransportClusterStatsAction extends TransportNodesAction<ClusterSta
|
||||||
@Override
|
@Override
|
||||||
protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeRequest) {
|
protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeRequest) {
|
||||||
NodeInfo nodeInfo = nodeService.info(false, true, false, true, false, true, false, true, false);
|
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<>();
|
List<ShardStats> shardsStats = new ArrayList<>();
|
||||||
for (IndexService indexService : indicesService) {
|
for (IndexService indexService : indicesService) {
|
||||||
for (IndexShard indexShard : indexService) {
|
for (IndexShard indexShard : indexService) {
|
||||||
|
|
|
@ -112,7 +112,7 @@ public final class IngestActionFilter extends AbstractComponent implements Actio
|
||||||
logger.error("failed to execute pipeline for a bulk request", throwable);
|
logger.error("failed to execute pipeline for a bulk request", throwable);
|
||||||
listener.onFailure(throwable);
|
listener.onFailure(throwable);
|
||||||
} else {
|
} else {
|
||||||
long ingestTookInMillis = TimeUnit.MILLISECONDS.convert(System.nanoTime() - ingestStartTimeInNanos, TimeUnit.NANOSECONDS);
|
long ingestTookInMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - ingestStartTimeInNanos);
|
||||||
BulkRequest bulkRequest = bulkRequestModifier.getBulkRequest();
|
BulkRequest bulkRequest = bulkRequestModifier.getBulkRequest();
|
||||||
ActionListener<BulkResponse> actionListener = bulkRequestModifier.wrapActionListenerIfNeeded(ingestTookInMillis, listener);
|
ActionListener<BulkResponse> actionListener = bulkRequestModifier.wrapActionListenerIfNeeded(ingestTookInMillis, listener);
|
||||||
if (bulkRequest.requests().isEmpty()) {
|
if (bulkRequest.requests().isEmpty()) {
|
||||||
|
|
|
@ -136,6 +136,7 @@ public class ClusterModule extends AbstractModule {
|
||||||
bind(AllocationService.class).asEagerSingleton();
|
bind(AllocationService.class).asEagerSingleton();
|
||||||
bind(DiscoveryNodeService.class).asEagerSingleton();
|
bind(DiscoveryNodeService.class).asEagerSingleton();
|
||||||
bind(ClusterService.class).to(InternalClusterService.class).asEagerSingleton();
|
bind(ClusterService.class).to(InternalClusterService.class).asEagerSingleton();
|
||||||
|
bind(NodeConnectionsService.class).asEagerSingleton();
|
||||||
bind(OperationRouting.class).asEagerSingleton();
|
bind(OperationRouting.class).asEagerSingleton();
|
||||||
bind(MetaDataCreateIndexService.class).asEagerSingleton();
|
bind(MetaDataCreateIndexService.class).asEagerSingleton();
|
||||||
bind(MetaDataDeleteIndexService.class).asEagerSingleton();
|
bind(MetaDataDeleteIndexService.class).asEagerSingleton();
|
||||||
|
|
|
@ -26,7 +26,6 @@ import org.elasticsearch.cluster.service.PendingClusterTask;
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
import org.elasticsearch.common.component.LifecycleComponent;
|
import org.elasticsearch.common.component.LifecycleComponent;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.tasks.TaskManager;
|
|
||||||
|
|
||||||
import java.util.List;
|
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
|
* @return A zero time value if the queue is empty, otherwise the time value oldest task waiting in the queue
|
||||||
*/
|
*/
|
||||||
TimeValue getMaxTaskWaitTime();
|
TimeValue getMaxTaskWaitTime();
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns task manager created in the cluster service
|
|
||||||
*/
|
|
||||||
TaskManager getTaskManager();
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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() {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -69,15 +69,17 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
|
||||||
private final State state;
|
private final State state;
|
||||||
private final SnapshotId snapshotId;
|
private final SnapshotId snapshotId;
|
||||||
private final boolean includeGlobalState;
|
private final boolean includeGlobalState;
|
||||||
|
private final boolean partial;
|
||||||
private final ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards;
|
private final ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards;
|
||||||
private final List<String> indices;
|
private final List<String> indices;
|
||||||
private final ImmutableOpenMap<String, List<ShardId>> waitingIndices;
|
private final ImmutableOpenMap<String, List<ShardId>> waitingIndices;
|
||||||
private final long startTime;
|
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.state = state;
|
||||||
this.snapshotId = snapshotId;
|
this.snapshotId = snapshotId;
|
||||||
this.includeGlobalState = includeGlobalState;
|
this.includeGlobalState = includeGlobalState;
|
||||||
|
this.partial = partial;
|
||||||
this.indices = indices;
|
this.indices = indices;
|
||||||
this.startTime = startTime;
|
this.startTime = startTime;
|
||||||
if (shards == null) {
|
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) {
|
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) {
|
public Entry(Entry entry, ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards) {
|
||||||
|
@ -121,6 +123,10 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
|
||||||
return includeGlobalState;
|
return includeGlobalState;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean partial() {
|
||||||
|
return partial;
|
||||||
|
}
|
||||||
|
|
||||||
public long startTime() {
|
public long startTime() {
|
||||||
return startTime;
|
return startTime;
|
||||||
}
|
}
|
||||||
|
@ -133,6 +139,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
|
||||||
Entry entry = (Entry) o;
|
Entry entry = (Entry) o;
|
||||||
|
|
||||||
if (includeGlobalState != entry.includeGlobalState) return false;
|
if (includeGlobalState != entry.includeGlobalState) return false;
|
||||||
|
if (partial != entry.partial) return false;
|
||||||
if (startTime != entry.startTime) return false;
|
if (startTime != entry.startTime) return false;
|
||||||
if (!indices.equals(entry.indices)) return false;
|
if (!indices.equals(entry.indices)) return false;
|
||||||
if (!shards.equals(entry.shards)) 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();
|
int result = state.hashCode();
|
||||||
result = 31 * result + snapshotId.hashCode();
|
result = 31 * result + snapshotId.hashCode();
|
||||||
result = 31 * result + (includeGlobalState ? 1 : 0);
|
result = 31 * result + (includeGlobalState ? 1 : 0);
|
||||||
|
result = 31 * result + (partial ? 1 : 0);
|
||||||
result = 31 * result + shards.hashCode();
|
result = 31 * result + shards.hashCode();
|
||||||
result = 31 * result + indices.hashCode();
|
result = 31 * result + indices.hashCode();
|
||||||
result = 31 * result + waitingIndices.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++) {
|
for (int i = 0; i < entries.length; i++) {
|
||||||
SnapshotId snapshotId = SnapshotId.readSnapshotId(in);
|
SnapshotId snapshotId = SnapshotId.readSnapshotId(in);
|
||||||
boolean includeGlobalState = in.readBoolean();
|
boolean includeGlobalState = in.readBoolean();
|
||||||
|
boolean partial = in.readBoolean();
|
||||||
State state = State.fromValue(in.readByte());
|
State state = State.fromValue(in.readByte());
|
||||||
int indices = in.readVInt();
|
int indices = in.readVInt();
|
||||||
List<String> indexBuilder = new ArrayList<>();
|
List<String> indexBuilder = new ArrayList<>();
|
||||||
|
@ -375,7 +384,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
|
||||||
State shardState = State.fromValue(in.readByte());
|
State shardState = State.fromValue(in.readByte());
|
||||||
builder.put(shardId, new ShardSnapshotStatus(nodeId, shardState));
|
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);
|
return new SnapshotsInProgress(entries);
|
||||||
}
|
}
|
||||||
|
@ -386,6 +395,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
|
||||||
for (Entry entry : entries) {
|
for (Entry entry : entries) {
|
||||||
entry.snapshotId().writeTo(out);
|
entry.snapshotId().writeTo(out);
|
||||||
out.writeBoolean(entry.includeGlobalState());
|
out.writeBoolean(entry.includeGlobalState());
|
||||||
|
out.writeBoolean(entry.partial());
|
||||||
out.writeByte(entry.state().value());
|
out.writeByte(entry.state().value());
|
||||||
out.writeVInt(entry.indices().size());
|
out.writeVInt(entry.indices().size());
|
||||||
for (String index : entry.indices()) {
|
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 SNAPSHOTS = new XContentBuilderString("snapshots");
|
||||||
static final XContentBuilderString SNAPSHOT = new XContentBuilderString("snapshot");
|
static final XContentBuilderString SNAPSHOT = new XContentBuilderString("snapshot");
|
||||||
static final XContentBuilderString INCLUDE_GLOBAL_STATE = new XContentBuilderString("include_global_state");
|
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 STATE = new XContentBuilderString("state");
|
||||||
static final XContentBuilderString INDICES = new XContentBuilderString("indices");
|
static final XContentBuilderString INDICES = new XContentBuilderString("indices");
|
||||||
static final XContentBuilderString START_TIME_MILLIS = new XContentBuilderString("start_time_millis");
|
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.REPOSITORY, entry.snapshotId().getRepository());
|
||||||
builder.field(Fields.SNAPSHOT, entry.snapshotId().getSnapshot());
|
builder.field(Fields.SNAPSHOT, entry.snapshotId().getSnapshot());
|
||||||
builder.field(Fields.INCLUDE_GLOBAL_STATE, entry.includeGlobalState());
|
builder.field(Fields.INCLUDE_GLOBAL_STATE, entry.includeGlobalState());
|
||||||
|
builder.field(Fields.PARTIAL, entry.partial());
|
||||||
builder.field(Fields.STATE, entry.state());
|
builder.field(Fields.STATE, entry.state());
|
||||||
builder.startArray(Fields.INDICES);
|
builder.startArray(Fields.INDICES);
|
||||||
{
|
{
|
||||||
|
|
|
@ -34,11 +34,12 @@ import org.elasticsearch.common.inject.Inject;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.common.util.concurrent.FutureUtils;
|
import org.elasticsearch.common.util.concurrent.FutureUtils;
|
||||||
|
import org.elasticsearch.common.util.set.Sets;
|
||||||
import org.elasticsearch.index.IndexNotFoundException;
|
import org.elasticsearch.index.IndexNotFoundException;
|
||||||
|
import org.elasticsearch.snapshots.SnapshotsService;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Set;
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.concurrent.ScheduledFuture;
|
import java.util.concurrent.ScheduledFuture;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
@ -67,7 +68,7 @@ public class MetaDataDeleteIndexService extends AbstractComponent {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void deleteIndices(final Request request, final Listener userListener) {
|
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);
|
final DeleteIndexListener listener = new DeleteIndexListener(userListener);
|
||||||
|
|
||||||
clusterService.submitStateUpdateTask("delete-index " + indices, new ClusterStateUpdateTask(Priority.URGENT) {
|
clusterService.submitStateUpdateTask("delete-index " + indices, new ClusterStateUpdateTask(Priority.URGENT) {
|
||||||
|
@ -84,6 +85,9 @@ public class MetaDataDeleteIndexService extends AbstractComponent {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ClusterState execute(final ClusterState currentState) {
|
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());
|
RoutingTable.Builder routingTableBuilder = RoutingTable.builder(currentState.routingTable());
|
||||||
MetaData.Builder metaDataBuilder = MetaData.builder(currentState.metaData());
|
MetaData.Builder metaDataBuilder = MetaData.builder(currentState.metaData());
|
||||||
ClusterBlocks.Builder clusterBlocksBuilder = ClusterBlocks.builder().blocks(currentState.blocks());
|
ClusterBlocks.Builder clusterBlocksBuilder = ClusterBlocks.builder().blocks(currentState.blocks());
|
||||||
|
|
|
@ -19,14 +19,12 @@
|
||||||
|
|
||||||
package org.elasticsearch.cluster.metadata;
|
package org.elasticsearch.cluster.metadata;
|
||||||
|
|
||||||
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
|
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.admin.indices.close.CloseIndexClusterStateUpdateRequest;
|
import org.elasticsearch.action.admin.indices.close.CloseIndexClusterStateUpdateRequest;
|
||||||
import org.elasticsearch.action.admin.indices.open.OpenIndexClusterStateUpdateRequest;
|
import org.elasticsearch.action.admin.indices.open.OpenIndexClusterStateUpdateRequest;
|
||||||
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
|
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
|
||||||
import org.elasticsearch.cluster.ClusterService;
|
import org.elasticsearch.cluster.ClusterService;
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.RestoreInProgress;
|
|
||||||
import org.elasticsearch.cluster.ack.ClusterStateUpdateResponse;
|
import org.elasticsearch.cluster.ack.ClusterStateUpdateResponse;
|
||||||
import org.elasticsearch.cluster.block.ClusterBlock;
|
import org.elasticsearch.cluster.block.ClusterBlock;
|
||||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
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.inject.Inject;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.index.IndexNotFoundException;
|
import org.elasticsearch.index.IndexNotFoundException;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
|
||||||
import org.elasticsearch.rest.RestStatus;
|
import org.elasticsearch.rest.RestStatus;
|
||||||
|
import org.elasticsearch.snapshots.RestoreService;
|
||||||
|
import org.elasticsearch.snapshots.SnapshotsService;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
@ -99,27 +98,10 @@ public class MetaDataIndexStateService extends AbstractComponent {
|
||||||
return currentState;
|
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
|
// Check if index closing conflicts with any running restores
|
||||||
// is found as closing an index that is being restored makes the index unusable (it cannot be recovered).
|
RestoreService.checkIndexClosing(currentState, indicesToClose);
|
||||||
RestoreInProgress restore = currentState.custom(RestoreInProgress.TYPE);
|
// Check if index closing conflicts with any running snapshots
|
||||||
if (restore != null) {
|
SnapshotsService.checkIndexClosing(currentState, indicesToClose);
|
||||||
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);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
logger.info("closing indices [{}]", indicesAsString);
|
logger.info("closing indices [{}]", indicesAsString);
|
||||||
|
|
||||||
|
|
|
@ -19,24 +19,40 @@
|
||||||
|
|
||||||
package org.elasticsearch.cluster.node;
|
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.component.AbstractComponent;
|
||||||
import org.elasticsearch.common.inject.Inject;
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.common.settings.Setting;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.transport.TransportAddress;
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Random;
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class DiscoveryNodeService extends AbstractComponent {
|
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 List<CustomAttributesProvider> customAttributesProviders = new CopyOnWriteArrayList<>();
|
||||||
|
private final Version version;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public DiscoveryNodeService(Settings settings) {
|
public DiscoveryNodeService(Settings settings, Version version) {
|
||||||
super(settings);
|
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) {
|
public DiscoveryNodeService addCustomAttributeProvider(CustomAttributesProvider customAttributesProvider) {
|
||||||
|
@ -44,7 +60,7 @@ public class DiscoveryNodeService extends AbstractComponent {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Map<String, String> buildAttributes() {
|
public DiscoveryNode buildLocalNode(TransportAddress publishAddress) {
|
||||||
Map<String, String> attributes = new HashMap<>(settings.getByPrefix("node.").getAsMap());
|
Map<String, String> attributes = new HashMap<>(settings.getByPrefix("node.").getAsMap());
|
||||||
attributes.remove("name"); // name is extracted in other places
|
attributes.remove("name"); // name is extracted in other places
|
||||||
if (attributes.containsKey("client")) {
|
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();
|
Map<String, String> buildAttributes();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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> {
|
public static final class UnassignedShards implements Iterable<ShardRouting> {
|
||||||
|
|
||||||
private final RoutingNodes nodes;
|
private final RoutingNodes nodes;
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.elasticsearch.cluster.routing.allocation;
|
package org.elasticsearch.cluster.routing.allocation;
|
||||||
|
|
||||||
import com.carrotsearch.hppc.cursors.ObjectCursor;
|
import com.carrotsearch.hppc.cursors.ObjectCursor;
|
||||||
import org.apache.lucene.util.ArrayUtil;
|
|
||||||
import org.elasticsearch.cluster.ClusterInfoService;
|
import org.elasticsearch.cluster.ClusterInfoService;
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.health.ClusterHealthStatus;
|
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.RoutingTable;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.cluster.routing.UnassignedInfo;
|
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.command.AllocationCommands;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
|
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.component.AbstractComponent;
|
||||||
import org.elasticsearch.common.inject.Inject;
|
import org.elasticsearch.common.inject.Inject;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.gateway.GatewayAllocator;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
@ -63,14 +62,17 @@ import java.util.stream.Collectors;
|
||||||
public class AllocationService extends AbstractComponent {
|
public class AllocationService extends AbstractComponent {
|
||||||
|
|
||||||
private final AllocationDeciders allocationDeciders;
|
private final AllocationDeciders allocationDeciders;
|
||||||
|
private final GatewayAllocator gatewayAllocator;
|
||||||
|
private final ShardsAllocator shardsAllocator;
|
||||||
private final ClusterInfoService clusterInfoService;
|
private final ClusterInfoService clusterInfoService;
|
||||||
private final ShardsAllocators shardsAllocators;
|
|
||||||
|
|
||||||
@Inject
|
@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);
|
super(settings);
|
||||||
this.allocationDeciders = allocationDeciders;
|
this.allocationDeciders = allocationDeciders;
|
||||||
this.shardsAllocators = shardsAllocators;
|
this.gatewayAllocator = gatewayAllocator;
|
||||||
|
this.shardsAllocator = shardsAllocator;
|
||||||
this.clusterInfoService = clusterInfoService;
|
this.clusterInfoService = clusterInfoService;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -92,7 +94,7 @@ public class AllocationService extends AbstractComponent {
|
||||||
if (!changed) {
|
if (!changed) {
|
||||||
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
|
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
|
||||||
}
|
}
|
||||||
shardsAllocators.applyStartedShards(allocation);
|
gatewayAllocator.applyStartedShards(allocation);
|
||||||
if (withReroute) {
|
if (withReroute) {
|
||||||
reroute(allocation);
|
reroute(allocation);
|
||||||
}
|
}
|
||||||
|
@ -192,7 +194,7 @@ public class AllocationService extends AbstractComponent {
|
||||||
if (!changed) {
|
if (!changed) {
|
||||||
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
|
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
|
||||||
}
|
}
|
||||||
shardsAllocators.applyFailedShards(allocation);
|
gatewayAllocator.applyFailedShards(allocation);
|
||||||
reroute(allocation);
|
reroute(allocation);
|
||||||
final RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes);
|
final RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes);
|
||||||
String failedShardsAsString = firstListElementsToCommaDelimitedString(failedShards, s -> s.shard.shardId().toString());
|
String failedShardsAsString = firstListElementsToCommaDelimitedString(failedShards, s -> s.shard.shardId().toString());
|
||||||
|
@ -306,14 +308,10 @@ public class AllocationService extends AbstractComponent {
|
||||||
if (allocation.routingNodes().unassigned().size() > 0) {
|
if (allocation.routingNodes().unassigned().size() > 0) {
|
||||||
updateLeftDelayOfUnassignedShards(allocation, settings);
|
updateLeftDelayOfUnassignedShards(allocation, settings);
|
||||||
|
|
||||||
changed |= shardsAllocators.allocateUnassigned(allocation);
|
changed |= gatewayAllocator.allocateUnassigned(allocation);
|
||||||
}
|
}
|
||||||
|
|
||||||
// move shards that no longer can be allocated
|
changed |= shardsAllocator.allocate(allocation);
|
||||||
changed |= shardsAllocators.moveShards(allocation);
|
|
||||||
|
|
||||||
// rebalance
|
|
||||||
changed |= shardsAllocators.rebalance(allocation);
|
|
||||||
assert RoutingNodes.assertShardStats(allocation.routingNodes());
|
assert RoutingNodes.assertShardStats(allocation.routingNodes());
|
||||||
return changed;
|
return changed;
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.elasticsearch.cluster.routing.allocation.allocator;
|
package org.elasticsearch.cluster.routing.allocation.allocator;
|
||||||
|
|
||||||
import com.carrotsearch.hppc.cursors.ObjectCursor;
|
|
||||||
import org.apache.lucene.util.ArrayUtil;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
import org.apache.lucene.util.IntroSorter;
|
import org.apache.lucene.util.IntroSorter;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
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.RoutingNodes;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
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.RoutingAllocation;
|
||||||
import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation;
|
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
|
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
|
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type;
|
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.common.settings.Settings;
|
||||||
import org.elasticsearch.gateway.PriorityComparator;
|
import org.elasticsearch.gateway.PriorityComparator;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.IdentityHashMap;
|
import java.util.IdentityHashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.function.Predicate;
|
|
||||||
|
|
||||||
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
|
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
|
||||||
|
|
||||||
|
@ -103,27 +96,16 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void applyStartedShards(StartedRerouteAllocation allocation) { /* ONLY FOR GATEWAYS */ }
|
public boolean allocate(RoutingAllocation allocation) {
|
||||||
|
if (allocation.routingNodes().size() == 0) {
|
||||||
@Override
|
/* with no nodes this is pointless */
|
||||||
public void applyFailedShards(FailedRerouteAllocation allocation) { /* ONLY FOR GATEWAYS */ }
|
return false;
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean allocateUnassigned(RoutingAllocation allocation) {
|
|
||||||
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);
|
final Balancer balancer = new Balancer(logger, allocation, weightFunction, threshold);
|
||||||
return balancer.balance();
|
boolean changed = balancer.allocateUnassigned();
|
||||||
}
|
changed |= balancer.moveShards();
|
||||||
|
changed |= balancer.balance();
|
||||||
@Override
|
return changed;
|
||||||
public boolean moveShards(RoutingAllocation allocation) {
|
|
||||||
final Balancer balancer = new Balancer(logger, allocation, weightFunction, threshold);
|
|
||||||
return balancer.moveShards();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -203,8 +185,8 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
}
|
}
|
||||||
|
|
||||||
private float weight(Balancer balancer, ModelNode node, String index, int numAdditionalShards) {
|
private float weight(Balancer balancer, ModelNode node, String index, int numAdditionalShards) {
|
||||||
final float weightShard = (node.numShards() + numAdditionalShards - balancer.avgShardsPerNode());
|
final float weightShard = node.numShards() + numAdditionalShards - balancer.avgShardsPerNode();
|
||||||
final float weightIndex = (node.numShards(index) + numAdditionalShards - balancer.avgShardsPerNode(index));
|
final float weightIndex = node.numShards(index) + numAdditionalShards - balancer.avgShardsPerNode(index);
|
||||||
return theta0 * weightShard + theta1 * weightIndex;
|
return theta0 * weightShard + theta1 * weightIndex;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -216,7 +198,6 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
public static class Balancer {
|
public static class Balancer {
|
||||||
private final ESLogger logger;
|
private final ESLogger logger;
|
||||||
private final Map<String, ModelNode> nodes = new HashMap<>();
|
private final Map<String, ModelNode> nodes = new HashMap<>();
|
||||||
private final HashSet<String> indices = new HashSet<>();
|
|
||||||
private final RoutingAllocation allocation;
|
private final RoutingAllocation allocation;
|
||||||
private final RoutingNodes routingNodes;
|
private final RoutingNodes routingNodes;
|
||||||
private final WeightFunction weight;
|
private final WeightFunction weight;
|
||||||
|
@ -225,19 +206,15 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
private final MetaData metaData;
|
private final MetaData metaData;
|
||||||
private final float avgShardsPerNode;
|
private final float avgShardsPerNode;
|
||||||
|
|
||||||
private final Predicate<ShardRouting> assignedFilter = shard -> shard.assignedToNode();
|
|
||||||
|
|
||||||
public Balancer(ESLogger logger, RoutingAllocation allocation, WeightFunction weight, float threshold) {
|
public Balancer(ESLogger logger, RoutingAllocation allocation, WeightFunction weight, float threshold) {
|
||||||
this.logger = logger;
|
this.logger = logger;
|
||||||
this.allocation = allocation;
|
this.allocation = allocation;
|
||||||
this.weight = weight;
|
this.weight = weight;
|
||||||
this.threshold = threshold;
|
this.threshold = threshold;
|
||||||
this.routingNodes = allocation.routingNodes();
|
this.routingNodes = allocation.routingNodes();
|
||||||
for (RoutingNode node : routingNodes) {
|
|
||||||
nodes.put(node.nodeId(), new ModelNode(node.nodeId()));
|
|
||||||
}
|
|
||||||
metaData = routingNodes.metaData();
|
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);
|
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) {
|
private static float absDelta(float lower, float higher) {
|
||||||
assert higher >= lower : higher + " lt " + lower +" but was expected to be gte";
|
assert higher >= lower : higher + " lt " + lower +" but was expected to be gte";
|
||||||
return Math.abs(higher - lower);
|
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
|
* @return <code>true</code> if the current configuration has been
|
||||||
* changed, otherwise <code>false</code>
|
* changed, otherwise <code>false</code>
|
||||||
*/
|
*/
|
||||||
final boolean allocateUnassigned() {
|
private boolean balance() {
|
||||||
return balance(true);
|
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,28 +307,10 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
* @return <code>true</code> if the current configuration has been
|
* @return <code>true</code> if the current configuration has been
|
||||||
* changed, otherwise <code>false</code>
|
* changed, otherwise <code>false</code>
|
||||||
*/
|
*/
|
||||||
public boolean balance() {
|
private boolean balanceByWeights() {
|
||||||
return balance(false);
|
boolean changed = false;
|
||||||
}
|
final NodeSorter sorter = newNodeSorter();
|
||||||
|
final AllocationDeciders deciders = allocation.deciders();
|
||||||
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");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
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 ModelNode[] modelNodes = sorter.modelNodes;
|
||||||
final float[] weights = sorter.weights;
|
final float[] weights = sorter.weights;
|
||||||
for (String index : buildWeightOrderedIndices(sorter)) {
|
for (String index : buildWeightOrderedIndices(sorter)) {
|
||||||
|
@ -350,7 +322,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
for (int i = 0; i < modelNodes.length; i++) {
|
for (int i = 0; i < modelNodes.length; i++) {
|
||||||
ModelNode modelNode = modelNodes[i];
|
ModelNode modelNode = modelNodes[i];
|
||||||
if (modelNode.getIndex(index) != null
|
if (modelNode.getIndex(index) != null
|
||||||
|| deciders.canAllocate(indexMetaData, modelNode.getRoutingNode(routingNodes), allocation).type() != Type.NO) {
|
|| deciders.canAllocate(indexMetaData, modelNode.getRoutingNode(), allocation).type() != Type.NO) {
|
||||||
// swap nodes at position i and relevantNodes
|
// swap nodes at position i and relevantNodes
|
||||||
modelNodes[i] = modelNodes[relevantNodes];
|
modelNodes[i] = modelNodes[relevantNodes];
|
||||||
modelNodes[relevantNodes] = modelNode;
|
modelNodes[relevantNodes] = modelNode;
|
||||||
|
@ -432,8 +404,6 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
}
|
|
||||||
return changed;
|
return changed;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -451,7 +421,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
* to the nodes we relocated them from.
|
* to the nodes we relocated them from.
|
||||||
*/
|
*/
|
||||||
private String[] buildWeightOrderedIndices(NodeSorter sorter) {
|
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];
|
final float[] deltas = new float[indices.length];
|
||||||
for (int i = 0; i < deltas.length; i++) {
|
for (int i = 0; i < deltas.length; i++) {
|
||||||
sorter.reset(indices[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>
|
* @return <code>true</code> if the allocation has changed, otherwise <code>false</code>
|
||||||
*/
|
*/
|
||||||
public boolean moveShards() {
|
public boolean moveShards() {
|
||||||
if (nodes.isEmpty()) {
|
// Iterate over the started shards interleaving between nodes, and check if they can remain. In the presence of throttling
|
||||||
/* 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
|
|
||||||
// shard movements, the goal of this iteration order is to achieve a fairer movement of shards from the nodes that are
|
// shard movements, the goal of this iteration order is to achieve a fairer movement of shards from the nodes that are
|
||||||
// offloading the shards.
|
// offloading the shards.
|
||||||
List<ShardRouting> shards = new ArrayList<>();
|
boolean changed = false;
|
||||||
int index = 0;
|
int index = 0;
|
||||||
boolean found = true;
|
boolean found = true;
|
||||||
|
final NodeSorter sorter = newNodeSorter();
|
||||||
while (found) {
|
while (found) {
|
||||||
found = false;
|
found = false;
|
||||||
for (RoutingNode routingNode : routingNodes) {
|
for (RoutingNode routingNode : allocation.routingNodes()) {
|
||||||
if (index >= routingNode.size()) {
|
if (index >= routingNode.size()) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
@ -524,26 +490,26 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
ShardRouting shardRouting = routingNode.get(index);
|
ShardRouting shardRouting = routingNode.get(index);
|
||||||
// we can only move started shards...
|
// we can only move started shards...
|
||||||
if (shardRouting.started()) {
|
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++;
|
index++;
|
||||||
}
|
}
|
||||||
if (shards.isEmpty()) {
|
|
||||||
return false;
|
return changed;
|
||||||
}
|
}
|
||||||
|
|
||||||
final RoutingNodes.UnassignedShards unassigned = routingNodes.unassigned();
|
/**
|
||||||
boolean changed = initialize(routingNodes, unassigned);
|
* Move started shard to the minimal eligible node with respect to the weight function
|
||||||
if (changed == false) {
|
*
|
||||||
final NodeSorter sorter = newNodeSorter();
|
* @return <code>true</code> if the shard was moved successfully, otherwise <code>false</code>
|
||||||
final ModelNode[] modelNodes = sorter.modelNodes;
|
*/
|
||||||
for (ShardRouting shardRouting : shards) {
|
private boolean moveShard(NodeSorter sorter, ShardRouting shardRouting, ModelNode sourceNode, RoutingNode routingNode) {
|
||||||
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());
|
logger.debug("[{}][{}] allocated on [{}], but can no longer be allocated on it, moving...", shardRouting.index(), shardRouting.id(), routingNode.node());
|
||||||
sorter.reset(shardRouting.getIndexName());
|
sorter.reset(shardRouting.getIndexName());
|
||||||
/*
|
/*
|
||||||
|
@ -552,36 +518,24 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
* This is not guaranteed to be balanced after this operation we still try best effort to
|
* This is not guaranteed to be balanced after this operation we still try best effort to
|
||||||
* allocate on the minimal eligible node.
|
* allocate on the minimal eligible node.
|
||||||
*/
|
*/
|
||||||
boolean moved = false;
|
for (ModelNode currentNode : sorter.modelNodes) {
|
||||||
for (ModelNode currentNode : modelNodes) {
|
if (currentNode != sourceNode) {
|
||||||
if (currentNode == sourceNode) {
|
RoutingNode target = currentNode.getRoutingNode();
|
||||||
continue;
|
|
||||||
}
|
|
||||||
RoutingNode target = currentNode.getRoutingNode(routingNodes);
|
|
||||||
Decision allocationDecision = allocation.deciders().canAllocate(shardRouting, target, allocation);
|
Decision allocationDecision = allocation.deciders().canAllocate(shardRouting, target, allocation);
|
||||||
Decision rebalanceDecision = allocation.deciders().canRebalance(shardRouting, 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?
|
if (allocationDecision.type() == Type.YES && rebalanceDecision.type() == Type.YES) { // TODO maybe we can respect throttling here too?
|
||||||
Decision sourceDecision = sourceNode.removeShard(shardRouting);
|
sourceNode.removeShard(shardRouting);
|
||||||
ShardRouting targetRelocatingShard = routingNodes.relocate(shardRouting, target.nodeId(), allocation.clusterInfo().getShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
|
ShardRouting targetRelocatingShard = routingNodes.relocate(shardRouting, target.nodeId(), allocation.clusterInfo().getShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
|
||||||
// re-add (now relocating shard) to source node
|
currentNode.addShard(targetRelocatingShard);
|
||||||
sourceNode.addShard(shardRouting, sourceDecision);
|
|
||||||
Decision targetDecision = new Decision.Multi().add(allocationDecision).add(rebalanceDecision);
|
|
||||||
currentNode.addShard(targetRelocatingShard, targetDecision);
|
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("Moved shard [{}] to node [{}]", shardRouting, routingNode.node());
|
logger.trace("Moved shard [{}] to node [{}]", shardRouting, routingNode.node());
|
||||||
}
|
}
|
||||||
moved = true;
|
return true;
|
||||||
changed = true;
|
}
|
||||||
break;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (moved == false) {
|
|
||||||
logger.debug("[{}][{}] can't move", shardRouting.index(), shardRouting.id());
|
logger.debug("[{}][{}] can't move", shardRouting.index(), shardRouting.id());
|
||||||
}
|
return false;
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return changed;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -593,27 +547,31 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
* on the target node which we respect during the allocation / balancing
|
* on the target node which we respect during the allocation / balancing
|
||||||
* process. In short, this method recreates the status-quo in the cluster.
|
* process. In short, this method recreates the status-quo in the cluster.
|
||||||
*/
|
*/
|
||||||
private void buildModelFromAssigned(Iterable<ShardRouting> shards) {
|
private void buildModelFromAssigned() {
|
||||||
for (ShardRouting shard : shards) {
|
for (RoutingNode rn : routingNodes) {
|
||||||
assert shard.assignedToNode();
|
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 */
|
/* we skip relocating shards here since we expect an initializing shard with the same id coming in */
|
||||||
if (shard.state() == RELOCATING) {
|
if (shard.state() != RELOCATING) {
|
||||||
continue;
|
node.addShard(shard);
|
||||||
}
|
|
||||||
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()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("Assigned shard [{}] to node [{}]", shard, node.getNodeId());
|
logger.trace("Assigned shard [{}] to node [{}]", shard, node.getNodeId());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Allocates all given shards on the minimal eligible node for the shards index
|
* 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.
|
* 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();
|
assert !nodes.isEmpty();
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("Start allocating unassigned shards");
|
logger.trace("Start allocating unassigned shards");
|
||||||
|
@ -657,7 +615,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
int secondaryLength = 0;
|
int secondaryLength = 0;
|
||||||
int primaryLength = primary.length;
|
int primaryLength = primary.length;
|
||||||
ArrayUtil.timSort(primary, comparator);
|
ArrayUtil.timSort(primary, comparator);
|
||||||
final Set<ModelNode> throttledNodes = Collections.newSetFromMap(new IdentityHashMap<ModelNode, Boolean>());
|
final Set<ModelNode> throttledNodes = Collections.newSetFromMap(new IdentityHashMap<>());
|
||||||
do {
|
do {
|
||||||
for (int i = 0; i < primaryLength; i++) {
|
for (int i = 0; i < primaryLength; i++) {
|
||||||
ShardRouting shard = primary[i];
|
ShardRouting shard = primary[i];
|
||||||
|
@ -695,7 +653,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
* don't check deciders
|
* don't check deciders
|
||||||
*/
|
*/
|
||||||
if (currentWeight <= minWeight) {
|
if (currentWeight <= minWeight) {
|
||||||
Decision currentDecision = deciders.canAllocate(shard, node.getRoutingNode(routingNodes), allocation);
|
Decision currentDecision = deciders.canAllocate(shard, node.getRoutingNode(), allocation);
|
||||||
NOUPDATE:
|
NOUPDATE:
|
||||||
if (currentDecision.type() == Type.YES || currentDecision.type() == Type.THROTTLE) {
|
if (currentDecision.type() == Type.YES || currentDecision.type() == Type.THROTTLE) {
|
||||||
if (currentWeight == minWeight) {
|
if (currentWeight == minWeight) {
|
||||||
|
@ -736,7 +694,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
}
|
}
|
||||||
assert decision != null && minNode != null || decision == null && minNode == null;
|
assert decision != null && minNode != null || decision == null && minNode == null;
|
||||||
if (minNode != null) {
|
if (minNode != null) {
|
||||||
minNode.addShard(shard, decision);
|
minNode.addShard(shard);
|
||||||
if (decision.type() == Type.YES) {
|
if (decision.type() == Type.YES) {
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("Assigned shard [{}] to [{}]", shard, minNode.getNodeId());
|
logger.trace("Assigned shard [{}] to [{}]", shard, minNode.getNodeId());
|
||||||
|
@ -745,7 +703,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
changed = true;
|
changed = true;
|
||||||
continue; // don't add to ignoreUnassigned
|
continue; // don't add to ignoreUnassigned
|
||||||
} else {
|
} else {
|
||||||
final RoutingNode node = minNode.getRoutingNode(routingNodes);
|
final RoutingNode node = minNode.getRoutingNode();
|
||||||
if (deciders.canAllocate(node, allocation).type() != Type.YES) {
|
if (deciders.canAllocate(node, allocation).type() != Type.YES) {
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("Can not allocate on node [{}] remove from round decision [{}]", node, decision.type());
|
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;
|
ShardRouting candidate = null;
|
||||||
final AllocationDeciders deciders = allocation.deciders();
|
final AllocationDeciders deciders = allocation.deciders();
|
||||||
for (ShardRouting shard : index.getAllShards()) {
|
for (ShardRouting shard : index) {
|
||||||
if (shard.started()) {
|
if (shard.started()) {
|
||||||
// skip initializing, unassigned and relocating shards we can't relocate them anyway
|
// 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);
|
Decision rebalanceDecision = deciders.canRebalance(shard, allocation);
|
||||||
if (((allocationDecision.type() == Type.YES) || (allocationDecision.type() == Type.THROTTLE))
|
if (((allocationDecision.type() == Type.YES) || (allocationDecision.type() == Type.THROTTLE))
|
||||||
&& ((rebalanceDecision.type() == Type.YES) || (rebalanceDecision.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) {
|
if (candidate != null) {
|
||||||
|
|
||||||
/* allocate on the model even if not throttled */
|
/* allocate on the model even if not throttled */
|
||||||
maxNode.removeShard(candidate);
|
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 (decision.type() == Type.YES) { /* only allocate on the cluster if we are not throttled */
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("Relocate shard [{}] from node [{}] to node [{}]", candidate, maxNode.getNodeId(),
|
logger.trace("Relocate shard [{}] from node [{}] to node [{}]", candidate, maxNode.getNodeId(),
|
||||||
minNode.getNodeId());
|
minNode.getNodeId());
|
||||||
}
|
}
|
||||||
/* now allocate on the cluster - if we are started we need to relocate the shard */
|
/* now allocate on the cluster */
|
||||||
if (candidate.started()) {
|
|
||||||
routingNodes.relocate(candidate, minNode.getNodeId(), allocation.clusterInfo().getShardSize(candidate, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
|
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));
|
|
||||||
}
|
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -846,14 +797,12 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
}
|
}
|
||||||
|
|
||||||
static class ModelNode implements Iterable<ModelIndex> {
|
static class ModelNode implements Iterable<ModelIndex> {
|
||||||
private final String id;
|
|
||||||
private final Map<String, ModelIndex> indices = new HashMap<>();
|
private final Map<String, ModelIndex> indices = new HashMap<>();
|
||||||
private int numShards = 0;
|
private int numShards = 0;
|
||||||
// lazily calculated
|
private final RoutingNode routingNode;
|
||||||
private RoutingNode routingNode;
|
|
||||||
|
|
||||||
public ModelNode(String id) {
|
public ModelNode(RoutingNode routingNode) {
|
||||||
this.id = id;
|
this.routingNode = routingNode;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ModelIndex getIndex(String indexId) {
|
public ModelIndex getIndex(String indexId) {
|
||||||
|
@ -861,13 +810,10 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getNodeId() {
|
public String getNodeId() {
|
||||||
return id;
|
return routingNode.nodeId();
|
||||||
}
|
}
|
||||||
|
|
||||||
public RoutingNode getRoutingNode(RoutingNodes routingNodes) {
|
public RoutingNode getRoutingNode() {
|
||||||
if (routingNode == null) {
|
|
||||||
routingNode = routingNodes.node(id);
|
|
||||||
}
|
|
||||||
return routingNode;
|
return routingNode;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -888,33 +834,31 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
return -1;
|
return -1;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void addShard(ShardRouting shard, Decision decision) {
|
public void addShard(ShardRouting shard) {
|
||||||
ModelIndex index = indices.get(shard.getIndexName());
|
ModelIndex index = indices.get(shard.getIndexName());
|
||||||
if (index == null) {
|
if (index == null) {
|
||||||
index = new ModelIndex(shard.getIndexName());
|
index = new ModelIndex(shard.getIndexName());
|
||||||
indices.put(index.getIndexId(), index);
|
indices.put(index.getIndexId(), index);
|
||||||
}
|
}
|
||||||
index.addShard(shard, decision);
|
index.addShard(shard);
|
||||||
numShards++;
|
numShards++;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Decision removeShard(ShardRouting shard) {
|
public void removeShard(ShardRouting shard) {
|
||||||
ModelIndex index = indices.get(shard.getIndexName());
|
ModelIndex index = indices.get(shard.getIndexName());
|
||||||
Decision removed = null;
|
|
||||||
if (index != null) {
|
if (index != null) {
|
||||||
removed = index.removeShard(shard);
|
index.removeShard(shard);
|
||||||
if (removed != null && index.numShards() == 0) {
|
if (index.numShards() == 0) {
|
||||||
indices.remove(shard.getIndexName());
|
indices.remove(shard.getIndexName());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
numShards--;
|
numShards--;
|
||||||
return removed;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
StringBuilder sb = new StringBuilder();
|
StringBuilder sb = new StringBuilder();
|
||||||
sb.append("Node(").append(id).append(")");
|
sb.append("Node(").append(routingNode.nodeId()).append(")");
|
||||||
return sb.toString();
|
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 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;
|
private int highestPrimary = -1;
|
||||||
|
|
||||||
public ModelIndex(String id) {
|
public ModelIndex(String id) {
|
||||||
|
@ -942,7 +886,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
public int highestPrimary() {
|
public int highestPrimary() {
|
||||||
if (highestPrimary == -1) {
|
if (highestPrimary == -1) {
|
||||||
int maxId = -1;
|
int maxId = -1;
|
||||||
for (ShardRouting shard : shards.keySet()) {
|
for (ShardRouting shard : shards) {
|
||||||
if (shard.primary()) {
|
if (shard.primary()) {
|
||||||
maxId = Math.max(maxId, shard.id());
|
maxId = Math.max(maxId, shard.id());
|
||||||
}
|
}
|
||||||
|
@ -960,24 +904,25 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
||||||
return shards.size();
|
return shards.size();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Collection<ShardRouting> getAllShards() {
|
@Override
|
||||||
return shards.keySet();
|
public Iterator<ShardRouting> iterator() {
|
||||||
|
return shards.iterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Decision removeShard(ShardRouting shard) {
|
public void removeShard(ShardRouting shard) {
|
||||||
highestPrimary = -1;
|
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;
|
highestPrimary = -1;
|
||||||
assert decision != null;
|
assert !shards.contains(shard) : "Shard already allocated on current node: " + shard;
|
||||||
assert !shards.containsKey(shard) : "Shard already allocated on current node: " + shards.get(shard) + " " + shard;
|
shards.add(shard);
|
||||||
shards.put(shard, decision);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean containsShard(ShardRouting shard) {
|
public boolean containsShard(ShardRouting shard) {
|
||||||
return shards.containsKey(shard);
|
return shards.contains(shard);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,56 +19,25 @@
|
||||||
|
|
||||||
package org.elasticsearch.cluster.routing.allocation.allocator;
|
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.RoutingAllocation;
|
||||||
import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p>
|
||||||
* A {@link ShardsAllocator} is the main entry point for shard allocation on nodes in the cluster.
|
* 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
|
* 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>
|
* </p>
|
||||||
*/
|
*/
|
||||||
public interface ShardsAllocator {
|
public interface ShardsAllocator {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Applies changes on started nodes based on the implemented algorithm. For example if a
|
* Allocates shards to nodes in the cluster. An implementation of this method should:
|
||||||
* shard has changed to {@link ShardRoutingState#STARTED} from {@link ShardRoutingState#RELOCATING}
|
* - assign unassigned shards
|
||||||
* this allocator might apply some cleanups on the node that used to hold the shard.
|
* - relocate shards that cannot stay on a node anymore
|
||||||
* @param allocation all started {@link ShardRouting shards}
|
* - relocate shards to find a good shard balance in the cluster
|
||||||
*/
|
|
||||||
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
|
|
||||||
*
|
*
|
||||||
* @param allocation current node allocation
|
* @param allocation current node allocation
|
||||||
* @return <code>true</code> if the allocation has changed, otherwise <code>false</code>
|
* @return <code>true</code> if the allocation has changed, otherwise <code>false</code>
|
||||||
*/
|
*/
|
||||||
boolean allocateUnassigned(RoutingAllocation allocation);
|
boolean allocate(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);
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.elasticsearch.cluster.service;
|
package org.elasticsearch.cluster.service;
|
||||||
|
|
||||||
import org.elasticsearch.Version;
|
|
||||||
import org.elasticsearch.cluster.AckedClusterStateTaskListener;
|
import org.elasticsearch.cluster.AckedClusterStateTaskListener;
|
||||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||||
import org.elasticsearch.cluster.ClusterName;
|
import org.elasticsearch.cluster.ClusterName;
|
||||||
|
@ -32,19 +31,18 @@ import org.elasticsearch.cluster.ClusterStateTaskExecutor;
|
||||||
import org.elasticsearch.cluster.ClusterStateTaskListener;
|
import org.elasticsearch.cluster.ClusterStateTaskListener;
|
||||||
import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
||||||
import org.elasticsearch.cluster.LocalNodeMasterListener;
|
import org.elasticsearch.cluster.LocalNodeMasterListener;
|
||||||
|
import org.elasticsearch.cluster.NodeConnectionsService;
|
||||||
import org.elasticsearch.cluster.TimeoutClusterStateListener;
|
import org.elasticsearch.cluster.TimeoutClusterStateListener;
|
||||||
import org.elasticsearch.cluster.block.ClusterBlock;
|
import org.elasticsearch.cluster.block.ClusterBlock;
|
||||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
import org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException;
|
import org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNodeService;
|
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||||
import org.elasticsearch.cluster.routing.OperationRouting;
|
import org.elasticsearch.cluster.routing.OperationRouting;
|
||||||
import org.elasticsearch.cluster.routing.RoutingTable;
|
import org.elasticsearch.cluster.routing.RoutingTable;
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
import org.elasticsearch.common.Priority;
|
import org.elasticsearch.common.Priority;
|
||||||
import org.elasticsearch.common.Randomness;
|
|
||||||
import org.elasticsearch.common.Strings;
|
import org.elasticsearch.common.Strings;
|
||||||
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
||||||
import org.elasticsearch.common.inject.Inject;
|
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.Setting;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.text.Text;
|
import org.elasticsearch.common.text.Text;
|
||||||
import org.elasticsearch.common.transport.TransportAddress;
|
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||||
import org.elasticsearch.common.util.concurrent.CountDown;
|
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.concurrent.PrioritizedRunnable;
|
||||||
import org.elasticsearch.common.util.iterable.Iterables;
|
import org.elasticsearch.common.util.iterable.Iterables;
|
||||||
import org.elasticsearch.discovery.Discovery;
|
import org.elasticsearch.discovery.Discovery;
|
||||||
import org.elasticsearch.tasks.TaskManager;
|
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
import org.elasticsearch.transport.TransportService;
|
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
@ -78,8 +73,6 @@ import java.util.Locale;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Queue;
|
import java.util.Queue;
|
||||||
import java.util.Random;
|
|
||||||
import java.util.concurrent.ConcurrentMap;
|
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
import java.util.concurrent.Executor;
|
import java.util.concurrent.Executor;
|
||||||
import java.util.concurrent.Future;
|
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 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_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 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 final ThreadPool threadPool;
|
||||||
|
|
||||||
private BiConsumer<ClusterChangedEvent, Discovery.AckListener> clusterStatePublisher;
|
private BiConsumer<ClusterChangedEvent, Discovery.AckListener> clusterStatePublisher;
|
||||||
|
|
||||||
private final OperationRouting operationRouting;
|
private final OperationRouting operationRouting;
|
||||||
|
|
||||||
private final TransportService transportService;
|
|
||||||
|
|
||||||
private final ClusterSettings clusterSettings;
|
private final ClusterSettings clusterSettings;
|
||||||
private final DiscoveryNodeService discoveryNodeService;
|
|
||||||
private final Version version;
|
|
||||||
|
|
||||||
private final TimeValue reconnectInterval;
|
|
||||||
|
|
||||||
private TimeValue slowTaskLoggingThreshold;
|
private TimeValue slowTaskLoggingThreshold;
|
||||||
|
|
||||||
|
@ -140,47 +123,49 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
||||||
|
|
||||||
private final ClusterBlocks.Builder initialBlocks;
|
private final ClusterBlocks.Builder initialBlocks;
|
||||||
|
|
||||||
private final TaskManager taskManager;
|
private NodeConnectionsService nodeConnectionsService;
|
||||||
|
|
||||||
private volatile ScheduledFuture reconnectToNodes;
|
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public InternalClusterService(Settings settings, OperationRouting operationRouting, TransportService transportService,
|
public InternalClusterService(Settings settings, OperationRouting operationRouting,
|
||||||
ClusterSettings clusterSettings, ThreadPool threadPool, ClusterName clusterName, DiscoveryNodeService discoveryNodeService, Version version) {
|
ClusterSettings clusterSettings, ThreadPool threadPool, ClusterName clusterName) {
|
||||||
super(settings);
|
super(settings);
|
||||||
this.operationRouting = operationRouting;
|
this.operationRouting = operationRouting;
|
||||||
this.transportService = transportService;
|
|
||||||
this.threadPool = threadPool;
|
this.threadPool = threadPool;
|
||||||
this.clusterSettings = clusterSettings;
|
this.clusterSettings = clusterSettings;
|
||||||
this.discoveryNodeService = discoveryNodeService;
|
|
||||||
this.version = version;
|
|
||||||
|
|
||||||
// will be replaced on doStart.
|
// will be replaced on doStart.
|
||||||
this.clusterState = ClusterState.builder(clusterName).build();
|
this.clusterState = ClusterState.builder(clusterName).build();
|
||||||
|
|
||||||
this.clusterSettings.addSettingsUpdateConsumer(CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, this::setSlowTaskLoggingThreshold);
|
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);
|
this.slowTaskLoggingThreshold = CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get(settings);
|
||||||
|
|
||||||
localNodeMasterListeners = new LocalNodeMasterListeners(threadPool);
|
localNodeMasterListeners = new LocalNodeMasterListeners(threadPool);
|
||||||
|
|
||||||
initialBlocks = ClusterBlocks.builder();
|
initialBlocks = ClusterBlocks.builder();
|
||||||
|
|
||||||
taskManager = transportService.getTaskManager();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) {
|
private void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) {
|
||||||
this.slowTaskLoggingThreshold = slowTaskLoggingThreshold;
|
this.slowTaskLoggingThreshold = slowTaskLoggingThreshold;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setClusterStatePublisher(BiConsumer<ClusterChangedEvent, Discovery.AckListener> publisher) {
|
synchronized public void setClusterStatePublisher(BiConsumer<ClusterChangedEvent, Discovery.AckListener> publisher) {
|
||||||
clusterStatePublisher = 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
|
@Override
|
||||||
public void addInitialStateBlock(ClusterBlock block) throws IllegalStateException {
|
synchronized public void addInitialStateBlock(ClusterBlock block) throws IllegalStateException {
|
||||||
if (lifecycle.started()) {
|
if (lifecycle.started()) {
|
||||||
throw new IllegalStateException("can't set initial block when started");
|
throw new IllegalStateException("can't set initial block when started");
|
||||||
}
|
}
|
||||||
|
@ -188,12 +173,12 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void removeInitialStateBlock(ClusterBlock block) throws IllegalStateException {
|
synchronized public void removeInitialStateBlock(ClusterBlock block) throws IllegalStateException {
|
||||||
removeInitialStateBlock(block.id());
|
removeInitialStateBlock(block.id());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void removeInitialStateBlock(int blockId) throws IllegalStateException {
|
synchronized public void removeInitialStateBlock(int blockId) throws IllegalStateException {
|
||||||
if (lifecycle.started()) {
|
if (lifecycle.started()) {
|
||||||
throw new IllegalStateException("can't set initial block when started");
|
throw new IllegalStateException("can't set initial block when started");
|
||||||
}
|
}
|
||||||
|
@ -201,26 +186,18 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void doStart() {
|
synchronized protected void doStart() {
|
||||||
Objects.requireNonNull(clusterStatePublisher, "please set a cluster state publisher before starting");
|
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(localNodeMasterListeners);
|
||||||
add(taskManager);
|
|
||||||
this.clusterState = ClusterState.builder(clusterState).blocks(initialBlocks).build();
|
this.clusterState = ClusterState.builder(clusterState).blocks(initialBlocks).build();
|
||||||
this.updateTasksExecutor = EsExecutors.newSinglePrioritizing(UPDATE_THREAD_NAME, daemonThreadFactory(settings, UPDATE_THREAD_NAME), threadPool.getThreadContext());
|
this.updateTasksExecutor = EsExecutors.newSinglePrioritizing(UPDATE_THREAD_NAME, daemonThreadFactory(settings, UPDATE_THREAD_NAME), threadPool.getThreadContext());
|
||||||
this.reconnectToNodes = threadPool.schedule(reconnectInterval, ThreadPool.Names.GENERIC, new ReconnectToNodes());
|
this.clusterState = ClusterState.builder(clusterState).blocks(initialBlocks).build();
|
||||||
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);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void doStop() {
|
synchronized protected void doStop() {
|
||||||
FutureUtils.cancel(this.reconnectToNodes);
|
|
||||||
for (NotifyTimeout onGoingTimeout : onGoingTimeouts) {
|
for (NotifyTimeout onGoingTimeout : onGoingTimeouts) {
|
||||||
onGoingTimeout.cancel();
|
onGoingTimeout.cancel();
|
||||||
onGoingTimeout.listener.onClose();
|
onGoingTimeout.listener.onClose();
|
||||||
|
@ -230,7 +207,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void doClose() {
|
synchronized protected void doClose() {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -400,11 +377,6 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
||||||
return updateTasksExecutor.getMaxTaskWaitTime();
|
return updateTasksExecutor.getMaxTaskWaitTime();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public TaskManager getTaskManager() {
|
|
||||||
return taskManager;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** asserts that the current thread is the cluster state update thread */
|
/** asserts that the current thread is the cluster state update thread */
|
||||||
public boolean assertClusterStateThread() {
|
public boolean assertClusterStateThread() {
|
||||||
assert Thread.currentThread().getName().contains(InternalClusterService.UPDATE_THREAD_NAME) : "not called from the cluster state update thread";
|
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;
|
return;
|
||||||
}
|
}
|
||||||
ClusterStateTaskExecutor.BatchResult<T> batchResult;
|
ClusterStateTaskExecutor.BatchResult<T> batchResult;
|
||||||
long startTimeNS = System.nanoTime();
|
long startTimeNS = currentTimeInNanos();
|
||||||
try {
|
try {
|
||||||
List<T> inputs = toExecute.stream().map(tUpdateTask -> tUpdateTask.task).collect(Collectors.toList());
|
List<T> inputs = toExecute.stream().map(tUpdateTask -> tUpdateTask.task).collect(Collectors.toList());
|
||||||
batchResult = executor.execute(previousClusterState, inputs);
|
batchResult = executor.execute(previousClusterState, inputs);
|
||||||
} catch (Throwable e) {
|
} 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()) {
|
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.nodes().prettyPrint());
|
||||||
sb.append(previousClusterState.routingTable().prettyPrint());
|
sb.append(previousClusterState.routingTable().prettyPrint());
|
||||||
sb.append(previousClusterState.getRoutingNodes().prettyPrint());
|
sb.append(previousClusterState.getRoutingNodes().prettyPrint());
|
||||||
|
@ -509,8 +481,8 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
||||||
}
|
}
|
||||||
task.listener.clusterStateProcessed(task.source, previousClusterState, newClusterState);
|
task.listener.clusterStateProcessed(task.source, previousClusterState, newClusterState);
|
||||||
}
|
}
|
||||||
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(System.nanoTime() - startTimeNS)));
|
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
|
||||||
logger.debug("processing [{}]: took {} no change in cluster_state", source, executionTime);
|
logger.debug("processing [{}]: took [{}] no change in cluster_state", source, executionTime);
|
||||||
warnAboutSlowTaskIfNeeded(executionTime, source);
|
warnAboutSlowTaskIfNeeded(executionTime, source);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -568,15 +540,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO, do this in parallel (and wait)
|
nodeConnectionsService.connectToAddedNodes(clusterChangedEvent);
|
||||||
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);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// if we are the master, publish the new state to all nodes
|
// 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
|
// 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()) {
|
nodeConnectionsService.disconnectFromRemovedNodes(clusterChangedEvent);
|
||||||
try {
|
|
||||||
transportService.disconnectFromNode(node);
|
|
||||||
} catch (Throwable e) {
|
|
||||||
logger.warn("failed to disconnect to node [" + node + "]", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
newClusterState.status(ClusterState.ClusterStateStatus.APPLIED);
|
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);
|
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)));
|
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());
|
logger.debug("processing [{}]: took [{}] done applying updated cluster_state (version: {}, uuid: {})", source, executionTime, newClusterState.version(), newClusterState.stateUUID());
|
||||||
warnAboutSlowTaskIfNeeded(executionTime, source);
|
warnAboutSlowTaskIfNeeded(executionTime, source);
|
||||||
} catch (Throwable t) {
|
} 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");
|
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.nodes().prettyPrint());
|
||||||
sb.append(newClusterState.routingTable().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) {
|
private static SafeClusterStateTaskListener safe(ClusterStateTaskListener listener, ESLogger logger) {
|
||||||
if (listener instanceof AckedClusterStateTaskListener) {
|
if (listener instanceof AckedClusterStateTaskListener) {
|
||||||
return new SafeAckedClusterStateTaskListener((AckedClusterStateTaskListener) listener, logger);
|
return new SafeAckedClusterStateTaskListener((AckedClusterStateTaskListener) listener, logger);
|
||||||
|
@ -777,7 +738,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
||||||
|
|
||||||
private void warnAboutSlowTaskIfNeeded(TimeValue executionTime, String source) {
|
private void warnAboutSlowTaskIfNeeded(TimeValue executionTime, String source) {
|
||||||
if (executionTime.getMillis() > slowTaskLoggingThreshold.getMillis()) {
|
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 static class LocalNodeMasterListeners implements ClusterStateListener {
|
||||||
|
|
||||||
private final List<LocalNodeMasterListener> listeners = new CopyOnWriteArrayList<>();
|
private final List<LocalNodeMasterListener> listeners = new CopyOnWriteArrayList<>();
|
||||||
|
|
|
@ -37,6 +37,7 @@ import org.elasticsearch.common.geo.builders.ShapeBuilder;
|
||||||
import org.elasticsearch.common.text.Text;
|
import org.elasticsearch.common.text.Text;
|
||||||
import org.elasticsearch.index.query.QueryBuilder;
|
import org.elasticsearch.index.query.QueryBuilder;
|
||||||
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
|
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
|
||||||
|
import org.elasticsearch.ingest.IngestStats;
|
||||||
import org.elasticsearch.search.rescore.RescoreBuilder;
|
import org.elasticsearch.search.rescore.RescoreBuilder;
|
||||||
import org.elasticsearch.search.suggest.SuggestionBuilder;
|
import org.elasticsearch.search.suggest.SuggestionBuilder;
|
||||||
import org.elasticsearch.search.suggest.completion.context.QueryContext;
|
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 {
|
public <T extends Throwable> T readThrowable() throws IOException {
|
||||||
if (readBoolean()) {
|
if (readBoolean()) {
|
||||||
int key = readVInt();
|
int key = readVInt();
|
||||||
|
|
|
@ -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 {
|
public void writeThrowable(Throwable throwable) throws IOException {
|
||||||
if (throwable == null) {
|
if (throwable == null) {
|
||||||
writeBoolean(false);
|
writeBoolean(false);
|
||||||
|
|
|
@ -29,8 +29,10 @@ import org.elasticsearch.client.transport.TransportClientNodesService;
|
||||||
import org.elasticsearch.cluster.ClusterModule;
|
import org.elasticsearch.cluster.ClusterModule;
|
||||||
import org.elasticsearch.cluster.ClusterName;
|
import org.elasticsearch.cluster.ClusterName;
|
||||||
import org.elasticsearch.cluster.InternalClusterInfoService;
|
import org.elasticsearch.cluster.InternalClusterInfoService;
|
||||||
|
import org.elasticsearch.cluster.NodeConnectionsService;
|
||||||
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
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.allocator.BalancedShardsAllocator;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.AwarenessAllocationDecider;
|
import org.elasticsearch.cluster.routing.allocation.decider.AwarenessAllocationDecider;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
|
import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
|
||||||
|
@ -259,7 +261,7 @@ public final class ClusterSettings extends AbstractScopedSettings {
|
||||||
TransportService.TRACE_LOG_INCLUDE_SETTING,
|
TransportService.TRACE_LOG_INCLUDE_SETTING,
|
||||||
TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING,
|
TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING,
|
||||||
ShardsLimitAllocationDecider.CLUSTER_TOTAL_SHARDS_PER_NODE_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.FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING,
|
||||||
HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_TYPE_SETTING,
|
HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_TYPE_SETTING,
|
||||||
Transport.TRANSPORT_TCP_COMPRESS,
|
Transport.TRANSPORT_TCP_COMPRESS,
|
||||||
|
@ -326,7 +328,7 @@ public final class ClusterSettings extends AbstractScopedSettings {
|
||||||
Environment.PATH_SCRIPTS_SETTING,
|
Environment.PATH_SCRIPTS_SETTING,
|
||||||
Environment.PATH_SHARED_DATA_SETTING,
|
Environment.PATH_SHARED_DATA_SETTING,
|
||||||
Environment.PIDFILE_SETTING,
|
Environment.PIDFILE_SETTING,
|
||||||
InternalClusterService.NODE_ID_SEED_SETTING,
|
DiscoveryNodeService.NODE_ID_SEED_SETTING,
|
||||||
DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING,
|
DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING,
|
||||||
DiscoveryModule.DISCOVERY_TYPE_SETTING,
|
DiscoveryModule.DISCOVERY_TYPE_SETTING,
|
||||||
DiscoveryModule.ZEN_MASTER_SERVICE_TYPE_SETTING,
|
DiscoveryModule.ZEN_MASTER_SERVICE_TYPE_SETTING,
|
||||||
|
|
|
@ -20,7 +20,10 @@
|
||||||
package org.elasticsearch.common.util.concurrent;
|
package org.elasticsearch.common.util.concurrent;
|
||||||
|
|
||||||
|
|
||||||
|
import org.elasticsearch.common.lease.Releasable;
|
||||||
|
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.concurrent.locks.ReentrantLock;
|
import java.util.concurrent.locks.ReentrantLock;
|
||||||
|
|
||||||
|
@ -30,7 +33,6 @@ import java.util.concurrent.locks.ReentrantLock;
|
||||||
* lock. The latter is important to assure that the list of locks does not grow
|
* lock. The latter is important to assure that the list of locks does not grow
|
||||||
* infinitely.
|
* infinitely.
|
||||||
*
|
*
|
||||||
* A Thread can acquire a lock only once.
|
|
||||||
*
|
*
|
||||||
* */
|
* */
|
||||||
public class KeyedLock<T> {
|
public class KeyedLock<T> {
|
||||||
|
@ -50,48 +52,38 @@ public class KeyedLock<T> {
|
||||||
|
|
||||||
private final ConcurrentMap<T, KeyLock> map = ConcurrentCollections.newConcurrentMap();
|
private final ConcurrentMap<T, KeyLock> map = ConcurrentCollections.newConcurrentMap();
|
||||||
|
|
||||||
protected final ThreadLocal<KeyLock> threadLocal = new ThreadLocal<>();
|
public Releasable acquire(T key) {
|
||||||
|
assert isHeldByCurrentThread(key) == false : "lock for " + key + " is already heald by this thread";
|
||||||
public void acquire(T key) {
|
|
||||||
while (true) {
|
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);
|
KeyLock perNodeLock = map.get(key);
|
||||||
if (perNodeLock == null) {
|
if (perNodeLock == null) {
|
||||||
KeyLock newLock = new KeyLock(fair);
|
KeyLock newLock = new KeyLock(fair);
|
||||||
perNodeLock = map.putIfAbsent(key, newLock);
|
perNodeLock = map.putIfAbsent(key, newLock);
|
||||||
if (perNodeLock == null) {
|
if (perNodeLock == null) {
|
||||||
newLock.lock();
|
newLock.lock();
|
||||||
threadLocal.set(newLock);
|
return new ReleasableLock(key, newLock);
|
||||||
return;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
assert perNodeLock != null;
|
assert perNodeLock != null;
|
||||||
int i = perNodeLock.count.get();
|
int i = perNodeLock.count.get();
|
||||||
if (i > 0 && perNodeLock.count.compareAndSet(i, i + 1)) {
|
if (i > 0 && perNodeLock.count.compareAndSet(i, i + 1)) {
|
||||||
perNodeLock.lock();
|
perNodeLock.lock();
|
||||||
threadLocal.set(perNodeLock);
|
return new ReleasableLock(key, perNodeLock);
|
||||||
return;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void release(T key) {
|
public boolean isHeldByCurrentThread(T key) {
|
||||||
KeyLock lock = threadLocal.get();
|
KeyLock lock = map.get(key);
|
||||||
if (lock == null) {
|
if (lock == null) {
|
||||||
throw new IllegalStateException("Lock not acquired");
|
return false;
|
||||||
}
|
}
|
||||||
release(key, lock);
|
return lock.isHeldByCurrentThread();
|
||||||
}
|
}
|
||||||
|
|
||||||
void release(T key, KeyLock lock) {
|
void release(T key, KeyLock lock) {
|
||||||
assert lock.isHeldByCurrentThread();
|
|
||||||
assert lock == map.get(key);
|
assert lock == map.get(key);
|
||||||
lock.unlock();
|
lock.unlock();
|
||||||
threadLocal.set(null);
|
|
||||||
int decrementAndGet = lock.count.decrementAndGet();
|
int decrementAndGet = lock.count.decrementAndGet();
|
||||||
if (decrementAndGet == 0) {
|
if (decrementAndGet == 0) {
|
||||||
map.remove(key, lock);
|
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")
|
@SuppressWarnings("serial")
|
||||||
private final static class KeyLock extends ReentrantLock {
|
private final static class KeyLock extends ReentrantLock {
|
||||||
KeyLock(boolean fair) {
|
KeyLock(boolean fair) {
|
||||||
|
|
|
@ -19,8 +19,14 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.mapper;
|
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.document.Field;
|
||||||
import org.apache.lucene.index.IndexOptions;
|
|
||||||
import org.apache.lucene.index.IndexableField;
|
import org.apache.lucene.index.IndexableField;
|
||||||
import org.apache.lucene.util.CloseableThreadLocal;
|
import org.apache.lucene.util.CloseableThreadLocal;
|
||||||
import org.elasticsearch.common.Strings;
|
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.ObjectMapper;
|
||||||
import org.elasticsearch.index.mapper.object.RootObjectMapper;
|
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 */
|
/** 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>() {
|
private CloseableThreadLocal<ParseContext.InternalParseContext> cache = new CloseableThreadLocal<ParseContext.InternalParseContext>() {
|
||||||
@Override
|
@Override
|
||||||
|
@ -99,7 +98,7 @@ class DocumentParser implements Closeable {
|
||||||
|
|
||||||
reverseOrder(context);
|
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
|
// reset the context to free up memory
|
||||||
context.reset(null, null, null);
|
context.reset(null, null, null);
|
||||||
return doc;
|
return doc;
|
||||||
|
@ -116,10 +115,7 @@ class DocumentParser implements Closeable {
|
||||||
// entire type is disabled
|
// entire type is disabled
|
||||||
parser.skipChildren();
|
parser.skipChildren();
|
||||||
} else if (emptyDoc == false) {
|
} else if (emptyDoc == false) {
|
||||||
Mapper update = parseObject(context, mapping.root, true);
|
parseObjectOrNested(context, mapping.root, true);
|
||||||
if (update != null) {
|
|
||||||
context.addDynamicMappingsUpdate(update);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) {
|
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) {
|
private static MapperParsingException wrapInMapperParsingException(SourceToParse source, Throwable e) {
|
||||||
// if its already a mapper parsing exception, no need to wrap it...
|
// if its already a mapper parsing exception, no need to wrap it...
|
||||||
if (e instanceof MapperParsingException) {
|
if (e instanceof MapperParsingException) {
|
||||||
|
@ -220,10 +211,156 @@ class DocumentParser implements Closeable {
|
||||||
return new MapperParsingException("failed to parse", e);
|
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) {
|
if (mapper.isEnabled() == false) {
|
||||||
context.parser().skipChildren();
|
context.parser().skipChildren();
|
||||||
return null;
|
return;
|
||||||
}
|
}
|
||||||
XContentParser parser = context.parser();
|
XContentParser parser = context.parser();
|
||||||
|
|
||||||
|
@ -234,7 +371,7 @@ class DocumentParser implements Closeable {
|
||||||
XContentParser.Token token = parser.currentToken();
|
XContentParser.Token token = parser.currentToken();
|
||||||
if (token == XContentParser.Token.VALUE_NULL) {
|
if (token == XContentParser.Token.VALUE_NULL) {
|
||||||
// the object is null ("obj1" : null), simply bail
|
// the object is null ("obj1" : null), simply bail
|
||||||
return null;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (token.isValue()) {
|
if (token.isValue()) {
|
||||||
|
@ -256,21 +393,19 @@ class DocumentParser implements Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
ObjectMapper update = null;
|
ObjectMapper update = null;
|
||||||
update = innerParseObject(context, mapper, parser, currentFieldName, token, update);
|
innerParseObject(context, mapper, parser, currentFieldName, token);
|
||||||
// restore the enable path flag
|
// restore the enable path flag
|
||||||
if (nested.isNested()) {
|
if (nested.isNested()) {
|
||||||
nested(context, nested);
|
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) {
|
while (token != XContentParser.Token.END_OBJECT) {
|
||||||
ObjectMapper newUpdate = null;
|
|
||||||
if (token == XContentParser.Token.START_OBJECT) {
|
if (token == XContentParser.Token.START_OBJECT) {
|
||||||
newUpdate = parseObject(context, mapper, currentFieldName);
|
parseObject(context, mapper, currentFieldName);
|
||||||
} else if (token == XContentParser.Token.START_ARRAY) {
|
} else if (token == XContentParser.Token.START_ARRAY) {
|
||||||
newUpdate = parseArray(context, mapper, currentFieldName);
|
parseArray(context, mapper, currentFieldName);
|
||||||
} else if (token == XContentParser.Token.FIELD_NAME) {
|
} else if (token == XContentParser.Token.FIELD_NAME) {
|
||||||
currentFieldName = parser.currentName();
|
currentFieldName = parser.currentName();
|
||||||
} else if (token == XContentParser.Token.VALUE_NULL) {
|
} else if (token == XContentParser.Token.VALUE_NULL) {
|
||||||
|
@ -278,19 +413,11 @@ class DocumentParser implements Closeable {
|
||||||
} else if (token == null) {
|
} 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?");
|
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()) {
|
} else if (token.isValue()) {
|
||||||
newUpdate = parseValue(context, mapper, currentFieldName, token);
|
parseValue(context, mapper, currentFieldName, token);
|
||||||
}
|
}
|
||||||
token = parser.nextToken();
|
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) {
|
private static void nested(ParseContext context, ObjectMapper.Nested nested) {
|
||||||
ParseContext.Document nestedDoc = context.doc();
|
ParseContext.Document nestedDoc = context.doc();
|
||||||
|
@ -335,33 +462,29 @@ class DocumentParser implements Closeable {
|
||||||
return context;
|
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) {
|
if (mapper instanceof ObjectMapper) {
|
||||||
return parseObject(context, (ObjectMapper) mapper, false);
|
parseObjectOrNested(context, (ObjectMapper) mapper, false);
|
||||||
} else {
|
} else {
|
||||||
FieldMapper fieldMapper = (FieldMapper)mapper;
|
FieldMapper fieldMapper = (FieldMapper)mapper;
|
||||||
Mapper update = fieldMapper.parse(context);
|
Mapper update = fieldMapper.parse(context);
|
||||||
|
if (update != null) {
|
||||||
|
context.addDynamicMapper(update);
|
||||||
|
}
|
||||||
if (fieldMapper.copyTo() != null) {
|
if (fieldMapper.copyTo() != null) {
|
||||||
parseCopyFields(context, fieldMapper, fieldMapper.copyTo().copyToFields());
|
parseCopyFields(context, fieldMapper, fieldMapper.copyTo().copyToFields());
|
||||||
}
|
}
|
||||||
return update;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static ObjectMapper parseObject(final ParseContext context, ObjectMapper mapper, String currentFieldName) throws IOException {
|
private static ObjectMapper parseObject(final ParseContext context, ObjectMapper mapper, String currentFieldName) throws IOException {
|
||||||
if (currentFieldName == null) {
|
assert 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() + "]");
|
|
||||||
}
|
|
||||||
context.path().add(currentFieldName);
|
context.path().add(currentFieldName);
|
||||||
|
|
||||||
ObjectMapper update = null;
|
ObjectMapper update = null;
|
||||||
Mapper objectMapper = mapper.getMapper(currentFieldName);
|
Mapper objectMapper = mapper.getMapper(currentFieldName);
|
||||||
if (objectMapper != null) {
|
if (objectMapper != null) {
|
||||||
final Mapper subUpdate = parseObjectOrField(context, objectMapper);
|
parseObjectOrField(context, objectMapper);
|
||||||
if (subUpdate != null) {
|
|
||||||
// propagate mapping update
|
|
||||||
update = mapper.mappingUpdate(subUpdate);
|
|
||||||
}
|
|
||||||
} else {
|
} else {
|
||||||
ObjectMapper.Dynamic dynamic = mapper.dynamic();
|
ObjectMapper.Dynamic dynamic = mapper.dynamic();
|
||||||
if (dynamic == null) {
|
if (dynamic == null) {
|
||||||
|
@ -382,8 +505,9 @@ class DocumentParser implements Closeable {
|
||||||
}
|
}
|
||||||
Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
|
Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
|
||||||
objectMapper = builder.build(builderContext);
|
objectMapper = builder.build(builderContext);
|
||||||
|
context.addDynamicMapper(objectMapper);
|
||||||
context.path().add(currentFieldName);
|
context.path().add(currentFieldName);
|
||||||
update = mapper.mappingUpdate(parseAndMergeUpdate(objectMapper, context));
|
parseObjectOrField(context, objectMapper);
|
||||||
} else {
|
} else {
|
||||||
// not dynamic, read everything up to end object
|
// not dynamic, read everything up to end object
|
||||||
context.parser().skipChildren();
|
context.parser().skipChildren();
|
||||||
|
@ -394,7 +518,7 @@ class DocumentParser implements Closeable {
|
||||||
return update;
|
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;
|
String arrayFieldName = lastFieldName;
|
||||||
Mapper mapper = parentMapper.getMapper(lastFieldName);
|
Mapper mapper = parentMapper.getMapper(lastFieldName);
|
||||||
if (mapper != null) {
|
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
|
// expects an array, if so we pass the context straight to the mapper and if not
|
||||||
// we serialize the array components
|
// we serialize the array components
|
||||||
if (mapper instanceof ArrayValueMapperParser) {
|
if (mapper instanceof ArrayValueMapperParser) {
|
||||||
final Mapper subUpdate = parseObjectOrField(context, mapper);
|
parseObjectOrField(context, mapper);
|
||||||
if (subUpdate != null) {
|
|
||||||
// propagate the mapping update
|
|
||||||
return parentMapper.mappingUpdate(subUpdate);
|
|
||||||
} else {
|
} else {
|
||||||
return null;
|
parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
|
||||||
}
|
|
||||||
} else {
|
|
||||||
return parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
|
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
|
||||||
|
@ -423,31 +541,34 @@ class DocumentParser implements Closeable {
|
||||||
} else if (dynamic == ObjectMapper.Dynamic.TRUE) {
|
} else if (dynamic == ObjectMapper.Dynamic.TRUE) {
|
||||||
Mapper.Builder builder = context.root().findTemplateBuilder(context, arrayFieldName, "object");
|
Mapper.Builder builder = context.root().findTemplateBuilder(context, arrayFieldName, "object");
|
||||||
if (builder == null) {
|
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.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
|
||||||
mapper = builder.build(builderContext);
|
mapper = builder.build(builderContext);
|
||||||
if (mapper != null && mapper instanceof ArrayValueMapperParser) {
|
assert mapper != null;
|
||||||
|
if (mapper instanceof ArrayValueMapperParser) {
|
||||||
|
context.addDynamicMapper(mapper);
|
||||||
context.path().add(arrayFieldName);
|
context.path().add(arrayFieldName);
|
||||||
mapper = parseAndMergeUpdate(mapper, context);
|
parseObjectOrField(context, mapper);
|
||||||
return parentMapper.mappingUpdate(mapper);
|
|
||||||
} else {
|
} else {
|
||||||
return parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
|
parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
|
||||||
}
|
}
|
||||||
} else {
|
} 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 parser = context.parser();
|
||||||
XContentParser.Token token;
|
XContentParser.Token token;
|
||||||
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
|
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
|
||||||
if (token == XContentParser.Token.START_OBJECT) {
|
if (token == XContentParser.Token.START_OBJECT) {
|
||||||
return parseObject(context, mapper, lastFieldName);
|
parseObject(context, mapper, lastFieldName);
|
||||||
} else if (token == XContentParser.Token.START_ARRAY) {
|
} else if (token == XContentParser.Token.START_ARRAY) {
|
||||||
return parseArray(context, mapper, lastFieldName);
|
parseArray(context, mapper, lastFieldName);
|
||||||
} else if (token == XContentParser.Token.FIELD_NAME) {
|
} else if (token == XContentParser.Token.FIELD_NAME) {
|
||||||
lastFieldName = parser.currentName();
|
lastFieldName = parser.currentName();
|
||||||
} else if (token == XContentParser.Token.VALUE_NULL) {
|
} else if (token == XContentParser.Token.VALUE_NULL) {
|
||||||
|
@ -455,25 +576,20 @@ class DocumentParser implements Closeable {
|
||||||
} else if (token == null) {
|
} 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?");
|
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 {
|
} 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) {
|
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() + "]");
|
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);
|
Mapper mapper = parentMapper.getMapper(currentFieldName);
|
||||||
if (mapper != null) {
|
if (mapper != null) {
|
||||||
Mapper subUpdate = parseObjectOrField(context, mapper);
|
parseObjectOrField(context, mapper);
|
||||||
if (subUpdate == null) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
return parentMapper.mappingUpdate(subUpdate);
|
|
||||||
} else {
|
} 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 + "]");
|
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();
|
ObjectMapper.Dynamic dynamic = parentMapper.dynamic();
|
||||||
if (dynamic == null) {
|
if (dynamic == null) {
|
||||||
dynamic = dynamicOrDefault(context.root().dynamic());
|
dynamic = dynamicOrDefault(context.root().dynamic());
|
||||||
|
@ -650,7 +766,7 @@ class DocumentParser implements Closeable {
|
||||||
throw new StrictDynamicMappingException(parentMapper.fullPath(), currentFieldName);
|
throw new StrictDynamicMappingException(parentMapper.fullPath(), currentFieldName);
|
||||||
}
|
}
|
||||||
if (dynamic == ObjectMapper.Dynamic.FALSE) {
|
if (dynamic == ObjectMapper.Dynamic.FALSE) {
|
||||||
return null;
|
return;
|
||||||
}
|
}
|
||||||
final String path = context.path().pathAsText(currentFieldName);
|
final String path = context.path().pathAsText(currentFieldName);
|
||||||
final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
|
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
|
// try to not introduce a conflict
|
||||||
mapper = mapper.updateFieldType(Collections.singletonMap(path, existingFieldType));
|
mapper = mapper.updateFieldType(Collections.singletonMap(path, existingFieldType));
|
||||||
}
|
}
|
||||||
|
context.addDynamicMapper(mapper);
|
||||||
|
|
||||||
mapper = parseAndMergeUpdate(mapper, context);
|
parseObjectOrField(context, mapper);
|
||||||
|
|
||||||
ObjectMapper update = null;
|
|
||||||
if (mapper != null) {
|
|
||||||
update = parentMapper.mappingUpdate(mapper);
|
|
||||||
}
|
|
||||||
return update;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Creates instances of the fields that the current field should be copied to */
|
/** 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
|
// 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));
|
context = context.overridePath(new ContentPath(0));
|
||||||
|
|
||||||
String[] paths = Strings.splitStringToArray(field, '.');
|
// TODO: why Strings.splitStringToArray instead of String.split?
|
||||||
String fieldName = paths[paths.length-1];
|
final String[] paths = Strings.splitStringToArray(field, '.');
|
||||||
|
final String fieldName = paths[paths.length-1];
|
||||||
ObjectMapper mapper = context.root();
|
ObjectMapper mapper = context.root();
|
||||||
ObjectMapper[] mappers = new ObjectMapper[paths.length-1];
|
ObjectMapper[] mappers = new ObjectMapper[paths.length-1];
|
||||||
if (paths.length > 1) {
|
if (paths.length > 1) {
|
||||||
|
@ -745,6 +857,7 @@ class DocumentParser implements Closeable {
|
||||||
if (mapper.nested() != ObjectMapper.Nested.NO) {
|
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`");
|
throw new MapperParsingException("It is forbidden to create dynamic nested objects ([" + context.path().pathAsText(paths[i]) + "]) through `copy_to`");
|
||||||
}
|
}
|
||||||
|
context.addDynamicMapper(mapper);
|
||||||
break;
|
break;
|
||||||
case FALSE:
|
case FALSE:
|
||||||
// Maybe we should log something to tell the user that the copy_to is ignored in this case.
|
// Maybe we should log something to tell the user that the copy_to is ignored in this case.
|
||||||
|
@ -759,34 +872,8 @@ class DocumentParser implements Closeable {
|
||||||
parent = mapper;
|
parent = mapper;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
ObjectMapper update = parseDynamicValue(context, mapper, fieldName, context.parser().currentToken());
|
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);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* 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) {
|
private static ObjectMapper.Dynamic dynamicOrDefault(ObjectMapper.Dynamic dynamic) {
|
||||||
|
|
|
@ -374,7 +374,8 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
|
||||||
// this can happen if this mapper represents a mapping update
|
// this can happen if this mapper represents a mapping update
|
||||||
return this;
|
return this;
|
||||||
} else if (fieldType.getClass() != newFieldType.getClass()) {
|
} 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);
|
MultiFields updatedMultiFields = multiFields.updateFieldType(fullNameToFieldType);
|
||||||
if (fieldType == newFieldType && multiFields == updatedMultiFields) {
|
if (fieldType == newFieldType && multiFields == updatedMultiFields) {
|
||||||
|
|
|
@ -76,6 +76,7 @@ public abstract class Mapper implements ToXContent, Iterable<Mapper> {
|
||||||
return this.name;
|
return this.name;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Returns a newly built mapper. */
|
||||||
public abstract Y build(BuilderContext context);
|
public abstract Y build(BuilderContext context);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -331,13 +331,13 @@ public abstract class ParseContext {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void addDynamicMappingsUpdate(Mapper update) {
|
public void addDynamicMapper(Mapper update) {
|
||||||
in.addDynamicMappingsUpdate(update);
|
in.addDynamicMapper(update);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mapper dynamicMappingsUpdate() {
|
public List<Mapper> getDynamicMappers() {
|
||||||
return in.dynamicMappingsUpdate();
|
return in.getDynamicMappers();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -369,7 +369,7 @@ public abstract class ParseContext {
|
||||||
|
|
||||||
private AllEntries allEntries = new AllEntries();
|
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) {
|
public InternalParseContext(@Nullable Settings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, ContentPath path) {
|
||||||
this.indexSettings = indexSettings;
|
this.indexSettings = indexSettings;
|
||||||
|
@ -394,7 +394,7 @@ public abstract class ParseContext {
|
||||||
this.source = source == null ? null : sourceToParse.source();
|
this.source = source == null ? null : sourceToParse.source();
|
||||||
this.path.reset();
|
this.path.reset();
|
||||||
this.allEntries = new AllEntries();
|
this.allEntries = new AllEntries();
|
||||||
this.dynamicMappingsUpdate = null;
|
this.dynamicMappers = new ArrayList<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -536,18 +536,13 @@ public abstract class ParseContext {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void addDynamicMappingsUpdate(Mapper mapper) {
|
public void addDynamicMapper(Mapper mapper) {
|
||||||
assert mapper instanceof RootObjectMapper : mapper;
|
dynamicMappers.add(mapper);
|
||||||
if (dynamicMappingsUpdate == null) {
|
|
||||||
dynamicMappingsUpdate = mapper;
|
|
||||||
} else {
|
|
||||||
dynamicMappingsUpdate = dynamicMappingsUpdate.merge(mapper, false);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mapper dynamicMappingsUpdate() {
|
public List<Mapper> getDynamicMappers() {
|
||||||
return dynamicMappingsUpdate;
|
return dynamicMappers;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -747,12 +742,12 @@ public abstract class ParseContext {
|
||||||
public abstract StringBuilder stringBuilder();
|
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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -26,6 +26,9 @@ import org.apache.lucene.search.Query;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.common.Strings;
|
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.settings.Settings;
|
||||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
import org.elasticsearch.common.xcontent.XContentParser;
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
@ -39,9 +42,12 @@ import org.elasticsearch.index.mapper.ParseContext;
|
||||||
import org.elasticsearch.index.mapper.internal.AllFieldMapper;
|
import org.elasticsearch.index.mapper.internal.AllFieldMapper;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import static org.apache.lucene.index.IndexOptions.NONE;
|
import static org.apache.lucene.index.IndexOptions.NONE;
|
||||||
import static org.elasticsearch.index.mapper.core.TypeParsers.parseMultiField;
|
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";
|
public static final String CONTENT_TYPE = "string";
|
||||||
private static final int POSITION_INCREMENT_GAP_USE_ANALYZER = -1;
|
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 class Defaults {
|
||||||
public static final MappedFieldType FIELD_TYPE = new StringFieldType();
|
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 {
|
public static class TypeParser implements Mapper.TypeParser {
|
||||||
|
private final DeprecationLogger deprecationLogger;
|
||||||
|
|
||||||
|
public TypeParser() {
|
||||||
|
ESLogger logger = Loggers.getLogger(getClass());
|
||||||
|
this.deprecationLogger = new DeprecationLogger(logger);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mapper.Builder parse(String fieldName, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
|
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] "
|
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 + "]");
|
+ "or [keyword] field instead for field [" + fieldName + "]");
|
||||||
}*/
|
}
|
||||||
StringFieldMapper.Builder builder = new StringFieldMapper.Builder(fieldName);
|
StringFieldMapper.Builder builder = new StringFieldMapper.Builder(fieldName);
|
||||||
// hack for the fact that string can't just accept true/false for
|
// hack for the fact that string can't just accept true/false for
|
||||||
// the index property and still accepts no/not_analyzed/analyzed
|
// 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,
|
int positionIncrementGap, int ignoreAbove,
|
||||||
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
|
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
|
||||||
super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, 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] "
|
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() + "]");
|
+ "or [keyword] field instead for field [" + fieldType.name() + "]");
|
||||||
}*/
|
}
|
||||||
if (fieldType.tokenized() && fieldType.indexOptions() != NONE && fieldType().hasDocValues()) {
|
if (fieldType.tokenized() && fieldType.indexOptions() != NONE && fieldType().hasDocValues()) {
|
||||||
throw new MapperParsingException("Field [" + fieldType.name() + "] cannot be analyzed and have doc values");
|
throw new MapperParsingException("Field [" + fieldType.name() + "] cannot be analyzed and have doc values");
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,7 +25,9 @@ import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.common.Strings;
|
import org.elasticsearch.common.Strings;
|
||||||
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
|
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
|
||||||
import org.elasticsearch.common.joda.Joda;
|
import org.elasticsearch.common.joda.Joda;
|
||||||
|
import org.elasticsearch.common.logging.DeprecationLogger;
|
||||||
import org.elasticsearch.common.logging.ESLoggerFactory;
|
import org.elasticsearch.common.logging.ESLoggerFactory;
|
||||||
|
import org.elasticsearch.common.logging.Loggers;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.settings.loader.SettingsLoader;
|
import org.elasticsearch.common.settings.loader.SettingsLoader;
|
||||||
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
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.SimilarityProvider;
|
||||||
import org.elasticsearch.index.similarity.SimilarityService;
|
import org.elasticsearch.index.similarity.SimilarityService;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Map.Entry;
|
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.isArray;
|
||||||
import static org.elasticsearch.common.xcontent.support.XContentMapValues.lenientNodeBooleanValue;
|
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_POSITIONS = "positions";
|
||||||
public static final String INDEX_OPTIONS_OFFSETS = "offsets";
|
public static final String INDEX_OPTIONS_OFFSETS = "offsets";
|
||||||
|
|
||||||
private static boolean nodeBooleanValue(Object node, Mapper.TypeParser.ParserContext parserContext) {
|
private static final DeprecationLogger DEPRECATION_LOGGER = new DeprecationLogger(Loggers.getLogger(TypeParsers.class));
|
||||||
if (parserContext.indexVersionCreated().onOrAfter(Version.V_5_0_0)) {
|
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);
|
return XContentMapValues.nodeBooleanValue(node);
|
||||||
} else {
|
} 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);
|
return XContentMapValues.lenientNodeBooleanValue(node);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -81,13 +94,13 @@ public class TypeParsers {
|
||||||
builder.precisionStep(nodeIntegerValue(propNode));
|
builder.precisionStep(nodeIntegerValue(propNode));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("ignore_malformed")) {
|
} else if (propName.equals("ignore_malformed")) {
|
||||||
builder.ignoreMalformed(nodeBooleanValue(propNode, parserContext));
|
builder.ignoreMalformed(nodeBooleanValue("ignore_malformed", propNode, parserContext));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("coerce")) {
|
} else if (propName.equals("coerce")) {
|
||||||
builder.coerce(nodeBooleanValue(propNode, parserContext));
|
builder.coerce(nodeBooleanValue("coerce", propNode, parserContext));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("omit_norms")) {
|
} else if (propName.equals("omit_norms")) {
|
||||||
builder.omitNorms(nodeBooleanValue(propNode, parserContext));
|
builder.omitNorms(nodeBooleanValue("omit_norms", propNode, parserContext));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("similarity")) {
|
} else if (propName.equals("similarity")) {
|
||||||
SimilarityProvider similarityProvider = resolveSimilarity(parserContext, name, propNode.toString());
|
SimilarityProvider similarityProvider = resolveSimilarity(parserContext, name, propNode.toString());
|
||||||
|
@ -112,16 +125,16 @@ public class TypeParsers {
|
||||||
parseTermVector(name, propNode.toString(), builder);
|
parseTermVector(name, propNode.toString(), builder);
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("store_term_vectors")) {
|
} else if (propName.equals("store_term_vectors")) {
|
||||||
builder.storeTermVectors(nodeBooleanValue(propNode, parserContext));
|
builder.storeTermVectors(nodeBooleanValue("store_term_vectors", propNode, parserContext));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("store_term_vector_offsets")) {
|
} else if (propName.equals("store_term_vector_offsets")) {
|
||||||
builder.storeTermVectorOffsets(nodeBooleanValue(propNode, parserContext));
|
builder.storeTermVectorOffsets(nodeBooleanValue("store_term_vector_offsets", propNode, parserContext));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("store_term_vector_positions")) {
|
} else if (propName.equals("store_term_vector_positions")) {
|
||||||
builder.storeTermVectorPositions(nodeBooleanValue(propNode, parserContext));
|
builder.storeTermVectorPositions(nodeBooleanValue("store_term_vector_positions", propNode, parserContext));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("store_term_vector_payloads")) {
|
} else if (propName.equals("store_term_vector_payloads")) {
|
||||||
builder.storeTermVectorPayloads(nodeBooleanValue(propNode, parserContext));
|
builder.storeTermVectorPayloads(nodeBooleanValue("store_term_vector_payloads", propNode, parserContext));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("analyzer")) {
|
} else if (propName.equals("analyzer")) {
|
||||||
NamedAnalyzer analyzer = parserContext.analysisService().analyzer(propNode.toString());
|
NamedAnalyzer analyzer = parserContext.analysisService().analyzer(propNode.toString());
|
||||||
|
@ -199,13 +212,13 @@ public class TypeParsers {
|
||||||
builder.index(parseIndex(name, propNode.toString(), parserContext));
|
builder.index(parseIndex(name, propNode.toString(), parserContext));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals(DOC_VALUES)) {
|
} else if (propName.equals(DOC_VALUES)) {
|
||||||
builder.docValues(nodeBooleanValue(propNode, parserContext));
|
builder.docValues(nodeBooleanValue(DOC_VALUES, propNode, parserContext));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("boost")) {
|
} else if (propName.equals("boost")) {
|
||||||
builder.boost(nodeFloatValue(propNode));
|
builder.boost(nodeFloatValue(propNode));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("omit_norms")) {
|
} else if (propName.equals("omit_norms")) {
|
||||||
builder.omitNorms(nodeBooleanValue(propNode, parserContext));
|
builder.omitNorms(nodeBooleanValue("omit_norms", propNode, parserContext));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("norms")) {
|
} else if (propName.equals("norms")) {
|
||||||
final Map<String, Object> properties = nodeMapValue(propNode, "norms");
|
final Map<String, Object> properties = nodeMapValue(propNode, "norms");
|
||||||
|
@ -227,7 +240,7 @@ public class TypeParsers {
|
||||||
builder.indexOptions(nodeIndexOptionValue(propNode));
|
builder.indexOptions(nodeIndexOptionValue(propNode));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("include_in_all")) {
|
} else if (propName.equals("include_in_all")) {
|
||||||
builder.includeInAll(nodeBooleanValue(propNode, parserContext));
|
builder.includeInAll(nodeBooleanValue("include_in_all", propNode, parserContext));
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
} else if (propName.equals("similarity")) {
|
} else if (propName.equals("similarity")) {
|
||||||
SimilarityProvider similarityProvider = resolveSimilarity(parserContext, name, propNode.toString());
|
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 {
|
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) {
|
switch (index) {
|
||||||
case "true":
|
case "true":
|
||||||
return true;
|
return true;
|
||||||
case "false":
|
case "false":
|
||||||
return false;
|
return false;
|
||||||
default:
|
|
||||||
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 "not_analyzed":
|
||||||
case "analyzed":
|
case "analyzed":
|
||||||
return true;
|
|
||||||
case "false":
|
|
||||||
case "no":
|
case "no":
|
||||||
return false;
|
if (parserContext.parseFieldMatcher().isStrict() == false) {
|
||||||
default:
|
DEPRECATION_LOGGER.deprecated("Expected a boolean for property [index] but got [{}]", index);
|
||||||
throw new IllegalArgumentException("Can't parse [index] value [" + index + "] for field [" + fieldName + "], expected [true], [false], [no], [not_analyzed] or [analyzed]");
|
return "no".equals(index) == false;
|
||||||
|
} else {
|
||||||
|
throw new IllegalArgumentException("Can't parse [index] value [" + index + "] for field [" + fieldName + "], expected [true] or [false]");
|
||||||
}
|
}
|
||||||
|
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 {
|
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);
|
return XContentMapValues.nodeBooleanValue(store);
|
||||||
} else {
|
} else {
|
||||||
|
if (BOOLEAN_STRINGS.contains(store) == false) {
|
||||||
|
DEPRECATION_LOGGER.deprecated("Expected a boolean for property [store] but got [{}]", store);
|
||||||
|
}
|
||||||
if ("no".equals(store)) {
|
if ("no".equals(store)) {
|
||||||
return false;
|
return false;
|
||||||
} else if ("yes".equals(store)) {
|
} else if ("yes".equals(store)) {
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -19,23 +19,36 @@
|
||||||
|
|
||||||
package org.elasticsearch.ingest;
|
package org.elasticsearch.ingest;
|
||||||
|
|
||||||
|
import org.elasticsearch.ElasticsearchParseException;
|
||||||
import org.elasticsearch.action.ActionRequest;
|
import org.elasticsearch.action.ActionRequest;
|
||||||
import org.elasticsearch.action.index.IndexRequest;
|
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.Strings;
|
||||||
|
import org.elasticsearch.common.metrics.CounterMetric;
|
||||||
|
import org.elasticsearch.common.metrics.MeanMetric;
|
||||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||||
import org.elasticsearch.ingest.core.IngestDocument;
|
import org.elasticsearch.ingest.core.IngestDocument;
|
||||||
import org.elasticsearch.ingest.core.Pipeline;
|
import org.elasticsearch.ingest.core.Pipeline;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.function.BiConsumer;
|
import java.util.function.BiConsumer;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
|
|
||||||
public class PipelineExecutionService {
|
public class PipelineExecutionService implements ClusterStateListener {
|
||||||
|
|
||||||
private final PipelineStore store;
|
private final PipelineStore store;
|
||||||
private final ThreadPool threadPool;
|
private final ThreadPool threadPool;
|
||||||
|
|
||||||
|
private final StatsHolder totalStats = new StatsHolder();
|
||||||
|
private volatile Map<String, StatsHolder> statsHolderPerPipeline = Collections.emptyMap();
|
||||||
|
|
||||||
public PipelineExecutionService(PipelineStore store, ThreadPool threadPool) {
|
public PipelineExecutionService(PipelineStore store, ThreadPool threadPool) {
|
||||||
this.store = store;
|
this.store = store;
|
||||||
this.threadPool = threadPool;
|
this.threadPool = threadPool;
|
||||||
|
@ -89,7 +102,54 @@ public class PipelineExecutionService {
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public IngestStats stats() {
|
||||||
|
Map<String, StatsHolder> statsHolderPerPipeline = this.statsHolderPerPipeline;
|
||||||
|
|
||||||
|
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 {
|
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 index = indexRequest.index();
|
||||||
String type = indexRequest.type();
|
String type = indexRequest.type();
|
||||||
String id = indexRequest.id();
|
String id = indexRequest.id();
|
||||||
|
@ -112,6 +172,15 @@ public class PipelineExecutionService {
|
||||||
indexRequest.timestamp(metadataMap.get(IngestDocument.MetaData.TIMESTAMP));
|
indexRequest.timestamp(metadataMap.get(IngestDocument.MetaData.TIMESTAMP));
|
||||||
indexRequest.ttl(metadataMap.get(IngestDocument.MetaData.TTL));
|
indexRequest.ttl(metadataMap.get(IngestDocument.MetaData.TTL));
|
||||||
indexRequest.source(ingestDocument.getSourceAndMetadata());
|
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) {
|
private Pipeline getPipeline(String pipelineId) {
|
||||||
|
@ -121,4 +190,30 @@ public class PipelineExecutionService {
|
||||||
}
|
}
|
||||||
return pipeline;
|
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());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,8 +34,10 @@ import org.elasticsearch.cluster.ClusterService;
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.ClusterStateObserver;
|
import org.elasticsearch.cluster.ClusterStateObserver;
|
||||||
import org.elasticsearch.cluster.MasterNodeChangePredicate;
|
import org.elasticsearch.cluster.MasterNodeChangePredicate;
|
||||||
|
import org.elasticsearch.cluster.NodeConnectionsService;
|
||||||
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
|
import org.elasticsearch.cluster.node.DiscoveryNodeService;
|
||||||
import org.elasticsearch.cluster.routing.RoutingService;
|
import org.elasticsearch.cluster.routing.RoutingService;
|
||||||
import org.elasticsearch.cluster.service.InternalClusterService;
|
import org.elasticsearch.cluster.service.InternalClusterService;
|
||||||
import org.elasticsearch.common.StopWatch;
|
import org.elasticsearch.common.StopWatch;
|
||||||
|
@ -294,6 +296,10 @@ public class Node implements Closeable {
|
||||||
"node cluster service implementation must inherit from InternalClusterService";
|
"node cluster service implementation must inherit from InternalClusterService";
|
||||||
final InternalClusterService clusterService = (InternalClusterService) injector.getInstance(ClusterService.class);
|
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
|
// TODO hack around circular dependencies problems
|
||||||
injector.getInstance(GatewayAllocator.class).setReallocation(clusterService, injector.getInstance(RoutingService.class));
|
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
|
// 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 transportService = injector.getInstance(TransportService.class);
|
||||||
transportService.start();
|
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();
|
clusterService.start();
|
||||||
|
|
||||||
// start after cluster service so the local disco is known
|
// 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(RoutingService.class).stop();
|
||||||
injector.getInstance(ClusterService.class).stop();
|
injector.getInstance(ClusterService.class).stop();
|
||||||
injector.getInstance(Discovery.class).stop();
|
injector.getInstance(Discovery.class).stop();
|
||||||
|
injector.getInstance(NodeConnectionsService.class).stop();
|
||||||
injector.getInstance(MonitorService.class).stop();
|
injector.getInstance(MonitorService.class).stop();
|
||||||
injector.getInstance(GatewayService.class).stop();
|
injector.getInstance(GatewayService.class).stop();
|
||||||
injector.getInstance(SearchService.class).stop();
|
injector.getInstance(SearchService.class).stop();
|
||||||
|
@ -449,6 +465,8 @@ public class Node implements Closeable {
|
||||||
toClose.add(injector.getInstance(RoutingService.class));
|
toClose.add(injector.getInstance(RoutingService.class));
|
||||||
toClose.add(() -> stopWatch.stop().start("cluster"));
|
toClose.add(() -> stopWatch.stop().start("cluster"));
|
||||||
toClose.add(injector.getInstance(ClusterService.class));
|
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(() -> stopWatch.stop().start("discovery"));
|
||||||
toClose.add(injector.getInstance(Discovery.class));
|
toClose.add(injector.getInstance(Discovery.class));
|
||||||
toClose.add(() -> stopWatch.stop().start("monitor"));
|
toClose.add(() -> stopWatch.stop().start("monitor"));
|
||||||
|
|
|
@ -90,6 +90,7 @@ public class NodeService extends AbstractComponent implements Closeable {
|
||||||
this.ingestService = new IngestService(settings, threadPool, processorsRegistryBuilder);
|
this.ingestService = new IngestService(settings, threadPool, processorsRegistryBuilder);
|
||||||
this.settingsFilter = settingsFilter;
|
this.settingsFilter = settingsFilter;
|
||||||
clusterService.add(ingestService.getPipelineStore());
|
clusterService.add(ingestService.getPipelineStore());
|
||||||
|
clusterService.add(ingestService.getPipelineExecutionService());
|
||||||
}
|
}
|
||||||
|
|
||||||
// can not use constructor injection or there will be a circular dependency
|
// 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(),
|
httpServer == null ? null : httpServer.stats(),
|
||||||
circuitBreakerService.stats(),
|
circuitBreakerService.stats(),
|
||||||
scriptService.stats(),
|
scriptService.stats(),
|
||||||
discovery.stats()
|
discovery.stats(),
|
||||||
|
ingestService.getPipelineExecutionService().stats()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
public NodeStats stats(CommonStatsFlags indices, boolean os, boolean process, boolean jvm, boolean threadPool,
|
public NodeStats stats(CommonStatsFlags indices, boolean os, boolean process, boolean jvm, boolean threadPool,
|
||||||
boolean fs, boolean transport, boolean http, boolean circuitBreaker,
|
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
|
// 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)
|
// only be applied to the sensible ones to use, like refresh/merge/flush/indexing stats)
|
||||||
return new NodeStats(discovery.localNode(), System.currentTimeMillis(),
|
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,
|
http ? (httpServer == null ? null : httpServer.stats()) : null,
|
||||||
circuitBreaker ? circuitBreakerService.stats() : null,
|
circuitBreaker ? circuitBreakerService.stats() : null,
|
||||||
script ? scriptService.stats() : null,
|
script ? scriptService.stats() : null,
|
||||||
discoveryStats ? discovery.stats() : null
|
discoveryStats ? discovery.stats() : null,
|
||||||
|
ingest ? ingestService.getPipelineExecutionService().stats() : null
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -81,6 +81,7 @@ public class RestNodesStatsAction extends BaseRestHandler {
|
||||||
nodesStatsRequest.breaker(metrics.contains("breaker"));
|
nodesStatsRequest.breaker(metrics.contains("breaker"));
|
||||||
nodesStatsRequest.script(metrics.contains("script"));
|
nodesStatsRequest.script(metrics.contains("script"));
|
||||||
nodesStatsRequest.discovery(metrics.contains("discovery"));
|
nodesStatsRequest.discovery(metrics.contains("discovery"));
|
||||||
|
nodesStatsRequest.ingest(metrics.contains("ingest"));
|
||||||
|
|
||||||
// check for index specific metrics
|
// check for index specific metrics
|
||||||
if (metrics.contains("indices")) {
|
if (metrics.contains("indices")) {
|
||||||
|
@ -113,6 +114,6 @@ public class RestNodesStatsAction extends BaseRestHandler {
|
||||||
nodesStatsRequest.indices().includeSegmentFileSizes(true);
|
nodesStatsRequest.indices().includeSegmentFileSizes(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
client.admin().cluster().nodesStats(nodesStatsRequest, new RestToXContentListener<NodesStatsResponse>(channel));
|
client.admin().cluster().nodesStats(nodesStatsRequest, new RestToXContentListener<>(channel));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -92,7 +92,7 @@ public class DerivativePipelineAggregator extends PipelineAggregator {
|
||||||
for (InternalHistogram.Bucket bucket : buckets) {
|
for (InternalHistogram.Bucket bucket : buckets) {
|
||||||
Long thisBucketKey = resolveBucketKeyAsLong(bucket);
|
Long thisBucketKey = resolveBucketKeyAsLong(bucket);
|
||||||
Double thisBucketValue = resolveBucketValue(histo, bucket, bucketsPaths()[0], gapPolicy);
|
Double thisBucketValue = resolveBucketValue(histo, bucket, bucketsPaths()[0], gapPolicy);
|
||||||
if (lastBucketValue != null) {
|
if (lastBucketValue != null && thisBucketValue != null) {
|
||||||
double gradient = thisBucketValue - lastBucketValue;
|
double gradient = thisBucketValue - lastBucketValue;
|
||||||
double xDiff = -1;
|
double xDiff = -1;
|
||||||
if (xAxisUnits != null) {
|
if (xAxisUnits != null) {
|
||||||
|
|
|
@ -774,6 +774,32 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
|
||||||
return false;
|
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
|
* Adds restore completion listener
|
||||||
* <p>
|
* <p>
|
||||||
|
|
|
@ -206,7 +206,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
||||||
// Store newSnapshot here to be processed in clusterStateProcessed
|
// Store newSnapshot here to be processed in clusterStateProcessed
|
||||||
List<String> indices = Arrays.asList(indexNameExpressionResolver.concreteIndices(currentState, request.indicesOptions(), request.indices()));
|
List<String> indices = Arrays.asList(indexNameExpressionResolver.concreteIndices(currentState, request.indicesOptions(), request.indices()));
|
||||||
logger.trace("[{}][{}] creating snapshot for indices [{}]", request.repository(), request.name(), 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);
|
snapshots = new SnapshotsInProgress(newSnapshot);
|
||||||
} else {
|
} else {
|
||||||
// TODO: What should we do if a snapshot is already running?
|
// 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() {
|
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new Runnable() {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
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();
|
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
|
* Adds snapshot completion listener
|
||||||
*
|
*
|
||||||
|
@ -1302,6 +1359,15 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
||||||
return includeGlobalState;
|
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
|
* Returns master node timeout
|
||||||
*
|
*
|
||||||
|
|
|
@ -33,6 +33,7 @@ import org.elasticsearch.common.inject.Inject;
|
||||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||||
import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
|
import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.lease.Releasable;
|
||||||
import org.elasticsearch.common.lease.Releasables;
|
import org.elasticsearch.common.lease.Releasables;
|
||||||
import org.elasticsearch.common.math.MathUtils;
|
import org.elasticsearch.common.math.MathUtils;
|
||||||
import org.elasticsearch.common.metrics.CounterMetric;
|
import org.elasticsearch.common.metrics.CounterMetric;
|
||||||
|
@ -943,8 +944,8 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
|
||||||
}
|
}
|
||||||
globalLock.readLock().lock();
|
globalLock.readLock().lock();
|
||||||
try {
|
try {
|
||||||
connectionLock.acquire(node.id());
|
|
||||||
try {
|
try (Releasable ignored = connectionLock.acquire(node.id())) {
|
||||||
if (!lifecycle.started()) {
|
if (!lifecycle.started()) {
|
||||||
throw new IllegalStateException("can't add nodes to a stopped transport");
|
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) {
|
} catch (Exception e) {
|
||||||
throw new ConnectTransportException(node, "general node connection failure", e);
|
throw new ConnectTransportException(node, "general node connection failure", e);
|
||||||
}
|
}
|
||||||
} finally {
|
|
||||||
connectionLock.release(node.id());
|
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
globalLock.readLock().unlock();
|
globalLock.readLock().unlock();
|
||||||
|
@ -1103,8 +1102,8 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void disconnectFromNode(DiscoveryNode node) {
|
public void disconnectFromNode(DiscoveryNode node) {
|
||||||
connectionLock.acquire(node.id());
|
|
||||||
try {
|
try (Releasable ignored = connectionLock.acquire(node.id())) {
|
||||||
NodeChannels nodeChannels = connectedNodes.remove(node);
|
NodeChannels nodeChannels = connectedNodes.remove(node);
|
||||||
if (nodeChannels != null) {
|
if (nodeChannels != null) {
|
||||||
try {
|
try {
|
||||||
|
@ -1115,8 +1114,6 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
|
||||||
transportServiceAdapter.raiseNodeDisconnected(node);
|
transportServiceAdapter.raiseNodeDisconnected(node);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
|
||||||
connectionLock.release(node.id());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1128,8 +1125,7 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
|
||||||
// check outside of the lock
|
// check outside of the lock
|
||||||
NodeChannels nodeChannels = connectedNodes.get(node);
|
NodeChannels nodeChannels = connectedNodes.get(node);
|
||||||
if (nodeChannels != null && nodeChannels.hasChannel(channel)) {
|
if (nodeChannels != null && nodeChannels.hasChannel(channel)) {
|
||||||
connectionLock.acquire(node.id());
|
try (Releasable ignored = connectionLock.acquire(node.id())) {
|
||||||
try {
|
|
||||||
nodeChannels = connectedNodes.get(node);
|
nodeChannels = connectedNodes.get(node);
|
||||||
// check again within the connection lock, if its still applicable to remove it
|
// check again within the connection lock, if its still applicable to remove it
|
||||||
if (nodeChannels != null && nodeChannels.hasChannel(channel)) {
|
if (nodeChannels != null && nodeChannels.hasChannel(channel)) {
|
||||||
|
@ -1143,8 +1139,6 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
|
||||||
}
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
} finally {
|
|
||||||
connectionLock.release(node.id());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
|
|
|
@ -194,7 +194,7 @@ public abstract class TaskManagerTestCase extends ESTestCase {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
transportService.start();
|
transportService.start();
|
||||||
clusterService = new TestClusterService(threadPool, transportService);
|
clusterService = new TestClusterService(threadPool);
|
||||||
clusterService.add(transportService.getTaskManager());
|
clusterService.add(transportService.getTaskManager());
|
||||||
discoveryNode = new DiscoveryNode(name, transportService.boundAddress().publishAddress(), Version.CURRENT);
|
discoveryNode = new DiscoveryNode(name, transportService.boundAddress().publishAddress(), Version.CURRENT);
|
||||||
IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(settings);
|
IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(settings);
|
||||||
|
@ -238,7 +238,7 @@ public abstract class TaskManagerTestCase extends ESTestCase {
|
||||||
RecordingTaskManagerListener[] listeners = new RecordingTaskManagerListener[nodes.length];
|
RecordingTaskManagerListener[] listeners = new RecordingTaskManagerListener[nodes.length];
|
||||||
for (int i = 0; i < nodes.length; i++) {
|
for (int i = 0; i < nodes.length; i++) {
|
||||||
listeners[i] = new RecordingTaskManagerListener(nodes[i].discoveryNode, actionMasks);
|
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;
|
return listeners;
|
||||||
}
|
}
|
||||||
|
|
|
@ -43,6 +43,7 @@ import org.elasticsearch.test.tasks.MockTaskManager;
|
||||||
import org.elasticsearch.test.tasks.MockTaskManagerListener;
|
import org.elasticsearch.test.tasks.MockTaskManagerListener;
|
||||||
import org.elasticsearch.test.transport.MockTransportService;
|
import org.elasticsearch.test.transport.MockTransportService;
|
||||||
import org.elasticsearch.transport.ReceiveTimeoutTransportException;
|
import org.elasticsearch.transport.ReceiveTimeoutTransportException;
|
||||||
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -263,8 +264,8 @@ public class TasksIT extends ESIntegTestCase {
|
||||||
ReentrantLock taskFinishLock = new ReentrantLock();
|
ReentrantLock taskFinishLock = new ReentrantLock();
|
||||||
taskFinishLock.lock();
|
taskFinishLock.lock();
|
||||||
CountDownLatch taskRegistered = new CountDownLatch(1);
|
CountDownLatch taskRegistered = new CountDownLatch(1);
|
||||||
for (ClusterService clusterService : internalCluster().getInstances(ClusterService.class)) {
|
for (TransportService transportService : internalCluster().getInstances(TransportService.class)) {
|
||||||
((MockTaskManager)clusterService.getTaskManager()).addListener(new MockTaskManagerListener() {
|
((MockTaskManager) transportService.getTaskManager()).addListener(new MockTaskManagerListener() {
|
||||||
@Override
|
@Override
|
||||||
public void onTaskRegistered(Task task) {
|
public void onTaskRegistered(Task task) {
|
||||||
if (task.getAction().startsWith(IndexAction.NAME)) {
|
if (task.getAction().startsWith(IndexAction.NAME)) {
|
||||||
|
@ -408,7 +409,7 @@ public class TasksIT extends ESIntegTestCase {
|
||||||
@Override
|
@Override
|
||||||
public void tearDown() throws Exception {
|
public void tearDown() throws Exception {
|
||||||
for (Map.Entry<Tuple<String, String>, RecordingTaskManagerListener> entry : listeners.entrySet()) {
|
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();
|
listeners.clear();
|
||||||
super.tearDown();
|
super.tearDown();
|
||||||
|
@ -418,10 +419,10 @@ public class TasksIT extends ESIntegTestCase {
|
||||||
* Registers recording task event listeners with the given action mask on all nodes
|
* Registers recording task event listeners with the given action mask on all nodes
|
||||||
*/
|
*/
|
||||||
private void registerTaskManageListeners(String actionMasks) {
|
private void registerTaskManageListeners(String actionMasks) {
|
||||||
for (ClusterService clusterService : internalCluster().getInstances(ClusterService.class)) {
|
for (String nodeName : internalCluster().getNodeNames()) {
|
||||||
DiscoveryNode node = clusterService.localNode();
|
DiscoveryNode node = internalCluster().getInstance(ClusterService.class, nodeName).localNode();
|
||||||
RecordingTaskManagerListener listener = new RecordingTaskManagerListener(node, Strings.splitStringToArray(actionMasks, ','));
|
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);
|
RecordingTaskManagerListener oldListener = listeners.put(new Tuple<>(node.name(), actionMasks), listener);
|
||||||
assertNull(oldListener);
|
assertNull(oldListener);
|
||||||
}
|
}
|
||||||
|
|
|
@ -48,19 +48,7 @@ public class ClusterModuleTests extends ModuleTestCase {
|
||||||
|
|
||||||
static class FakeShardsAllocator implements ShardsAllocator {
|
static class FakeShardsAllocator implements ShardsAllocator {
|
||||||
@Override
|
@Override
|
||||||
public void applyStartedShards(StartedRerouteAllocation allocation) {}
|
public boolean allocate(RoutingAllocation 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) {
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,16 +18,12 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.cluster;
|
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.health.ClusterHealthResponse;
|
||||||
import org.elasticsearch.action.admin.cluster.tasks.PendingClusterTasksResponse;
|
import org.elasticsearch.action.admin.cluster.tasks.PendingClusterTasksResponse;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.cluster.service.InternalClusterService;
|
|
||||||
import org.elasticsearch.cluster.service.PendingClusterTask;
|
import org.elasticsearch.cluster.service.PendingClusterTask;
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
import org.elasticsearch.common.Priority;
|
import org.elasticsearch.common.Priority;
|
||||||
import org.elasticsearch.common.collect.Tuple;
|
|
||||||
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
||||||
import org.elasticsearch.common.component.LifecycleComponent;
|
import org.elasticsearch.common.component.LifecycleComponent;
|
||||||
import org.elasticsearch.common.inject.Inject;
|
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.settings.Settings;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
||||||
import org.elasticsearch.node.Node;
|
|
||||||
import org.elasticsearch.plugins.Plugin;
|
import org.elasticsearch.plugins.Plugin;
|
||||||
import org.elasticsearch.test.ESIntegTestCase;
|
import org.elasticsearch.test.ESIntegTestCase;
|
||||||
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
|
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
|
||||||
import org.elasticsearch.test.ESIntegTestCase.Scope;
|
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.test.junit.annotations.TestLogging;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
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.CountDownLatch;
|
||||||
import java.util.concurrent.CyclicBarrier;
|
|
||||||
import java.util.concurrent.Semaphore;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
|
||||||
|
|
||||||
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
|
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.equalTo;
|
||||||
import static org.hamcrest.Matchers.greaterThan;
|
import static org.hamcrest.Matchers.greaterThan;
|
||||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||||
|
@ -85,74 +67,6 @@ public class ClusterServiceIT extends ESIntegTestCase {
|
||||||
return pluginList(TestPlugin.class);
|
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 {
|
public void testAckedUpdateTask() throws Exception {
|
||||||
Settings settings = settingsBuilder()
|
Settings settings = settingsBuilder()
|
||||||
.put("discovery.type", "local")
|
.put("discovery.type", "local")
|
||||||
|
@ -299,63 +213,6 @@ public class ClusterServiceIT extends ESIntegTestCase {
|
||||||
|
|
||||||
assertThat(processedLatch.await(1, TimeUnit.SECONDS), equalTo(true));
|
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 {
|
public void testAckedUpdateTaskNoAckExpected() throws Exception {
|
||||||
Settings settings = settingsBuilder()
|
Settings settings = settingsBuilder()
|
||||||
.put("discovery.type", "local")
|
.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 {
|
public static class TestPlugin extends Plugin {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -639,6 +639,7 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
|
||||||
return new SnapshotsInProgress(new SnapshotsInProgress.Entry(
|
return new SnapshotsInProgress(new SnapshotsInProgress.Entry(
|
||||||
new SnapshotId(randomName("repo"), randomName("snap")),
|
new SnapshotId(randomName("repo"), randomName("snap")),
|
||||||
randomBoolean(),
|
randomBoolean(),
|
||||||
|
randomBoolean(),
|
||||||
SnapshotsInProgress.State.fromValue((byte) randomIntBetween(0, 6)),
|
SnapshotsInProgress.State.fromValue((byte) randomIntBetween(0, 6)),
|
||||||
Collections.<String>emptyList(),
|
Collections.<String>emptyList(),
|
||||||
Math.abs(randomLong()),
|
Math.abs(randomLong()),
|
||||||
|
|
|
@ -143,11 +143,11 @@ public class DiskUsageTests extends ESTestCase {
|
||||||
};
|
};
|
||||||
NodeStats[] nodeStats = new NodeStats[] {
|
NodeStats[] nodeStats = new NodeStats[] {
|
||||||
new NodeStats(new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, Version.CURRENT), 0,
|
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,
|
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,
|
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);
|
InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvaiableUsages, newMostAvaiableUsages);
|
||||||
DiskUsage leastNode_1 = newLeastAvaiableUsages.get("node_1");
|
DiskUsage leastNode_1 = newLeastAvaiableUsages.get("node_1");
|
||||||
|
@ -184,11 +184,11 @@ public class DiskUsageTests extends ESTestCase {
|
||||||
};
|
};
|
||||||
NodeStats[] nodeStats = new NodeStats[] {
|
NodeStats[] nodeStats = new NodeStats[] {
|
||||||
new NodeStats(new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, Version.CURRENT), 0,
|
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,
|
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,
|
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);
|
InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvailableUsages, newMostAvailableUsages);
|
||||||
DiskUsage leastNode_1 = newLeastAvailableUsages.get("node_1");
|
DiskUsage leastNode_1 = newLeastAvailableUsages.get("node_1");
|
||||||
|
|
|
@ -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() {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -27,6 +27,7 @@ import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.ClusterStateTaskExecutor;
|
import org.elasticsearch.cluster.ClusterStateTaskExecutor;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
import org.elasticsearch.cluster.node.DiscoveryNodeService;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||||
import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
||||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
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.FailedRerouteAllocation;
|
||||||
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
|
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
|
import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
|
||||||
import org.elasticsearch.cluster.service.InternalClusterService;
|
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.index.Index;
|
import org.elasticsearch.index.Index;
|
||||||
import org.elasticsearch.test.ESAllocationTestCase;
|
import org.elasticsearch.test.ESAllocationTestCase;
|
||||||
|
@ -305,7 +305,7 @@ public class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
|
||||||
return randomSubsetOf(1, shards.toArray(new ShardRouting[0])).get(0);
|
return randomSubsetOf(1, shards.toArray(new ShardRouting[0])).get(0);
|
||||||
} else {
|
} else {
|
||||||
return
|
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()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -35,7 +35,6 @@ import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
||||||
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
|
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
|
||||||
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocator;
|
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.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
|
||||||
import org.elasticsearch.common.logging.ESLogger;
|
import org.elasticsearch.common.logging.ESLogger;
|
||||||
import org.elasticsearch.common.logging.Loggers;
|
import org.elasticsearch.common.logging.Loggers;
|
||||||
|
@ -311,29 +310,9 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
|
||||||
public void testNoRebalanceOnPrimaryOverload() {
|
public void testNoRebalanceOnPrimaryOverload() {
|
||||||
Settings.Builder settings = settingsBuilder();
|
Settings.Builder settings = settingsBuilder();
|
||||||
AllocationService strategy = new AllocationService(settings.build(), randomAllocationDeciders(settings.build(),
|
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() {
|
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
|
* // this allocator tries to rebuild this scenario where a rebalance is
|
||||||
* // triggered solely by the primary overload on node [1] where a shard
|
* // triggered solely by the primary overload on node [1] where a shard
|
||||||
|
@ -355,8 +334,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
|
||||||
--------[test][3], node[3], [P], s[STARTED]
|
--------[test][3], node[3], [P], s[STARTED]
|
||||||
---- unassigned
|
---- unassigned
|
||||||
*/
|
*/
|
||||||
@Override
|
public boolean allocate(RoutingAllocation allocation) {
|
||||||
public boolean allocateUnassigned(RoutingAllocation allocation) {
|
|
||||||
RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned();
|
RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned();
|
||||||
boolean changed = !unassigned.isEmpty();
|
boolean changed = !unassigned.isEmpty();
|
||||||
ShardRouting[] drain = unassigned.drain();
|
ShardRouting[] drain = unassigned.drain();
|
||||||
|
@ -403,7 +381,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
|
||||||
}
|
}
|
||||||
return changed;
|
return changed;
|
||||||
}
|
}
|
||||||
}), EmptyClusterInfoService.INSTANCE);
|
}, EmptyClusterInfoService.INSTANCE);
|
||||||
MetaData.Builder metaDataBuilder = MetaData.builder();
|
MetaData.Builder metaDataBuilder = MetaData.builder();
|
||||||
RoutingTable.Builder routingTableBuilder = RoutingTable.builder();
|
RoutingTable.Builder routingTableBuilder = RoutingTable.builder();
|
||||||
IndexMetaData.Builder indexMeta = IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1);
|
IndexMetaData.Builder indexMeta = IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1);
|
||||||
|
|
|
@ -36,7 +36,7 @@ import org.elasticsearch.cluster.routing.RoutingTable;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
||||||
import org.elasticsearch.cluster.routing.TestShardRouting;
|
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.command.AllocationCommands;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
|
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
|
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)});
|
AllocationDeciders allocationDeciders = new AllocationDeciders(Settings.EMPTY, new AllocationDecider[] {new NodeVersionAllocationDecider(Settings.EMPTY)});
|
||||||
AllocationService strategy = new MockAllocationService(Settings.EMPTY,
|
AllocationService strategy = new MockAllocationService(Settings.EMPTY,
|
||||||
allocationDeciders,
|
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);
|
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
|
// 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();
|
state = ClusterState.builder(state).routingResult(result).build();
|
||||||
|
@ -363,7 +363,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
|
||||||
new NodeVersionAllocationDecider(Settings.EMPTY)});
|
new NodeVersionAllocationDecider(Settings.EMPTY)});
|
||||||
AllocationService strategy = new MockAllocationService(Settings.EMPTY,
|
AllocationService strategy = new MockAllocationService(Settings.EMPTY,
|
||||||
allocationDeciders,
|
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);
|
RoutingAllocation.Result result = strategy.reroute(state, new AllocationCommands(), true);
|
||||||
|
|
||||||
// Make sure that primary shards are only allocated on the new node
|
// Make sure that primary shards are only allocated on the new node
|
||||||
|
|
|
@ -30,7 +30,7 @@ import org.elasticsearch.cluster.routing.RoutingNode;
|
||||||
import org.elasticsearch.cluster.routing.RoutingTable;
|
import org.elasticsearch.cluster.routing.RoutingTable;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
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.AllocationDecider;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
|
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
|
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
|
||||||
|
@ -59,7 +59,7 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase {
|
||||||
RandomAllocationDecider randomAllocationDecider = new RandomAllocationDecider(getRandom());
|
RandomAllocationDecider randomAllocationDecider = new RandomAllocationDecider(getRandom());
|
||||||
AllocationService strategy = new AllocationService(settingsBuilder().build(), new AllocationDeciders(Settings.EMPTY,
|
AllocationService strategy = new AllocationService(settingsBuilder().build(), new AllocationDeciders(Settings.EMPTY,
|
||||||
new HashSet<>(Arrays.asList(new SameShardAllocationDecider(Settings.EMPTY), new ReplicaAfterPrimaryActiveAllocationDecider(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);
|
int indices = scaledRandomIntBetween(1, 20);
|
||||||
Builder metaBuilder = MetaData.builder();
|
Builder metaBuilder = MetaData.builder();
|
||||||
int maxNumReplicas = 1;
|
int maxNumReplicas = 1;
|
||||||
|
|
|
@ -39,7 +39,7 @@ import org.elasticsearch.cluster.routing.ShardRoutingState;
|
||||||
import org.elasticsearch.cluster.routing.TestShardRouting;
|
import org.elasticsearch.cluster.routing.TestShardRouting;
|
||||||
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
||||||
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
|
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.AllocationCommand;
|
||||||
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
|
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
|
||||||
import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
|
import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
|
||||||
|
@ -65,10 +65,6 @@ import static org.hamcrest.Matchers.nullValue;
|
||||||
|
|
||||||
public class DiskThresholdDeciderTests extends ESAllocationTestCase {
|
public class DiskThresholdDeciderTests extends ESAllocationTestCase {
|
||||||
|
|
||||||
private static ShardsAllocators makeShardsAllocators() {
|
|
||||||
return new ShardsAllocators(NoopGatewayAllocator.INSTANCE);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testDiskThreshold() {
|
public void testDiskThreshold() {
|
||||||
Settings diskSettings = settingsBuilder()
|
Settings diskSettings = settingsBuilder()
|
||||||
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
|
.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("cluster.routing.allocation.node_concurrent_recoveries", 10)
|
||||||
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
||||||
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
|
.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()
|
MetaData metaData = MetaData.builder()
|
||||||
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
|
.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("cluster.routing.allocation.node_concurrent_recoveries", 10)
|
||||||
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
||||||
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
|
.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();
|
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
|
||||||
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
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("cluster.routing.allocation.node_concurrent_recoveries", 10)
|
||||||
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
||||||
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
|
.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();
|
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
|
||||||
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
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("cluster.routing.allocation.node_concurrent_recoveries", 10)
|
||||||
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
||||||
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
|
.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()
|
MetaData metaData = MetaData.builder()
|
||||||
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(2))
|
.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("cluster.routing.allocation.node_concurrent_recoveries", 10)
|
||||||
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
||||||
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
|
.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();
|
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
|
||||||
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
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("cluster.routing.allocation.node_concurrent_recoveries", 10)
|
||||||
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
||||||
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
|
.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();
|
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
|
||||||
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
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("cluster.routing.allocation.node_concurrent_recoveries", 10)
|
||||||
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
||||||
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
|
.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();
|
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
|
||||||
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
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("cluster.routing.allocation.node_concurrent_recoveries", 10)
|
||||||
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
||||||
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
|
.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()
|
MetaData metaData = MetaData.builder()
|
||||||
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0))
|
.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("cluster.routing.allocation.node_concurrent_recoveries", 10)
|
||||||
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
||||||
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
|
.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()
|
MetaData metaData = MetaData.builder()
|
||||||
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0))
|
.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("cluster.routing.allocation.node_concurrent_recoveries", 10)
|
||||||
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
||||||
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
|
.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()
|
MetaData metaData = MetaData.builder()
|
||||||
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
|
.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("cluster.routing.allocation.node_concurrent_recoveries", 10)
|
||||||
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
||||||
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
|
.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
|
// 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.
|
// and therefor we will have sufficient disk space on node1.
|
||||||
RoutingAllocation.Result result = strategy.reroute(clusterState, "reroute");
|
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(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
|
||||||
|
|
||||||
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
|
.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");
|
RoutingAllocation.Result result = strategy.reroute(clusterState, "reroute");
|
||||||
|
|
||||||
assertThat(result.routingTable().index("test").getShards().get(0).primaryShard().state(), equalTo(STARTED));
|
assertThat(result.routingTable().index("test").getShards().get(0).primaryShard().state(), equalTo(STARTED));
|
||||||
|
|
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -492,7 +492,7 @@ public class NodeJoinControllerTests extends ESTestCase {
|
||||||
static class NoopAllocationService extends AllocationService {
|
static class NoopAllocationService extends AllocationService {
|
||||||
|
|
||||||
public NoopAllocationService(Settings settings) {
|
public NoopAllocationService(Settings settings) {
|
||||||
super(settings, null, null, null);
|
super(settings, null, null, null, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,12 +19,20 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.mapper;
|
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.bytes.BytesReference;
|
||||||
import org.elasticsearch.common.compress.CompressedXContent;
|
import org.elasticsearch.common.compress.CompressedXContent;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
||||||
|
import org.elasticsearch.index.mapper.object.ObjectMapper;
|
||||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||||
|
|
||||||
// TODO: make this a real unit test
|
// TODO: make this a real unit test
|
||||||
public class DocumentParserTests extends ESSingleNodeTestCase {
|
public class DocumentParserTests extends ESSingleNodeTestCase {
|
||||||
|
|
||||||
|
@ -61,4 +69,113 @@ public class DocumentParserTests extends ESSingleNodeTestCase {
|
||||||
assertNotNull(doc.rootDoc().getField("bar"));
|
assertNotNull(doc.rootDoc().getField("bar"));
|
||||||
assertNotNull(doc.rootDoc().getField(UidFieldMapper.NAME));
|
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"));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -42,6 +42,7 @@ import org.elasticsearch.index.mapper.core.TextFieldMapper;
|
||||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
import static java.util.Collections.emptyMap;
|
import static java.util.Collections.emptyMap;
|
||||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
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);
|
ctx.reset(XContentHelper.createParser(source.source()), new ParseContext.Document(), source);
|
||||||
assertEquals(XContentParser.Token.START_OBJECT, ctx.parser().nextToken());
|
assertEquals(XContentParser.Token.START_OBJECT, ctx.parser().nextToken());
|
||||||
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 {
|
public void testDynamicMappingsNotNeeded() throws Exception {
|
||||||
|
|
|
@ -19,12 +19,8 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.mapper;
|
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 org.elasticsearch.test.ESTestCase;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
@ -59,7 +55,7 @@ public class FieldTypeLookupTests extends ESTestCase {
|
||||||
|
|
||||||
public void testAddNewField() {
|
public void testAddNewField() {
|
||||||
FieldTypeLookup lookup = new FieldTypeLookup();
|
FieldTypeLookup lookup = new FieldTypeLookup();
|
||||||
FakeFieldMapper f = new FakeFieldMapper("foo");
|
MockFieldMapper f = new MockFieldMapper("foo");
|
||||||
FieldTypeLookup lookup2 = lookup.copyAndAddAll("type", newList(f), randomBoolean());
|
FieldTypeLookup lookup2 = lookup.copyAndAddAll("type", newList(f), randomBoolean());
|
||||||
assertNull(lookup.get("foo"));
|
assertNull(lookup.get("foo"));
|
||||||
assertNull(lookup.get("bar"));
|
assertNull(lookup.get("bar"));
|
||||||
|
@ -73,8 +69,8 @@ public class FieldTypeLookupTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testAddExistingField() {
|
public void testAddExistingField() {
|
||||||
FakeFieldMapper f = new FakeFieldMapper("foo");
|
MockFieldMapper f = new MockFieldMapper("foo");
|
||||||
FakeFieldMapper f2 = new FakeFieldMapper("foo");
|
MockFieldMapper f2 = new MockFieldMapper("foo");
|
||||||
FieldTypeLookup lookup = new FieldTypeLookup();
|
FieldTypeLookup lookup = new FieldTypeLookup();
|
||||||
lookup = lookup.copyAndAddAll("type1", newList(f), randomBoolean());
|
lookup = lookup.copyAndAddAll("type1", newList(f), randomBoolean());
|
||||||
FieldTypeLookup lookup2 = lookup.copyAndAddAll("type2", newList(f2), randomBoolean());
|
FieldTypeLookup lookup2 = lookup.copyAndAddAll("type2", newList(f2), randomBoolean());
|
||||||
|
@ -84,8 +80,8 @@ public class FieldTypeLookupTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testAddExistingIndexName() {
|
public void testAddExistingIndexName() {
|
||||||
FakeFieldMapper f = new FakeFieldMapper("foo");
|
MockFieldMapper f = new MockFieldMapper("foo");
|
||||||
FakeFieldMapper f2 = new FakeFieldMapper("bar");
|
MockFieldMapper f2 = new MockFieldMapper("bar");
|
||||||
FieldTypeLookup lookup = new FieldTypeLookup();
|
FieldTypeLookup lookup = new FieldTypeLookup();
|
||||||
lookup = lookup.copyAndAddAll("type1", newList(f), randomBoolean());
|
lookup = lookup.copyAndAddAll("type1", newList(f), randomBoolean());
|
||||||
FieldTypeLookup lookup2 = lookup.copyAndAddAll("type2", newList(f2), randomBoolean());
|
FieldTypeLookup lookup2 = lookup.copyAndAddAll("type2", newList(f2), randomBoolean());
|
||||||
|
@ -96,8 +92,8 @@ public class FieldTypeLookupTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testAddExistingFullName() {
|
public void testAddExistingFullName() {
|
||||||
FakeFieldMapper f = new FakeFieldMapper("foo");
|
MockFieldMapper f = new MockFieldMapper("foo");
|
||||||
FakeFieldMapper f2 = new FakeFieldMapper("foo");
|
MockFieldMapper f2 = new MockFieldMapper("foo");
|
||||||
FieldTypeLookup lookup = new FieldTypeLookup();
|
FieldTypeLookup lookup = new FieldTypeLookup();
|
||||||
try {
|
try {
|
||||||
lookup.copyAndAddAll("type2", newList(f2), randomBoolean());
|
lookup.copyAndAddAll("type2", newList(f2), randomBoolean());
|
||||||
|
@ -107,12 +103,13 @@ public class FieldTypeLookupTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testCheckCompatibilityMismatchedTypes() {
|
public void testCheckCompatibilityMismatchedTypes() {
|
||||||
FieldMapper f1 = new FakeFieldMapper("foo");
|
FieldMapper f1 = new MockFieldMapper("foo");
|
||||||
FieldTypeLookup lookup = new FieldTypeLookup();
|
FieldTypeLookup lookup = new FieldTypeLookup();
|
||||||
lookup = lookup.copyAndAddAll("type", newList(f1), randomBoolean());
|
lookup = lookup.copyAndAddAll("type", newList(f1), randomBoolean());
|
||||||
|
|
||||||
MappedFieldType ft2 = FakeFieldMapper.makeOtherFieldType("foo");
|
OtherFakeFieldType ft2 = new OtherFakeFieldType();
|
||||||
FieldMapper f2 = new FakeFieldMapper("foo", ft2);
|
ft2.setName("foo");
|
||||||
|
FieldMapper f2 = new MockFieldMapper("foo", ft2);
|
||||||
try {
|
try {
|
||||||
lookup.copyAndAddAll("type2", newList(f2), false);
|
lookup.copyAndAddAll("type2", newList(f2), false);
|
||||||
fail("expected type mismatch");
|
fail("expected type mismatch");
|
||||||
|
@ -129,13 +126,14 @@ public class FieldTypeLookupTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testCheckCompatibilityConflict() {
|
public void testCheckCompatibilityConflict() {
|
||||||
FieldMapper f1 = new FakeFieldMapper("foo");
|
FieldMapper f1 = new MockFieldMapper("foo");
|
||||||
FieldTypeLookup lookup = new FieldTypeLookup();
|
FieldTypeLookup lookup = new FieldTypeLookup();
|
||||||
lookup = lookup.copyAndAddAll("type", newList(f1), randomBoolean());
|
lookup = lookup.copyAndAddAll("type", newList(f1), randomBoolean());
|
||||||
|
|
||||||
MappedFieldType ft2 = FakeFieldMapper.makeFieldType("foo");
|
MappedFieldType ft2 = new MockFieldMapper.FakeFieldType();
|
||||||
|
ft2.setName("foo");
|
||||||
ft2.setBoost(2.0f);
|
ft2.setBoost(2.0f);
|
||||||
FieldMapper f2 = new FakeFieldMapper("foo", ft2);
|
FieldMapper f2 = new MockFieldMapper("foo", ft2);
|
||||||
try {
|
try {
|
||||||
// different type
|
// different type
|
||||||
lookup.copyAndAddAll("type2", newList(f2), false);
|
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("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
|
lookup.copyAndAddAll("type2", newList(f2), true); // boost is updateable, so ok if forcing
|
||||||
// now with a non changeable setting
|
// now with a non changeable setting
|
||||||
MappedFieldType ft3 = FakeFieldMapper.makeFieldType("foo");
|
MappedFieldType ft3 = new MockFieldMapper.FakeFieldType();
|
||||||
|
ft3.setName("foo");
|
||||||
ft3.setStored(true);
|
ft3.setStored(true);
|
||||||
FieldMapper f3 = new FakeFieldMapper("foo", ft3);
|
FieldMapper f3 = new MockFieldMapper("foo", ft3);
|
||||||
try {
|
try {
|
||||||
lookup.copyAndAddAll("type2", newList(f3), false);
|
lookup.copyAndAddAll("type2", newList(f3), false);
|
||||||
fail("expected conflict");
|
fail("expected conflict");
|
||||||
|
@ -165,8 +164,8 @@ public class FieldTypeLookupTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testSimpleMatchFullNames() {
|
public void testSimpleMatchFullNames() {
|
||||||
FakeFieldMapper f1 = new FakeFieldMapper("foo");
|
MockFieldMapper f1 = new MockFieldMapper("foo");
|
||||||
FakeFieldMapper f2 = new FakeFieldMapper("bar");
|
MockFieldMapper f2 = new MockFieldMapper("bar");
|
||||||
FieldTypeLookup lookup = new FieldTypeLookup();
|
FieldTypeLookup lookup = new FieldTypeLookup();
|
||||||
lookup = lookup.copyAndAddAll("type", newList(f1, f2), randomBoolean());
|
lookup = lookup.copyAndAddAll("type", newList(f1, f2), randomBoolean());
|
||||||
Collection<String> names = lookup.simpleMatchToFullName("b*");
|
Collection<String> names = lookup.simpleMatchToFullName("b*");
|
||||||
|
@ -175,7 +174,7 @@ public class FieldTypeLookupTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testIteratorImmutable() {
|
public void testIteratorImmutable() {
|
||||||
FakeFieldMapper f1 = new FakeFieldMapper("foo");
|
MockFieldMapper f1 = new MockFieldMapper("foo");
|
||||||
FieldTypeLookup lookup = new FieldTypeLookup();
|
FieldTypeLookup lookup = new FieldTypeLookup();
|
||||||
lookup = lookup.copyAndAddAll("type", newList(f1), randomBoolean());
|
lookup = lookup.copyAndAddAll("type", newList(f1), randomBoolean());
|
||||||
|
|
||||||
|
@ -194,59 +193,6 @@ public class FieldTypeLookupTests extends ESTestCase {
|
||||||
return Arrays.asList(mapper);
|
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) {
|
private int size(Iterator<MappedFieldType> iterator) {
|
||||||
if (iterator == null) {
|
if (iterator == null) {
|
||||||
throw new NullPointerException("iterator");
|
throw new NullPointerException("iterator");
|
||||||
|
@ -258,4 +204,23 @@ public class FieldTypeLookupTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
return count;
|
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";
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -28,15 +28,13 @@ import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||||
|
|
||||||
import static org.hamcrest.Matchers.containsString;
|
import static org.hamcrest.Matchers.containsString;
|
||||||
|
|
||||||
/**
|
|
||||||
*/
|
|
||||||
public class SimpleObjectMappingTests extends ESSingleNodeTestCase {
|
public class SimpleObjectMappingTests extends ESSingleNodeTestCase {
|
||||||
public void testDifferentInnerObjectTokenFailure() throws Exception {
|
public void testDifferentInnerObjectTokenFailure() throws Exception {
|
||||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||||
.endObject().endObject().string();
|
.endObject().endObject().string();
|
||||||
|
|
||||||
DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse("type", new CompressedXContent(mapping));
|
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" +
|
defaultMapper.parse("test", "type", "1", new BytesArray(" {\n" +
|
||||||
" \"object\": {\n" +
|
" \"object\": {\n" +
|
||||||
" \"array\":[\n" +
|
" \"array\":[\n" +
|
||||||
|
@ -50,10 +48,8 @@ public class SimpleObjectMappingTests extends ESSingleNodeTestCase {
|
||||||
" },\n" +
|
" },\n" +
|
||||||
" \"value\":\"value\"\n" +
|
" \"value\":\"value\"\n" +
|
||||||
" }"));
|
" }"));
|
||||||
fail();
|
});
|
||||||
} catch (MapperParsingException e) {
|
assertTrue(e.getMessage(), e.getMessage().contains("different type"));
|
||||||
// all is well
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testEmptyArrayProperties() throws Exception {
|
public void testEmptyArrayProperties() throws Exception {
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.action.bulk.BulkRequest;
|
||||||
import org.elasticsearch.action.delete.DeleteRequest;
|
import org.elasticsearch.action.delete.DeleteRequest;
|
||||||
import org.elasticsearch.action.index.IndexRequest;
|
import org.elasticsearch.action.index.IndexRequest;
|
||||||
import org.elasticsearch.action.update.UpdateRequest;
|
import org.elasticsearch.action.update.UpdateRequest;
|
||||||
|
import org.elasticsearch.common.bytes.BytesArray;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.ingest.core.CompoundProcessor;
|
import org.elasticsearch.ingest.core.CompoundProcessor;
|
||||||
import org.elasticsearch.ingest.core.IngestDocument;
|
import org.elasticsearch.ingest.core.IngestDocument;
|
||||||
|
@ -38,15 +39,16 @@ import org.mockito.ArgumentMatcher;
|
||||||
import org.mockito.invocation.InvocationOnMock;
|
import org.mockito.invocation.InvocationOnMock;
|
||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.function.BiConsumer;
|
import java.util.function.BiConsumer;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
|
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
import static org.mockito.Matchers.anyBoolean;
|
||||||
import static org.mockito.Matchers.eq;
|
import static org.mockito.Matchers.eq;
|
||||||
import static org.mockito.Matchers.any;
|
import static org.mockito.Matchers.any;
|
||||||
import static org.mockito.Matchers.anyBoolean;
|
|
||||||
import static org.mockito.Matchers.anyString;
|
import static org.mockito.Matchers.anyString;
|
||||||
import static org.mockito.Matchers.argThat;
|
import static org.mockito.Matchers.argThat;
|
||||||
import static org.mockito.Mockito.doAnswer;
|
import static org.mockito.Mockito.doAnswer;
|
||||||
|
@ -341,6 +343,43 @@ public class PipelineExecutionServiceTests extends ESTestCase {
|
||||||
verify(completionHandler, times(1)).accept(null);
|
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) {
|
private IngestDocument eqID(String index, String type, String id, Map<String, Object> source) {
|
||||||
return argThat(new IngestDocumentMatcher(index, type, id, source));
|
return argThat(new IngestDocumentMatcher(index, type, id, source));
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.metrics.sum.Sum;
|
||||||
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
|
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
|
||||||
import org.elasticsearch.search.aggregations.pipeline.derivative.Derivative;
|
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.search.aggregations.support.AggregationPath;
|
||||||
import org.elasticsearch.test.ESIntegTestCase;
|
import org.elasticsearch.test.ESIntegTestCase;
|
||||||
import org.hamcrest.Matchers;
|
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.histogram;
|
||||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.stats;
|
import static org.elasticsearch.search.aggregations.AggregationBuilders.stats;
|
||||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
|
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.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.assertAcked;
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
|
||||||
import static org.hamcrest.Matchers.closeTo;
|
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,
|
private void checkBucketKeyAndDocCount(final String msg, final Histogram.Bucket bucket, final long expectedKey,
|
||||||
final long expectedDocCount) {
|
final long expectedDocCount) {
|
||||||
assertThat(msg, bucket, notNullValue());
|
assertThat(msg, bucket, notNullValue());
|
||||||
|
|
|
@ -1813,12 +1813,15 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testDeleteIndexDuringSnapshot() throws Exception {
|
public void testCloseOrDeleteIndexDuringSnapshot() throws Exception {
|
||||||
Client client = client();
|
Client client = client();
|
||||||
|
|
||||||
boolean allowPartial = randomBoolean();
|
boolean allowPartial = randomBoolean();
|
||||||
|
|
||||||
logger.info("--> creating repository");
|
logger.info("--> creating repository");
|
||||||
|
|
||||||
|
// 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")
|
assertAcked(client.admin().cluster().preparePutRepository("test-repo")
|
||||||
.setType("mock").setSettings(Settings.settingsBuilder()
|
.setType("mock").setSettings(Settings.settingsBuilder()
|
||||||
.put("location", randomRepoPath())
|
.put("location", randomRepoPath())
|
||||||
|
@ -1826,6 +1829,15 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
|
||||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
|
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
|
||||||
.put("block_on_init", true)
|
.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");
|
createIndex("test-idx-1", "test-idx-2", "test-idx-3");
|
||||||
ensureGreen();
|
ensureGreen();
|
||||||
|
@ -1845,23 +1857,59 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
|
||||||
ListenableActionFuture<CreateSnapshotResponse> future = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap")
|
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");
|
logger.info("--> wait for block to kick in");
|
||||||
|
if (initBlocking) {
|
||||||
waitForBlock(internalCluster().getMasterName(), "test-repo", TimeValue.timeValueMinutes(1));
|
waitForBlock(internalCluster().getMasterName(), "test-repo", TimeValue.timeValueMinutes(1));
|
||||||
logger.info("--> delete some indices while snapshot is running");
|
} else {
|
||||||
client.admin().indices().prepareDelete("test-idx-1", "test-idx-2").get();
|
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");
|
logger.info("--> unblock running master node");
|
||||||
unblockNode(internalCluster().getMasterName());
|
unblockNode(internalCluster().getMasterName());
|
||||||
|
} else {
|
||||||
|
logger.info("--> unblock all data nodes");
|
||||||
|
unblockAllDataNodes("test-repo");
|
||||||
|
}
|
||||||
logger.info("--> waiting for snapshot to finish");
|
logger.info("--> waiting for snapshot to finish");
|
||||||
CreateSnapshotResponse createSnapshotResponse = future.get();
|
CreateSnapshotResponse createSnapshotResponse = future.get();
|
||||||
|
|
||||||
if (allowPartial) {
|
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().state(), equalTo((SnapshotState.PARTIAL)));
|
||||||
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
|
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
|
||||||
assertThat(createSnapshotResponse.getSnapshotInfo().failedShards(), greaterThan(0));
|
assertThat(createSnapshotResponse.getSnapshotInfo().failedShards(), greaterThan(0));
|
||||||
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), lessThan(createSnapshotResponse.getSnapshotInfo().totalShards()));
|
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), lessThan(createSnapshotResponse.getSnapshotInfo().totalShards()));
|
||||||
} else {
|
} else {
|
||||||
logger.info("Deleted index during snapshot and doesn't allow partial");
|
logger.info("Snapshot successfully completed");
|
||||||
assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo((SnapshotState.FAILED)));
|
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_", 1), new ShardSnapshotStatus("unknown-node", State.ABORTED));
|
||||||
shards.put(new ShardId("test-idx", "_na_", 2), 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<>();
|
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();
|
return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(Collections.unmodifiableList(entries))).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -80,7 +80,7 @@ public class MockLogAppender extends AppenderSkeleton {
|
||||||
protected final String logger;
|
protected final String logger;
|
||||||
protected final Level level;
|
protected final Level level;
|
||||||
protected final String message;
|
protected final String message;
|
||||||
protected boolean saw;
|
volatile boolean saw;
|
||||||
|
|
||||||
public AbstractEventExpectation(String name, String logger, Level level, String message) {
|
public AbstractEventExpectation(String name, String logger, Level level, String message) {
|
||||||
this.name = name;
|
this.name = name;
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.elasticsearch.transport.netty;
|
package org.elasticsearch.transport.netty;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.lease.Releasable;
|
||||||
import org.elasticsearch.common.util.concurrent.KeyedLock;
|
import org.elasticsearch.common.util.concurrent.KeyedLock;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import org.hamcrest.Matchers;
|
import org.hamcrest.Matchers;
|
||||||
|
@ -29,9 +30,7 @@ import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
import static org.hamcrest.Matchers.containsString;
|
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
import static org.hamcrest.Matchers.is;
|
|
||||||
import static org.hamcrest.Matchers.not;
|
import static org.hamcrest.Matchers.not;
|
||||||
|
|
||||||
public class KeyedLockTests extends ESTestCase {
|
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 {
|
public static class AcquireAndReleaseThread extends Thread {
|
||||||
private CountDownLatch startLatch;
|
private CountDownLatch startLatch;
|
||||||
|
@ -117,16 +94,16 @@ public class KeyedLockTests extends ESTestCase {
|
||||||
int numRuns = scaledRandomIntBetween(5000, 50000);
|
int numRuns = scaledRandomIntBetween(5000, 50000);
|
||||||
for (int i = 0; i < numRuns; i++) {
|
for (int i = 0; i < numRuns; i++) {
|
||||||
String curName = names[randomInt(names.length - 1)];
|
String curName = names[randomInt(names.length - 1)];
|
||||||
connectionLock.acquire(curName);
|
assert connectionLock.isHeldByCurrentThread(curName) == false;
|
||||||
try {
|
try (Releasable ignored = connectionLock.acquire(curName)) {
|
||||||
|
assert connectionLock.isHeldByCurrentThread(curName);
|
||||||
|
assert connectionLock.isHeldByCurrentThread(curName + "bla") == false;
|
||||||
Integer integer = counter.get(curName);
|
Integer integer = counter.get(curName);
|
||||||
if (integer == null) {
|
if (integer == null) {
|
||||||
counter.put(curName, 1);
|
counter.put(curName, 1);
|
||||||
} else {
|
} else {
|
||||||
counter.put(curName, integer.intValue() + 1);
|
counter.put(curName, integer.intValue() + 1);
|
||||||
}
|
}
|
||||||
} finally {
|
|
||||||
connectionLock.release(curName);
|
|
||||||
}
|
}
|
||||||
AtomicInteger atomicInteger = new AtomicInteger(0);
|
AtomicInteger atomicInteger = new AtomicInteger(0);
|
||||||
AtomicInteger value = safeCounter.putIfAbsent(curName, atomicInteger);
|
AtomicInteger value = safeCounter.putIfAbsent(curName, atomicInteger);
|
||||||
|
|
|
@ -29,7 +29,7 @@ my $Issue_URL = "http://github.com/${User_Repo}issues/";
|
||||||
|
|
||||||
my @Groups = qw(
|
my @Groups = qw(
|
||||||
breaking deprecation feature
|
breaking deprecation feature
|
||||||
enhancement bug regression upgrade build doc test
|
enhancement bug regression upgrade non-issue build docs test
|
||||||
);
|
);
|
||||||
my %Group_Labels = (
|
my %Group_Labels = (
|
||||||
breaking => 'Breaking changes',
|
breaking => 'Breaking changes',
|
||||||
|
@ -42,6 +42,7 @@ my %Group_Labels = (
|
||||||
regression => 'Regressions',
|
regression => 'Regressions',
|
||||||
test => 'Tests',
|
test => 'Tests',
|
||||||
upgrade => 'Upgrades',
|
upgrade => 'Upgrades',
|
||||||
|
"non-issue" => 'Non-issue',
|
||||||
other => 'NOT CLASSIFIED',
|
other => 'NOT CLASSIFIED',
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -157,6 +158,8 @@ sub fetch_issues {
|
||||||
ISSUE:
|
ISSUE:
|
||||||
for my $issue (@issues) {
|
for my $issue (@issues) {
|
||||||
next if $seen{ $issue->{number} } && !$issue->{pull_request};
|
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 %labels = map { $_->{name} => 1 } @{ $issue->{labels} };
|
||||||
my ($header) = map { substr( $_, 1 ) } grep {/^:/} keys %labels;
|
my ($header) = map { substr( $_, 1 ) } grep {/^:/} keys %labels;
|
||||||
$header ||= 'NOT CLASSIFIED';
|
$header ||= 'NOT CLASSIFIED';
|
||||||
|
|
|
@ -107,6 +107,7 @@ export ES_DIRECT_SIZE
|
||||||
export ES_JAVA_OPTS
|
export ES_JAVA_OPTS
|
||||||
export ES_GC_LOG_FILE
|
export ES_GC_LOG_FILE
|
||||||
export JAVA_HOME
|
export JAVA_HOME
|
||||||
|
export ES_INCLUDE
|
||||||
|
|
||||||
# Check DAEMON exists
|
# Check DAEMON exists
|
||||||
test -x $DAEMON || exit 0
|
test -x $DAEMON || exit 0
|
||||||
|
|
|
@ -66,6 +66,7 @@ export ES_JAVA_OPTS
|
||||||
export ES_GC_LOG_FILE
|
export ES_GC_LOG_FILE
|
||||||
export ES_STARTUP_SLEEP_TIME
|
export ES_STARTUP_SLEEP_TIME
|
||||||
export JAVA_HOME
|
export JAVA_HOME
|
||||||
|
export ES_INCLUDE
|
||||||
|
|
||||||
lockfile=/var/lock/subsys/$prog
|
lockfile=/var/lock/subsys/$prog
|
||||||
|
|
||||||
|
|
|
@ -85,6 +85,10 @@ REM JAVA_OPTS=%JAVA_OPTS% -XX:HeapDumpPath=$ES_HOME/logs/heapdump.hprof
|
||||||
REM Disables explicit GC
|
REM Disables explicit GC
|
||||||
set JAVA_OPTS=%JAVA_OPTS% -XX:+DisableExplicitGC
|
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)
|
REM Ensure UTF-8 encoding by default (e.g. filenames)
|
||||||
set JAVA_OPTS=%JAVA_OPTS% -Dfile.encoding=UTF-8
|
set JAVA_OPTS=%JAVA_OPTS% -Dfile.encoding=UTF-8
|
||||||
|
|
||||||
|
|
|
@ -81,6 +81,10 @@ JAVA_OPTS="$JAVA_OPTS -XX:+HeapDumpOnOutOfMemoryError"
|
||||||
# Disables explicit GC
|
# Disables explicit GC
|
||||||
JAVA_OPTS="$JAVA_OPTS -XX:+DisableExplicitGC"
|
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)
|
# Ensure UTF-8 encoding by default (e.g. filenames)
|
||||||
JAVA_OPTS="$JAVA_OPTS -Dfile.encoding=UTF-8"
|
JAVA_OPTS="$JAVA_OPTS -Dfile.encoding=UTF-8"
|
||||||
|
|
||||||
|
|
|
@ -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`>>
|
get a client is by creating a <<transport-client,`TransportClient`>>
|
||||||
that connects to a cluster.
|
that connects to a cluster.
|
||||||
|
|
||||||
*Important:*
|
[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
|
The client must have the same major version (e.g. `2.x`, or `5.x`) as the
|
||||||
major versions.
|
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]]
|
[[transport-client]]
|
||||||
|
@ -53,11 +57,23 @@ Client client = TransportClient.builder().settings(settings).build();
|
||||||
//Add transport addresses and do something with the client...
|
//Add transport addresses and do something with the client...
|
||||||
--------------------------------------------------
|
--------------------------------------------------
|
||||||
|
|
||||||
The client allows sniffing the rest of the cluster, which adds data nodes
|
The Transport client comes with a cluster sniffing feature which
|
||||||
into its list of machines to use. In this case, note that the IP addresses
|
allows it to dynamically add new hosts and remove old ones.
|
||||||
used will be the ones that the other nodes were started with (the
|
When sniffing is enabled the the transport client will connect to the nodes in its
|
||||||
"publish" address). In order to enable it, set the
|
internal node list, which is built via calls to addTransportAddress.
|
||||||
`client.transport.sniff` to `true`:
|
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]
|
[source,java]
|
||||||
--------------------------------------------------
|
--------------------------------------------------
|
||||||
|
|
|
@ -176,7 +176,7 @@ need to specify the `type` (like `string` or `date`) since it is already known.
|
||||||
[[mapper-attachments-copy-to]]
|
[[mapper-attachments-copy-to]]
|
||||||
==== Copy To feature
|
==== 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:
|
feature, you need to define it on each sub-field you want to copy to another field:
|
||||||
|
|
||||||
[source,js]
|
[source,js]
|
||||||
|
|
|
@ -60,6 +60,9 @@ of `indices`, `os`, `process`, `jvm`, `transport`, `http`,
|
||||||
`discovery`::
|
`discovery`::
|
||||||
Statistics about the discovery
|
Statistics about the discovery
|
||||||
|
|
||||||
|
`ingest`::
|
||||||
|
Statistics about ingest preprocessing
|
||||||
|
|
||||||
[source,js]
|
[source,js]
|
||||||
--------------------------------------------------
|
--------------------------------------------------
|
||||||
# return indices and os
|
# 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
|
# Some groups from just the indices stats
|
||||||
curl -XGET 'http://localhost:9200/_nodes/stats/indices?pretty&groups=foo,bar'
|
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.
|
|
@ -83,6 +83,16 @@ Cluster wide settings can be returned using:
|
||||||
curl -XGET localhost:9200/_cluster/settings
|
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
|
A list of dynamically updatable settings can be found in the
|
||||||
<<modules,Modules>> documentation.
|
<<modules,Modules>> documentation.
|
||||||
|
|
|
@ -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
|
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
|
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`
|
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` &
|
there is no guarantee the document didn't change).
|
||||||
`external_gte` are not supported.
|
|
||||||
|
[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.
|
||||||
|
|
||||||
|
=====================================================
|
||||||
|
|
|
@ -634,6 +634,9 @@ plugin.mandatory: ingest-attachment,ingest-geoip
|
||||||
|
|
||||||
A node will not start if either of these plugins are not available.
|
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-procesesor]]
|
||||||
=== Append Processor
|
=== Append Processor
|
||||||
Appends one or more values to an existing array if the field already exists and it is an array.
|
Appends one or more values to an existing array if the field already exists and it is an array.
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
[[mapping-store]]
|
[[mapping-store]]
|
||||||
=== `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
|
but they are not _stored_. This means that the field can be queried, but the
|
||||||
original field value cannot be retrieved.
|
original field value cannot be retrieved.
|
||||||
|
|
||||||
|
|
|
@ -10,7 +10,7 @@ your application to Elasticsearch 2.2.
|
||||||
==== Geo Point Type
|
==== Geo Point Type
|
||||||
|
|
||||||
The `geo_point` format has been changed to reduce index size and the time required to both index and query
|
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
|
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.
|
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
|
to inherit this setting (these are the defaults for systemd). These
|
||||||
settings can be modified by editing the `elasticsearch.service` file.
|
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]
|
[float]
|
||||||
=== Cloud AWS Plugin
|
=== Cloud AWS Plugin
|
||||||
|
|
||||||
|
|
|
@ -21,6 +21,7 @@ your application to Elasticsearch 5.0.
|
||||||
* <<breaking_50_scripting>>
|
* <<breaking_50_scripting>>
|
||||||
* <<breaking_50_term_vectors>>
|
* <<breaking_50_term_vectors>>
|
||||||
* <<breaking_50_security>>
|
* <<breaking_50_security>>
|
||||||
|
* <<breaking_50_snapshot_restore>>
|
||||||
|
|
||||||
[[breaking_50_search_changes]]
|
[[breaking_50_search_changes]]
|
||||||
=== Warmers
|
=== 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
|
to inherit this setting (these are the defaults for systemd). These
|
||||||
settings can be modified by editing the elasticsearch.service file.
|
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]]
|
[[breaking_50_scripting]]
|
||||||
=== 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
|
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.
|
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.
|
||||||
|
|
|
@ -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`.
|
`log1p`, `log2p`, `ln`, `ln1p`, `ln2p`, `square`, `sqrt`, or `reciprocal`.
|
||||||
Defaults to `none`.
|
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`::
|
`missing`::
|
||||||
|
|
||||||
Value used if the document doesn't have that field. The modifier
|
Value used if the document doesn't have that field. The modifier
|
||||||
|
|
|
@ -185,10 +185,10 @@ values separately.
|
||||||
"filter" : {
|
"filter" : {
|
||||||
"geo_bounding_box" : {
|
"geo_bounding_box" : {
|
||||||
"pin.location" : {
|
"pin.location" : {
|
||||||
"top" : -74.1,
|
"top" : 40.73,
|
||||||
"left" : 40.73,
|
"left" : -74.1,
|
||||||
"bottom" : -71.12,
|
"bottom" : 40.01,
|
||||||
"right" : 40.01
|
"right" : -71.12
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -35,7 +35,7 @@ inside the `has_child` query:
|
||||||
{
|
{
|
||||||
"has_child" : {
|
"has_child" : {
|
||||||
"type" : "blog_tag",
|
"type" : "blog_tag",
|
||||||
"score_mode" : "sum",
|
"score_mode" : "min",
|
||||||
"query" : {
|
"query" : {
|
||||||
"term" : {
|
"term" : {
|
||||||
"tag" : "something"
|
"tag" : "something"
|
||||||
|
@ -57,7 +57,7 @@ a match:
|
||||||
{
|
{
|
||||||
"has_child" : {
|
"has_child" : {
|
||||||
"type" : "blog_tag",
|
"type" : "blog_tag",
|
||||||
"score_mode" : "sum",
|
"score_mode" : "min",
|
||||||
"min_children": 2, <1>
|
"min_children": 2, <1>
|
||||||
"max_children": 10, <1>
|
"max_children": 10, <1>
|
||||||
"query" : {
|
"query" : {
|
||||||
|
|
|
@ -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."
|
"tweet": "You got no idea what I'd... what I'd give to be invisible."
|
||||||
}
|
}
|
||||||
}
|
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"_index" : "marvel",
|
"_index" : "marvel",
|
||||||
|
|
|
@ -47,6 +47,7 @@ import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -103,7 +104,8 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
|
|
||||||
protected abstract BulkRequest buildBulk(Iterable<SearchHit> docs);
|
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() {
|
public void start() {
|
||||||
initialSearch();
|
initialSearch();
|
||||||
|
@ -161,8 +163,13 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
setScroll(searchResponse.getScrollId());
|
setScroll(searchResponse.getScrollId());
|
||||||
if (searchResponse.getShardFailures() != null && searchResponse.getShardFailures().length > 0) {
|
if ( // If any of the shards failed that should abort the request.
|
||||||
startNormalTermination(emptyList(), unmodifiableList(Arrays.asList(searchResponse.getShardFailures())));
|
(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;
|
return;
|
||||||
}
|
}
|
||||||
long total = searchResponse.getHits().totalHits();
|
long total = searchResponse.getHits().totalHits();
|
||||||
|
@ -176,7 +183,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
SearchHit[] docs = searchResponse.getHits().getHits();
|
SearchHit[] docs = searchResponse.getHits().getHits();
|
||||||
logger.debug("scroll returned [{}] documents with a scroll id of [{}]", docs.length, searchResponse.getScrollId());
|
logger.debug("scroll returned [{}] documents with a scroll id of [{}]", docs.length, searchResponse.getScrollId());
|
||||||
if (docs.length == 0) {
|
if (docs.length == 0) {
|
||||||
startNormalTermination(emptyList(), emptyList());
|
startNormalTermination(emptyList(), emptyList(), false);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
task.countBatch();
|
task.countBatch();
|
||||||
|
@ -261,18 +268,18 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
throw new IllegalArgumentException("Unknown op type: " + item.getOpType());
|
throw new IllegalArgumentException("Unknown op type: " + item.getOpType());
|
||||||
}
|
}
|
||||||
// Track the indexes we've seen so we can refresh them if requested
|
// 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()) {
|
if (false == failures.isEmpty()) {
|
||||||
startNormalTermination(unmodifiableList(failures), emptyList());
|
startNormalTermination(unmodifiableList(failures), emptyList(), false);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (mainRequest.getSize() != SIZE_ALL_MATCHES && task.getSuccessfullyProcessed() >= mainRequest.getSize()) {
|
if (mainRequest.getSize() != SIZE_ALL_MATCHES && task.getSuccessfullyProcessed() >= mainRequest.getSize()) {
|
||||||
// We've processed all the requested docs.
|
// We've processed all the requested docs.
|
||||||
startNormalTermination(emptyList(), emptyList());
|
startNormalTermination(emptyList(), emptyList(), false);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
startNextScroll();
|
startNextScroll();
|
||||||
|
@ -311,9 +318,9 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
failures.add(failure);
|
failures.add(failure);
|
||||||
}
|
}
|
||||||
|
|
||||||
void startNormalTermination(List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures) {
|
void startNormalTermination(List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures, boolean timedOut) {
|
||||||
if (false == mainRequest.isRefresh()) {
|
if (task.isCancelled() || false == mainRequest.isRefresh()) {
|
||||||
finishHim(null, indexingFailures, searchFailures);
|
finishHim(null, indexingFailures, searchFailures, timedOut);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
RefreshRequest refresh = new RefreshRequest();
|
RefreshRequest refresh = new RefreshRequest();
|
||||||
|
@ -321,7 +328,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
client.admin().indices().refresh(refresh, new ActionListener<RefreshResponse>() {
|
client.admin().indices().refresh(refresh, new ActionListener<RefreshResponse>() {
|
||||||
@Override
|
@Override
|
||||||
public void onResponse(RefreshResponse response) {
|
public void onResponse(RefreshResponse response) {
|
||||||
finishHim(null, indexingFailures, searchFailures);
|
finishHim(null, indexingFailures, searchFailures, timedOut);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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
|
* @param failure if non null then the request failed catastrophically with this exception
|
||||||
*/
|
*/
|
||||||
void finishHim(Throwable failure) {
|
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 failure if non null then the request failed catastrophically with this exception
|
||||||
* @param indexingFailures any indexing failures accumulated during the request
|
* @param indexingFailures any indexing failures accumulated during the request
|
||||||
* @param searchFailures any search 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();
|
String scrollId = scroll.get();
|
||||||
if (Strings.hasLength(scrollId)) {
|
if (Strings.hasLength(scrollId)) {
|
||||||
/*
|
/*
|
||||||
|
@ -369,7 +377,8 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
if (failure == null) {
|
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 {
|
} else {
|
||||||
listener.onFailure(failure);
|
listener.onFailure(failure);
|
||||||
}
|
}
|
||||||
|
@ -382,6 +391,14 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
return exponentialBackoff(mainRequest.getRetryBackoffInitialTime(), mainRequest.getMaxRetries());
|
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.
|
* Wraps a backoffPolicy in another policy that counts the number of backoffs acquired.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -45,16 +45,18 @@ public class BulkIndexByScrollResponse extends ActionResponse implements ToXCont
|
||||||
private BulkByScrollTask.Status status;
|
private BulkByScrollTask.Status status;
|
||||||
private List<Failure> indexingFailures;
|
private List<Failure> indexingFailures;
|
||||||
private List<ShardSearchFailure> searchFailures;
|
private List<ShardSearchFailure> searchFailures;
|
||||||
|
private boolean timedOut;
|
||||||
|
|
||||||
public BulkIndexByScrollResponse() {
|
public BulkIndexByScrollResponse() {
|
||||||
}
|
}
|
||||||
|
|
||||||
public BulkIndexByScrollResponse(TimeValue took, BulkByScrollTask.Status status, List<Failure> indexingFailures,
|
public BulkIndexByScrollResponse(TimeValue took, BulkByScrollTask.Status status, List<Failure> indexingFailures,
|
||||||
List<ShardSearchFailure> searchFailures) {
|
List<ShardSearchFailure> searchFailures, boolean timedOut) {
|
||||||
this.took = took;
|
this.took = took;
|
||||||
this.status = requireNonNull(status, "Null status not supported");
|
this.status = requireNonNull(status, "Null status not supported");
|
||||||
this.indexingFailures = indexingFailures;
|
this.indexingFailures = indexingFailures;
|
||||||
this.searchFailures = searchFailures;
|
this.searchFailures = searchFailures;
|
||||||
|
this.timedOut = timedOut;
|
||||||
}
|
}
|
||||||
|
|
||||||
public TimeValue getTook() {
|
public TimeValue getTook() {
|
||||||
|
@ -103,6 +105,13 @@ public class BulkIndexByScrollResponse extends ActionResponse implements ToXCont
|
||||||
return searchFailures;
|
return searchFailures;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Did any of the sub-requests that were part of this request timeout?
|
||||||
|
*/
|
||||||
|
public boolean isTimedOut() {
|
||||||
|
return timedOut;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void writeTo(StreamOutput out) throws IOException {
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
super.writeTo(out);
|
super.writeTo(out);
|
||||||
|
@ -116,6 +125,7 @@ public class BulkIndexByScrollResponse extends ActionResponse implements ToXCont
|
||||||
for (ShardSearchFailure failure: searchFailures) {
|
for (ShardSearchFailure failure: searchFailures) {
|
||||||
failure.writeTo(out);
|
failure.writeTo(out);
|
||||||
}
|
}
|
||||||
|
out.writeBoolean(timedOut);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -135,11 +145,13 @@ public class BulkIndexByScrollResponse extends ActionResponse implements ToXCont
|
||||||
searchFailures.add(readShardSearchFailure(in));
|
searchFailures.add(readShardSearchFailure(in));
|
||||||
}
|
}
|
||||||
this.searchFailures = unmodifiableList(searchFailures);
|
this.searchFailures = unmodifiableList(searchFailures);
|
||||||
|
this.timedOut = in.readBoolean();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
builder.field("took", took.millis());
|
builder.field("took", took.millis());
|
||||||
|
builder.field("timed_out", timedOut);
|
||||||
status.innerXContent(builder, params, false, false);
|
status.innerXContent(builder, params, false, false);
|
||||||
builder.startArray("failures");
|
builder.startArray("failures");
|
||||||
for (Failure failure: indexingFailures) {
|
for (Failure failure: indexingFailures) {
|
||||||
|
|
|
@ -19,7 +19,9 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.reindex;
|
package org.elasticsearch.index.reindex;
|
||||||
|
|
||||||
|
import org.elasticsearch.ExceptionsHelper;
|
||||||
import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
|
import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
|
||||||
|
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||||
import org.elasticsearch.rest.RestChannel;
|
import org.elasticsearch.rest.RestChannel;
|
||||||
import org.elasticsearch.rest.RestStatus;
|
import org.elasticsearch.rest.RestStatus;
|
||||||
import org.elasticsearch.rest.action.support.RestToXContentListener;
|
import org.elasticsearch.rest.action.support.RestToXContentListener;
|
||||||
|
@ -35,12 +37,25 @@ public class BulkIndexByScrollResponseContentListener<R extends BulkIndexByScrol
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected RestStatus getStatus(R response) {
|
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;
|
RestStatus status = RestStatus.OK;
|
||||||
|
if (response.isTimedOut()) {
|
||||||
|
status = RestStatus.REQUEST_TIMEOUT;
|
||||||
|
}
|
||||||
for (Failure failure : response.getIndexingFailures()) {
|
for (Failure failure : response.getIndexingFailures()) {
|
||||||
if (failure.getStatus().getStatus() > status.getStatus()) {
|
if (failure.getStatus().getStatus() > status.getStatus()) {
|
||||||
status = failure.getStatus();
|
status = failure.getStatus();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
for (ShardSearchFailure failure: response.getSearchFailures()) {
|
||||||
|
RestStatus failureStatus = ExceptionsHelper.status(failure.getCause());
|
||||||
|
if (failureStatus.getStatus() > status.getStatus()) {
|
||||||
|
status = failureStatus;
|
||||||
|
}
|
||||||
|
}
|
||||||
return status;
|
return status;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -35,8 +35,9 @@ public class ReindexResponse extends BulkIndexByScrollResponse {
|
||||||
public ReindexResponse() {
|
public ReindexResponse() {
|
||||||
}
|
}
|
||||||
|
|
||||||
public ReindexResponse(TimeValue took, Status status, List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures) {
|
public ReindexResponse(TimeValue took, Status status, List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures,
|
||||||
super(took, status, indexingFailures, searchFailures);
|
boolean timedOut) {
|
||||||
|
super(took, status, indexingFailures, searchFailures, timedOut);
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getCreated() {
|
public long getCreated() {
|
||||||
|
@ -46,6 +47,7 @@ public class ReindexResponse extends BulkIndexByScrollResponse {
|
||||||
@Override
|
@Override
|
||||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
builder.field("took", getTook());
|
builder.field("took", getTook());
|
||||||
|
builder.field("timed_out", isTimedOut());
|
||||||
getStatus().innerXContent(builder, params, true, false);
|
getStatus().innerXContent(builder, params, true, false);
|
||||||
builder.startArray("failures");
|
builder.startArray("failures");
|
||||||
for (Failure failure: getIndexingFailures()) {
|
for (Failure failure: getIndexingFailures()) {
|
||||||
|
|
|
@ -107,7 +107,10 @@ public class RestUpdateByQueryAction extends
|
||||||
internalRequest.setSize(internalRequest.getSearchRequest().source().size());
|
internalRequest.setSize(internalRequest.getSearchRequest().source().size());
|
||||||
internalRequest.setPipeline(request.param("pipeline"));
|
internalRequest.setPipeline(request.param("pipeline"));
|
||||||
internalRequest.getSearchRequest().source().size(request.paramAsInt("scroll_size", scrollSize));
|
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);
|
execute(request, internalRequest, channel);
|
||||||
}
|
}
|
||||||
|
|
|
@ -191,8 +191,9 @@ public class TransportReindexAction extends HandledTransportAction<ReindexReques
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReindexResponse buildResponse(TimeValue took, List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures) {
|
protected ReindexResponse buildResponse(TimeValue took, List<Failure> indexingFailures, List<ShardSearchFailure> searchFailures,
|
||||||
return new ReindexResponse(took, task.getStatus(), indexingFailures, searchFailures);
|
boolean timedOut) {
|
||||||
|
return new ReindexResponse(took, task.getStatus(), indexingFailures, searchFailures, timedOut);
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|
|
@ -96,8 +96,8 @@ public class TransportUpdateByQueryAction extends HandledTransportAction<UpdateB
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected BulkIndexByScrollResponse buildResponse(TimeValue took, List<Failure> indexingFailures,
|
protected BulkIndexByScrollResponse buildResponse(TimeValue took, List<Failure> indexingFailures,
|
||||||
List<ShardSearchFailure> searchFailures) {
|
List<ShardSearchFailure> searchFailures, boolean timedOut) {
|
||||||
return new BulkIndexByScrollResponse(took, task.getStatus(), indexingFailures, searchFailures);
|
return new BulkIndexByScrollResponse(took, task.getStatus(), indexingFailures, searchFailures, timedOut);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.action.ActionRequest;
|
||||||
import org.elasticsearch.action.ActionRequestBuilder;
|
import org.elasticsearch.action.ActionRequestBuilder;
|
||||||
import org.elasticsearch.action.ActionResponse;
|
import org.elasticsearch.action.ActionResponse;
|
||||||
import org.elasticsearch.action.DocWriteResponse;
|
import org.elasticsearch.action.DocWriteResponse;
|
||||||
|
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
|
||||||
import org.elasticsearch.action.bulk.BackoffPolicy;
|
import org.elasticsearch.action.bulk.BackoffPolicy;
|
||||||
import org.elasticsearch.action.bulk.BulkItemResponse;
|
import org.elasticsearch.action.bulk.BulkItemResponse;
|
||||||
import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
|
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.Executor;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
|
|
||||||
import static java.util.Collections.emptyList;
|
import static java.util.Collections.emptyList;
|
||||||
import static java.util.Collections.emptyMap;
|
import static java.util.Collections.emptyMap;
|
||||||
|
import static java.util.Collections.singleton;
|
||||||
import static org.apache.lucene.util.TestUtil.randomSimpleString;
|
import static org.apache.lucene.util.TestUtil.randomSimpleString;
|
||||||
import static org.elasticsearch.action.bulk.BackoffPolicy.constantBackoff;
|
import static org.elasticsearch.action.bulk.BackoffPolicy.constantBackoff;
|
||||||
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
|
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
|
||||||
|
@ -248,15 +251,33 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
*/
|
*/
|
||||||
public void testShardFailuresAbortRequest() throws Exception {
|
public void testShardFailuresAbortRequest() throws Exception {
|
||||||
ShardSearchFailure shardFailure = new ShardSearchFailure(new RuntimeException("test"));
|
ShardSearchFailure shardFailure = new ShardSearchFailure(new RuntimeException("test"));
|
||||||
new DummyAbstractAsyncBulkByScrollAction()
|
InternalSearchResponse internalResponse = new InternalSearchResponse(null, null, null, null, false, null);
|
||||||
.onScrollResponse(new SearchResponse(null, scrollId(), 5, 4, randomLong(), new ShardSearchFailure[] { shardFailure }));
|
new DummyAbstractAsyncBulkByScrollAction().onScrollResponse(
|
||||||
|
new SearchResponse(internalResponse, scrollId(), 5, 4, randomLong(), new ShardSearchFailure[] { shardFailure }));
|
||||||
BulkIndexByScrollResponse response = listener.get();
|
BulkIndexByScrollResponse response = listener.get();
|
||||||
assertThat(response.getIndexingFailures(), emptyCollectionOf(Failure.class));
|
assertThat(response.getIndexingFailures(), emptyCollectionOf(Failure.class));
|
||||||
assertThat(response.getSearchFailures(), contains(shardFailure));
|
assertThat(response.getSearchFailures(), contains(shardFailure));
|
||||||
|
assertFalse(response.isTimedOut());
|
||||||
assertNull(response.getReasonCancelled());
|
assertNull(response.getReasonCancelled());
|
||||||
assertThat(client.scrollsCleared, contains(scrollId));
|
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.
|
* Mimicks bulk indexing failures.
|
||||||
*/
|
*/
|
||||||
|
@ -370,6 +391,32 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
assertEquals(defaultBackoffBeforeFailing, millis);
|
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 {
|
public void testCancelBeforeInitialSearch() throws Exception {
|
||||||
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.initialSearch());
|
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.initialSearch());
|
||||||
}
|
}
|
||||||
|
@ -396,8 +443,8 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
public void testCancelBeforeStartNormalTermination() throws Exception {
|
public void testCancelBeforeStartNormalTermination() throws Exception {
|
||||||
// Refresh or not doesn't matter - we don't try to refresh.
|
// Refresh or not doesn't matter - we don't try to refresh.
|
||||||
mainRequest.setRefresh(usually());
|
mainRequest.setRefresh(usually());
|
||||||
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.startNormalTermination(emptyList(), emptyList()));
|
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.startNormalTermination(emptyList(), emptyList(), false));
|
||||||
// This wouldn't return if we called refresh - the action would hang waiting for the refresh that we haven't mocked.
|
assertNull("No refresh was attempted", client.lastRefreshRequest.get());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void cancelTaskCase(Consumer<DummyAbstractAsyncBulkByScrollAction> testMe) throws Exception {
|
private void cancelTaskCase(Consumer<DummyAbstractAsyncBulkByScrollAction> testMe) throws Exception {
|
||||||
|
@ -430,8 +477,8 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected BulkIndexByScrollResponse buildResponse(TimeValue took, List<Failure> indexingFailures,
|
protected BulkIndexByScrollResponse buildResponse(TimeValue took, List<Failure> indexingFailures,
|
||||||
List<ShardSearchFailure> searchFailures) {
|
List<ShardSearchFailure> searchFailures, boolean timedOut) {
|
||||||
return new BulkIndexByScrollResponse(took, task.getStatus(), indexingFailures, searchFailures);
|
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 static class MyMockClient extends FilterClient {
|
||||||
private final List<String> scrollsCleared = new ArrayList<>();
|
private final List<String> scrollsCleared = new ArrayList<>();
|
||||||
private final AtomicInteger bulksAttempts = new AtomicInteger();
|
private final AtomicInteger bulksAttempts = new AtomicInteger();
|
||||||
|
private final AtomicReference<RefreshRequest> lastRefreshRequest = new AtomicReference<>();
|
||||||
|
|
||||||
private int bulksToReject = 0;
|
private int bulksToReject = 0;
|
||||||
|
|
||||||
|
@ -457,6 +505,11 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
protected <Request extends ActionRequest<Request>, Response extends ActionResponse,
|
protected <Request extends ActionRequest<Request>, Response extends ActionResponse,
|
||||||
RequestBuilder extends ActionRequestBuilder<Request, Response, RequestBuilder>> void doExecute(
|
RequestBuilder extends ActionRequestBuilder<Request, Response, RequestBuilder>> void doExecute(
|
||||||
Action<Request, Response, RequestBuilder> action, Request request, ActionListener<Response> listener) {
|
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) {
|
if (request instanceof ClearScrollRequest) {
|
||||||
ClearScrollRequest clearScroll = (ClearScrollRequest) request;
|
ClearScrollRequest clearScroll = (ClearScrollRequest) request;
|
||||||
scrollsCleared.addAll(clearScroll.getScrollIds());
|
scrollsCleared.addAll(clearScroll.getScrollIds());
|
||||||
|
|
|
@ -19,14 +19,12 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.reindex;
|
package org.elasticsearch.index.reindex;
|
||||||
|
|
||||||
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
|
|
||||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||||
|
|
||||||
public class ReindexBasicTests extends ReindexTestCase {
|
public class ReindexBasicTests extends ReindexTestCase {
|
||||||
|
@ -84,40 +82,4 @@ public class ReindexBasicTests extends ReindexTestCase {
|
||||||
assertThat(copy.get(), responseMatcher().created(half).batches(half, 5));
|
assertThat(copy.get(), responseMatcher().created(half).batches(half, 5));
|
||||||
assertHitCount(client().prepareSearch("dest").setTypes("half").setSize(0).get(), half);
|
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);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -102,7 +102,7 @@ public class RoundTripTests extends ESTestCase {
|
||||||
|
|
||||||
public void testReindexResponse() throws IOException {
|
public void testReindexResponse() throws IOException {
|
||||||
ReindexResponse response = new ReindexResponse(timeValueMillis(randomPositiveLong()), randomStatus(), randomIndexingFailures(),
|
ReindexResponse response = new ReindexResponse(timeValueMillis(randomPositiveLong()), randomStatus(), randomIndexingFailures(),
|
||||||
randomSearchFailures());
|
randomSearchFailures(), randomBoolean());
|
||||||
ReindexResponse tripped = new ReindexResponse();
|
ReindexResponse tripped = new ReindexResponse();
|
||||||
roundTrip(response, tripped);
|
roundTrip(response, tripped);
|
||||||
assertResponseEquals(response, tripped);
|
assertResponseEquals(response, tripped);
|
||||||
|
@ -110,7 +110,7 @@ public class RoundTripTests extends ESTestCase {
|
||||||
|
|
||||||
public void testBulkIndexByScrollResponse() throws IOException {
|
public void testBulkIndexByScrollResponse() throws IOException {
|
||||||
BulkIndexByScrollResponse response = new BulkIndexByScrollResponse(timeValueMillis(randomPositiveLong()), randomStatus(),
|
BulkIndexByScrollResponse response = new BulkIndexByScrollResponse(timeValueMillis(randomPositiveLong()), randomStatus(),
|
||||||
randomIndexingFailures(), randomSearchFailures());
|
randomIndexingFailures(), randomSearchFailures(), randomBoolean());
|
||||||
BulkIndexByScrollResponse tripped = new BulkIndexByScrollResponse();
|
BulkIndexByScrollResponse tripped = new BulkIndexByScrollResponse();
|
||||||
roundTrip(response, tripped);
|
roundTrip(response, tripped);
|
||||||
assertResponseEquals(response, tripped);
|
assertResponseEquals(response, tripped);
|
||||||
|
|
|
@ -19,12 +19,9 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.reindex;
|
package org.elasticsearch.index.reindex;
|
||||||
|
|
||||||
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
|
|
||||||
import org.elasticsearch.search.sort.SortOrder;
|
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.index.query.QueryBuilders.termQuery;
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||||
|
|
||||||
public class UpdateByQueryBasicTests extends UpdateByQueryTestCase {
|
public class UpdateByQueryBasicTests extends UpdateByQueryTestCase {
|
||||||
|
@ -64,44 +61,4 @@ public class UpdateByQueryBasicTests extends UpdateByQueryTestCase {
|
||||||
assertEquals(3, client().prepareGet("test", "test", "3").get().getVersion());
|
assertEquals(3, client().prepareGet("test", "test", "3").get().getVersion());
|
||||||
assertEquals(2, client().prepareGet("test", "test", "4").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);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -75,6 +75,7 @@
|
||||||
index: source
|
index: source
|
||||||
dest:
|
dest:
|
||||||
index: dest
|
index: dest
|
||||||
|
- is_false: timed_out
|
||||||
- match: {task: '/.+:\d+/'}
|
- match: {task: '/.+:\d+/'}
|
||||||
- set: {task: task}
|
- set: {task: task}
|
||||||
- is_false: updated
|
- is_false: updated
|
||||||
|
@ -125,7 +126,8 @@
|
||||||
- match: {failures.0.id: "1"}
|
- match: {failures.0.id: "1"}
|
||||||
- match: {failures.0.status: 409}
|
- match: {failures.0.status: 409}
|
||||||
- match: {failures.0.cause.type: version_conflict_engine_exception}
|
- 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.shard: /\d+/}
|
||||||
- match: {failures.0.cause.index: dest}
|
- match: {failures.0.cause.index: dest}
|
||||||
- is_true: took
|
- is_true: took
|
||||||
|
|
|
@ -12,6 +12,7 @@
|
||||||
- do:
|
- do:
|
||||||
update-by-query:
|
update-by-query:
|
||||||
index: test
|
index: test
|
||||||
|
- is_false: timed_out
|
||||||
- match: {updated: 1}
|
- match: {updated: 1}
|
||||||
- match: {version_conflicts: 0}
|
- match: {version_conflicts: 0}
|
||||||
- match: {batches: 1}
|
- match: {batches: 1}
|
||||||
|
@ -86,7 +87,8 @@
|
||||||
- match: {failures.0.id: "1"}
|
- match: {failures.0.id: "1"}
|
||||||
- match: {failures.0.status: 409}
|
- match: {failures.0.status: 409}
|
||||||
- match: {failures.0.cause.type: version_conflict_engine_exception}
|
- 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.shard: /\d+/}
|
||||||
- match: {failures.0.cause.index: test}
|
- match: {failures.0.cause.index: test}
|
||||||
- is_true: took
|
- is_true: took
|
||||||
|
|
|
@ -47,7 +47,7 @@ public class GceDiscoveryPlugin extends Plugin {
|
||||||
static {
|
static {
|
||||||
/*
|
/*
|
||||||
* GCE's http client changes access levels because its silly and we
|
* 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
|
* 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
|
* 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
|
* our plugin permissions don't allow core to "reach through" plugins to
|
||||||
|
|
|
@ -137,6 +137,8 @@ final class TikaImpl {
|
||||||
perms.add(new SecurityPermission("putProviderProperty.BC"));
|
perms.add(new SecurityPermission("putProviderProperty.BC"));
|
||||||
perms.add(new SecurityPermission("insertProvider"));
|
perms.add(new SecurityPermission("insertProvider"));
|
||||||
perms.add(new ReflectPermission("suppressAccessChecks"));
|
perms.add(new ReflectPermission("suppressAccessChecks"));
|
||||||
|
// xmlbeans, use by POI, needs to get the context classloader
|
||||||
|
perms.add(new RuntimePermission("getClassLoader"));
|
||||||
perms.setReadOnly();
|
perms.setReadOnly();
|
||||||
return perms;
|
return perms;
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,4 +27,6 @@ grant {
|
||||||
permission java.security.SecurityPermission "insertProvider";
|
permission java.security.SecurityPermission "insertProvider";
|
||||||
// TODO: fix POI XWPF to not do this: https://bz.apache.org/bugzilla/show_bug.cgi?id=58597
|
// TODO: fix POI XWPF to not do this: https://bz.apache.org/bugzilla/show_bug.cgi?id=58597
|
||||||
permission java.lang.reflect.ReflectPermission "suppressAccessChecks";
|
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
|
@ -137,6 +137,8 @@ final class TikaImpl {
|
||||||
perms.add(new SecurityPermission("putProviderProperty.BC"));
|
perms.add(new SecurityPermission("putProviderProperty.BC"));
|
||||||
perms.add(new SecurityPermission("insertProvider"));
|
perms.add(new SecurityPermission("insertProvider"));
|
||||||
perms.add(new ReflectPermission("suppressAccessChecks"));
|
perms.add(new ReflectPermission("suppressAccessChecks"));
|
||||||
|
// xmlbeans, use by POI, needs to get the context classloader
|
||||||
|
perms.add(new RuntimePermission("getClassLoader"));
|
||||||
perms.setReadOnly();
|
perms.setReadOnly();
|
||||||
return perms;
|
return perms;
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,4 +27,6 @@ grant {
|
||||||
permission java.security.SecurityPermission "insertProvider";
|
permission java.security.SecurityPermission "insertProvider";
|
||||||
// TODO: fix POI XWPF to not do this: https://bz.apache.org/bugzilla/show_bug.cgi?id=58597
|
// TODO: fix POI XWPF to not do this: https://bz.apache.org/bugzilla/show_bug.cgi?id=58597
|
||||||
permission java.lang.reflect.ReflectPermission "suppressAccessChecks";
|
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
Loading…
Reference in New Issue