From d862453d6822525fc7b14ec60a5496767dd7fa8e Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Tue, 5 Feb 2019 07:52:17 -0700 Subject: [PATCH 01/23] Support unknown fields in ingest pipeline map configuration (#38352) We already support unknown objects in the list of pipelines, this changes the `PipelineConfiguration` to support fields other than just `id` and `config`. Relates to #36938 --- .../action/ingest/GetPipelineResponse.java | 19 +++++++----- .../ingest/PipelineConfiguration.java | 8 ++++- .../ingest/PipelineConfigurationTests.java | 31 +++++++++++++++++-- 3 files changed, 48 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/ingest/GetPipelineResponse.java b/server/src/main/java/org/elasticsearch/action/ingest/GetPipelineResponse.java index 297a7f0efc1d2..fdbd241431394 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/GetPipelineResponse.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/GetPipelineResponse.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.ingest; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -109,13 +110,12 @@ public static GetPipelineResponse fromXContent(XContentParser parser) throws IOE while(parser.nextToken().equals(Token.FIELD_NAME)) { String pipelineId = parser.currentName(); parser.nextToken(); - XContentBuilder contentBuilder = XContentBuilder.builder(parser.contentType().xContent()); - contentBuilder.generator().copyCurrentStructure(parser); - PipelineConfiguration pipeline = - new PipelineConfiguration( - pipelineId, BytesReference.bytes(contentBuilder), contentBuilder.contentType() - ); - pipelines.add(pipeline); + try (XContentBuilder contentBuilder = XContentBuilder.builder(parser.contentType().xContent())) { + contentBuilder.generator().copyCurrentStructure(parser); + PipelineConfiguration pipeline = + new PipelineConfiguration(pipelineId, BytesReference.bytes(contentBuilder), contentBuilder.contentType()); + pipelines.add(pipeline); + } } ensureExpectedToken(XContentParser.Token.END_OBJECT, parser.currentToken(), parser::getTokenLocation); return new GetPipelineResponse(pipelines); @@ -148,6 +148,11 @@ public boolean equals(Object other) { } } + @Override + public String toString() { + return Strings.toString(this); + } + @Override public int hashCode() { int result = 1; diff --git a/server/src/main/java/org/elasticsearch/ingest/PipelineConfiguration.java b/server/src/main/java/org/elasticsearch/ingest/PipelineConfiguration.java index 6778f3d1eaa6a..81ef55ecf40a6 100644 --- a/server/src/main/java/org/elasticsearch/ingest/PipelineConfiguration.java +++ b/server/src/main/java/org/elasticsearch/ingest/PipelineConfiguration.java @@ -22,6 +22,7 @@ import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.Diff; import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -41,7 +42,7 @@ */ public final class PipelineConfiguration extends AbstractDiffable implements ToXContentObject { - private static final ObjectParser PARSER = new ObjectParser<>("pipeline_config", Builder::new); + private static final ObjectParser PARSER = new ObjectParser<>("pipeline_config", true, Builder::new); static { PARSER.declareString(Builder::setId, new ParseField("id")); PARSER.declareField((parser, builder, aVoid) -> { @@ -123,6 +124,11 @@ public static Diff readDiffFrom(StreamInput in) throws IO return readDiffFrom(PipelineConfiguration::readFrom, in); } + @Override + public String toString() { + return Strings.toString(this); + } + @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(id); diff --git a/server/src/test/java/org/elasticsearch/ingest/PipelineConfigurationTests.java b/server/src/test/java/org/elasticsearch/ingest/PipelineConfigurationTests.java index 7b134879cda45..eb1171f66a597 100644 --- a/server/src/test/java/org/elasticsearch/ingest/PipelineConfigurationTests.java +++ b/server/src/test/java/org/elasticsearch/ingest/PipelineConfigurationTests.java @@ -31,12 +31,13 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.AbstractXContentTestCase; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.function.Predicate; -public class PipelineConfigurationTests extends ESTestCase { +public class PipelineConfigurationTests extends AbstractXContentTestCase { public void testSerialization() throws IOException { PipelineConfiguration configuration = new PipelineConfiguration("1", @@ -68,4 +69,30 @@ public void testParser() throws IOException { assertEquals("{}", XContentHelper.convertToJson(parsed.getConfig(), false, parsed.getXContentType())); assertEquals("1", parsed.getId()); } + + @Override + protected PipelineConfiguration createTestInstance() { + BytesArray config; + if (randomBoolean()) { + config = new BytesArray("{}".getBytes(StandardCharsets.UTF_8)); + } else { + config = new BytesArray("{\"foo\": \"bar\"}".getBytes(StandardCharsets.UTF_8)); + } + return new PipelineConfiguration(randomAlphaOfLength(4), config, XContentType.JSON); + } + + @Override + protected PipelineConfiguration doParseInstance(XContentParser parser) throws IOException { + return PipelineConfiguration.getParser().parse(parser, null); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + return field -> field.equals("config"); + } } From 2f6afd290e4a884ea4b653fe4522917eceeb8d39 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 5 Feb 2019 16:44:18 +0100 Subject: [PATCH 02/23] Fix Concurrent Snapshot Ending And Stabilize Snapshot Finalization (#38368) * The problem in #38226 is that in some corner cases multiple calls to `endSnapshot` were made concurrently, leading to non-deterministic behavior (`beginSnapshot` was triggering a repository finalization while one that was triggered by a `deleteSnapshot` was already in progress) * Fixed by: * Making all `endSnapshot` calls originate from the cluster state being in a "completed" state (apart from on short-circuit on initializing an empty snapshot). This forced putting the failure string into `SnapshotsInProgress.Entry`. * Adding deduplication logic to `endSnapshot` * Also: * Streamlined the init behavior to work the same way (keep state on the `SnapshotsService` to decide which snapshot entries are stale) * closes #38226 --- .../cluster/SnapshotsInProgress.java | 36 +- .../snapshots/SnapshotShardsService.java | 2 - .../snapshots/SnapshotsService.java | 436 ++++++++---------- .../discovery/SnapshotDisruptionIT.java | 5 - .../DedicatedClusterSnapshotRestoreIT.java | 1 - .../SharedClusterSnapshotRestoreIT.java | 12 +- 6 files changed, 234 insertions(+), 258 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java index 565c5134d1b38..c839acf45fc80 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java @@ -24,6 +24,7 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterState.Custom; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -87,9 +88,11 @@ public static class Entry { private final ImmutableOpenMap> waitingIndices; private final long startTime; private final long repositoryStateId; + @Nullable private final String failure; public Entry(Snapshot snapshot, boolean includeGlobalState, boolean partial, State state, List indices, - long startTime, long repositoryStateId, ImmutableOpenMap shards) { + long startTime, long repositoryStateId, ImmutableOpenMap shards, + String failure) { this.state = state; this.snapshot = snapshot; this.includeGlobalState = includeGlobalState; @@ -104,15 +107,26 @@ public Entry(Snapshot snapshot, boolean includeGlobalState, boolean partial, Sta this.waitingIndices = findWaitingIndices(shards); } this.repositoryStateId = repositoryStateId; + this.failure = failure; + } + + public Entry(Snapshot snapshot, boolean includeGlobalState, boolean partial, State state, List indices, + long startTime, long repositoryStateId, ImmutableOpenMap shards) { + this(snapshot, includeGlobalState, partial, state, indices, startTime, repositoryStateId, shards, null); } public Entry(Entry entry, State state, ImmutableOpenMap shards) { this(entry.snapshot, entry.includeGlobalState, entry.partial, state, entry.indices, entry.startTime, - entry.repositoryStateId, shards); + entry.repositoryStateId, shards, entry.failure); + } + + public Entry(Entry entry, State state, ImmutableOpenMap shards, String failure) { + this(entry.snapshot, entry.includeGlobalState, entry.partial, state, entry.indices, entry.startTime, + entry.repositoryStateId, shards, failure); } public Entry(Entry entry, ImmutableOpenMap shards) { - this(entry, entry.state, shards); + this(entry, entry.state, shards, entry.failure); } public Snapshot snapshot() { @@ -151,6 +165,10 @@ public long getRepositoryStateId() { return repositoryStateId; } + public String failure() { + return failure; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -427,6 +445,12 @@ public SnapshotsInProgress(StreamInput in) throws IOException { } } long repositoryStateId = in.readLong(); + final String failure; + if (in.getVersion().onOrAfter(Version.V_7_0_0)) { + failure = in.readOptionalString(); + } else { + failure = null; + } entries[i] = new Entry(snapshot, includeGlobalState, partial, @@ -434,7 +458,8 @@ public SnapshotsInProgress(StreamInput in) throws IOException { Collections.unmodifiableList(indexBuilder), startTime, repositoryStateId, - builder.build()); + builder.build(), + failure); } this.entries = Arrays.asList(entries); } @@ -463,6 +488,9 @@ public void writeTo(StreamOutput out) throws IOException { } } out.writeLong(entry.repositoryStateId); + if (out.getVersion().onOrAfter(Version.V_7_0_0)) { + out.writeOptionalString(entry.failure); + } } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 93b078977357a..fbb0a876e8f29 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -591,8 +591,6 @@ private class SnapshotStateExecutor implements ClusterStateTaskExecutor>> snapshotCompletionListeners = new ConcurrentHashMap<>(); + // Set of snapshots that are currently being initialized by this node + private final Set initializingSnapshots = Collections.synchronizedSet(new HashSet<>()); + + // Set of snapshots that are currently being ended by this node + private final Set endingSnapshots = Collections.synchronizedSet(new HashSet<>()); + @Inject public SnapshotsService(Settings settings, ClusterService clusterService, IndexNameExpressionResolver indexNameExpressionResolver, RepositoriesService repositoriesService, ThreadPool threadPool) { @@ -207,7 +215,7 @@ public List snapshots(final String repositoryName, } final ArrayList snapshotList = new ArrayList<>(snapshotSet); CollectionUtil.timSort(snapshotList); - return Collections.unmodifiableList(snapshotList); + return unmodifiableList(snapshotList); } /** @@ -223,7 +231,7 @@ public List currentSnapshots(final String repositoryName) { snapshotList.add(inProgressSnapshot(entry)); } CollectionUtil.timSort(snapshotList); - return Collections.unmodifiableList(snapshotList); + return unmodifiableList(snapshotList); } /** @@ -280,6 +288,7 @@ public ClusterState execute(ClusterState currentState) { System.currentTimeMillis(), repositoryData.getGenId(), null); + initializingSnapshots.add(newSnapshot.snapshot()); snapshots = new SnapshotsInProgress(newSnapshot); } else { throw new ConcurrentSnapshotExecutionException(repositoryName, snapshotName, " a snapshot is already running"); @@ -290,6 +299,9 @@ public ClusterState execute(ClusterState currentState) { @Override public void onFailure(String source, Exception e) { logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to create snapshot", repositoryName, snapshotName), e); + if (newSnapshot != null) { + initializingSnapshots.remove(newSnapshot.snapshot()); + } newSnapshot = null; listener.onFailure(e); } @@ -297,7 +309,21 @@ public void onFailure(String source, Exception e) { @Override public void clusterStateProcessed(String source, ClusterState oldState, final ClusterState newState) { if (newSnapshot != null) { - beginSnapshot(newState, newSnapshot, request.partial(), listener); + final Snapshot current = newSnapshot.snapshot(); + assert initializingSnapshots.contains(current); + beginSnapshot(newState, newSnapshot, request.partial(), new ActionListener() { + @Override + public void onResponse(final Snapshot snapshot) { + initializingSnapshots.remove(snapshot); + listener.onResponse(snapshot); + } + + @Override + public void onFailure(final Exception e) { + initializingSnapshots.remove(current); + listener.onFailure(e); + } + }); } } @@ -370,6 +396,7 @@ private void beginSnapshot(final ClusterState clusterState, @Override protected void doRun() { + assert initializingSnapshots.contains(snapshot.snapshot()); Repository repository = repositoriesService.repository(snapshot.snapshot().getRepository()); MetaData metaData = clusterState.metaData(); @@ -394,9 +421,6 @@ protected void doRun() { } clusterService.submitStateUpdateTask("update_snapshot [" + snapshot.snapshot() + "]", new ClusterStateUpdateTask() { - SnapshotsInProgress.Entry endSnapshot; - String failure; - @Override public ClusterState execute(ClusterState currentState) { SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE); @@ -407,19 +431,18 @@ public ClusterState execute(ClusterState currentState) { continue; } - if (entry.state() != State.ABORTED) { - // Replace the snapshot that was just intialized - ImmutableOpenMap shards = - shards(currentState, entry.indices()); + if (entry.state() == State.ABORTED) { + entries.add(entry); + } else { + // Replace the snapshot that was just initialized + ImmutableOpenMap shards = + shards(currentState, entry.indices()); if (!partial) { Tuple, Set> indicesWithMissingShards = indicesWithMissingShards(shards, currentState.metaData()); Set missing = indicesWithMissingShards.v1(); Set closed = indicesWithMissingShards.v2(); if (missing.isEmpty() == false || closed.isEmpty() == false) { - endSnapshot = new SnapshotsInProgress.Entry(entry, State.FAILED, shards); - entries.add(endSnapshot); - final StringBuilder failureMessage = new StringBuilder(); if (missing.isEmpty() == false) { failureMessage.append("Indices don't have primary shards "); @@ -432,24 +455,15 @@ public ClusterState execute(ClusterState currentState) { failureMessage.append("Indices are closed "); failureMessage.append(closed); } - failure = failureMessage.toString(); + entries.add(new SnapshotsInProgress.Entry(entry, State.FAILED, shards, failureMessage.toString())); continue; } } - SnapshotsInProgress.Entry updatedSnapshot = new SnapshotsInProgress.Entry(entry, State.STARTED, shards); - entries.add(updatedSnapshot); - if (completed(shards.values())) { - endSnapshot = updatedSnapshot; - } - } else { - assert entry.state() == State.ABORTED : "expecting snapshot to be aborted during initialization"; - failure = "snapshot was aborted during initialization"; - endSnapshot = entry; - entries.add(endSnapshot); + entries.add(new SnapshotsInProgress.Entry(entry, State.STARTED, shards)); } } return ClusterState.builder(currentState) - .putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(Collections.unmodifiableList(entries))) + .putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(unmodifiableList(entries))) .build(); } @@ -477,14 +491,6 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS // completion listener in this method. For the snapshot completion to work properly, the snapshot // should still exist when listener is registered. userCreateSnapshotListener.onResponse(snapshot.snapshot()); - - // Now that snapshot completion listener is registered we can end the snapshot if needed - // We should end snapshot only if 1) we didn't accept it for processing (which happens when there - // is nothing to do) and 2) there was a snapshot in metadata that we should end. Otherwise we should - // go ahead and continue working on this snapshot rather then end here. - if (endSnapshot != null) { - endSnapshot(endSnapshot, failure); - } } }); } @@ -552,7 +558,7 @@ private void cleanupAfterError(Exception exception) { } - private SnapshotInfo inProgressSnapshot(SnapshotsInProgress.Entry entry) { + private static SnapshotInfo inProgressSnapshot(SnapshotsInProgress.Entry entry) { return new SnapshotInfo(entry.snapshot().getSnapshotId(), entry.indices().stream().map(IndexId::getName).collect(Collectors.toList()), entry.startTime(), entry.includeGlobalState()); @@ -610,7 +616,7 @@ public List currentSnapshots(final String repository, builder.add(entry); } } - return Collections.unmodifiableList(builder); + return unmodifiableList(builder); } /** @@ -666,7 +672,7 @@ public Map snapshotShards(final String reposi return unmodifiableMap(shardStatus); } - private SnapshotShardFailure findShardFailure(List shardFailures, ShardId shardId) { + private static SnapshotShardFailure findShardFailure(List shardFailures, ShardId shardId) { for (SnapshotShardFailure shardFailure : shardFailures) { if (shardId.getIndexName().equals(shardFailure.index()) && shardId.getId() == shardFailure.shardId()) { return shardFailure; @@ -680,14 +686,27 @@ public void applyClusterState(ClusterChangedEvent event) { try { if (event.localNodeMaster()) { // We don't remove old master when master flips anymore. So, we need to check for change in master - if (event.nodesRemoved() || event.previousState().nodes().isLocalNodeElectedMaster() == false) { - processSnapshotsOnRemovedNodes(event); + final SnapshotsInProgress snapshotsInProgress = event.state().custom(SnapshotsInProgress.TYPE); + if (snapshotsInProgress != null) { + if (removedNodesCleanupNeeded(snapshotsInProgress, event.nodesDelta().removedNodes())) { + processSnapshotsOnRemovedNodes(); + } + if (event.routingTableChanged() && waitingShardsStartedOrUnassigned(snapshotsInProgress, event)) { + processStartedShards(); + } + // Cleanup all snapshots that have no more work left: + // 1. Completed snapshots + // 2. Snapshots in state INIT that the previous master failed to start + // 3. Snapshots in any other state that have all their shard tasks completed + snapshotsInProgress.entries().stream().filter( + entry -> entry.state().completed() + || entry.state() == State.INIT && initializingSnapshots.contains(entry.snapshot()) == false + || entry.state() != State.INIT && completed(entry.shards().values()) + ).forEach(this::endSnapshot); } - if (event.routingTableChanged()) { - processStartedShards(event); + if (event.previousState().nodes().isLocalNodeElectedMaster() == false) { + finalizeSnapshotDeletionFromPreviousMaster(event); } - removeFinishedSnapshotFromClusterState(event); - finalizeSnapshotDeletionFromPreviousMaster(event); } } catch (Exception e) { logger.warn("Failed to update snapshot state ", e); @@ -706,166 +725,134 @@ public void applyClusterState(ClusterChangedEvent event) { * snapshot was deleted and a call to GET snapshots would reveal that the snapshot no longer exists. */ private void finalizeSnapshotDeletionFromPreviousMaster(ClusterChangedEvent event) { - if (event.localNodeMaster() && event.previousState().nodes().isLocalNodeElectedMaster() == false) { - SnapshotDeletionsInProgress deletionsInProgress = event.state().custom(SnapshotDeletionsInProgress.TYPE); - if (deletionsInProgress != null && deletionsInProgress.hasDeletionsInProgress()) { - assert deletionsInProgress.getEntries().size() == 1 : "only one in-progress deletion allowed per cluster"; - SnapshotDeletionsInProgress.Entry entry = deletionsInProgress.getEntries().get(0); - deleteSnapshotFromRepository(entry.getSnapshot(), null, entry.getRepositoryStateId()); - } + SnapshotDeletionsInProgress deletionsInProgress = event.state().custom(SnapshotDeletionsInProgress.TYPE); + if (deletionsInProgress != null && deletionsInProgress.hasDeletionsInProgress()) { + assert deletionsInProgress.getEntries().size() == 1 : "only one in-progress deletion allowed per cluster"; + SnapshotDeletionsInProgress.Entry entry = deletionsInProgress.getEntries().get(0); + deleteSnapshotFromRepository(entry.getSnapshot(), null, entry.getRepositoryStateId()); } } /** - * Removes a finished snapshot from the cluster state. This can happen if the previous - * master node processed a cluster state update that marked the snapshot as finished, - * but the previous master node died before removing the snapshot in progress from the - * cluster state. It is then the responsibility of the new master node to end the - * snapshot and remove it from the cluster state. + * Cleans up shard snapshots that were running on removed nodes */ - private void removeFinishedSnapshotFromClusterState(ClusterChangedEvent event) { - if (event.localNodeMaster() && !event.previousState().nodes().isLocalNodeElectedMaster()) { - SnapshotsInProgress snapshotsInProgress = event.state().custom(SnapshotsInProgress.TYPE); - if (snapshotsInProgress != null && !snapshotsInProgress.entries().isEmpty()) { - for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { - if (entry.state().completed()) { - endSnapshot(entry); + private void processSnapshotsOnRemovedNodes() { + clusterService.submitStateUpdateTask("update snapshot state after node removal", new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + DiscoveryNodes nodes = currentState.nodes(); + SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE); + if (snapshots == null) { + return currentState; + } + boolean changed = false; + ArrayList entries = new ArrayList<>(); + for (final SnapshotsInProgress.Entry snapshot : snapshots.entries()) { + SnapshotsInProgress.Entry updatedSnapshot = snapshot; + if (snapshot.state() == State.STARTED || snapshot.state() == State.ABORTED) { + ImmutableOpenMap.Builder shards = ImmutableOpenMap.builder(); + boolean snapshotChanged = false; + for (ObjectObjectCursor shardEntry : snapshot.shards()) { + ShardSnapshotStatus shardStatus = shardEntry.value; + if (!shardStatus.state().completed() && shardStatus.nodeId() != null) { + if (nodes.nodeExists(shardStatus.nodeId())) { + shards.put(shardEntry.key, shardEntry.value); + } else { + // TODO: Restart snapshot on another node? + snapshotChanged = true; + logger.warn("failing snapshot of shard [{}] on closed node [{}]", + shardEntry.key, shardStatus.nodeId()); + shards.put(shardEntry.key, + new ShardSnapshotStatus(shardStatus.nodeId(), State.FAILED, "node shutdown")); + } + } + } + if (snapshotChanged) { + changed = true; + ImmutableOpenMap shardsMap = shards.build(); + if (!snapshot.state().completed() && completed(shardsMap.values())) { + updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, State.SUCCESS, shardsMap); + } else { + updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, snapshot.state(), shardsMap); + } + } + entries.add(updatedSnapshot); + } else if (snapshot.state() == State.INIT && initializingSnapshots.contains(snapshot.snapshot()) == false) { + changed = true; + // Mark the snapshot as aborted as it failed to start from the previous master + updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, State.ABORTED, snapshot.shards()); + entries.add(updatedSnapshot); + + // Clean up the snapshot that failed to start from the old master + deleteSnapshot(snapshot.snapshot(), new ActionListener() { + @Override + public void onResponse(Void aVoid) { + logger.debug("cleaned up abandoned snapshot {} in INIT state", snapshot.snapshot()); + } + + @Override + public void onFailure(Exception e) { + logger.warn("failed to clean up abandoned snapshot {} in INIT state", snapshot.snapshot()); + } + }, updatedSnapshot.getRepositoryStateId(), false); } } + if (changed) { + return ClusterState.builder(currentState) + .putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(unmodifiableList(entries))).build(); + } + return currentState; } - } + + @Override + public void onFailure(String source, Exception e) { + logger.warn("failed to update snapshot state after node removal"); + } + }); } - /** - * Cleans up shard snapshots that were running on removed nodes - * - * @param event cluster changed event - */ - private void processSnapshotsOnRemovedNodes(ClusterChangedEvent event) { - if (removedNodesCleanupNeeded(event)) { - // Check if we just became the master - final boolean newMaster = !event.previousState().nodes().isLocalNodeElectedMaster(); - clusterService.submitStateUpdateTask("update snapshot state after node removal", new ClusterStateUpdateTask() { - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - DiscoveryNodes nodes = currentState.nodes(); - SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE); - if (snapshots == null) { - return currentState; - } + private void processStartedShards() { + clusterService.submitStateUpdateTask("update snapshot state after shards started", new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + RoutingTable routingTable = currentState.routingTable(); + SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE); + if (snapshots != null) { boolean changed = false; ArrayList entries = new ArrayList<>(); for (final SnapshotsInProgress.Entry snapshot : snapshots.entries()) { SnapshotsInProgress.Entry updatedSnapshot = snapshot; - boolean snapshotChanged = false; - if (snapshot.state() == State.STARTED || snapshot.state() == State.ABORTED) { - ImmutableOpenMap.Builder shards = ImmutableOpenMap.builder(); - for (ObjectObjectCursor shardEntry : snapshot.shards()) { - ShardSnapshotStatus shardStatus = shardEntry.value; - if (!shardStatus.state().completed() && shardStatus.nodeId() != null) { - if (nodes.nodeExists(shardStatus.nodeId())) { - shards.put(shardEntry.key, shardEntry.value); - } else { - // TODO: Restart snapshot on another node? - snapshotChanged = true; - logger.warn("failing snapshot of shard [{}] on closed node [{}]", - shardEntry.key, shardStatus.nodeId()); - shards.put(shardEntry.key, - new ShardSnapshotStatus(shardStatus.nodeId(), State.FAILED, "node shutdown")); - } - } - } - if (snapshotChanged) { + if (snapshot.state() == State.STARTED) { + ImmutableOpenMap shards = processWaitingShards(snapshot.shards(), + routingTable); + if (shards != null) { changed = true; - ImmutableOpenMap shardsMap = shards.build(); - if (!snapshot.state().completed() && completed(shardsMap.values())) { - updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, State.SUCCESS, shardsMap); - endSnapshot(updatedSnapshot); + if (!snapshot.state().completed() && completed(shards.values())) { + updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, State.SUCCESS, shards); } else { - updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, snapshot.state(), shardsMap); + updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, shards); } } entries.add(updatedSnapshot); - } else if (snapshot.state() == State.INIT && newMaster) { - changed = true; - // Mark the snapshot as aborted as it failed to start from the previous master - updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, State.ABORTED, snapshot.shards()); - entries.add(updatedSnapshot); - - // Clean up the snapshot that failed to start from the old master - deleteSnapshot(snapshot.snapshot(), new ActionListener() { - @Override - public void onResponse(Void aVoid) { - logger.debug("cleaned up abandoned snapshot {} in INIT state", snapshot.snapshot()); - } - - @Override - public void onFailure(Exception e) { - logger.warn("failed to clean up abandoned snapshot {} in INIT state", snapshot.snapshot()); - } - }, updatedSnapshot.getRepositoryStateId(), false); } } if (changed) { - snapshots = new SnapshotsInProgress(entries.toArray(new SnapshotsInProgress.Entry[entries.size()])); - return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, snapshots).build(); - } - return currentState; - } - - @Override - public void onFailure(String source, Exception e) { - logger.warn("failed to update snapshot state after node removal"); - } - }); - } - } - - private void processStartedShards(ClusterChangedEvent event) { - if (waitingShardsStartedOrUnassigned(event)) { - clusterService.submitStateUpdateTask("update snapshot state after shards started", new ClusterStateUpdateTask() { - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - RoutingTable routingTable = currentState.routingTable(); - SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE); - if (snapshots != null) { - boolean changed = false; - ArrayList entries = new ArrayList<>(); - for (final SnapshotsInProgress.Entry snapshot : snapshots.entries()) { - SnapshotsInProgress.Entry updatedSnapshot = snapshot; - if (snapshot.state() == State.STARTED) { - ImmutableOpenMap shards = processWaitingShards(snapshot.shards(), - routingTable); - if (shards != null) { - changed = true; - if (!snapshot.state().completed() && completed(shards.values())) { - updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, State.SUCCESS, shards); - endSnapshot(updatedSnapshot); - } else { - updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, shards); - } - } - entries.add(updatedSnapshot); - } - } - if (changed) { - snapshots = new SnapshotsInProgress(entries.toArray(new SnapshotsInProgress.Entry[entries.size()])); - return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, snapshots).build(); - } + return ClusterState.builder(currentState) + .putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(unmodifiableList(entries))).build(); } - return currentState; } + return currentState; + } - @Override - public void onFailure(String source, Exception e) { - logger.warn(() -> - new ParameterizedMessage("failed to update snapshot state after shards started from [{}] ", source), e); - } - }); - } + @Override + public void onFailure(String source, Exception e) { + logger.warn(() -> + new ParameterizedMessage("failed to update snapshot state after shards started from [{}] ", source), e); + } + }); } - private ImmutableOpenMap processWaitingShards( + private static ImmutableOpenMap processWaitingShards( ImmutableOpenMap snapshotShards, RoutingTable routingTable) { boolean snapshotChanged = false; ImmutableOpenMap.Builder shards = ImmutableOpenMap.builder(); @@ -905,19 +892,16 @@ private ImmutableOpenMap processWaitingShards( } } - private boolean waitingShardsStartedOrUnassigned(ClusterChangedEvent event) { - SnapshotsInProgress curr = event.state().custom(SnapshotsInProgress.TYPE); - if (curr != null) { - for (SnapshotsInProgress.Entry entry : curr.entries()) { - if (entry.state() == State.STARTED && !entry.waitingIndices().isEmpty()) { - for (ObjectCursor index : entry.waitingIndices().keys()) { - if (event.indexRoutingTableChanged(index.value)) { - IndexRoutingTable indexShardRoutingTable = event.state().getRoutingTable().index(index.value); - for (ShardId shardId : entry.waitingIndices().get(index.value)) { - ShardRouting shardRouting = indexShardRoutingTable.shard(shardId.id()).primaryShard(); - if (shardRouting != null && (shardRouting.started() || shardRouting.unassigned())) { - return true; - } + private static boolean waitingShardsStartedOrUnassigned(SnapshotsInProgress snapshotsInProgress, ClusterChangedEvent event) { + for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { + if (entry.state() == State.STARTED) { + for (ObjectCursor index : entry.waitingIndices().keys()) { + if (event.indexRoutingTableChanged(index.value)) { + IndexRoutingTable indexShardRoutingTable = event.state().getRoutingTable().index(index.value); + for (ShardId shardId : entry.waitingIndices().get(index.value)) { + ShardRouting shardRouting = indexShardRoutingTable.shard(shardId.id()).primaryShard(); + if (shardRouting != null && (shardRouting.started() || shardRouting.unassigned())) { + return true; } } } @@ -927,28 +911,12 @@ private boolean waitingShardsStartedOrUnassigned(ClusterChangedEvent event) { return false; } - private boolean removedNodesCleanupNeeded(ClusterChangedEvent event) { - SnapshotsInProgress snapshotsInProgress = event.state().custom(SnapshotsInProgress.TYPE); - if (snapshotsInProgress == null) { - return false; - } - // Check if we just became the master - boolean newMaster = !event.previousState().nodes().isLocalNodeElectedMaster(); - for (SnapshotsInProgress.Entry snapshot : snapshotsInProgress.entries()) { - if (newMaster && (snapshot.state() == State.SUCCESS || snapshot.state() == State.INIT)) { - // We just replaced old master and snapshots in intermediate states needs to be cleaned - return true; - } - for (DiscoveryNode node : event.nodesDelta().removedNodes()) { - for (ObjectCursor shardStatus : snapshot.shards().values()) { - if (!shardStatus.value.state().completed() && node.getId().equals(shardStatus.value.nodeId())) { - // At least one shard was running on the removed node - we need to fail it - return true; - } - } - } - } - return false; + private static boolean removedNodesCleanupNeeded(SnapshotsInProgress snapshotsInProgress, List removedNodes) { + // If at least one shard was running on a removed node - we need to fail it + return removedNodes.isEmpty() == false && snapshotsInProgress.entries().stream().flatMap(snapshot -> + StreamSupport.stream(((Iterable) () -> snapshot.shards().valuesIt()).spliterator(), false) + .filter(s -> s.state().completed() == false).map(ShardSnapshotStatus::nodeId)) + .anyMatch(removedNodes.stream().map(DiscoveryNode::getId).collect(Collectors.toSet())::contains); } /** @@ -981,25 +949,16 @@ private Tuple, Set> indicesWithMissingShards( * * @param entry snapshot */ - void endSnapshot(final SnapshotsInProgress.Entry entry) { - endSnapshot(entry, null); - } - - - /** - * Finalizes the shard in repository and then removes it from cluster state - *

- * This is non-blocking method that runs on a thread from SNAPSHOT thread pool - * - * @param entry snapshot - * @param failure failure reason or null if snapshot was successful - */ - private void endSnapshot(final SnapshotsInProgress.Entry entry, final String failure) { + private void endSnapshot(final SnapshotsInProgress.Entry entry) { + if (endingSnapshots.add(entry.snapshot()) == false) { + return; + } threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new AbstractRunnable() { @Override protected void doRun() { final Snapshot snapshot = entry.snapshot(); final Repository repository = repositoriesService.repository(snapshot.getRepository()); + final String failure = entry.failure(); logger.trace("[{}] finalizing snapshot in repository, state: [{}], failure[{}]", snapshot, entry.state(), failure); ArrayList shardFailures = new ArrayList<>(); for (ObjectObjectCursor shardStatus : entry.shards()) { @@ -1015,7 +974,7 @@ protected void doRun() { entry.startTime(), failure, entry.shards().size(), - Collections.unmodifiableList(shardFailures), + unmodifiableList(shardFailures), entry.getRepositoryStateId(), entry.includeGlobalState()); removeSnapshotFromClusterState(snapshot, snapshotInfo, null); @@ -1033,7 +992,7 @@ public void onFailure(final Exception e) { /** * Removes record of running snapshot from cluster state - * @param snapshot snapshot + * @param snapshot snapshot * @param snapshotInfo snapshot info if snapshot was successful * @param e exception if snapshot failed */ @@ -1043,11 +1002,11 @@ private void removeSnapshotFromClusterState(final Snapshot snapshot, final Snaps /** * Removes record of running snapshot from cluster state and notifies the listener when this action is complete - * @param snapshot snapshot + * @param snapshot snapshot * @param failure exception if snapshot failed * @param listener listener to notify when snapshot information is removed from the cluster state */ - private void removeSnapshotFromClusterState(final Snapshot snapshot, final SnapshotInfo snapshotInfo, final Exception failure, + private void removeSnapshotFromClusterState(final Snapshot snapshot, @Nullable SnapshotInfo snapshotInfo, final Exception failure, @Nullable CleanupAfterErrorListener listener) { clusterService.submitStateUpdateTask("remove snapshot metadata", new ClusterStateUpdateTask() { @@ -1065,8 +1024,8 @@ public ClusterState execute(ClusterState currentState) { } } if (changed) { - snapshots = new SnapshotsInProgress(entries.toArray(new SnapshotsInProgress.Entry[entries.size()])); - return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, snapshots).build(); + return ClusterState.builder(currentState) + .putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(unmodifiableList(entries))).build(); } } return currentState; @@ -1075,6 +1034,7 @@ public ClusterState execute(ClusterState currentState) { @Override public void onFailure(String source, Exception e) { logger.warn(() -> new ParameterizedMessage("[{}] failed to remove snapshot metadata", snapshot), e); + endingSnapshots.remove(snapshot); if (listener != null) { listener.onFailure(e); } @@ -1082,6 +1042,7 @@ public void onFailure(String source, Exception e) { @Override public void onNoLongerMaster(String source) { + endingSnapshots.remove(snapshot); if (listener != null) { listener.onNoLongerMaster(); } @@ -1101,6 +1062,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS logger.warn("Failed to notify listeners", e); } } + endingSnapshots.remove(snapshot); if (listener != null) { listener.onResponse(snapshotInfo); } @@ -1207,13 +1169,12 @@ public ClusterState execute(ClusterState currentState) throws Exception { final ImmutableOpenMap shards; final State state = snapshotEntry.state(); + final String failure; if (state == State.INIT) { // snapshot is still initializing, mark it as aborted shards = snapshotEntry.shards(); assert shards.isEmpty(); - // No shards in this snapshot, we delete it right away since the SnapshotShardsService - // has no work to do. - endSnapshot(snapshotEntry); + failure = "Snapshot was aborted during initialization"; } else if (state == State.STARTED) { // snapshot is started - mark every non completed shard as aborted final ImmutableOpenMap.Builder shardsBuilder = ImmutableOpenMap.builder(); @@ -1225,7 +1186,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { shardsBuilder.put(shardEntry.key, status); } shards = shardsBuilder.build(); - + failure = "Snapshot was aborted by deletion"; } else { boolean hasUncompletedShards = false; // Cleanup in case a node gone missing and snapshot wasn't updated for some reason @@ -1246,10 +1207,10 @@ public ClusterState execute(ClusterState currentState) throws Exception { // where we force to finish the snapshot logger.debug("trying to delete completed snapshot with no finalizing shards - can delete immediately"); shards = snapshotEntry.shards(); - endSnapshot(snapshotEntry); } + failure = snapshotEntry.failure(); } - SnapshotsInProgress.Entry newSnapshot = new SnapshotsInProgress.Entry(snapshotEntry, State.ABORTED, shards); + SnapshotsInProgress.Entry newSnapshot = new SnapshotsInProgress.Entry(snapshotEntry, State.ABORTED, shards, failure); clusterStateBuilder.putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(newSnapshot)); } return clusterStateBuilder.build(); @@ -1400,7 +1361,8 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS * @param indices list of indices to be snapshotted * @return list of shard to be included into current snapshot */ - private ImmutableOpenMap shards(ClusterState clusterState, List indices) { + private static ImmutableOpenMap shards(ClusterState clusterState, + List indices) { ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); MetaData metaData = clusterState.metaData(); for (IndexId index : indices) { diff --git a/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java index d732f51bd542a..8548b332c1ad1 100644 --- a/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java @@ -29,7 +29,6 @@ import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.SnapshotDeletionsInProgress; import org.elasticsearch.cluster.SnapshotsInProgress; -import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -52,7 +51,6 @@ import org.elasticsearch.test.transport.MockTransportService; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.instanceOf; /** * Tests snapshot operations during disruptions. @@ -156,9 +154,6 @@ public void clusterChanged(ClusterChangedEvent event) { logger.info("--> got exception from race in master operation retries"); } else { logger.info("--> got exception from hanged master", ex); - assertThat(cause, instanceOf(MasterNotDiscoveredException.class)); - cause = cause.getCause(); - assertThat(cause, instanceOf(FailedToCommitClusterStateException.class)); } } diff --git a/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java index e3254d785f5fc..b118d3a3d4933 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java @@ -988,7 +988,6 @@ public void testMasterShutdownDuringFailedSnapshot() throws Exception { * can be restored when the node the shrunken index was created on is no longer part of * the cluster. */ - @AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/38226") public void testRestoreShrinkIndex() throws Exception { logger.info("--> starting a master node and a data node"); internalCluster().startMasterOnlyNode(); diff --git a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index ba821b49559b4..5ca7cbc6aef50 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -3637,7 +3637,6 @@ public void testParallelRestoreOperationsFromSingleSnapshot() throws Exception { } @TestLogging("org.elasticsearch.snapshots:TRACE") - @AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/38226") public void testAbortedSnapshotDuringInitDoesNotStart() throws Exception { final Client client = client(); @@ -3684,14 +3683,9 @@ public void testAbortedSnapshotDuringInitDoesNotStart() throws Exception { // The deletion must set the snapshot in the ABORTED state assertBusy(() -> { - try { - SnapshotsStatusResponse status = - client.admin().cluster().prepareSnapshotStatus("repository").setSnapshots("snap").get(); - assertThat(status.getSnapshots().iterator().next().getState(), equalTo(State.ABORTED)); - } catch (Exception e) { - // Force assertBusy to retry on every exception - throw new AssertionError(e); - } + SnapshotsStatusResponse status = + client.admin().cluster().prepareSnapshotStatus("repository").setSnapshots("snap").get(); + assertThat(status.getSnapshots().iterator().next().getState(), equalTo(State.ABORTED)); }); // Now unblock the repository From 0beb3c93d1ca6dd891269d902bde0e32b9983bbc Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 5 Feb 2019 17:05:19 +0100 Subject: [PATCH 03/23] Clean up duplicate follow config parameter code (#37688) Introduced FollowParameters class that put follow, resume follow, put auto follow pattern requests and follow info response classes reuse. The FollowParameters class had the fields, getters etc. for the common parameters that all these APIs have. Also binary and xcontent serialization / parsing is handled by this class. The follow, resume follow, put auto follow pattern request classes originally used optional non primitive fields, so FollowParameters has that too and the follow info api can handle that now too. Also the followerIndex field can in production only be specified via the url path. If it is also specified via the request body then it must have the same value as is specified in the url path. This option only existed to xcontent testing. However the AbstractSerializingTestCase base class now also supports createXContextTestInstance() to provide a different test instance when testing xcontent, so allowing followerIndex to be specified via the request body is no longer needed. By moving the followerIndex field from Body to ResumeFollowAction.Request class and not allowing the followerIndex field to be specified via the request body the Body class is redundant and can be removed. The ResumeFollowAction.Request class can then directly use the FollowParameters class. For consistency I also removed the ability to specified followerIndex in the put follow api and the name in put auto follow pattern api via the request body. --- .../client/ccr/PutFollowRequest.java | 2 - .../client/ccr/ResumeFollowRequest.java | 3 - .../client/ccr/PutFollowRequestTests.java | 5 +- .../client/ccr/ResumeFollowRequestTests.java | 8 +- .../ccr/action/AutoFollowCoordinator.java | 26 +- .../ccr/action/TransportFollowInfoAction.java | 25 +- .../TransportPutAutoFollowPatternAction.java | 23 +- .../ccr/action/TransportPutFollowAction.java | 19 +- .../action/TransportResumeFollowAction.java | 53 +-- .../elasticsearch/xpack/CcrIntegTestCase.java | 8 +- .../xpack/CcrSingleNodeTestCase.java | 8 +- .../elasticsearch/xpack/ccr/AutoFollowIT.java | 69 ++-- .../xpack/ccr/FollowerFailOverIT.java | 30 +- .../xpack/ccr/IndexFollowingIT.java | 14 +- .../xpack/ccr/LocalIndexFollowingIT.java | 4 +- .../action/AutoFollowCoordinatorTests.java | 6 +- .../ccr/action/FollowInfoResponseTests.java | 79 +---- .../PutAutoFollowPatternRequestTests.java | 54 +-- .../action/PutFollowActionRequestTests.java | 21 +- .../ResumeFollowActionRequestTests.java | 48 ++- .../core/ccr/action/FollowInfoAction.java | 158 +-------- .../core/ccr/action/FollowParameters.java | 314 ++++++++++++++++++ .../action/PutAutoFollowPatternAction.java | 305 ++++------------- .../core/ccr/action/PutFollowAction.java | 142 +++----- .../core/ccr/action/ResumeFollowAction.java | 297 ++--------------- 25 files changed, 705 insertions(+), 1016 deletions(-) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/FollowParameters.java diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/PutFollowRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/PutFollowRequest.java index 8307b04bd7087..9c9e3f92b8173 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/PutFollowRequest.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/PutFollowRequest.java @@ -32,7 +32,6 @@ public final class PutFollowRequest extends FollowConfig implements Validatable, static final ParseField REMOTE_CLUSTER_FIELD = new ParseField("remote_cluster"); static final ParseField LEADER_INDEX_FIELD = new ParseField("leader_index"); - static final ParseField FOLLOWER_INDEX_FIELD = new ParseField("follower_index"); private final String remoteCluster; private final String leaderIndex; @@ -55,7 +54,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(); builder.field(REMOTE_CLUSTER_FIELD.getPreferredName(), remoteCluster); builder.field(LEADER_INDEX_FIELD.getPreferredName(), leaderIndex); - builder.field(FOLLOWER_INDEX_FIELD.getPreferredName(), followerIndex); toXContentFragment(builder, params); builder.endObject(); return builder; diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/ResumeFollowRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/ResumeFollowRequest.java index d9ceb666afd2f..972f327134749 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/ResumeFollowRequest.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/ResumeFollowRequest.java @@ -26,8 +26,6 @@ import java.io.IOException; import java.util.Objects; -import static org.elasticsearch.client.ccr.PutFollowRequest.FOLLOWER_INDEX_FIELD; - public final class ResumeFollowRequest extends FollowConfig implements Validatable, ToXContentObject { private final String followerIndex; @@ -39,7 +37,6 @@ public ResumeFollowRequest(String followerIndex) { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); - builder.field(FOLLOWER_INDEX_FIELD.getPreferredName(), followerIndex); toXContentFragment(builder, params); builder.endObject(); return builder; diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/PutFollowRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/PutFollowRequestTests.java index 35353ce4a96f9..1f6a3d9f0ac28 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/PutFollowRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/PutFollowRequestTests.java @@ -31,12 +31,11 @@ public class PutFollowRequestTests extends AbstractXContentTestCase { private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("test_parser", - true, (args) -> new PutFollowRequest((String) args[0], (String) args[1], (String) args[2])); + true, (args) -> new PutFollowRequest((String) args[0], (String) args[1], "followerIndex")); static { PARSER.declareString(ConstructingObjectParser.constructorArg(), PutFollowRequest.REMOTE_CLUSTER_FIELD); PARSER.declareString(ConstructingObjectParser.constructorArg(), PutFollowRequest.LEADER_INDEX_FIELD); - PARSER.declareString(ConstructingObjectParser.constructorArg(), PutFollowRequest.FOLLOWER_INDEX_FIELD); PARSER.declareInt(PutFollowRequest::setMaxReadRequestOperationCount, PutFollowRequest.MAX_READ_REQUEST_OPERATION_COUNT); PARSER.declareField( PutFollowRequest::setMaxReadRequestSize, @@ -82,7 +81,7 @@ protected boolean supportsUnknownFields() { @Override protected PutFollowRequest createTestInstance() { PutFollowRequest putFollowRequest = - new PutFollowRequest(randomAlphaOfLength(4), randomAlphaOfLength(4), randomAlphaOfLength(4)); + new PutFollowRequest(randomAlphaOfLength(4), randomAlphaOfLength(4), "followerIndex"); if (randomBoolean()) { putFollowRequest.setMaxOutstandingReadRequests(randomIntBetween(0, Integer.MAX_VALUE)); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/ResumeFollowRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/ResumeFollowRequestTests.java index 3f00891331839..d5d2b7e25539f 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/ResumeFollowRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/ResumeFollowRequestTests.java @@ -21,7 +21,6 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractXContentTestCase; @@ -30,11 +29,10 @@ public class ResumeFollowRequestTests extends AbstractXContentTestCase { - private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("test_parser", - true, (args) -> new ResumeFollowRequest((String) args[0])); + private static final ObjectParser PARSER = new ObjectParser<>("test_parser", + true, () -> new ResumeFollowRequest("followerIndex")); static { - PARSER.declareString(ConstructingObjectParser.constructorArg(), PutFollowRequest.FOLLOWER_INDEX_FIELD); PARSER.declareInt(ResumeFollowRequest::setMaxReadRequestOperationCount, FollowConfig.MAX_READ_REQUEST_OPERATION_COUNT); PARSER.declareField( ResumeFollowRequest::setMaxReadRequestSize, @@ -79,7 +77,7 @@ protected boolean supportsUnknownFields() { @Override protected ResumeFollowRequest createTestInstance() { - ResumeFollowRequest resumeFollowRequest = new ResumeFollowRequest(randomAlphaOfLength(4)); + ResumeFollowRequest resumeFollowRequest = new ResumeFollowRequest("followerIndex"); if (randomBoolean()) { resumeFollowRequest.setMaxOutstandingReadRequests(randomIntBetween(0, Integer.MAX_VALUE)); } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java index 82153e77fc35e..03e936ca8c2ea 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java @@ -41,7 +41,6 @@ import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata.AutoFollowPattern; import org.elasticsearch.xpack.core.ccr.AutoFollowStats; import org.elasticsearch.xpack.core.ccr.action.PutFollowAction; -import org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction; import java.util.ArrayList; import java.util.Collections; @@ -514,23 +513,20 @@ private void followLeaderIndex(String autoFollowPattenName, final String leaderIndexName = indexToFollow.getName(); final String followIndexName = getFollowerIndexName(pattern, leaderIndexName); - ResumeFollowAction.Request followRequest = new ResumeFollowAction.Request(); - followRequest.setFollowerIndex(followIndexName); - followRequest.setMaxReadRequestOperationCount(pattern.getMaxReadRequestOperationCount()); - followRequest.setMaxReadRequestSize(pattern.getMaxReadRequestSize()); - followRequest.setMaxOutstandingReadRequests(pattern.getMaxOutstandingReadRequests()); - followRequest.setMaxWriteRequestOperationCount(pattern.getMaxWriteRequestOperationCount()); - followRequest.setMaxWriteRequestSize(pattern.getMaxWriteRequestSize()); - followRequest.setMaxOutstandingWriteRequests(pattern.getMaxOutstandingWriteRequests()); - followRequest.setMaxWriteBufferCount(pattern.getMaxWriteBufferCount()); - followRequest.setMaxWriteBufferSize(pattern.getMaxWriteBufferSize()); - followRequest.setMaxRetryDelay(pattern.getMaxRetryDelay()); - followRequest.setReadPollTimeout(pattern.getPollTimeout()); - PutFollowAction.Request request = new PutFollowAction.Request(); request.setRemoteCluster(remoteCluster); request.setLeaderIndex(indexToFollow.getName()); - request.setFollowRequest(followRequest); + request.setFollowerIndex(followIndexName); + request.getParameters().setMaxReadRequestOperationCount(pattern.getMaxReadRequestOperationCount()); + request.getParameters().setMaxReadRequestSize(pattern.getMaxReadRequestSize()); + request.getParameters().setMaxOutstandingReadRequests(pattern.getMaxOutstandingReadRequests()); + request.getParameters().setMaxWriteRequestOperationCount(pattern.getMaxWriteRequestOperationCount()); + request.getParameters().setMaxWriteRequestSize(pattern.getMaxWriteRequestSize()); + request.getParameters().setMaxOutstandingWriteRequests(pattern.getMaxOutstandingWriteRequests()); + request.getParameters().setMaxWriteBufferCount(pattern.getMaxWriteBufferCount()); + request.getParameters().setMaxWriteBufferSize(pattern.getMaxWriteBufferSize()); + request.getParameters().setMaxRetryDelay(pattern.getMaxRetryDelay()); + request.getParameters().setReadPollTimeout(pattern.getPollTimeout()); // Execute if the create and follow api call succeeds: Runnable successHandler = () -> { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportFollowInfoAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportFollowInfoAction.java index df227639137ae..cb96b8bb29851 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportFollowInfoAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportFollowInfoAction.java @@ -22,7 +22,7 @@ import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.ccr.Ccr; import org.elasticsearch.xpack.core.ccr.action.FollowInfoAction; -import org.elasticsearch.xpack.core.ccr.action.FollowInfoAction.Response.FollowParameters; +import org.elasticsearch.xpack.core.ccr.action.FollowParameters; import org.elasticsearch.xpack.core.ccr.action.FollowInfoAction.Response.FollowerInfo; import org.elasticsearch.xpack.core.ccr.action.FollowInfoAction.Response.Status; @@ -97,18 +97,17 @@ static List getFollowInfos(List concreteFollowerIndices, C String leaderIndex = ccrCustomData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_NAME_KEY); if (result.isPresent()) { ShardFollowTask params = result.get(); - FollowParameters followParameters = new FollowParameters( - params.getMaxReadRequestOperationCount(), - params.getMaxReadRequestSize(), - params.getMaxOutstandingReadRequests(), - params.getMaxWriteRequestOperationCount(), - params.getMaxWriteRequestSize(), - params.getMaxOutstandingWriteRequests(), - params.getMaxWriteBufferCount(), - params.getMaxWriteBufferSize(), - params.getMaxRetryDelay(), - params.getReadPollTimeout() - ); + FollowParameters followParameters = new FollowParameters(); + followParameters.setMaxOutstandingReadRequests(params.getMaxOutstandingReadRequests()); + followParameters.setMaxOutstandingWriteRequests(params.getMaxOutstandingWriteRequests()); + followParameters.setMaxReadRequestOperationCount(params.getMaxReadRequestOperationCount()); + followParameters.setMaxWriteRequestOperationCount(params.getMaxWriteRequestOperationCount()); + followParameters.setMaxReadRequestSize(params.getMaxReadRequestSize()); + followParameters.setMaxWriteRequestSize(params.getMaxWriteRequestSize()); + followParameters.setMaxWriteBufferCount(params.getMaxWriteBufferCount()); + followParameters.setMaxWriteBufferSize(params.getMaxWriteBufferSize()); + followParameters.setMaxRetryDelay(params.getMaxRetryDelay()); + followParameters.setReadPollTimeout(params.getReadPollTimeout()); followerInfos.add(new FollowerInfo(followerIndex, remoteCluster, leaderIndex, Status.ACTIVE, followParameters)); } else { followerInfos.add(new FollowerInfo(followerIndex, remoteCluster, leaderIndex, Status.PAUSED, null)); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java index 8c722942d19b0..d5127cbb74d4b 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java @@ -147,8 +147,7 @@ static ClusterState innerPut(PutAutoFollowPatternAction.Request request, markExistingIndicesAsAutoFollowedForNewPatterns(request.getLeaderIndexPatterns(), remoteClusterState.metaData(), previousPattern, followedIndexUUIDs); } else { - markExistingIndicesAsAutoFollowed(request.getLeaderIndexPatterns(), remoteClusterState.metaData(), - followedIndexUUIDs); + markExistingIndicesAsAutoFollowed(request.getLeaderIndexPatterns(), remoteClusterState.metaData(), followedIndexUUIDs); } if (filteredHeaders != null) { @@ -159,16 +158,16 @@ static ClusterState innerPut(PutAutoFollowPatternAction.Request request, request.getRemoteCluster(), request.getLeaderIndexPatterns(), request.getFollowIndexNamePattern(), - request.getMaxReadRequestOperationCount(), - request.getMaxReadRequestSize(), - request.getMaxConcurrentReadBatches(), - request.getMaxWriteRequestOperationCount(), - request.getMaxWriteRequestSize(), - request.getMaxConcurrentWriteBatches(), - request.getMaxWriteBufferCount(), - request.getMaxWriteBufferSize(), - request.getMaxRetryDelay(), - request.getReadPollTimeout()); + request.getParameters().getMaxReadRequestOperationCount(), + request.getParameters().getMaxReadRequestSize(), + request.getParameters().getMaxOutstandingReadRequests(), + request.getParameters().getMaxWriteRequestOperationCount(), + request.getParameters().getMaxWriteRequestSize(), + request.getParameters().getMaxOutstandingWriteRequests(), + request.getParameters().getMaxWriteBufferCount(), + request.getParameters().getMaxWriteBufferSize(), + request.getParameters().getMaxRetryDelay(), + request.getParameters().getReadPollTimeout()); patterns.put(request.getName(), autoFollowPattern); ClusterState.Builder newState = ClusterState.builder(localState); newState.metaData(MetaData.builder(localState.getMetaData()) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutFollowAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutFollowAction.java index 27f3b60fb5291..84250baaeaa21 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutFollowAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutFollowAction.java @@ -38,6 +38,7 @@ import org.elasticsearch.xpack.ccr.CcrLicenseChecker; import org.elasticsearch.xpack.ccr.CcrSettings; import org.elasticsearch.xpack.ccr.repository.CcrRepository; +import org.elasticsearch.xpack.core.ccr.action.FollowParameters; import org.elasticsearch.xpack.core.ccr.action.PutFollowAction; import org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction; @@ -126,18 +127,18 @@ private void createFollowerIndex( // soft deletes are enabled by default on indices created on 7.0.0 or later if (leaderIndexMetaData.getSettings().getAsBoolean(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), IndexMetaData.SETTING_INDEX_VERSION_CREATED.get(leaderIndexMetaData.getSettings()).onOrAfter(Version.V_7_0_0)) == false) { - listener.onFailure( - new IllegalArgumentException("leader index [" + request.getLeaderIndex() + "] does not have soft deletes enabled")); + listener.onFailure(new IllegalArgumentException("leader index [" + request.getLeaderIndex() + + "] does not have soft deletes enabled")); return; } final Settings.Builder settingsBuilder = Settings.builder() - .put(IndexMetaData.SETTING_INDEX_PROVIDED_NAME, request.getFollowRequest().getFollowerIndex()) + .put(IndexMetaData.SETTING_INDEX_PROVIDED_NAME, request.getFollowerIndex()) .put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true); final String leaderClusterRepoName = CcrRepository.NAME_PREFIX + request.getRemoteCluster(); final RestoreSnapshotRequest restoreRequest = new RestoreSnapshotRequest(leaderClusterRepoName, CcrRepository.LATEST) .indices(request.getLeaderIndex()).indicesOptions(request.indicesOptions()).renamePattern("^(.*)$") - .renameReplacement(request.getFollowRequest().getFollowerIndex()).masterNodeTimeout(request.masterNodeTimeout()) + .renameReplacement(request.getFollowerIndex()).masterNodeTimeout(request.masterNodeTimeout()) .indexSettings(settingsBuilder); final Client clientWithHeaders = CcrLicenseChecker.wrapClient(this.client, threadPool.getThreadContext().getHeaders()); @@ -217,10 +218,14 @@ private void initiateFollowing( final PutFollowAction.Request request, final ActionListener listener) { assert request.waitForActiveShards() != ActiveShardCount.DEFAULT : "PutFollowAction does not support DEFAULT."; - activeShardsObserver.waitForActiveShards(new String[]{request.getFollowRequest().getFollowerIndex()}, + activeShardsObserver.waitForActiveShards(new String[]{request.getFollowerIndex()}, request.waitForActiveShards(), request.timeout(), result -> { if (result) { - client.execute(ResumeFollowAction.INSTANCE, request.getFollowRequest(), ActionListener.wrap( + FollowParameters parameters = request.getParameters(); + ResumeFollowAction.Request resumeFollowRequest = new ResumeFollowAction.Request(); + resumeFollowRequest.setFollowerIndex(request.getFollowerIndex()); + resumeFollowRequest.setParameters(new FollowParameters(parameters)); + client.execute(ResumeFollowAction.INSTANCE, resumeFollowRequest, ActionListener.wrap( r -> listener.onResponse(new PutFollowAction.Response(true, true, r.isAcknowledged())), listener::onFailure )); @@ -232,6 +237,6 @@ private void initiateFollowing( @Override protected ClusterBlockException checkBlock(final PutFollowAction.Request request, final ClusterState state) { - return state.blocks().indexBlockedException(ClusterBlockLevel.METADATA_WRITE, request.getFollowRequest().getFollowerIndex()); + return state.blocks().indexBlockedException(ClusterBlockLevel.METADATA_WRITE, request.getFollowerIndex()); } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java index 0a1a22215a04b..150e1df7a3bae 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java @@ -16,8 +16,8 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider; @@ -47,6 +47,7 @@ import org.elasticsearch.xpack.ccr.Ccr; import org.elasticsearch.xpack.ccr.CcrLicenseChecker; import org.elasticsearch.xpack.ccr.CcrSettings; +import org.elasticsearch.xpack.core.ccr.action.FollowParameters; import org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction; import java.io.IOException; @@ -177,8 +178,7 @@ void start( for (int shardId = 0; shardId < numShards; shardId++) { String taskId = followIndexMetadata.getIndexUUID() + "-" + shardId; - - final ShardFollowTask shardFollowTask = createShardFollowTask(shardId, clusterNameAlias, request, + final ShardFollowTask shardFollowTask = createShardFollowTask(shardId, clusterNameAlias, request.getParameters(), leaderIndexMetadata, followIndexMetadata, filteredHeaders); persistentTasksService.sendStartRequest(taskId, ShardFollowTask.NAME, shardFollowTask, handler.getActionListener(shardId)); } @@ -190,6 +190,8 @@ static void validate( final IndexMetaData followIndex, final String[] leaderIndexHistoryUUID, final MapperService followerMapperService) { + FollowParameters parameters = request.getParameters(); + Map ccrIndexMetadata = followIndex.getCustomData(Ccr.CCR_CUSTOM_METADATA_KEY); if (ccrIndexMetadata == null) { throw new IllegalArgumentException("follow index ["+ followIndex.getIndex().getName() + "] does not have ccr metadata"); @@ -197,8 +199,8 @@ static void validate( String leaderIndexUUID = leaderIndex.getIndex().getUUID(); String recordedLeaderIndexUUID = ccrIndexMetadata.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_UUID_KEY); if (leaderIndexUUID.equals(recordedLeaderIndexUUID) == false) { - throw new IllegalArgumentException("follow index [" + request.getFollowerIndex() + "] should reference [" + leaderIndexUUID + - "] as leader index but instead reference [" + recordedLeaderIndexUUID + "] as leader index"); + throw new IllegalArgumentException("follow index [" + request.getFollowerIndex() + "] should reference [" + + leaderIndexUUID + "] as leader index but instead reference [" + recordedLeaderIndexUUID + "] as leader index"); } String[] recordedHistoryUUIDs = extractLeaderShardHistoryUUIDs(ccrIndexMetadata); @@ -219,7 +221,8 @@ static void validate( "] does not have soft deletes enabled"); } if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(followIndex.getSettings()) == false) { - throw new IllegalArgumentException("follower index [" + request.getFollowerIndex() + "] does not have soft deletes enabled"); + throw new IllegalArgumentException("follower index [" + request.getFollowerIndex() + + "] does not have soft deletes enabled"); } if (leaderIndex.getNumberOfShards() != followIndex.getNumberOfShards()) { throw new IllegalArgumentException("leader index primary shards [" + leaderIndex.getNumberOfShards() + @@ -251,69 +254,69 @@ static void validate( private static ShardFollowTask createShardFollowTask( int shardId, String clusterAliasName, - ResumeFollowAction.Request request, + FollowParameters parameters, IndexMetaData leaderIndexMetadata, IndexMetaData followIndexMetadata, Map filteredHeaders ) { int maxReadRequestOperationCount; - if (request.getMaxReadRequestOperationCount() != null) { - maxReadRequestOperationCount = request.getMaxReadRequestOperationCount(); + if (parameters.getMaxReadRequestOperationCount() != null) { + maxReadRequestOperationCount = parameters.getMaxReadRequestOperationCount(); } else { maxReadRequestOperationCount = DEFAULT_MAX_READ_REQUEST_OPERATION_COUNT; } ByteSizeValue maxReadRequestSize; - if (request.getMaxReadRequestSize() != null) { - maxReadRequestSize = request.getMaxReadRequestSize(); + if (parameters.getMaxReadRequestSize() != null) { + maxReadRequestSize = parameters.getMaxReadRequestSize(); } else { maxReadRequestSize = DEFAULT_MAX_READ_REQUEST_SIZE; } int maxOutstandingReadRequests; - if (request.getMaxOutstandingReadRequests() != null){ - maxOutstandingReadRequests = request.getMaxOutstandingReadRequests(); + if (parameters.getMaxOutstandingReadRequests() != null){ + maxOutstandingReadRequests = parameters.getMaxOutstandingReadRequests(); } else { maxOutstandingReadRequests = DEFAULT_MAX_OUTSTANDING_READ_REQUESTS; } final int maxWriteRequestOperationCount; - if (request.getMaxWriteRequestOperationCount() != null) { - maxWriteRequestOperationCount = request.getMaxWriteRequestOperationCount(); + if (parameters.getMaxWriteRequestOperationCount() != null) { + maxWriteRequestOperationCount = parameters.getMaxWriteRequestOperationCount(); } else { maxWriteRequestOperationCount = DEFAULT_MAX_WRITE_REQUEST_OPERATION_COUNT; } final ByteSizeValue maxWriteRequestSize; - if (request.getMaxWriteRequestSize() != null) { - maxWriteRequestSize = request.getMaxWriteRequestSize(); + if (parameters.getMaxWriteRequestSize() != null) { + maxWriteRequestSize = parameters.getMaxWriteRequestSize(); } else { maxWriteRequestSize = DEFAULT_MAX_WRITE_REQUEST_SIZE; } int maxOutstandingWriteRequests; - if (request.getMaxOutstandingWriteRequests() != null) { - maxOutstandingWriteRequests = request.getMaxOutstandingWriteRequests(); + if (parameters.getMaxOutstandingWriteRequests() != null) { + maxOutstandingWriteRequests = parameters.getMaxOutstandingWriteRequests(); } else { maxOutstandingWriteRequests = DEFAULT_MAX_OUTSTANDING_WRITE_REQUESTS; } int maxWriteBufferCount; - if (request.getMaxWriteBufferCount() != null) { - maxWriteBufferCount = request.getMaxWriteBufferCount(); + if (parameters.getMaxWriteBufferCount() != null) { + maxWriteBufferCount = parameters.getMaxWriteBufferCount(); } else { maxWriteBufferCount = DEFAULT_MAX_WRITE_BUFFER_COUNT; } ByteSizeValue maxWriteBufferSize; - if (request.getMaxWriteBufferSize() != null) { - maxWriteBufferSize = request.getMaxWriteBufferSize(); + if (parameters.getMaxWriteBufferSize() != null) { + maxWriteBufferSize = parameters.getMaxWriteBufferSize(); } else { maxWriteBufferSize = DEFAULT_MAX_WRITE_BUFFER_SIZE; } - TimeValue maxRetryDelay = request.getMaxRetryDelay() == null ? DEFAULT_MAX_RETRY_DELAY : request.getMaxRetryDelay(); - TimeValue readPollTimeout = request.getReadPollTimeout() == null ? DEFAULT_READ_POLL_TIMEOUT : request.getReadPollTimeout(); + TimeValue maxRetryDelay = parameters.getMaxRetryDelay() == null ? DEFAULT_MAX_RETRY_DELAY : parameters.getMaxRetryDelay(); + TimeValue readPollTimeout = parameters.getReadPollTimeout() == null ? DEFAULT_READ_POLL_TIMEOUT : parameters.getReadPollTimeout(); return new ShardFollowTask( clusterAliasName, diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java index d28969bc10c8e..48ea50af9990b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java @@ -426,7 +426,9 @@ public static PutFollowAction.Request putFollow(String leaderIndex, String follo PutFollowAction.Request request = new PutFollowAction.Request(); request.setRemoteCluster("leader_cluster"); request.setLeaderIndex(leaderIndex); - request.setFollowRequest(resumeFollow(followerIndex)); + request.setFollowerIndex(followerIndex); + request.getParameters().setMaxRetryDelay(TimeValue.timeValueMillis(10)); + request.getParameters().setReadPollTimeout(TimeValue.timeValueMillis(10)); request.waitForActiveShards(waitForActiveShards); return request; } @@ -434,8 +436,8 @@ public static PutFollowAction.Request putFollow(String leaderIndex, String follo public static ResumeFollowAction.Request resumeFollow(String followerIndex) { ResumeFollowAction.Request request = new ResumeFollowAction.Request(); request.setFollowerIndex(followerIndex); - request.setMaxRetryDelay(TimeValue.timeValueMillis(10)); - request.setReadPollTimeout(TimeValue.timeValueMillis(10)); + request.getParameters().setMaxRetryDelay(TimeValue.timeValueMillis(10)); + request.getParameters().setReadPollTimeout(TimeValue.timeValueMillis(10)); return request; } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java index 48531c7d28f9a..c2760aa5efd6b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java @@ -89,8 +89,8 @@ protected AutoFollowStats getAutoFollowStats() { protected ResumeFollowAction.Request getResumeFollowRequest(String followerIndex) { ResumeFollowAction.Request request = new ResumeFollowAction.Request(); request.setFollowerIndex(followerIndex); - request.setMaxRetryDelay(TimeValue.timeValueMillis(1)); - request.setReadPollTimeout(TimeValue.timeValueMillis(1)); + request.getParameters().setMaxRetryDelay(TimeValue.timeValueMillis(1)); + request.getParameters().setReadPollTimeout(TimeValue.timeValueMillis(1)); return request; } @@ -98,7 +98,9 @@ protected PutFollowAction.Request getPutFollowRequest(String leaderIndex, String PutFollowAction.Request request = new PutFollowAction.Request(); request.setRemoteCluster("local"); request.setLeaderIndex(leaderIndex); - request.setFollowRequest(getResumeFollowRequest(followerIndex)); + request.setFollowerIndex(followerIndex); + request.getParameters().setMaxRetryDelay(TimeValue.timeValueMillis(1)); + request.getParameters().setReadPollTimeout(TimeValue.timeValueMillis(1)); request.waitForActiveShards(ActiveShardCount.ONE); return request; } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java index 4025f647cb2a6..f12dcea4af9b5 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java @@ -24,7 +24,7 @@ import org.elasticsearch.xpack.core.ccr.action.CcrStatsAction; import org.elasticsearch.xpack.core.ccr.action.DeleteAutoFollowPatternAction; import org.elasticsearch.xpack.core.ccr.action.FollowInfoAction; -import org.elasticsearch.xpack.core.ccr.action.FollowInfoAction.Response.FollowParameters; +import org.elasticsearch.xpack.core.ccr.action.FollowParameters; import org.elasticsearch.xpack.core.ccr.action.FollowInfoAction.Response.FollowerInfo; import org.elasticsearch.xpack.core.ccr.action.PutAutoFollowPatternAction; @@ -186,41 +186,42 @@ public void testAutoFollowParameterAreDelegated() throws Exception { // Enabling auto following: PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); - request.setName("my-pattern"); request.setRemoteCluster("leader_cluster"); request.setLeaderIndexPatterns(Collections.singletonList("logs-*")); // Need to set this, because following an index in the same cluster request.setFollowIndexNamePattern("copy-{{leader_index}}"); if (randomBoolean()) { - request.setMaxWriteBufferCount(randomIntBetween(0, Integer.MAX_VALUE)); + request.getParameters().setMaxWriteBufferCount(randomIntBetween(0, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxConcurrentReadBatches(randomIntBetween(0, Integer.MAX_VALUE)); + request.getParameters().setMaxOutstandingReadRequests(randomIntBetween(0, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxConcurrentWriteBatches(randomIntBetween(0, Integer.MAX_VALUE)); + request.getParameters().setMaxOutstandingWriteRequests(randomIntBetween(0, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxReadRequestOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); + request.getParameters().setMaxReadRequestOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxReadRequestSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); + request.getParameters().setMaxReadRequestSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); } if (randomBoolean()) { - request.setMaxRetryDelay(TimeValue.timeValueMillis(500)); + request.getParameters().setMaxRetryDelay(TimeValue.timeValueMillis(500)); } if (randomBoolean()) { - request.setReadPollTimeout(TimeValue.timeValueMillis(500)); + request.getParameters().setReadPollTimeout(TimeValue.timeValueMillis(500)); } if (randomBoolean()) { - request.setMaxWriteRequestOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); + request.getParameters().setMaxWriteRequestOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxWriteBufferSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); + request.getParameters().setMaxWriteBufferSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); } if (randomBoolean()) { - request.setMaxWriteRequestSize(new ByteSizeValue(randomNonNegativeLong())); + request.getParameters().setMaxWriteRequestSize(new ByteSizeValue(randomNonNegativeLong())); } + + request.setName("my-pattern"); assertTrue(followerClient().execute(PutAutoFollowPatternAction.INSTANCE, request).actionGet().isAcknowledged()); createLeaderIndex("logs-201901", leaderIndexSettings); @@ -242,35 +243,39 @@ public void testAutoFollowParameterAreDelegated() throws Exception { FollowParameters followParameters = followerInfo.getParameters(); assertThat(followParameters, notNullValue()); - if (request.getMaxWriteBufferCount() != null) { - assertThat(followParameters.getMaxWriteBufferCount(), equalTo(request.getMaxWriteBufferCount())); + if (request.getParameters().getMaxWriteBufferCount() != null) { + assertThat(followParameters.getMaxWriteBufferCount(), equalTo(request.getParameters().getMaxWriteBufferCount())); } - if (request.getMaxWriteBufferSize() != null) { - assertThat(followParameters.getMaxWriteBufferSize(), equalTo(request.getMaxWriteBufferSize())); + if (request.getParameters().getMaxWriteBufferSize() != null) { + assertThat(followParameters.getMaxWriteBufferSize(), equalTo(request.getParameters().getMaxWriteBufferSize())); } - if (request.getMaxConcurrentReadBatches() != null) { - assertThat(followParameters.getMaxOutstandingReadRequests(), equalTo(request.getMaxConcurrentReadBatches())); + if (request.getParameters().getMaxOutstandingReadRequests() != null) { + assertThat(followParameters.getMaxOutstandingReadRequests(), + equalTo(request.getParameters().getMaxOutstandingReadRequests())); } - if (request.getMaxConcurrentWriteBatches() != null) { - assertThat(followParameters.getMaxOutstandingWriteRequests(), equalTo(request.getMaxConcurrentWriteBatches())); + if (request.getParameters().getMaxOutstandingWriteRequests() != null) { + assertThat(followParameters.getMaxOutstandingWriteRequests(), + equalTo(request.getParameters().getMaxOutstandingWriteRequests())); } - if (request.getMaxReadRequestOperationCount() != null) { - assertThat(followParameters.getMaxReadRequestOperationCount(), equalTo(request.getMaxReadRequestOperationCount())); + if (request.getParameters().getMaxReadRequestOperationCount() != null) { + assertThat(followParameters.getMaxReadRequestOperationCount(), + equalTo(request.getParameters().getMaxReadRequestOperationCount())); } - if (request.getMaxReadRequestSize() != null) { - assertThat(followParameters.getMaxReadRequestSize(), equalTo(request.getMaxReadRequestSize())); + if (request.getParameters().getMaxReadRequestSize() != null) { + assertThat(followParameters.getMaxReadRequestSize(), equalTo(request.getParameters().getMaxReadRequestSize())); } - if (request.getMaxRetryDelay() != null) { - assertThat(followParameters.getMaxRetryDelay(), equalTo(request.getMaxRetryDelay())); + if (request.getParameters().getMaxRetryDelay() != null) { + assertThat(followParameters.getMaxRetryDelay(), equalTo(request.getParameters().getMaxRetryDelay())); } - if (request.getReadPollTimeout() != null) { - assertThat(followParameters.getReadPollTimeout(), equalTo(request.getReadPollTimeout())); + if (request.getParameters().getReadPollTimeout() != null) { + assertThat(followParameters.getReadPollTimeout(), equalTo(request.getParameters().getReadPollTimeout())); } - if (request.getMaxWriteRequestOperationCount() != null) { - assertThat(followParameters.getMaxWriteRequestOperationCount(), equalTo(request.getMaxWriteRequestOperationCount())); + if (request.getParameters().getMaxWriteRequestOperationCount() != null) { + assertThat(followParameters.getMaxWriteRequestOperationCount(), + equalTo(request.getParameters().getMaxWriteRequestOperationCount())); } - if (request.getMaxWriteRequestSize() != null) { - assertThat(followParameters.getMaxWriteRequestSize(), equalTo(request.getMaxWriteRequestSize())); + if (request.getParameters().getMaxWriteRequestSize() != null) { + assertThat(followParameters.getMaxWriteRequestSize(), equalTo(request.getParameters().getMaxWriteRequestSize())); } }); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java index 3dd20c4385fee..707e44310b4b7 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java @@ -90,13 +90,13 @@ public void testFailOverOnFollower() throws Exception { } availableDocs.release(between(100, 200)); PutFollowAction.Request follow = putFollow("leader-index", "follower-index"); - follow.getFollowRequest().setMaxReadRequestOperationCount(randomIntBetween(32, 2048)); - follow.getFollowRequest().setMaxReadRequestSize(new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)); - follow.getFollowRequest().setMaxOutstandingReadRequests(randomIntBetween(1, 10)); - follow.getFollowRequest().setMaxWriteRequestOperationCount(randomIntBetween(32, 2048)); - follow.getFollowRequest().setMaxWriteRequestSize(new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)); - follow.getFollowRequest().setMaxOutstandingWriteRequests(randomIntBetween(1, 10)); - logger.info("--> follow params {}", Strings.toString(follow.getFollowRequest())); + follow.getParameters().setMaxReadRequestOperationCount(randomIntBetween(32, 2048)); + follow.getParameters().setMaxReadRequestSize(new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)); + follow.getParameters().setMaxOutstandingReadRequests(randomIntBetween(1, 10)); + follow.getParameters().setMaxWriteRequestOperationCount(randomIntBetween(32, 2048)); + follow.getParameters().setMaxWriteRequestSize(new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)); + follow.getParameters().setMaxOutstandingWriteRequests(randomIntBetween(1, 10)); + logger.info("--> follow request {}", Strings.toString(follow)); followerClient().execute(PutFollowAction.INSTANCE, follow).get(); disableDelayedAllocation("follower-index"); ensureFollowerGreen("follower-index"); @@ -151,17 +151,17 @@ public void testFollowIndexAndCloseNode() throws Exception { thread.start(); PutFollowAction.Request followRequest = putFollow("index1", "index2"); - followRequest.getFollowRequest().setMaxReadRequestOperationCount(randomIntBetween(32, 2048)); - followRequest.getFollowRequest().setMaxReadRequestSize(new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)); - followRequest.getFollowRequest().setMaxOutstandingReadRequests(randomIntBetween(1, 10)); - followRequest.getFollowRequest().setMaxWriteRequestOperationCount(randomIntBetween(32, 2048)); - followRequest.getFollowRequest().setMaxWriteRequestSize(new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)); - followRequest.getFollowRequest().setMaxOutstandingWriteRequests(randomIntBetween(1, 10)); + followRequest.getParameters().setMaxReadRequestOperationCount(randomIntBetween(32, 2048)); + followRequest.getParameters().setMaxReadRequestSize(new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)); + followRequest.getParameters().setMaxOutstandingReadRequests(randomIntBetween(1, 10)); + followRequest.getParameters().setMaxWriteRequestOperationCount(randomIntBetween(32, 2048)); + followRequest.getParameters().setMaxWriteRequestSize(new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)); + followRequest.getParameters().setMaxOutstandingWriteRequests(randomIntBetween(1, 10)); followerClient().execute(PutFollowAction.INSTANCE, followRequest).get(); disableDelayedAllocation("index2"); - logger.info("--> follow params {}", Strings.toString(followRequest.getFollowRequest())); + logger.info("--> follow request {}", Strings.toString(followRequest)); - int maxOpsPerRead = followRequest.getFollowRequest().getMaxReadRequestOperationCount(); + int maxOpsPerRead = followRequest.getParameters().getMaxReadRequestOperationCount(); int maxNumDocsReplicated = Math.min(between(50, 500), between(maxOpsPerRead, maxOpsPerRead * 10)); availableDocs.release(maxNumDocsReplicated / 2 + 1); atLeastDocsIndexed(followerClient(), "index2", maxNumDocsReplicated / 3); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index eee28b5875bcc..28f845fe7d463 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -180,7 +180,7 @@ public void testFollowIndex() throws Exception { } pauseFollow("index2"); - followerClient().execute(ResumeFollowAction.INSTANCE, followRequest.getFollowRequest()).get(); + followerClient().execute(ResumeFollowAction.INSTANCE, resumeFollow("index2")).get(); final int secondBatchNumDocs = randomIntBetween(2, 64); logger.info("Indexing [{}] docs as second batch", secondBatchNumDocs); for (int i = firstBatchNumDocs; i < firstBatchNumDocs + secondBatchNumDocs; i++) { @@ -446,10 +446,10 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) atLeastDocsIndexed(leaderClient(), "index1", numDocsIndexed / 3); PutFollowAction.Request followRequest = putFollow("index1", "index2"); - followRequest.getFollowRequest().setMaxReadRequestOperationCount(maxOpsPerRead); - followRequest.getFollowRequest().setMaxOutstandingReadRequests(randomIntBetween(1, 10)); - followRequest.getFollowRequest().setMaxOutstandingWriteRequests(randomIntBetween(1, 10)); - followRequest.getFollowRequest().setMaxWriteBufferCount(randomIntBetween(1024, 10240)); + followRequest.getParameters().setMaxReadRequestOperationCount(maxOpsPerRead); + followRequest.getParameters().setMaxOutstandingReadRequests(randomIntBetween(1, 10)); + followRequest.getParameters().setMaxOutstandingWriteRequests(randomIntBetween(1, 10)); + followRequest.getParameters().setMaxWriteBufferCount(randomIntBetween(1024, 10240)); followerClient().execute(PutFollowAction.INSTANCE, followRequest).get(); availableDocs.release(numDocsIndexed * 2 + bulkSize); atLeastDocsIndexed(leaderClient(), "index1", numDocsIndexed); @@ -544,7 +544,7 @@ public void testFollowIndexMaxOperationSizeInBytes() throws Exception { } PutFollowAction.Request followRequest = putFollow("index1", "index2"); - followRequest.getFollowRequest().setMaxReadRequestSize(new ByteSizeValue(1, ByteSizeUnit.BYTES)); + followRequest.getParameters().setMaxReadRequestSize(new ByteSizeValue(1, ByteSizeUnit.BYTES)); followerClient().execute(PutFollowAction.INSTANCE, followRequest).get(); final Map firstBatchNumDocsPerShard = new HashMap<>(); @@ -1016,7 +1016,7 @@ public void testIndexFallBehind() throws Exception { forceMergeRequest.maxNumSegments(1); leaderClient().admin().indices().forceMerge(forceMergeRequest).actionGet(); - followerClient().execute(ResumeFollowAction.INSTANCE, followRequest.getFollowRequest()).get(); + followerClient().execute(ResumeFollowAction.INSTANCE, resumeFollow("index2")).get(); assertBusy(() -> { List statuses = getFollowTaskStatuses("index2"); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/LocalIndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/LocalIndexFollowingIT.java index f50f17c9e296d..0df3f4ea47f43 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/LocalIndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/LocalIndexFollowingIT.java @@ -85,7 +85,7 @@ public void testDoNotCreateFollowerIfLeaderDoesNotHaveSoftDeletes() throws Excep followRequest.setFollowerIndex("follower-index"); PutFollowAction.Request putFollowRequest = getPutFollowRequest("leader", "follower"); putFollowRequest.setLeaderIndex("leader-index"); - putFollowRequest.setFollowRequest(followRequest); + putFollowRequest.setFollowerIndex("follower-index"); IllegalArgumentException error = expectThrows(IllegalArgumentException.class, () -> client().execute(PutFollowAction.INSTANCE, putFollowRequest).actionGet()); assertThat(error.getMessage(), equalTo("leader index [leader-index] does not have soft deletes enabled")); @@ -98,7 +98,7 @@ public void testRemoveRemoteConnection() throws Exception { request.setRemoteCluster("local"); request.setLeaderIndexPatterns(Collections.singletonList("logs-*")); request.setFollowIndexNamePattern("copy-{{leader_index}}"); - request.setReadPollTimeout(TimeValue.timeValueMillis(10)); + request.getParameters().setReadPollTimeout(TimeValue.timeValueMillis(10)); assertTrue(client().execute(PutAutoFollowPatternAction.INSTANCE, request).actionGet().isAcknowledged()); long previousNumberOfSuccessfulFollowedIndices = getAutoFollowStats().getNumberOfSuccessfulFollowIndices(); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java index 4d4603d022f7d..2037c7faaa7b4 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java @@ -109,7 +109,7 @@ void createAndFollow(Map headers, assertThat(headers, equalTo(autoFollowHeaders.get("remote"))); assertThat(followRequest.getRemoteCluster(), equalTo("remote")); assertThat(followRequest.getLeaderIndex(), equalTo("logs-20190101")); - assertThat(followRequest.getFollowRequest().getFollowerIndex(), equalTo("logs-20190101")); + assertThat(followRequest.getFollowerIndex(), equalTo("logs-20190101")); successHandler.run(); } @@ -227,7 +227,7 @@ void createAndFollow(Map headers, Consumer failureHandler) { assertThat(followRequest.getRemoteCluster(), equalTo("remote")); assertThat(followRequest.getLeaderIndex(), equalTo("logs-20190101")); - assertThat(followRequest.getFollowRequest().getFollowerIndex(), equalTo("logs-20190101")); + assertThat(followRequest.getFollowerIndex(), equalTo("logs-20190101")); successHandler.run(); } @@ -284,7 +284,7 @@ void createAndFollow(Map headers, Consumer failureHandler) { assertThat(followRequest.getRemoteCluster(), equalTo("remote")); assertThat(followRequest.getLeaderIndex(), equalTo("logs-20190101")); - assertThat(followRequest.getFollowRequest().getFollowerIndex(), equalTo("logs-20190101")); + assertThat(followRequest.getFollowerIndex(), equalTo("logs-20190101")); failureHandler.accept(failure); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowInfoResponseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowInfoResponseTests.java index d21098506a121..eceb37819d187 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowInfoResponseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowInfoResponseTests.java @@ -20,61 +20,13 @@ import java.util.List; import static org.elasticsearch.xpack.core.ccr.action.FollowInfoAction.Response.FOLLOWER_INDICES_FIELD; -import static org.elasticsearch.xpack.core.ccr.action.FollowInfoAction.Response.FollowParameters; + +import org.elasticsearch.xpack.core.ccr.action.FollowParameters; import static org.elasticsearch.xpack.core.ccr.action.FollowInfoAction.Response.Status; public class FollowInfoResponseTests extends AbstractSerializingTestCase { - static final ConstructingObjectParser PARAMETERS_PARSER = new ConstructingObjectParser<>( - "parameters_parser", - args -> { - return new FollowParameters( - (Integer) args[0], - (ByteSizeValue) args[1], - (Integer) args[2], - (Integer) args[3], - (ByteSizeValue) args[4], - (Integer) args[5], - (Integer) args[6], - (ByteSizeValue) args[7], - (TimeValue) args[8], - (TimeValue) args[9] - ); - }); - - static { - PARAMETERS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), ShardFollowTask.MAX_READ_REQUEST_OPERATION_COUNT); - PARAMETERS_PARSER.declareField( - ConstructingObjectParser.constructorArg(), - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), ShardFollowTask.MAX_READ_REQUEST_SIZE.getPreferredName()), - ShardFollowTask.MAX_READ_REQUEST_SIZE, - ObjectParser.ValueType.STRING); - PARAMETERS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), ShardFollowTask.MAX_OUTSTANDING_READ_REQUESTS); - PARAMETERS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), ShardFollowTask.MAX_WRITE_REQUEST_OPERATION_COUNT); - PARAMETERS_PARSER.declareField( - ConstructingObjectParser.constructorArg(), - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), ShardFollowTask.MAX_WRITE_REQUEST_SIZE.getPreferredName()), - ShardFollowTask.MAX_WRITE_REQUEST_SIZE, - ObjectParser.ValueType.STRING); - PARAMETERS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), ShardFollowTask.MAX_OUTSTANDING_WRITE_REQUESTS); - PARAMETERS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), ShardFollowTask.MAX_WRITE_BUFFER_COUNT); - PARAMETERS_PARSER.declareField( - ConstructingObjectParser.constructorArg(), - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), ShardFollowTask.MAX_WRITE_BUFFER_SIZE.getPreferredName()), - ShardFollowTask.MAX_WRITE_BUFFER_SIZE, - ObjectParser.ValueType.STRING); - PARAMETERS_PARSER.declareField( - ConstructingObjectParser.constructorArg(), - (p, c) -> TimeValue.parseTimeValue(p.text(), ShardFollowTask.MAX_RETRY_DELAY.getPreferredName()), - ShardFollowTask.MAX_RETRY_DELAY, - ObjectParser.ValueType.STRING); - PARAMETERS_PARSER.declareField( - ConstructingObjectParser.constructorArg(), - (p, c) -> TimeValue.parseTimeValue(p.text(), ShardFollowTask.READ_POLL_TIMEOUT.getPreferredName()), - ShardFollowTask.READ_POLL_TIMEOUT, - ObjectParser.ValueType.STRING); - } - + static final ObjectParser PARAMETERS_PARSER = new ObjectParser<>("parameters_parser", FollowParameters::new); static final ConstructingObjectParser INFO_PARSER = new ConstructingObjectParser<>( "info_parser", args -> { @@ -88,6 +40,8 @@ public class FollowInfoResponseTests extends AbstractSerializingTestCase instanceReader() protected PutAutoFollowPatternAction.Request createTestInstance() { PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); request.setName(randomAlphaOfLength(4)); + request.setRemoteCluster(randomAlphaOfLength(4)); request.setLeaderIndexPatterns(Arrays.asList(generateRandomStringArray(4, 4, false))); if (randomBoolean()) { request.setFollowIndexNamePattern(randomAlphaOfLength(4)); } + ResumeFollowActionRequestTests.generateFollowParameters(request.getParameters()); + return request; + } + + @Override + protected PutAutoFollowPatternAction.Request createXContextTestInstance(XContentType xContentType) { + // follower index parameter is not part of the request body and is provided in the url path. + // So this field cannot be used for creating a test instance for xcontent testing. + PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); + request.setRemoteCluster(randomAlphaOfLength(4)); + request.setLeaderIndexPatterns(Arrays.asList(generateRandomStringArray(4, 4, false))); if (randomBoolean()) { - request.setReadPollTimeout(TimeValue.timeValueMillis(500)); - } - if (randomBoolean()) { - request.setMaxRetryDelay(TimeValue.timeValueMillis(500)); - } - if (randomBoolean()) { - request.setMaxWriteRequestOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); - } - if (randomBoolean()) { - request.setMaxWriteBufferSize(new ByteSizeValue(randomNonNegativeLong())); - } - if (randomBoolean()) { - request.setMaxWriteRequestSize(new ByteSizeValue(randomNonNegativeLong())); - } - if (randomBoolean()) { - request.setMaxReadRequestOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); - } - if (randomBoolean()) { - request.setMaxConcurrentReadBatches(randomIntBetween(0, Integer.MAX_VALUE)); - } - if (randomBoolean()) { - request.setMaxConcurrentWriteBatches(randomIntBetween(0, Integer.MAX_VALUE)); - } - if (randomBoolean()) { - request.setMaxReadRequestSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); - } - if (randomBoolean()) { - request.setMaxWriteBufferCount(randomIntBetween(0, Integer.MAX_VALUE)); - } - if (randomBoolean()) { - request.setMaxWriteBufferSize(new ByteSizeValue(randomNonNegativeLong())); + request.setFollowIndexNamePattern(randomAlphaOfLength(4)); } + ResumeFollowActionRequestTests.generateFollowParameters(request.getParameters()); return request; } @@ -109,17 +91,17 @@ public void testValidate() { validationException = request.validate(); assertThat(validationException, nullValue()); - request.setMaxRetryDelay(TimeValue.ZERO); + request.getParameters().setMaxRetryDelay(TimeValue.ZERO); validationException = request.validate(); assertThat(validationException, notNullValue()); assertThat(validationException.getMessage(), containsString("[max_retry_delay] must be positive but was [0ms]")); - request.setMaxRetryDelay(TimeValue.timeValueMinutes(10)); + request.getParameters().setMaxRetryDelay(TimeValue.timeValueMinutes(10)); validationException = request.validate(); assertThat(validationException, notNullValue()); assertThat(validationException.getMessage(), containsString("[max_retry_delay] must be less than [5m] but was [10m]")); - request.setMaxRetryDelay(TimeValue.timeValueMinutes(1)); + request.getParameters().setMaxRetryDelay(TimeValue.timeValueMinutes(1)); validationException = request.validate(); assertThat(validationException, nullValue()); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutFollowActionRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutFollowActionRequestTests.java index d32a773ebe218..02b5eca08fa4a 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutFollowActionRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutFollowActionRequestTests.java @@ -8,6 +8,7 @@ import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.test.AbstractSerializingTestCase; import org.elasticsearch.xpack.core.ccr.action.PutFollowAction; @@ -22,16 +23,32 @@ protected Writeable.Reader instanceReader() { @Override protected PutFollowAction.Request createTestInstance() { + PutFollowAction.Request request = new PutFollowAction.Request(); + request.setFollowerIndex(randomAlphaOfLength(4)); + request.waitForActiveShards(randomFrom(ActiveShardCount.DEFAULT, ActiveShardCount.NONE, ActiveShardCount.ONE, + ActiveShardCount.ALL)); + + request.setRemoteCluster(randomAlphaOfLength(4)); + request.setLeaderIndex(randomAlphaOfLength(4)); + ResumeFollowActionRequestTests.generateFollowParameters(request.getParameters()); + return request; + } + + @Override + protected PutFollowAction.Request createXContextTestInstance(XContentType xContentType) { + // follower index parameter and wait for active shards params are not part of the request body and + // are provided in the url path. So these fields cannot be used for creating a test instance for xcontent testing. PutFollowAction.Request request = new PutFollowAction.Request(); request.setRemoteCluster(randomAlphaOfLength(4)); request.setLeaderIndex(randomAlphaOfLength(4)); - request.setFollowRequest(ResumeFollowActionRequestTests.createTestRequest()); + request.setFollowerIndex("followerIndex"); + ResumeFollowActionRequestTests.generateFollowParameters(request.getParameters()); return request; } @Override protected PutFollowAction.Request doParseInstance(XContentParser parser) throws IOException { - return PutFollowAction.Request.fromXContent(parser, null, ActiveShardCount.DEFAULT); + return PutFollowAction.Request.fromXContent(parser, "followerIndex", ActiveShardCount.DEFAULT); } @Override diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ResumeFollowActionRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ResumeFollowActionRequestTests.java index 3d3e869f53e8a..53efac70a7dc0 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ResumeFollowActionRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ResumeFollowActionRequestTests.java @@ -11,7 +11,9 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.test.AbstractSerializingTestCase; +import org.elasticsearch.xpack.core.ccr.action.FollowParameters; import org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction; import java.io.IOException; @@ -29,7 +31,20 @@ protected Writeable.Reader instanceReader() { @Override protected ResumeFollowAction.Request createTestInstance() { - return createTestRequest(); + ResumeFollowAction.Request request = new ResumeFollowAction.Request(); + request.setFollowerIndex(randomAlphaOfLength(4)); + + generateFollowParameters(request.getParameters()); + return request; + } + + @Override + protected ResumeFollowAction.Request createXContextTestInstance(XContentType type) { + // follower index parameter is not part of the request body and is provided in the url path. + // So this field cannot be used for creating a test instance for xcontent testing. + ResumeFollowAction.Request request = new ResumeFollowAction.Request(); + generateFollowParameters(request.getParameters()); + return request; } @Override @@ -42,57 +57,54 @@ protected boolean supportsUnknownFields() { return false; } - static ResumeFollowAction.Request createTestRequest() { - ResumeFollowAction.Request request = new ResumeFollowAction.Request(); - request.setFollowerIndex(randomAlphaOfLength(4)); + static void generateFollowParameters(FollowParameters followParameters) { if (randomBoolean()) { - request.setMaxReadRequestOperationCount(randomIntBetween(1, Integer.MAX_VALUE)); + followParameters.setMaxReadRequestOperationCount(randomIntBetween(1, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxOutstandingReadRequests(randomIntBetween(1, Integer.MAX_VALUE)); + followParameters.setMaxOutstandingReadRequests(randomIntBetween(1, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxOutstandingWriteRequests(randomIntBetween(1, Integer.MAX_VALUE)); + followParameters.setMaxOutstandingWriteRequests(randomIntBetween(1, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxReadRequestSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); + followParameters.setMaxReadRequestSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); } if (randomBoolean()) { - request.setMaxWriteBufferCount(randomIntBetween(1, Integer.MAX_VALUE)); + followParameters.setMaxWriteBufferCount(randomIntBetween(1, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxWriteRequestOperationCount(randomIntBetween(1, Integer.MAX_VALUE)); + followParameters.setMaxWriteRequestOperationCount(randomIntBetween(1, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxWriteRequestSize(new ByteSizeValue(randomNonNegativeLong())); + followParameters.setMaxWriteRequestSize(new ByteSizeValue(randomNonNegativeLong())); } if (randomBoolean()) { - request.setMaxWriteBufferSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); + followParameters.setMaxWriteBufferSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); } if (randomBoolean()) { - request.setMaxRetryDelay(TimeValue.timeValueMillis(500)); + followParameters.setMaxRetryDelay(TimeValue.timeValueMillis(500)); } if (randomBoolean()) { - request.setReadPollTimeout(TimeValue.timeValueMillis(500)); + followParameters.setReadPollTimeout(TimeValue.timeValueMillis(500)); } - return request; } public void testValidate() { ResumeFollowAction.Request request = new ResumeFollowAction.Request(); request.setFollowerIndex("index2"); - request.setMaxRetryDelay(TimeValue.ZERO); + request.getParameters().setMaxRetryDelay(TimeValue.ZERO); ActionRequestValidationException validationException = request.validate(); assertThat(validationException, notNullValue()); assertThat(validationException.getMessage(), containsString("[max_retry_delay] must be positive but was [0ms]")); - request.setMaxRetryDelay(TimeValue.timeValueMinutes(10)); + request.getParameters().setMaxRetryDelay(TimeValue.timeValueMinutes(10)); validationException = request.validate(); assertThat(validationException, notNullValue()); assertThat(validationException.getMessage(), containsString("[max_retry_delay] must be less than [5m] but was [10m]")); - request.setMaxRetryDelay(TimeValue.timeValueMinutes(1)); + request.getParameters().setMaxRetryDelay(TimeValue.timeValueMinutes(1)); validationException = request.validate(); assertThat(validationException, nullValue()); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/FollowInfoAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/FollowInfoAction.java index 11d4f22e1b7a8..4cbd575c67b30 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/FollowInfoAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/FollowInfoAction.java @@ -14,8 +14,6 @@ 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.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -24,17 +22,6 @@ import java.util.List; import java.util.Objects; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_OUTSTANDING_READ_REQUESTS; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_OUTSTANDING_WRITE_REQUESTS; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_READ_REQUEST_OPERATION_COUNT; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_READ_REQUEST_SIZE; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_RETRY_DELAY_FIELD; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_WRITE_BUFFER_COUNT; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_WRITE_BUFFER_SIZE; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_WRITE_REQUEST_OPERATION_COUNT; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_WRITE_REQUEST_SIZE; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.READ_POLL_TIMEOUT; - public class FollowInfoAction extends Action { public static final String NAME = "cluster:monitor/ccr/follow_info"; @@ -202,7 +189,7 @@ public FollowParameters getParameters() { remoteCluster = in.readString(); leaderIndex = in.readString(); status = Status.fromString(in.readString()); - parameters = in.readOptionalWriteable(FollowParameters::new); + parameters = in.readOptionalWriteable(innerIn -> new FollowParameters(in)); } @Override @@ -224,16 +211,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (parameters != null) { builder.startObject(PARAMETERS_FIELD.getPreferredName()); { - builder.field(MAX_READ_REQUEST_OPERATION_COUNT.getPreferredName(), parameters.maxReadRequestOperationCount); - builder.field(MAX_READ_REQUEST_SIZE.getPreferredName(), parameters.maxReadRequestSize.getStringRep()); - builder.field(MAX_OUTSTANDING_READ_REQUESTS.getPreferredName(), parameters.maxOutstandingReadRequests); - builder.field(MAX_WRITE_REQUEST_OPERATION_COUNT.getPreferredName(), parameters.maxWriteRequestOperationCount); - builder.field(MAX_WRITE_REQUEST_SIZE.getPreferredName(), parameters.maxWriteRequestSize.getStringRep()); - builder.field(MAX_OUTSTANDING_WRITE_REQUESTS.getPreferredName(), parameters.maxOutstandingWriteRequests); - builder.field(MAX_WRITE_BUFFER_COUNT.getPreferredName(), parameters.maxWriteBufferCount); - builder.field(MAX_WRITE_BUFFER_SIZE.getPreferredName(), parameters.maxWriteBufferSize.getStringRep()); - builder.field(MAX_RETRY_DELAY_FIELD.getPreferredName(), parameters.maxRetryDelay.getStringRep()); - builder.field(READ_POLL_TIMEOUT.getPreferredName(), parameters.readPollTimeout.getStringRep()); + parameters.toXContentFragment(builder); } builder.endObject(); } @@ -263,138 +241,6 @@ public String toString() { } } - public static class FollowParameters implements Writeable { - - private final int maxReadRequestOperationCount; - private final ByteSizeValue maxReadRequestSize; - private final int maxOutstandingReadRequests; - private final int maxWriteRequestOperationCount; - private final ByteSizeValue maxWriteRequestSize; - private final int maxOutstandingWriteRequests; - private final int maxWriteBufferCount; - private final ByteSizeValue maxWriteBufferSize; - private final TimeValue maxRetryDelay; - private final TimeValue readPollTimeout; - - public FollowParameters(int maxReadRequestOperationCount, - ByteSizeValue maxReadRequestSize, int maxOutstandingReadRequests, - int maxWriteRequestOperationCount, ByteSizeValue maxWriteRequestSize, - int maxOutstandingWriteRequests, int maxWriteBufferCount, - ByteSizeValue maxWriteBufferSize, TimeValue maxRetryDelay, TimeValue readPollTimeout) { - this.maxReadRequestOperationCount = maxReadRequestOperationCount; - this.maxReadRequestSize = maxReadRequestSize; - this.maxOutstandingReadRequests = maxOutstandingReadRequests; - this.maxWriteRequestOperationCount = maxWriteRequestOperationCount; - this.maxWriteRequestSize = maxWriteRequestSize; - this.maxOutstandingWriteRequests = maxOutstandingWriteRequests; - this.maxWriteBufferCount = maxWriteBufferCount; - this.maxWriteBufferSize = maxWriteBufferSize; - this.maxRetryDelay = maxRetryDelay; - this.readPollTimeout = readPollTimeout; - } - - public int getMaxReadRequestOperationCount() { - return maxReadRequestOperationCount; - } - - public ByteSizeValue getMaxReadRequestSize() { - return maxReadRequestSize; - } - - public int getMaxOutstandingReadRequests() { - return maxOutstandingReadRequests; - } - - public int getMaxWriteRequestOperationCount() { - return maxWriteRequestOperationCount; - } - - public ByteSizeValue getMaxWriteRequestSize() { - return maxWriteRequestSize; - } - - public int getMaxOutstandingWriteRequests() { - return maxOutstandingWriteRequests; - } - - public int getMaxWriteBufferCount() { - return maxWriteBufferCount; - } - - public ByteSizeValue getMaxWriteBufferSize() { - return maxWriteBufferSize; - } - - public TimeValue getMaxRetryDelay() { - return maxRetryDelay; - } - - public TimeValue getReadPollTimeout() { - return readPollTimeout; - } - - FollowParameters(StreamInput in) throws IOException { - this.maxReadRequestOperationCount = in.readVInt(); - this.maxReadRequestSize = new ByteSizeValue(in); - this.maxOutstandingReadRequests = in.readVInt(); - this.maxWriteRequestOperationCount = in.readVInt(); - this.maxWriteRequestSize = new ByteSizeValue(in); - this.maxOutstandingWriteRequests = in.readVInt(); - this.maxWriteBufferCount = in.readVInt(); - this.maxWriteBufferSize = new ByteSizeValue(in); - this.maxRetryDelay = in.readTimeValue(); - this.readPollTimeout = in.readTimeValue(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeVLong(maxReadRequestOperationCount); - maxReadRequestSize.writeTo(out); - out.writeVInt(maxOutstandingReadRequests); - out.writeVLong(maxWriteRequestOperationCount); - maxWriteRequestSize.writeTo(out); - out.writeVInt(maxOutstandingWriteRequests); - out.writeVInt(maxWriteBufferCount); - maxWriteBufferSize.writeTo(out); - out.writeTimeValue(maxRetryDelay); - out.writeTimeValue(readPollTimeout); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - FollowParameters that = (FollowParameters) o; - return maxReadRequestOperationCount == that.maxReadRequestOperationCount && - maxOutstandingReadRequests == that.maxOutstandingReadRequests && - maxWriteRequestOperationCount == that.maxWriteRequestOperationCount && - maxOutstandingWriteRequests == that.maxOutstandingWriteRequests && - maxWriteBufferCount == that.maxWriteBufferCount && - Objects.equals(maxReadRequestSize, that.maxReadRequestSize) && - Objects.equals(maxWriteRequestSize, that.maxWriteRequestSize) && - Objects.equals(maxWriteBufferSize, that.maxWriteBufferSize) && - Objects.equals(maxRetryDelay, that.maxRetryDelay) && - Objects.equals(readPollTimeout, that.readPollTimeout); - } - - @Override - public int hashCode() { - return Objects.hash( - maxReadRequestOperationCount, - maxReadRequestSize, - maxOutstandingReadRequests, - maxWriteRequestOperationCount, - maxWriteRequestSize, - maxOutstandingWriteRequests, - maxWriteBufferCount, - maxWriteBufferSize, - maxRetryDelay, - readPollTimeout - ); - } - - } - public enum Status { ACTIVE("active"), diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/FollowParameters.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/FollowParameters.java new file mode 100644 index 0000000000000..001a79323ab38 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/FollowParameters.java @@ -0,0 +1,314 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.core.ccr.action; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.common.ParseField; +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.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.AbstractObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; + +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +public class FollowParameters implements Writeable { + + static final TimeValue RETRY_DELAY_MAX = TimeValue.timeValueMinutes(5); + + static final ParseField MAX_READ_REQUEST_OPERATION_COUNT = new ParseField("max_read_request_operation_count"); + static final ParseField MAX_WRITE_REQUEST_OPERATION_COUNT = new ParseField("max_write_request_operation_count"); + static final ParseField MAX_OUTSTANDING_READ_REQUESTS = new ParseField("max_outstanding_read_requests"); + static final ParseField MAX_OUTSTANDING_WRITE_REQUESTS = new ParseField("max_outstanding_write_requests"); + static final ParseField MAX_READ_REQUEST_SIZE = new ParseField("max_read_request_size"); + static final ParseField MAX_WRITE_REQUEST_SIZE = new ParseField("max_write_request_size"); + static final ParseField MAX_WRITE_BUFFER_COUNT = new ParseField("max_write_buffer_count"); + static final ParseField MAX_WRITE_BUFFER_SIZE = new ParseField("max_write_buffer_size"); + static final ParseField MAX_RETRY_DELAY = new ParseField("max_retry_delay"); + static final ParseField READ_POLL_TIMEOUT = new ParseField("read_poll_timeout"); + + Integer maxReadRequestOperationCount; + Integer maxWriteRequestOperationCount; + Integer maxOutstandingReadRequests; + Integer maxOutstandingWriteRequests; + ByteSizeValue maxReadRequestSize; + ByteSizeValue maxWriteRequestSize; + Integer maxWriteBufferCount; + ByteSizeValue maxWriteBufferSize; + TimeValue maxRetryDelay; + TimeValue readPollTimeout; + + public FollowParameters() { + } + + public FollowParameters(FollowParameters source) { + this.maxReadRequestOperationCount = source.maxReadRequestOperationCount; + this.maxWriteRequestOperationCount = source.maxWriteRequestOperationCount; + this.maxOutstandingReadRequests = source.maxOutstandingReadRequests; + this.maxOutstandingWriteRequests = source.maxOutstandingWriteRequests; + this.maxReadRequestSize = source.maxReadRequestSize; + this.maxWriteRequestSize = source.maxWriteRequestSize; + this.maxWriteBufferCount = source.maxWriteBufferCount; + this.maxWriteBufferSize = source.maxWriteBufferSize; + this.maxRetryDelay = source.maxRetryDelay; + this.readPollTimeout = source.readPollTimeout; + } + + public Integer getMaxReadRequestOperationCount() { + return maxReadRequestOperationCount; + } + + public void setMaxReadRequestOperationCount(Integer maxReadRequestOperationCount) { + this.maxReadRequestOperationCount = maxReadRequestOperationCount; + } + + public ByteSizeValue getMaxReadRequestSize() { + return maxReadRequestSize; + } + + public void setMaxReadRequestSize(ByteSizeValue maxReadRequestSize) { + this.maxReadRequestSize = maxReadRequestSize; + } + + public Integer getMaxOutstandingReadRequests() { + return maxOutstandingReadRequests; + } + + public void setMaxOutstandingReadRequests(Integer maxOutstandingReadRequests) { + this.maxOutstandingReadRequests = maxOutstandingReadRequests; + } + + public Integer getMaxWriteRequestOperationCount() { + return maxWriteRequestOperationCount; + } + + public void setMaxWriteRequestOperationCount(Integer maxWriteRequestOperationCount) { + this.maxWriteRequestOperationCount = maxWriteRequestOperationCount; + } + + public ByteSizeValue getMaxWriteRequestSize() { + return maxWriteRequestSize; + } + + public void setMaxWriteRequestSize(ByteSizeValue maxWriteRequestSize) { + this.maxWriteRequestSize = maxWriteRequestSize; + } + + public Integer getMaxOutstandingWriteRequests() { + return maxOutstandingWriteRequests; + } + + public void setMaxOutstandingWriteRequests(Integer maxOutstandingWriteRequests) { + this.maxOutstandingWriteRequests = maxOutstandingWriteRequests; + } + + public Integer getMaxWriteBufferCount() { + return maxWriteBufferCount; + } + + public void setMaxWriteBufferCount(Integer maxWriteBufferCount) { + this.maxWriteBufferCount = maxWriteBufferCount; + } + + public ByteSizeValue getMaxWriteBufferSize() { + return maxWriteBufferSize; + } + + public void setMaxWriteBufferSize(ByteSizeValue maxWriteBufferSize) { + this.maxWriteBufferSize = maxWriteBufferSize; + } + + public TimeValue getMaxRetryDelay() { + return maxRetryDelay; + } + + public void setMaxRetryDelay(TimeValue maxRetryDelay) { + this.maxRetryDelay = maxRetryDelay; + } + + public TimeValue getReadPollTimeout() { + return readPollTimeout; + } + + public void setReadPollTimeout(TimeValue readPollTimeout) { + this.readPollTimeout = readPollTimeout; + } + + public ActionRequestValidationException validate() { + ActionRequestValidationException e = null; + + if (maxReadRequestOperationCount != null && maxReadRequestOperationCount < 1) { + e = addValidationError(MAX_READ_REQUEST_OPERATION_COUNT.getPreferredName() + " must be larger than 0", e); + } + if (maxReadRequestSize != null && maxReadRequestSize.compareTo(ByteSizeValue.ZERO) <= 0) { + e = addValidationError(MAX_READ_REQUEST_SIZE.getPreferredName() + " must be larger than 0", e); + } + if (maxOutstandingReadRequests != null && maxOutstandingReadRequests < 1) { + e = addValidationError(MAX_OUTSTANDING_READ_REQUESTS.getPreferredName() + " must be larger than 0", e); + } + if (maxWriteRequestOperationCount != null && maxWriteRequestOperationCount < 1) { + e = addValidationError(MAX_WRITE_REQUEST_OPERATION_COUNT.getPreferredName() + " must be larger than 0", e); + } + if (maxWriteRequestSize != null && maxWriteRequestSize.compareTo(ByteSizeValue.ZERO) <= 0) { + e = addValidationError(MAX_WRITE_REQUEST_SIZE.getPreferredName() + " must be larger than 0", e); + } + if (maxOutstandingWriteRequests != null && maxOutstandingWriteRequests < 1) { + e = addValidationError(MAX_OUTSTANDING_WRITE_REQUESTS.getPreferredName() + " must be larger than 0", e); + } + if (maxWriteBufferCount != null && maxWriteBufferCount < 1) { + e = addValidationError(MAX_WRITE_BUFFER_COUNT.getPreferredName() + " must be larger than 0", e); + } + if (maxWriteBufferSize != null && maxWriteBufferSize.compareTo(ByteSizeValue.ZERO) <= 0) { + e = addValidationError(MAX_WRITE_BUFFER_SIZE.getPreferredName() + " must be larger than 0", e); + } + if (maxRetryDelay != null && maxRetryDelay.millis() <= 0) { + String message = "[" + MAX_RETRY_DELAY.getPreferredName() + "] must be positive but was [" + + maxRetryDelay.getStringRep() + "]"; + e = addValidationError(message, e); + } + if (maxRetryDelay != null && maxRetryDelay.millis() > RETRY_DELAY_MAX.millis()) { + String message = "[" + MAX_RETRY_DELAY.getPreferredName() + "] must be less than [" + RETRY_DELAY_MAX.getStringRep() + + "] but was [" + maxRetryDelay.getStringRep() + "]"; + e = addValidationError(message, e); + } + + return e; + } + + FollowParameters(StreamInput in) throws IOException { + fromStreamInput(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalVInt(maxReadRequestOperationCount); + out.writeOptionalVInt(maxOutstandingReadRequests); + out.writeOptionalWriteable(maxReadRequestSize); + out.writeOptionalVInt(maxWriteRequestOperationCount); + out.writeOptionalWriteable(maxWriteRequestSize); + out.writeOptionalVInt(maxOutstandingWriteRequests); + out.writeOptionalVInt(maxWriteBufferCount); + out.writeOptionalWriteable(maxWriteBufferSize); + out.writeOptionalTimeValue(maxRetryDelay); + out.writeOptionalTimeValue(readPollTimeout); + } + + void fromStreamInput(StreamInput in) throws IOException { + maxReadRequestOperationCount = in.readOptionalVInt(); + maxOutstandingReadRequests = in.readOptionalVInt(); + maxReadRequestSize = in.readOptionalWriteable(ByteSizeValue::new); + maxWriteRequestOperationCount = in.readOptionalVInt(); + maxWriteRequestSize = in.readOptionalWriteable(ByteSizeValue::new); + maxOutstandingWriteRequests = in.readOptionalVInt(); + maxWriteBufferCount = in.readOptionalVInt(); + maxWriteBufferSize = in.readOptionalWriteable(ByteSizeValue::new); + maxRetryDelay = in.readOptionalTimeValue(); + readPollTimeout = in.readOptionalTimeValue(); + } + + XContentBuilder toXContentFragment(final XContentBuilder builder) throws IOException { + if (maxReadRequestOperationCount != null) { + builder.field(MAX_READ_REQUEST_OPERATION_COUNT.getPreferredName(), maxReadRequestOperationCount); + } + if (maxWriteRequestOperationCount != null) { + builder.field(MAX_WRITE_REQUEST_OPERATION_COUNT.getPreferredName(), maxWriteRequestOperationCount); + } + if (maxOutstandingReadRequests != null) { + builder.field(MAX_OUTSTANDING_READ_REQUESTS.getPreferredName(), maxOutstandingReadRequests); + } + if (maxOutstandingWriteRequests != null) { + builder.field(MAX_OUTSTANDING_WRITE_REQUESTS.getPreferredName(), maxOutstandingWriteRequests); + } + if (maxReadRequestSize != null) { + builder.field(MAX_READ_REQUEST_SIZE.getPreferredName(), maxReadRequestSize.getStringRep()); + } + if (maxWriteRequestSize != null) { + builder.field(MAX_WRITE_REQUEST_SIZE.getPreferredName(), maxWriteRequestSize.getStringRep()); + } + if (maxWriteBufferCount != null) { + builder.field(MAX_WRITE_BUFFER_COUNT.getPreferredName(), maxWriteBufferCount); + } + if (maxWriteBufferSize != null) { + builder.field(MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxWriteBufferSize.getStringRep()); + } + if (maxRetryDelay != null) { + builder.field(MAX_RETRY_DELAY.getPreferredName(), maxRetryDelay.getStringRep()); + } + if (readPollTimeout != null) { + builder.field(READ_POLL_TIMEOUT.getPreferredName(), readPollTimeout.getStringRep()); + } + return builder; + } + + public static

void initParser(AbstractObjectParser parser) { + parser.declareInt(FollowParameters::setMaxReadRequestOperationCount, MAX_READ_REQUEST_OPERATION_COUNT); + parser.declareInt(FollowParameters::setMaxWriteRequestOperationCount, MAX_WRITE_REQUEST_OPERATION_COUNT); + parser.declareInt(FollowParameters::setMaxOutstandingReadRequests, MAX_OUTSTANDING_READ_REQUESTS); + parser.declareInt(FollowParameters::setMaxOutstandingWriteRequests, MAX_OUTSTANDING_WRITE_REQUESTS); + parser.declareField( + FollowParameters::setMaxReadRequestSize, + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_READ_REQUEST_SIZE.getPreferredName()), + AutoFollowMetadata.AutoFollowPattern.MAX_READ_REQUEST_SIZE, + ObjectParser.ValueType.STRING); + parser.declareField( + FollowParameters::setMaxWriteRequestSize, + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_WRITE_REQUEST_SIZE.getPreferredName()), + AutoFollowMetadata.AutoFollowPattern.MAX_WRITE_REQUEST_SIZE, + ObjectParser.ValueType.STRING); + parser.declareInt(FollowParameters::setMaxWriteBufferCount, MAX_WRITE_BUFFER_COUNT); + parser.declareField( + FollowParameters::setMaxWriteBufferSize, + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_WRITE_BUFFER_SIZE.getPreferredName()), + MAX_WRITE_BUFFER_SIZE, + ObjectParser.ValueType.STRING); + parser.declareField(FollowParameters::setMaxRetryDelay, + (p, c) -> TimeValue.parseTimeValue(p.text(), MAX_RETRY_DELAY.getPreferredName()), + MAX_RETRY_DELAY, ObjectParser.ValueType.STRING); + parser.declareField(FollowParameters::setReadPollTimeout, + (p, c) -> TimeValue.parseTimeValue(p.text(), READ_POLL_TIMEOUT.getPreferredName()), + READ_POLL_TIMEOUT, ObjectParser.ValueType.STRING); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o instanceof FollowParameters == false) return false; + FollowParameters that = (FollowParameters) o; + return Objects.equals(maxReadRequestOperationCount, that.maxReadRequestOperationCount) && + Objects.equals(maxWriteRequestOperationCount, that.maxWriteRequestOperationCount) && + Objects.equals(maxOutstandingReadRequests, that.maxOutstandingReadRequests) && + Objects.equals(maxOutstandingWriteRequests, that.maxOutstandingWriteRequests) && + Objects.equals(maxReadRequestSize, that.maxReadRequestSize) && + Objects.equals(maxWriteRequestSize, that.maxWriteRequestSize) && + Objects.equals(maxWriteBufferCount, that.maxWriteBufferCount) && + Objects.equals(maxWriteBufferSize, that.maxWriteBufferSize) && + Objects.equals(maxRetryDelay, that.maxRetryDelay) && + Objects.equals(readPollTimeout, that.readPollTimeout); + } + + @Override + public int hashCode() { + return Objects.hash( + maxReadRequestOperationCount, + maxWriteRequestOperationCount, + maxOutstandingReadRequests, + maxOutstandingWriteRequests, + maxReadRequestSize, + maxWriteRequestSize, + maxWriteBufferCount, + maxWriteBufferSize, + maxRetryDelay, + readPollTimeout + ); + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java index 12d30e4d9f9b1..1ae9801916bce 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java @@ -5,15 +5,14 @@ */ package org.elasticsearch.xpack.core.ccr.action; +import org.elasticsearch.Version; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.support.master.AcknowledgedRequest; import org.elasticsearch.action.support.master.AcknowledgedResponse; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -32,6 +31,7 @@ public class PutAutoFollowPatternAction extends Action { public static final String NAME = "cluster:admin/xpack/ccr/auto_follow_pattern/put"; public static final PutAutoFollowPatternAction INSTANCE = new PutAutoFollowPatternAction(); + private static final int MAX_NAME_BYTES = 255; private PutAutoFollowPatternAction() { super(NAME); @@ -44,54 +44,27 @@ public AcknowledgedResponse newResponse() { public static class Request extends AcknowledgedRequest implements ToXContentObject { - private static final ObjectParser PARSER = new ObjectParser<>("put_auto_follow_pattern_request", Request::new); - private static final ParseField NAME_FIELD = new ParseField("name"); - private static final int MAX_NAME_BYTES = 255; + // Note that Request should be the Value class here for this parser with a 'parameters' field that maps to + // PutAutoFollowPatternParameters class. But since two minor version are already released with duplicate + // follow parameters in several APIs, PutAutoFollowPatternParameters is now the Value class here. + private static final ObjectParser PARSER = + new ObjectParser<>("put_auto_follow_pattern_request", PutAutoFollowPatternParameters::new); static { - PARSER.declareString(Request::setName, NAME_FIELD); - PARSER.declareString(Request::setRemoteCluster, REMOTE_CLUSTER_FIELD); - PARSER.declareStringArray(Request::setLeaderIndexPatterns, AutoFollowPattern.LEADER_PATTERNS_FIELD); - PARSER.declareString(Request::setFollowIndexNamePattern, AutoFollowPattern.FOLLOW_PATTERN_FIELD); - PARSER.declareInt(Request::setMaxReadRequestOperationCount, AutoFollowPattern.MAX_READ_REQUEST_OPERATION_COUNT); - PARSER.declareField( - Request::setMaxReadRequestSize, - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), AutoFollowPattern.MAX_READ_REQUEST_SIZE.getPreferredName()), - AutoFollowPattern.MAX_READ_REQUEST_SIZE, - ObjectParser.ValueType.STRING); - PARSER.declareInt(Request::setMaxConcurrentReadBatches, AutoFollowPattern.MAX_OUTSTANDING_READ_REQUESTS); - PARSER.declareInt(Request::setMaxWriteRequestOperationCount, AutoFollowPattern.MAX_WRITE_REQUEST_OPERATION_COUNT); - PARSER.declareField( - Request::setMaxWriteRequestSize, - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), AutoFollowPattern.MAX_WRITE_REQUEST_SIZE.getPreferredName()), - AutoFollowPattern.MAX_WRITE_REQUEST_SIZE, - ObjectParser.ValueType.STRING); - PARSER.declareInt(Request::setMaxConcurrentWriteBatches, AutoFollowPattern.MAX_OUTSTANDING_WRITE_REQUESTS); - PARSER.declareInt(Request::setMaxWriteBufferCount, AutoFollowPattern.MAX_WRITE_BUFFER_COUNT); - PARSER.declareField( - Request::setMaxWriteBufferSize, - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), AutoFollowPattern.MAX_WRITE_BUFFER_SIZE.getPreferredName()), - AutoFollowPattern.MAX_WRITE_BUFFER_SIZE, - ObjectParser.ValueType.STRING); - PARSER.declareField(Request::setMaxRetryDelay, - (p, c) -> TimeValue.parseTimeValue(p.text(), AutoFollowPattern.MAX_RETRY_DELAY.getPreferredName()), - AutoFollowPattern.MAX_RETRY_DELAY, ObjectParser.ValueType.STRING); - PARSER.declareField(Request::setReadPollTimeout, - (p, c) -> TimeValue.parseTimeValue(p.text(), AutoFollowPattern.READ_POLL_TIMEOUT.getPreferredName()), - AutoFollowPattern.READ_POLL_TIMEOUT, ObjectParser.ValueType.STRING); + PARSER.declareString((params, value) -> params.remoteCluster = value, REMOTE_CLUSTER_FIELD); + PARSER.declareStringArray((params, value) -> params.leaderIndexPatterns = value, AutoFollowPattern.LEADER_PATTERNS_FIELD); + PARSER.declareString((params, value) -> params.followIndexNamePattern = value, AutoFollowPattern.FOLLOW_PATTERN_FIELD); + FollowParameters.initParser(PARSER); } public static Request fromXContent(XContentParser parser, String name) throws IOException { - Request request = PARSER.parse(parser, null); - if (name != null) { - if (request.name == null) { - request.name = name; - } else { - if (request.name.equals(name) == false) { - throw new IllegalArgumentException("provided name is not equal"); - } - } - } + PutAutoFollowPatternParameters parameters = PARSER.parse(parser, null); + Request request = new Request(); + request.setName(name); + request.setRemoteCluster(parameters.remoteCluster); + request.setLeaderIndexPatterns(parameters.leaderIndexPatterns); + request.setFollowIndexNamePattern(parameters.followIndexNamePattern); + request.setParameters(parameters); return request; } @@ -99,40 +72,28 @@ public static Request fromXContent(XContentParser parser, String name) throws IO private String remoteCluster; private List leaderIndexPatterns; private String followIndexNamePattern; - - private Integer maxReadRequestOperationCount; - private ByteSizeValue maxReadRequestSize; - private Integer maxConcurrentReadBatches; - private Integer maxWriteRequestOperationCount; - private ByteSizeValue maxWriteRequestSize; - private Integer maxConcurrentWriteBatches; - private Integer maxWriteBufferCount; - private ByteSizeValue maxWriteBufferSize; - private TimeValue maxRetryDelay; - private TimeValue readPollTimeout; + private FollowParameters parameters = new FollowParameters(); public Request() { } @Override public ActionRequestValidationException validate() { - ActionRequestValidationException validationException = null; + ActionRequestValidationException validationException = parameters.validate(); if (name == null) { - validationException = addValidationError("[" + NAME_FIELD.getPreferredName() + "] is missing", validationException); + validationException = addValidationError("[name] is missing", validationException); } if (name != null) { if (name.contains(",")) { - validationException = addValidationError("[" + NAME_FIELD.getPreferredName() + "] name must not contain a ','", - validationException); + validationException = addValidationError("[name] name must not contain a ','", validationException); } if (name.startsWith("_")) { - validationException = addValidationError("[" + NAME_FIELD.getPreferredName() + "] name must not start with '_'", - validationException); + validationException = addValidationError("[name] name must not start with '_'", validationException); } int byteCount = name.getBytes(StandardCharsets.UTF_8).length; if (byteCount > MAX_NAME_BYTES) { - validationException = addValidationError("[" + NAME_FIELD.getPreferredName() + "] name is too long (" + - byteCount + " > " + MAX_NAME_BYTES + ")", validationException); + validationException = addValidationError("[name] name is too long (" + byteCount + " > " + MAX_NAME_BYTES + ")", + validationException); } } if (remoteCluster == null) { @@ -143,19 +104,6 @@ public ActionRequestValidationException validate() { validationException = addValidationError("[" + AutoFollowPattern.LEADER_PATTERNS_FIELD.getPreferredName() + "] is missing", validationException); } - if (maxRetryDelay != null) { - if (maxRetryDelay.millis() <= 0) { - String message = "[" + AutoFollowPattern.MAX_RETRY_DELAY.getPreferredName() + "] must be positive but was [" + - maxRetryDelay.getStringRep() + "]"; - validationException = addValidationError(message, validationException); - } - if (maxRetryDelay.millis() > ResumeFollowAction.MAX_RETRY_DELAY.millis()) { - String message = "[" + AutoFollowPattern.MAX_RETRY_DELAY.getPreferredName() + "] must be less than [" + - ResumeFollowAction.MAX_RETRY_DELAY + - "] but was [" + maxRetryDelay.getStringRep() + "]"; - validationException = addValidationError(message, validationException); - } - } return validationException; } @@ -191,84 +139,12 @@ public void setFollowIndexNamePattern(String followIndexNamePattern) { this.followIndexNamePattern = followIndexNamePattern; } - public Integer getMaxReadRequestOperationCount() { - return maxReadRequestOperationCount; - } - - public void setMaxReadRequestOperationCount(Integer maxReadRequestOperationCount) { - this.maxReadRequestOperationCount = maxReadRequestOperationCount; - } - - public Integer getMaxConcurrentReadBatches() { - return maxConcurrentReadBatches; - } - - public void setMaxConcurrentReadBatches(Integer maxConcurrentReadBatches) { - this.maxConcurrentReadBatches = maxConcurrentReadBatches; - } - - public ByteSizeValue getMaxReadRequestSize() { - return maxReadRequestSize; - } - - public void setMaxReadRequestSize(ByteSizeValue maxReadRequestSize) { - this.maxReadRequestSize = maxReadRequestSize; - } - - public Integer getMaxWriteRequestOperationCount() { - return maxWriteRequestOperationCount; - } - - public void setMaxWriteRequestOperationCount(Integer maxWriteRequestOperationCount) { - this.maxWriteRequestOperationCount = maxWriteRequestOperationCount; - } - - public ByteSizeValue getMaxWriteRequestSize() { - return maxWriteRequestSize; - } - - public void setMaxWriteRequestSize(ByteSizeValue maxWriteRequestSize) { - this.maxWriteRequestSize = maxWriteRequestSize; - } - - public Integer getMaxConcurrentWriteBatches() { - return maxConcurrentWriteBatches; - } - - public void setMaxConcurrentWriteBatches(Integer maxConcurrentWriteBatches) { - this.maxConcurrentWriteBatches = maxConcurrentWriteBatches; - } - - public Integer getMaxWriteBufferCount() { - return maxWriteBufferCount; - } - - public void setMaxWriteBufferCount(Integer maxWriteBufferCount) { - this.maxWriteBufferCount = maxWriteBufferCount; - } - - public ByteSizeValue getMaxWriteBufferSize() { - return maxWriteBufferSize; - } - - public void setMaxWriteBufferSize(ByteSizeValue maxWriteBufferSize) { - this.maxWriteBufferSize = maxWriteBufferSize; - } - - public TimeValue getMaxRetryDelay() { - return maxRetryDelay; - } - - public void setMaxRetryDelay(TimeValue maxRetryDelay) { - this.maxRetryDelay = maxRetryDelay; - } - - public TimeValue getReadPollTimeout() { - return readPollTimeout; + public FollowParameters getParameters() { + return parameters; } - public void setReadPollTimeout(TimeValue readPollTimeout) { - this.readPollTimeout = readPollTimeout; + public void setParameters(FollowParameters parameters) { + this.parameters = parameters; } public Request(StreamInput in) throws IOException { @@ -277,16 +153,21 @@ public Request(StreamInput in) throws IOException { remoteCluster = in.readString(); leaderIndexPatterns = in.readStringList(); followIndexNamePattern = in.readOptionalString(); - maxReadRequestOperationCount = in.readOptionalVInt(); - maxReadRequestSize = in.readOptionalWriteable(ByteSizeValue::new); - maxConcurrentReadBatches = in.readOptionalVInt(); - maxWriteRequestOperationCount = in.readOptionalVInt(); - maxWriteRequestSize = in.readOptionalWriteable(ByteSizeValue::new); - maxConcurrentWriteBatches = in.readOptionalVInt(); - maxWriteBufferCount = in.readOptionalVInt(); - maxWriteBufferSize = in.readOptionalWriteable(ByteSizeValue::new); - maxRetryDelay = in.readOptionalTimeValue(); - readPollTimeout = in.readOptionalTimeValue(); + if (in.getVersion().onOrAfter(Version.V_7_0_0)) { + parameters = new FollowParameters(in); + } else { + parameters = new FollowParameters(); + parameters.maxReadRequestOperationCount = in.readOptionalVInt(); + parameters.maxReadRequestSize = in.readOptionalWriteable(ByteSizeValue::new); + parameters.maxOutstandingReadRequests = in.readOptionalVInt(); + parameters.maxWriteRequestOperationCount = in.readOptionalVInt(); + parameters.maxWriteRequestSize = in.readOptionalWriteable(ByteSizeValue::new); + parameters.maxOutstandingWriteRequests = in.readOptionalVInt(); + parameters.maxWriteBufferCount = in.readOptionalVInt(); + parameters.maxWriteBufferSize = in.readOptionalWriteable(ByteSizeValue::new); + parameters.maxRetryDelay = in.readOptionalTimeValue(); + parameters.readPollTimeout = in.readOptionalTimeValue(); + } } @Override @@ -296,58 +177,32 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(remoteCluster); out.writeStringCollection(leaderIndexPatterns); out.writeOptionalString(followIndexNamePattern); - out.writeOptionalVInt(maxReadRequestOperationCount); - out.writeOptionalWriteable(maxReadRequestSize); - out.writeOptionalVInt(maxConcurrentReadBatches); - out.writeOptionalVInt(maxWriteRequestOperationCount); - out.writeOptionalWriteable(maxWriteRequestSize); - out.writeOptionalVInt(maxConcurrentWriteBatches); - out.writeOptionalVInt(maxWriteBufferCount); - out.writeOptionalWriteable(maxWriteBufferSize); - out.writeOptionalTimeValue(maxRetryDelay); - out.writeOptionalTimeValue(readPollTimeout); + if (out.getVersion().onOrAfter(Version.V_7_0_0)) { + parameters.writeTo(out); + } else { + out.writeOptionalVInt(parameters.maxReadRequestOperationCount); + out.writeOptionalWriteable(parameters.maxReadRequestSize); + out.writeOptionalVInt(parameters.maxOutstandingReadRequests); + out.writeOptionalVInt(parameters.maxWriteRequestOperationCount); + out.writeOptionalWriteable(parameters.maxWriteRequestSize); + out.writeOptionalVInt(parameters.maxOutstandingWriteRequests); + out.writeOptionalVInt(parameters.maxWriteBufferCount); + out.writeOptionalWriteable(parameters.maxWriteBufferSize); + out.writeOptionalTimeValue(parameters.maxRetryDelay); + out.writeOptionalTimeValue(parameters.readPollTimeout); + } } @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); { - builder.field(NAME_FIELD.getPreferredName(), name); builder.field(REMOTE_CLUSTER_FIELD.getPreferredName(), remoteCluster); builder.field(AutoFollowPattern.LEADER_PATTERNS_FIELD.getPreferredName(), leaderIndexPatterns); if (followIndexNamePattern != null) { builder.field(AutoFollowPattern.FOLLOW_PATTERN_FIELD.getPreferredName(), followIndexNamePattern); } - if (maxReadRequestOperationCount != null) { - builder.field(AutoFollowPattern.MAX_READ_REQUEST_OPERATION_COUNT.getPreferredName(), maxReadRequestOperationCount); - } - if (maxReadRequestSize != null) { - builder.field(AutoFollowPattern.MAX_READ_REQUEST_SIZE.getPreferredName(), maxReadRequestSize.getStringRep()); - } - if (maxWriteRequestOperationCount != null) { - builder.field(AutoFollowPattern.MAX_WRITE_REQUEST_OPERATION_COUNT.getPreferredName(), maxWriteRequestOperationCount); - } - if (maxWriteRequestSize != null) { - builder.field(AutoFollowPattern.MAX_WRITE_REQUEST_SIZE.getPreferredName(), maxWriteRequestSize.getStringRep()); - } - if (maxWriteBufferCount != null) { - builder.field(AutoFollowPattern.MAX_WRITE_BUFFER_COUNT.getPreferredName(), maxWriteBufferCount); - } - if (maxWriteBufferSize != null) { - builder.field(AutoFollowPattern.MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxWriteBufferSize.getStringRep()); - } - if (maxConcurrentReadBatches != null) { - builder.field(AutoFollowPattern.MAX_OUTSTANDING_READ_REQUESTS.getPreferredName(), maxConcurrentReadBatches); - } - if (maxConcurrentWriteBatches != null) { - builder.field(AutoFollowPattern.MAX_OUTSTANDING_WRITE_REQUESTS.getPreferredName(), maxConcurrentWriteBatches); - } - if (maxRetryDelay != null) { - builder.field(AutoFollowPattern.MAX_RETRY_DELAY.getPreferredName(), maxRetryDelay.getStringRep()); - } - if (readPollTimeout != null) { - builder.field(AutoFollowPattern.READ_POLL_TIMEOUT.getPreferredName(), readPollTimeout.getStringRep()); - } + parameters.toXContentFragment(builder); } builder.endObject(); return builder; @@ -359,39 +214,25 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; Request request = (Request) o; return Objects.equals(name, request.name) && - Objects.equals(remoteCluster, request.remoteCluster) && - Objects.equals(leaderIndexPatterns, request.leaderIndexPatterns) && - Objects.equals(followIndexNamePattern, request.followIndexNamePattern) && - Objects.equals(maxReadRequestOperationCount, request.maxReadRequestOperationCount) && - Objects.equals(maxReadRequestSize, request.maxReadRequestSize) && - Objects.equals(maxConcurrentReadBatches, request.maxConcurrentReadBatches) && - Objects.equals(maxWriteRequestOperationCount, request.maxWriteRequestOperationCount) && - Objects.equals(maxWriteRequestSize, request.maxWriteRequestSize) && - Objects.equals(maxConcurrentWriteBatches, request.maxConcurrentWriteBatches) && - Objects.equals(maxWriteBufferCount, request.maxWriteBufferCount) && - Objects.equals(maxWriteBufferSize, request.maxWriteBufferSize) && - Objects.equals(maxRetryDelay, request.maxRetryDelay) && - Objects.equals(readPollTimeout, request.readPollTimeout); + Objects.equals(remoteCluster, request.remoteCluster) && + Objects.equals(leaderIndexPatterns, request.leaderIndexPatterns) && + Objects.equals(followIndexNamePattern, request.followIndexNamePattern) && + Objects.equals(parameters, request.parameters); } @Override public int hashCode() { - return Objects.hash( - name, - remoteCluster, - leaderIndexPatterns, - followIndexNamePattern, - maxReadRequestOperationCount, - maxReadRequestSize, - maxConcurrentReadBatches, - maxWriteRequestOperationCount, - maxWriteRequestSize, - maxConcurrentWriteBatches, - maxWriteBufferCount, - maxWriteBufferSize, - maxRetryDelay, - readPollTimeout); + return Objects.hash(name, remoteCluster, leaderIndexPatterns, followIndexNamePattern, parameters); } + + // This class only exists for reuse of the FollowParameters class, see comment above the parser field. + private static class PutAutoFollowPatternParameters extends FollowParameters { + + private String remoteCluster; + private List leaderIndexPatterns; + private String followIndexNamePattern; + } + } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java index 13ec8b84f81a0..89c18a9824ab4 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java @@ -18,8 +18,6 @@ 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.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -29,17 +27,6 @@ import java.util.Objects; import static org.elasticsearch.action.ValidateActions.addValidationError; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.FOLLOWER_INDEX_FIELD; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_OUTSTANDING_READ_REQUESTS; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_OUTSTANDING_WRITE_REQUESTS; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_READ_REQUEST_OPERATION_COUNT; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_READ_REQUEST_SIZE; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_RETRY_DELAY_FIELD; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_WRITE_BUFFER_COUNT; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_WRITE_BUFFER_SIZE; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_WRITE_REQUEST_OPERATION_COUNT; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_WRITE_REQUEST_SIZE; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.READ_POLL_TIMEOUT; public final class PutFollowAction extends Action { @@ -65,72 +52,47 @@ public static class Request extends AcknowledgedRequest implements Indi private static final ParseField REMOTE_CLUSTER_FIELD = new ParseField("remote_cluster"); private static final ParseField LEADER_INDEX_FIELD = new ParseField("leader_index"); - private static final ObjectParser PARSER = new ObjectParser<>(NAME, () -> { - Request request = new Request(); - request.setFollowRequest(new ResumeFollowAction.Request()); - return request; - }); + // Note that Request should be the Value class here for this parser with a 'parameters' field that maps to + // PutFollowParameters class. But since two minor version are already released with duplicate follow parameters + // in several APIs, PutFollowParameters is now the Value class here. + private static final ObjectParser PARSER = new ObjectParser<>(NAME, PutFollowParameters::new); static { - PARSER.declareString(Request::setRemoteCluster, REMOTE_CLUSTER_FIELD); - PARSER.declareString(Request::setLeaderIndex, LEADER_INDEX_FIELD); - PARSER.declareString((req, val) -> req.followRequest.setFollowerIndex(val), FOLLOWER_INDEX_FIELD); - PARSER.declareInt((req, val) -> req.followRequest.setMaxReadRequestOperationCount(val), MAX_READ_REQUEST_OPERATION_COUNT); - PARSER.declareField( - (req, val) -> req.followRequest.setMaxReadRequestSize(val), - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_READ_REQUEST_SIZE.getPreferredName()), - MAX_READ_REQUEST_SIZE, - ObjectParser.ValueType.STRING); - PARSER.declareInt((req, val) -> req.followRequest.setMaxOutstandingReadRequests(val), MAX_OUTSTANDING_READ_REQUESTS); - PARSER.declareInt((req, val) -> req.followRequest.setMaxWriteRequestOperationCount(val), MAX_WRITE_REQUEST_OPERATION_COUNT); - PARSER.declareField( - (req, val) -> req.followRequest.setMaxWriteRequestSize(val), - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_WRITE_REQUEST_SIZE.getPreferredName()), - MAX_WRITE_REQUEST_SIZE, - ObjectParser.ValueType.STRING); - PARSER.declareInt((req, val) -> req.followRequest.setMaxOutstandingWriteRequests(val), MAX_OUTSTANDING_WRITE_REQUESTS); - PARSER.declareInt((req, val) -> req.followRequest.setMaxWriteBufferCount(val), MAX_WRITE_BUFFER_COUNT); - PARSER.declareField( - (req, val) -> req.followRequest.setMaxWriteBufferSize(val), - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_WRITE_BUFFER_SIZE.getPreferredName()), - MAX_WRITE_BUFFER_SIZE, - ObjectParser.ValueType.STRING); - PARSER.declareField( - (req, val) -> req.followRequest.setMaxRetryDelay(val), - (p, c) -> TimeValue.parseTimeValue(p.text(), MAX_RETRY_DELAY_FIELD.getPreferredName()), - MAX_RETRY_DELAY_FIELD, - ObjectParser.ValueType.STRING); - PARSER.declareField( - (req, val) -> req.followRequest.setReadPollTimeout(val), - (p, c) -> TimeValue.parseTimeValue(p.text(), READ_POLL_TIMEOUT.getPreferredName()), - READ_POLL_TIMEOUT, - ObjectParser.ValueType.STRING); + PARSER.declareString((putFollowParameters, value) -> putFollowParameters.remoteCluster = value, REMOTE_CLUSTER_FIELD); + PARSER.declareString((putFollowParameters, value) -> putFollowParameters.leaderIndex = value, LEADER_INDEX_FIELD); + FollowParameters.initParser(PARSER); } public static Request fromXContent(final XContentParser parser, final String followerIndex, ActiveShardCount waitForActiveShards) throws IOException { - Request request = PARSER.parse(parser, followerIndex); - if (followerIndex != null) { - if (request.getFollowRequest().getFollowerIndex() == null) { - request.getFollowRequest().setFollowerIndex(followerIndex); - } else { - if (request.getFollowRequest().getFollowerIndex().equals(followerIndex) == false) { - throw new IllegalArgumentException("provided follower_index is not equal"); - } - } - } + PutFollowParameters parameters = PARSER.parse(parser, null); + + Request request = new Request(); request.waitForActiveShards(waitForActiveShards); + request.setFollowerIndex(followerIndex); + request.setRemoteCluster(parameters.remoteCluster); + request.setLeaderIndex(parameters.leaderIndex); + request.setParameters(parameters); return request; } private String remoteCluster; private String leaderIndex; + private String followerIndex; + private FollowParameters parameters = new FollowParameters(); private ActiveShardCount waitForActiveShards = ActiveShardCount.NONE; - private ResumeFollowAction.Request followRequest; public Request() { } + public String getFollowerIndex() { + return followerIndex; + } + + public void setFollowerIndex(String followerIndex) { + this.followerIndex = followerIndex; + } + public String getRemoteCluster() { return remoteCluster; } @@ -147,6 +109,14 @@ public void setLeaderIndex(String leaderIndex) { this.leaderIndex = leaderIndex; } + public FollowParameters getParameters() { + return parameters; + } + + public void setParameters(FollowParameters parameters) { + this.parameters = parameters; + } + public ActiveShardCount waitForActiveShards() { return waitForActiveShards; } @@ -168,29 +138,24 @@ public void waitForActiveShards(ActiveShardCount waitForActiveShards) { } } - public ResumeFollowAction.Request getFollowRequest() { - return followRequest; - } - - public void setFollowRequest(ResumeFollowAction.Request followRequest) { - this.followRequest = followRequest; - } - @Override public ActionRequestValidationException validate() { - ActionRequestValidationException e = followRequest.validate(); + ActionRequestValidationException e = parameters.validate(); if (remoteCluster == null) { e = addValidationError(REMOTE_CLUSTER_FIELD.getPreferredName() + " is missing", e); } if (leaderIndex == null) { e = addValidationError(LEADER_INDEX_FIELD.getPreferredName() + " is missing", e); } + if (followerIndex == null) { + e = addValidationError("follower_index is missing", e); + } return e; } @Override public String[] indices() { - return new String[]{followRequest.getFollowerIndex()}; + return new String[]{followerIndex}; } @Override @@ -200,12 +165,13 @@ public IndicesOptions indicesOptions() { public Request(StreamInput in) throws IOException { super(in); - remoteCluster = in.readString(); - leaderIndex = in.readString(); + this.remoteCluster = in.readString(); + this.leaderIndex = in.readString(); + this.followerIndex = in.readString(); + this.parameters = new FollowParameters(in); if (in.getVersion().onOrAfter(Version.V_6_7_0)) { waitForActiveShards(ActiveShardCount.readFrom(in)); } - followRequest = new ResumeFollowAction.Request(in); } @Override @@ -213,10 +179,11 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(remoteCluster); out.writeString(leaderIndex); + out.writeString(followerIndex); + parameters.writeTo(out); if (out.getVersion().onOrAfter(Version.V_6_7_0)) { waitForActiveShards.writeTo(out); } - followRequest.writeTo(out); } @Override @@ -225,7 +192,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws { builder.field(REMOTE_CLUSTER_FIELD.getPreferredName(), remoteCluster); builder.field(LEADER_INDEX_FIELD.getPreferredName(), leaderIndex); - followRequest.toXContentFragment(builder, params); + parameters.toXContentFragment(builder); } builder.endObject(); return builder; @@ -238,24 +205,23 @@ public boolean equals(Object o) { Request request = (Request) o; return Objects.equals(remoteCluster, request.remoteCluster) && Objects.equals(leaderIndex, request.leaderIndex) && - Objects.equals(waitForActiveShards, request.waitForActiveShards) && - Objects.equals(followRequest, request.followRequest); + Objects.equals(followerIndex, request.followerIndex) && + Objects.equals(parameters, request.parameters) && + Objects.equals(waitForActiveShards, request.waitForActiveShards); } @Override public int hashCode() { - return Objects.hash(remoteCluster, leaderIndex, waitForActiveShards, followRequest); + return Objects.hash(remoteCluster, leaderIndex, followerIndex, parameters, waitForActiveShards); } - @Override - public String toString() { - return "PutFollowAction.Request{" + - "remoteCluster='" + remoteCluster + '\'' + - ", leaderIndex='" + leaderIndex + '\'' + - ", waitForActiveShards=" + waitForActiveShards + - ", followRequest=" + followRequest + - '}'; + // This class only exists for reuse of the FollowParameters class, see comment above the parser field. + private static class PutFollowParameters extends FollowParameters { + + private String remoteCluster; + private String leaderIndex; } + } public static class Response extends ActionResponse implements ToXContentObject { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/ResumeFollowAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/ResumeFollowAction.java index 41728928e098f..547f04889a669 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/ResumeFollowAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/ResumeFollowAction.java @@ -10,11 +10,8 @@ import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.MasterNodeRequest; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -30,8 +27,6 @@ public final class ResumeFollowAction extends Action { public static final ResumeFollowAction INSTANCE = new ResumeFollowAction(); public static final String NAME = "cluster:admin/xpack/ccr/resume_follow"; - public static final TimeValue MAX_RETRY_DELAY = TimeValue.timeValueMinutes(5); - private ResumeFollowAction() { super(NAME); } @@ -43,65 +38,28 @@ public AcknowledgedResponse newResponse() { public static class Request extends MasterNodeRequest implements ToXContentObject { - static final ParseField FOLLOWER_INDEX_FIELD = new ParseField("follower_index"); - static final ParseField MAX_READ_REQUEST_OPERATION_COUNT = new ParseField("max_read_request_operation_count"); - static final ParseField MAX_READ_REQUEST_SIZE = new ParseField("max_read_request_size"); - static final ParseField MAX_OUTSTANDING_READ_REQUESTS = new ParseField("max_outstanding_read_requests"); - static final ParseField MAX_WRITE_REQUEST_OPERATION_COUNT = new ParseField("max_write_request_operation_count"); - static final ParseField MAX_WRITE_REQUEST_SIZE = new ParseField("max_write_request_size"); - static final ParseField MAX_OUTSTANDING_WRITE_REQUESTS = new ParseField("max_outstanding_write_requests"); - static final ParseField MAX_WRITE_BUFFER_COUNT = new ParseField("max_write_buffer_count"); - static final ParseField MAX_WRITE_BUFFER_SIZE = new ParseField("max_write_buffer_size"); - static final ParseField MAX_RETRY_DELAY_FIELD = new ParseField("max_retry_delay"); - static final ParseField READ_POLL_TIMEOUT = new ParseField("read_poll_timeout"); - static final ObjectParser PARSER = new ObjectParser<>(NAME, Request::new); + // Note that Request should be the Value class here for this parser with a 'parameters' field that maps to FollowParameters class + // But since two minor version are already released with duplicate follow parameters in several APIs, FollowParameters + // is now the Value class here. + static final ObjectParser PARSER = new ObjectParser<>(NAME, FollowParameters::new); static { - PARSER.declareString(Request::setFollowerIndex, FOLLOWER_INDEX_FIELD); - PARSER.declareInt(Request::setMaxReadRequestOperationCount, MAX_READ_REQUEST_OPERATION_COUNT); - PARSER.declareField( - Request::setMaxReadRequestSize, - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_READ_REQUEST_SIZE.getPreferredName()), MAX_READ_REQUEST_SIZE, - ObjectParser.ValueType.STRING); - PARSER.declareInt(Request::setMaxOutstandingReadRequests, MAX_OUTSTANDING_READ_REQUESTS); - PARSER.declareInt(Request::setMaxWriteRequestOperationCount, MAX_WRITE_REQUEST_OPERATION_COUNT); - PARSER.declareField(Request::setMaxWriteRequestSize, - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_WRITE_REQUEST_SIZE.getPreferredName()), MAX_WRITE_REQUEST_SIZE, - ObjectParser.ValueType.STRING); - PARSER.declareInt(Request::setMaxOutstandingWriteRequests, MAX_OUTSTANDING_WRITE_REQUESTS); - PARSER.declareInt(Request::setMaxWriteBufferCount, MAX_WRITE_BUFFER_COUNT); - PARSER.declareField( - Request::setMaxWriteBufferSize, - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_WRITE_BUFFER_SIZE.getPreferredName()), - MAX_WRITE_BUFFER_SIZE, - ObjectParser.ValueType.STRING); - PARSER.declareField( - Request::setMaxRetryDelay, - (p, c) -> TimeValue.parseTimeValue(p.text(), MAX_RETRY_DELAY_FIELD.getPreferredName()), - MAX_RETRY_DELAY_FIELD, - ObjectParser.ValueType.STRING); - PARSER.declareField( - Request::setReadPollTimeout, - (p, c) -> TimeValue.parseTimeValue(p.text(), READ_POLL_TIMEOUT.getPreferredName()), - READ_POLL_TIMEOUT, - ObjectParser.ValueType.STRING); + FollowParameters.initParser(PARSER); } public static Request fromXContent(final XContentParser parser, final String followerIndex) throws IOException { - Request request = PARSER.parse(parser, followerIndex); - if (followerIndex != null) { - if (request.followerIndex == null) { - request.followerIndex = followerIndex; - } else { - if (request.followerIndex.equals(followerIndex) == false) { - throw new IllegalArgumentException("provided follower_index is not equal"); - } - } - } + FollowParameters parameters = PARSER.parse(parser, null); + Request request = new Request(); + request.setFollowerIndex(followerIndex); + request.setParameters(parameters); return request; } private String followerIndex; + private FollowParameters parameters = new FollowParameters(); + + public Request() { + } public String getFollowerIndex() { return followerIndex; @@ -111,261 +69,58 @@ public void setFollowerIndex(String followerIndex) { this.followerIndex = followerIndex; } - private Integer maxReadRequestOperationCount; - - public Integer getMaxReadRequestOperationCount() { - return maxReadRequestOperationCount; - } - - public void setMaxReadRequestOperationCount(Integer maxReadRequestOperationCount) { - this.maxReadRequestOperationCount = maxReadRequestOperationCount; - } - - private Integer maxOutstandingReadRequests; - - public Integer getMaxOutstandingReadRequests() { - return maxOutstandingReadRequests; - } - - public void setMaxOutstandingReadRequests(Integer maxOutstandingReadRequests) { - this.maxOutstandingReadRequests = maxOutstandingReadRequests; - } - - private ByteSizeValue maxReadRequestSize; - - public ByteSizeValue getMaxReadRequestSize() { - return maxReadRequestSize; - } - - public void setMaxReadRequestSize(ByteSizeValue maxReadRequestSize) { - this.maxReadRequestSize = maxReadRequestSize; - } - - private Integer maxWriteRequestOperationCount; - - public Integer getMaxWriteRequestOperationCount() { - return maxWriteRequestOperationCount; - } - - public void setMaxWriteRequestOperationCount(Integer maxWriteRequestOperationCount) { - this.maxWriteRequestOperationCount = maxWriteRequestOperationCount; - } - - private ByteSizeValue maxWriteRequestSize; - - public ByteSizeValue getMaxWriteRequestSize() { - return maxWriteRequestSize; - } - - public void setMaxWriteRequestSize(ByteSizeValue maxWriteRequestSize) { - this.maxWriteRequestSize = maxWriteRequestSize; - } - - private Integer maxOutstandingWriteRequests; - - public Integer getMaxOutstandingWriteRequests() { - return maxOutstandingWriteRequests; - } - - public void setMaxOutstandingWriteRequests(Integer maxOutstandingWriteRequests) { - this.maxOutstandingWriteRequests = maxOutstandingWriteRequests; - } - - private Integer maxWriteBufferCount; - - public Integer getMaxWriteBufferCount() { - return maxWriteBufferCount; - } - - public void setMaxWriteBufferCount(Integer maxWriteBufferCount) { - this.maxWriteBufferCount = maxWriteBufferCount; - } - - private ByteSizeValue maxWriteBufferSize; - - public ByteSizeValue getMaxWriteBufferSize() { - return maxWriteBufferSize; - } - - public void setMaxWriteBufferSize(ByteSizeValue maxWriteBufferSize) { - this.maxWriteBufferSize = maxWriteBufferSize; - } - - private TimeValue maxRetryDelay; - - public void setMaxRetryDelay(TimeValue maxRetryDelay) { - this.maxRetryDelay = maxRetryDelay; - } - - public TimeValue getMaxRetryDelay() { - return maxRetryDelay; - } - - private TimeValue readPollTimeout; - - public TimeValue getReadPollTimeout() { - return readPollTimeout; + public FollowParameters getParameters() { + return parameters; } - public void setReadPollTimeout(TimeValue readPollTimeout) { - this.readPollTimeout = readPollTimeout; - } - - public Request() { + public void setParameters(FollowParameters parameters) { + this.parameters = parameters; } @Override public ActionRequestValidationException validate() { - ActionRequestValidationException e = null; - + ActionRequestValidationException e = parameters.validate(); if (followerIndex == null) { - e = addValidationError(FOLLOWER_INDEX_FIELD.getPreferredName() + " is missing", e); - } - if (maxReadRequestOperationCount != null && maxReadRequestOperationCount < 1) { - e = addValidationError(MAX_READ_REQUEST_OPERATION_COUNT.getPreferredName() + " must be larger than 0", e); - } - if (maxReadRequestSize != null && maxReadRequestSize.compareTo(ByteSizeValue.ZERO) <= 0) { - e = addValidationError(MAX_READ_REQUEST_SIZE.getPreferredName() + " must be larger than 0", e); - } - if (maxOutstandingReadRequests != null && maxOutstandingReadRequests < 1) { - e = addValidationError(MAX_OUTSTANDING_READ_REQUESTS.getPreferredName() + " must be larger than 0", e); - } - if (maxWriteRequestOperationCount != null && maxWriteRequestOperationCount < 1) { - e = addValidationError(MAX_WRITE_REQUEST_OPERATION_COUNT.getPreferredName() + " must be larger than 0", e); + e = addValidationError("follower_index is missing", e); } - if (maxWriteRequestSize != null && maxWriteRequestSize.compareTo(ByteSizeValue.ZERO) <= 0) { - e = addValidationError(MAX_WRITE_REQUEST_SIZE.getPreferredName() + " must be larger than 0", e); - } - if (maxOutstandingWriteRequests != null && maxOutstandingWriteRequests < 1) { - e = addValidationError(MAX_OUTSTANDING_WRITE_REQUESTS.getPreferredName() + " must be larger than 0", e); - } - if (maxWriteBufferCount != null && maxWriteBufferCount < 1) { - e = addValidationError(MAX_WRITE_BUFFER_COUNT.getPreferredName() + " must be larger than 0", e); - } - if (maxWriteBufferSize != null && maxWriteBufferSize.compareTo(ByteSizeValue.ZERO) <= 0) { - e = addValidationError(MAX_WRITE_BUFFER_SIZE.getPreferredName() + " must be larger than 0", e); - } - if (maxRetryDelay != null && maxRetryDelay.millis() <= 0) { - String message = "[" + MAX_RETRY_DELAY_FIELD.getPreferredName() + "] must be positive but was [" + - maxRetryDelay.getStringRep() + "]"; - e = addValidationError(message, e); - } - if (maxRetryDelay != null && maxRetryDelay.millis() > ResumeFollowAction.MAX_RETRY_DELAY.millis()) { - String message = "[" + MAX_RETRY_DELAY_FIELD.getPreferredName() + "] must be less than [" + MAX_RETRY_DELAY + - "] but was [" + maxRetryDelay.getStringRep() + "]"; - e = addValidationError(message, e); - } - return e; } public Request(StreamInput in) throws IOException { super(in); followerIndex = in.readString(); - maxReadRequestOperationCount = in.readOptionalVInt(); - maxOutstandingReadRequests = in.readOptionalVInt(); - maxReadRequestSize = in.readOptionalWriteable(ByteSizeValue::new); - maxWriteRequestOperationCount = in.readOptionalVInt(); - maxWriteRequestSize = in.readOptionalWriteable(ByteSizeValue::new); - maxOutstandingWriteRequests = in.readOptionalVInt(); - maxWriteBufferCount = in.readOptionalVInt(); - maxWriteBufferSize = in.readOptionalWriteable(ByteSizeValue::new); - maxRetryDelay = in.readOptionalTimeValue(); - readPollTimeout = in.readOptionalTimeValue(); + parameters = new FollowParameters(in); } @Override public void writeTo(final StreamOutput out) throws IOException { super.writeTo(out); out.writeString(followerIndex); - out.writeOptionalVInt(maxReadRequestOperationCount); - out.writeOptionalVInt(maxOutstandingReadRequests); - out.writeOptionalWriteable(maxReadRequestSize); - out.writeOptionalVInt(maxWriteRequestOperationCount); - out.writeOptionalWriteable(maxWriteRequestSize); - out.writeOptionalVInt(maxOutstandingWriteRequests); - out.writeOptionalVInt(maxWriteBufferCount); - out.writeOptionalWriteable(maxWriteBufferSize); - out.writeOptionalTimeValue(maxRetryDelay); - out.writeOptionalTimeValue(readPollTimeout); + parameters.writeTo(out); } @Override public XContentBuilder toXContent(final XContentBuilder builder, final Params params) throws IOException { builder.startObject(); { - toXContentFragment(builder, params); + parameters.toXContentFragment(builder); } builder.endObject(); return builder; } - void toXContentFragment(final XContentBuilder builder, final Params params) throws IOException { - builder.field(FOLLOWER_INDEX_FIELD.getPreferredName(), followerIndex); - if (maxReadRequestOperationCount != null) { - builder.field(MAX_READ_REQUEST_OPERATION_COUNT.getPreferredName(), maxReadRequestOperationCount); - } - if (maxReadRequestSize != null) { - builder.field(MAX_READ_REQUEST_SIZE.getPreferredName(), maxReadRequestSize.getStringRep()); - } - if (maxWriteRequestOperationCount != null) { - builder.field(MAX_WRITE_REQUEST_OPERATION_COUNT.getPreferredName(), maxWriteRequestOperationCount); - } - if (maxWriteRequestSize != null) { - builder.field(MAX_WRITE_REQUEST_SIZE.getPreferredName(), maxWriteRequestSize.getStringRep()); - } - if (maxWriteBufferCount != null) { - builder.field(MAX_WRITE_BUFFER_COUNT.getPreferredName(), maxWriteBufferCount); - } - if (maxWriteBufferSize != null) { - builder.field(MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxWriteBufferSize.getStringRep()); - } - if (maxOutstandingReadRequests != null) { - builder.field(MAX_OUTSTANDING_READ_REQUESTS.getPreferredName(), maxOutstandingReadRequests); - } - if (maxOutstandingWriteRequests != null) { - builder.field(MAX_OUTSTANDING_WRITE_REQUESTS.getPreferredName(), maxOutstandingWriteRequests); - } - if (maxRetryDelay != null) { - builder.field(MAX_RETRY_DELAY_FIELD.getPreferredName(), maxRetryDelay.getStringRep()); - } - if (readPollTimeout != null) { - builder.field(READ_POLL_TIMEOUT.getPreferredName(), readPollTimeout.getStringRep()); - } - } - @Override - public boolean equals(final Object o) { + public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Request request = (Request) o; - return Objects.equals(maxReadRequestOperationCount, request.maxReadRequestOperationCount) && - Objects.equals(maxReadRequestSize, request.maxReadRequestSize) && - Objects.equals(maxOutstandingReadRequests, request.maxOutstandingReadRequests) && - Objects.equals(maxWriteRequestOperationCount, request.maxWriteRequestOperationCount) && - Objects.equals(maxWriteRequestSize, request.maxWriteRequestSize) && - Objects.equals(maxOutstandingWriteRequests, request.maxOutstandingWriteRequests) && - Objects.equals(maxWriteBufferCount, request.maxWriteBufferCount) && - Objects.equals(maxWriteBufferSize, request.maxWriteBufferSize) && - Objects.equals(maxRetryDelay, request.maxRetryDelay) && - Objects.equals(readPollTimeout, request.readPollTimeout) && - Objects.equals(followerIndex, request.followerIndex); + return Objects.equals(followerIndex, request.followerIndex) && + Objects.equals(parameters, request.parameters); } @Override public int hashCode() { - return Objects.hash( - followerIndex, - maxReadRequestOperationCount, - maxReadRequestSize, - maxOutstandingReadRequests, - maxWriteRequestOperationCount, - maxWriteRequestSize, - maxOutstandingWriteRequests, - maxWriteBufferCount, - maxWriteBufferSize, - maxRetryDelay, - readPollTimeout); + return Objects.hash(followerIndex, parameters); } } From 8742db3afe70d8dea9db8a1f9a281ea65f6e77c8 Mon Sep 17 00:00:00 2001 From: Michael Basnight Date: Tue, 5 Feb 2019 10:08:08 -0600 Subject: [PATCH 04/23] Update Rollup Caps to allow unknown fields (#38339) This commit ensures that the parts of rollup caps that can allow unknown fields will allow them. It also modifies the test such that we can use the features we need for disallowing fields in spots where they would not be allowed. Relates #36938 --- .../client/rollup/RollableIndexCaps.java | 2 +- .../client/rollup/RollupJobCaps.java | 26 +++++-------------- .../rollup/GetRollupCapsResponseTests.java | 10 +++++++ .../GetRollupIndexCapsResponseTests.java | 10 +++++++ .../rollup/RollupCapsResponseTestCase.java | 16 +++++++++--- 5 files changed, 40 insertions(+), 24 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/rollup/RollableIndexCaps.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/rollup/RollableIndexCaps.java index cf849e38dd0b4..8e0bea0996bbd 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/rollup/RollableIndexCaps.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/rollup/RollableIndexCaps.java @@ -44,7 +44,7 @@ public class RollableIndexCaps implements ToXContentFragment { public static final Function> PARSER = indexName -> { @SuppressWarnings("unchecked") ConstructingObjectParser p - = new ConstructingObjectParser<>(indexName, + = new ConstructingObjectParser<>(indexName, true, a -> new RollableIndexCaps(indexName, (List) a[0])); p.declareObjectArray(ConstructingObjectParser.constructorArg(), RollupJobCaps.PARSER::apply, diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/rollup/RollupJobCaps.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/rollup/RollupJobCaps.java index 7ba1aaa4d7c2b..15161069f7338 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/rollup/RollupJobCaps.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/rollup/RollupJobCaps.java @@ -33,7 +33,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.function.Function; +import java.util.stream.Collectors; /** * Represents the Rollup capabilities for a specific job on a single rollup index @@ -45,15 +45,12 @@ public class RollupJobCaps implements ToXContentObject { private static final ParseField FIELDS = new ParseField("fields"); private static final String NAME = "rollup_job_caps"; - public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, + public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, true, a -> { @SuppressWarnings("unchecked") List> caps = (List>) a[3]; - if (caps.isEmpty()) { - return new RollupJobCaps((String) a[0], (String) a[1], (String) a[2], Collections.emptyMap()); - } - Map mapCaps = new HashMap<>(caps.size()); - caps.forEach(c -> mapCaps.put(c.v1(), c.v2())); + Map mapCaps = + new HashMap<>(caps.stream().collect(Collectors.toMap(Tuple::v1, Tuple::v2))); return new RollupJobCaps((String) a[0], (String) a[1], (String) a[2], mapCaps); }); @@ -140,16 +137,6 @@ public static class RollupFieldCaps implements ToXContentFragment { private static final String NAME = "rollup_field_caps"; private final List> aggs; - public static final Function> PARSER = fieldName -> { - @SuppressWarnings("unchecked") - ConstructingObjectParser parser - = new ConstructingObjectParser<>(NAME, a -> new RollupFieldCaps((List>) a[0])); - - parser.declareObjectArray(ConstructingObjectParser.constructorArg(), - (p, c) -> p.map(), new ParseField(fieldName)); - return parser; - }; - RollupFieldCaps(final List> aggs) { this.aggs = Collections.unmodifiableList(Objects.requireNonNull(aggs)); } @@ -170,13 +157,12 @@ public static RollupFieldCaps fromXContent(XContentParser parser) throws IOExcep List> aggs = new ArrayList<>(); if (parser.nextToken().equals(XContentParser.Token.START_ARRAY)) { while (parser.nextToken().equals(XContentParser.Token.START_OBJECT)) { - aggs.add(Collections.unmodifiableMap(parser.map())); + aggs.add(parser.map()); } } - return new RollupFieldCaps(Collections.unmodifiableList(aggs)); + return new RollupFieldCaps(aggs); } - @Override public boolean equals(Object other) { if (this == other) { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/GetRollupCapsResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/GetRollupCapsResponseTests.java index a728b65cf64ce..a9c3a59faf5ae 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/GetRollupCapsResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/GetRollupCapsResponseTests.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.Map; +import java.util.function.Predicate; public class GetRollupCapsResponseTests extends RollupCapsResponseTestCase { @@ -40,6 +41,15 @@ protected void toXContent(GetRollupCapsResponse response, XContentBuilder builde builder.endObject(); } + @Override + protected Predicate randomFieldsExcludeFilter() { + return (field) -> + // base cannot have extra things in it + "".equals(field) + // the field list expects to be a nested object of a certain type + || field.contains("fields"); + } + @Override protected GetRollupCapsResponse fromXContent(XContentParser parser) throws IOException { return GetRollupCapsResponse.fromXContent(parser); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/GetRollupIndexCapsResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/GetRollupIndexCapsResponseTests.java index afd0e54f92b1f..20e29aef0df64 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/GetRollupIndexCapsResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/GetRollupIndexCapsResponseTests.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.Map; +import java.util.function.Predicate; public class GetRollupIndexCapsResponseTests extends RollupCapsResponseTestCase { @@ -40,6 +41,15 @@ protected void toXContent(GetRollupIndexCapsResponse response, XContentBuilder b builder.endObject(); } + @Override + protected Predicate randomFieldsExcludeFilter() { + return (field) -> + // base cannot have extra things in it + "".equals(field) + // the field list expects to be a nested object of a certain type + || field.contains("fields"); + } + @Override protected GetRollupIndexCapsResponse fromXContent(XContentParser parser) throws IOException { return GetRollupIndexCapsResponse.fromXContent(parser); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/RollupCapsResponseTestCase.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/RollupCapsResponseTestCase.java index 6d1c0359d172d..cdc4280dbff91 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/RollupCapsResponseTestCase.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/RollupCapsResponseTestCase.java @@ -25,6 +25,7 @@ import org.elasticsearch.client.rollup.job.config.RollupJobConfig; import org.elasticsearch.client.rollup.job.config.RollupJobConfigTests; import org.elasticsearch.client.rollup.job.config.TermsGroupConfig; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; @@ -40,6 +41,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.function.Predicate; import java.util.stream.Collectors; import static java.util.Collections.singletonMap; @@ -55,15 +57,23 @@ abstract class RollupCapsResponseTestCase extends ESTestCase { protected abstract T fromXContent(XContentParser parser) throws IOException; + protected Predicate randomFieldsExcludeFilter() { + return field -> false; + } + + protected String[] shuffleFieldsExceptions() { + return Strings.EMPTY_ARRAY; + } + public void testFromXContent() throws IOException { xContentTester( this::createParser, this::createTestInstance, this::toXContent, this::fromXContent) - .supportsUnknownFields(false) - .randomFieldsExcludeFilter(field -> - field.endsWith("job_id")) + .supportsUnknownFields(true) + .randomFieldsExcludeFilter(randomFieldsExcludeFilter()) + .shuffleFieldsExceptions(shuffleFieldsExceptions()) .test(); } From 92bc68170548dfa3587cbf5058bb10bc58503e6c Mon Sep 17 00:00:00 2001 From: David Roberts Date: Tue, 5 Feb 2019 16:10:00 +0000 Subject: [PATCH 05/23] [ML] Report index unavailable instead of waiting for lazy node (#38423) If a job cannot be assigned to a node because an index it requires is unavailable and there are lazy ML nodes then index unavailable should be reported as the assignment explanation rather than waiting for a lazy ML node. --- .../ml/action/TransportOpenJobAction.java | 54 ++++++------- .../action/TransportOpenJobActionTests.java | 76 ++++++++++++++----- 2 files changed, 84 insertions(+), 46 deletions(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportOpenJobAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportOpenJobAction.java index f873d8699b9b4..bfe0cdef41596 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportOpenJobAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportOpenJobAction.java @@ -89,7 +89,6 @@ public class TransportOpenJobAction extends TransportMasterNodeAction unavailableIndices = verifyIndicesPrimaryShardsAreActive(resultsWriteAlias, clusterState); - if (unavailableIndices.size() != 0) { - String reason = "Not opening job [" + jobId + "], because not all primary shards are active for the following indices [" + - String.join(",", unavailableIndices) + "]"; - logger.debug(reason); - return new PersistentTasksCustomMetaData.Assignment(null, reason); - } // Try to allocate jobs according to memory usage, but if that's not possible (maybe due to a mixed version cluster or maybe // because of some weird OS problem) then fall back to the old mechanism of only considering numbers of assigned jobs - boolean allocateByMemory = true; - - if (memoryTracker.isRecentlyRefreshed() == false) { - - boolean scheduledRefresh = memoryTracker.asyncRefresh(); - if (scheduledRefresh) { - String reason = "Not opening job [" + jobId + "] because job memory requirements are stale - refresh requested"; - logger.debug(reason); - return new PersistentTasksCustomMetaData.Assignment(null, reason); - } else { - allocateByMemory = false; - logger.warn("Falling back to allocating job [{}] by job counts because a memory requirement refresh could not be scheduled", - jobId); - } + boolean allocateByMemory = isMemoryTrackerRecentlyRefreshed; + if (isMemoryTrackerRecentlyRefreshed == false) { + logger.warn("Falling back to allocating job [{}] by job counts because a memory requirement refresh could not be scheduled", + jobId); } List reasons = new LinkedList<>(); @@ -592,12 +573,33 @@ public PersistentTasksCustomMetaData.Assignment getAssignment(OpenJobAction.JobP return AWAITING_UPGRADE; } - PersistentTasksCustomMetaData.Assignment assignment = selectLeastLoadedMlNode(params.getJobId(), + String jobId = params.getJobId(); + String resultsWriteAlias = AnomalyDetectorsIndex.resultsWriteAlias(jobId); + List unavailableIndices = verifyIndicesPrimaryShardsAreActive(resultsWriteAlias, clusterState); + if (unavailableIndices.size() != 0) { + String reason = "Not opening job [" + jobId + "], because not all primary shards are active for the following indices [" + + String.join(",", unavailableIndices) + "]"; + logger.debug(reason); + return new PersistentTasksCustomMetaData.Assignment(null, reason); + } + + boolean isMemoryTrackerRecentlyRefreshed = memoryTracker.isRecentlyRefreshed(); + if (isMemoryTrackerRecentlyRefreshed == false) { + boolean scheduledRefresh = memoryTracker.asyncRefresh(); + if (scheduledRefresh) { + String reason = "Not opening job [" + jobId + "] because job memory requirements are stale - refresh requested"; + logger.debug(reason); + return new PersistentTasksCustomMetaData.Assignment(null, reason); + } + } + + PersistentTasksCustomMetaData.Assignment assignment = selectLeastLoadedMlNode(jobId, params.getJob(), clusterState, maxConcurrentJobAllocations, maxMachineMemoryPercent, memoryTracker, + isMemoryTrackerRecentlyRefreshed, logger); if (assignment.getExecutorNode() == null) { int numMlNodes = 0; diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportOpenJobActionTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportOpenJobActionTests.java index 9b7673338f619..e489a6a9a7c4c 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportOpenJobActionTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportOpenJobActionTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; @@ -60,11 +61,9 @@ import java.net.InetAddress; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Date; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.SortedMap; @@ -79,11 +78,13 @@ public class TransportOpenJobActionTests extends ESTestCase { private MlMemoryTracker memoryTracker; + private boolean isMemoryTrackerRecentlyRefreshed; @Before public void setup() { memoryTracker = mock(MlMemoryTracker.class); - when(memoryTracker.isRecentlyRefreshed()).thenReturn(true); + isMemoryTrackerRecentlyRefreshed = true; + when(memoryTracker.isRecentlyRefreshed()).thenReturn(isMemoryTrackerRecentlyRefreshed); } public void testValidate_jobMissing() { @@ -141,7 +142,7 @@ public void testSelectLeastLoadedMlNode_byCount() { jobBuilder.setJobVersion(Version.CURRENT); Assignment result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id4", jobBuilder.build(), - cs.build(), 2, 30, memoryTracker, logger); + cs.build(), 2, 30, memoryTracker, isMemoryTrackerRecentlyRefreshed, logger); assertEquals("", result.getExplanation()); assertEquals("_node_id3", result.getExecutorNode()); } @@ -178,7 +179,7 @@ public void testSelectLeastLoadedMlNode_maxCapacity() { Job job = BaseMlIntegTestCase.createFareQuoteJob("job_id0", new ByteSizeValue(150, ByteSizeUnit.MB)).build(new Date()); Assignment result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id0", job, cs.build(), 2, - 30, memoryTracker, logger); + 30, memoryTracker, isMemoryTrackerRecentlyRefreshed, logger); assertNull(result.getExecutorNode()); assertTrue(result.getExplanation().contains("because this node is full. Number of opened jobs [" + maxRunningJobsPerNode + "], xpack.ml.max_open_jobs [" + maxRunningJobsPerNode + "]")); @@ -204,7 +205,8 @@ public void testSelectLeastLoadedMlNode_noMlNodes() { Job job = BaseMlIntegTestCase.createFareQuoteJob("job_id2", new ByteSizeValue(2, ByteSizeUnit.MB)).build(new Date()); - Assignment result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id2", job, cs.build(), 2, 30, memoryTracker, logger); + Assignment result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id2", job, cs.build(), 2, 30, memoryTracker, + isMemoryTrackerRecentlyRefreshed, logger); assertTrue(result.getExplanation().contains("because this node isn't a ml node")); assertNull(result.getExecutorNode()); } @@ -239,7 +241,8 @@ public void testSelectLeastLoadedMlNode_maxConcurrentOpeningJobs() { Job job = BaseMlIntegTestCase.createFareQuoteJob("job_id6", new ByteSizeValue(2, ByteSizeUnit.MB)).build(new Date()); ClusterState cs = csBuilder.build(); - Assignment result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id6", job, cs, 2, 30, memoryTracker, logger); + Assignment result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id6", job, cs, 2, 30, memoryTracker, + isMemoryTrackerRecentlyRefreshed, logger); assertEquals("_node_id3", result.getExecutorNode()); tasksBuilder = PersistentTasksCustomMetaData.builder(tasks); @@ -249,7 +252,8 @@ public void testSelectLeastLoadedMlNode_maxConcurrentOpeningJobs() { csBuilder = ClusterState.builder(cs); csBuilder.metaData(MetaData.builder(cs.metaData()).putCustom(PersistentTasksCustomMetaData.TYPE, tasks)); cs = csBuilder.build(); - result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id7", job, cs, 2, 30, memoryTracker, logger); + result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id7", job, cs, 2, 30, memoryTracker, isMemoryTrackerRecentlyRefreshed, + logger); assertNull("no node selected, because OPENING state", result.getExecutorNode()); assertTrue(result.getExplanation().contains("because node exceeds [2] the maximum number of jobs [2] in opening state")); @@ -260,7 +264,8 @@ public void testSelectLeastLoadedMlNode_maxConcurrentOpeningJobs() { csBuilder = ClusterState.builder(cs); csBuilder.metaData(MetaData.builder(cs.metaData()).putCustom(PersistentTasksCustomMetaData.TYPE, tasks)); cs = csBuilder.build(); - result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id7", job, cs, 2, 30, memoryTracker, logger); + result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id7", job, cs, 2, 30, memoryTracker, isMemoryTrackerRecentlyRefreshed, + logger); assertNull("no node selected, because stale task", result.getExecutorNode()); assertTrue(result.getExplanation().contains("because node exceeds [2] the maximum number of jobs [2] in opening state")); @@ -271,7 +276,8 @@ public void testSelectLeastLoadedMlNode_maxConcurrentOpeningJobs() { csBuilder = ClusterState.builder(cs); csBuilder.metaData(MetaData.builder(cs.metaData()).putCustom(PersistentTasksCustomMetaData.TYPE, tasks)); cs = csBuilder.build(); - result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id7", job, cs, 2, 30, memoryTracker, logger); + result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id7", job, cs, 2, 30, memoryTracker, isMemoryTrackerRecentlyRefreshed, + logger); assertNull("no node selected, because null state", result.getExecutorNode()); assertTrue(result.getExplanation().contains("because node exceeds [2] the maximum number of jobs [2] in opening state")); } @@ -310,7 +316,8 @@ public void testSelectLeastLoadedMlNode_concurrentOpeningJobsAndStaleFailedJob() Job job = BaseMlIntegTestCase.createFareQuoteJob("job_id7", new ByteSizeValue(2, ByteSizeUnit.MB)).build(new Date()); // Allocation won't be possible if the stale failed job is treated as opening - Assignment result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id7", job, cs, 2, 30, memoryTracker, logger); + Assignment result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id7", job, cs, 2, 30, memoryTracker, + isMemoryTrackerRecentlyRefreshed, logger); assertEquals("_node_id1", result.getExecutorNode()); tasksBuilder = PersistentTasksCustomMetaData.builder(tasks); @@ -320,7 +327,8 @@ public void testSelectLeastLoadedMlNode_concurrentOpeningJobsAndStaleFailedJob() csBuilder = ClusterState.builder(cs); csBuilder.metaData(MetaData.builder(cs.metaData()).putCustom(PersistentTasksCustomMetaData.TYPE, tasks)); cs = csBuilder.build(); - result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id8", job, cs, 2, 30, memoryTracker, logger); + result = TransportOpenJobAction.selectLeastLoadedMlNode("job_id8", job, cs, 2, 30, memoryTracker, isMemoryTrackerRecentlyRefreshed, + logger); assertNull("no node selected, because OPENING state", result.getExecutorNode()); assertTrue(result.getExplanation().contains("because node exceeds [2] the maximum number of jobs [2] in opening state")); } @@ -353,7 +361,7 @@ public void testSelectLeastLoadedMlNode_noCompatibleJobTypeNodes() { metaData.putCustom(PersistentTasksCustomMetaData.TYPE, tasks); cs.metaData(metaData); Assignment result = TransportOpenJobAction.selectLeastLoadedMlNode("incompatible_type_job", job, cs.build(), 2, 30, - memoryTracker, logger); + memoryTracker, isMemoryTrackerRecentlyRefreshed, logger); assertThat(result.getExplanation(), containsString("because this node does not support jobs of type [incompatible_type]")); assertNull(result.getExecutorNode()); } @@ -384,7 +392,7 @@ public void testSelectLeastLoadedMlNode_noNodesMatchingModelSnapshotMinVersion() metaData.putCustom(PersistentTasksCustomMetaData.TYPE, tasks); cs.metaData(metaData); Assignment result = TransportOpenJobAction.selectLeastLoadedMlNode("job_with_incompatible_model_snapshot", job, cs.build(), - 2, 30, memoryTracker, logger); + 2, 30, memoryTracker, isMemoryTrackerRecentlyRefreshed, logger); assertThat(result.getExplanation(), containsString( "because the job's model snapshot requires a node of version [6.3.0] or higher")); assertNull(result.getExecutorNode()); @@ -413,7 +421,7 @@ public void testSelectLeastLoadedMlNode_jobWithRulesButNoNodeMeetsRequiredVersio Job job = jobWithRules("job_with_rules"); Assignment result = TransportOpenJobAction.selectLeastLoadedMlNode("job_with_rules", job, cs.build(), 2, 30, memoryTracker, - logger); + isMemoryTrackerRecentlyRefreshed, logger); assertThat(result.getExplanation(), containsString( "because jobs using custom_rules require a node of version [6.4.0] or higher")); assertNull(result.getExecutorNode()); @@ -442,7 +450,7 @@ public void testSelectLeastLoadedMlNode_jobWithRulesAndNodeMeetsRequiredVersion( Job job = jobWithRules("job_with_rules"); Assignment result = TransportOpenJobAction.selectLeastLoadedMlNode("job_with_rules", job, cs.build(), 2, 30, memoryTracker, - logger); + isMemoryTrackerRecentlyRefreshed, logger); assertNotNull(result.getExecutorNode()); } @@ -529,10 +537,10 @@ public void testJobTaskMatcherMatch() { public void testGetAssignment_GivenJobThatRequiresMigration() { ClusterService clusterService = mock(ClusterService.class); - ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, new HashSet<>( - Arrays.asList(MachineLearning.CONCURRENT_JOB_ALLOCATIONS, MachineLearning.MAX_MACHINE_MEMORY_PERCENT, - MachineLearning.MAX_LAZY_ML_NODES) - )); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, + Sets.newHashSet(MachineLearning.CONCURRENT_JOB_ALLOCATIONS, MachineLearning.MAX_MACHINE_MEMORY_PERCENT, + MachineLearning.MAX_LAZY_ML_NODES) + ); when(clusterService.getClusterSettings()).thenReturn(clusterSettings); TransportOpenJobAction.OpenJobPersistentTasksExecutor executor = new TransportOpenJobAction.OpenJobPersistentTasksExecutor( @@ -542,6 +550,34 @@ public void testGetAssignment_GivenJobThatRequiresMigration() { assertEquals(TransportOpenJobAction.AWAITING_MIGRATION, executor.getAssignment(params, mock(ClusterState.class))); } + // An index being unavailable should take precedence over waiting for a lazy node + public void testGetAssignment_GivenUnavailableIndicesWithLazyNode() { + Settings settings = Settings.builder().put(MachineLearning.MAX_LAZY_ML_NODES.getKey(), 1).build(); + ClusterService clusterService = mock(ClusterService.class); + ClusterSettings clusterSettings = new ClusterSettings(settings, + Sets.newHashSet(MachineLearning.CONCURRENT_JOB_ALLOCATIONS, MachineLearning.MAX_MACHINE_MEMORY_PERCENT, + MachineLearning.MAX_LAZY_ML_NODES) + ); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + + ClusterState.Builder csBuilder = ClusterState.builder(new ClusterName("_name")); + MetaData.Builder metaData = MetaData.builder(); + RoutingTable.Builder routingTable = RoutingTable.builder(); + addIndices(metaData, routingTable); + routingTable.remove(".ml-state"); + csBuilder.metaData(metaData); + csBuilder.routingTable(routingTable.build()); + + TransportOpenJobAction.OpenJobPersistentTasksExecutor executor = new TransportOpenJobAction.OpenJobPersistentTasksExecutor( + settings, clusterService, mock(AutodetectProcessManager.class), mock(MlMemoryTracker.class), mock(Client.class)); + + OpenJobAction.JobParams params = new OpenJobAction.JobParams("unavailable_index_with_lazy_node"); + params.setJob(mock(Job.class)); + assertEquals("Not opening job [unavailable_index_with_lazy_node], " + + "because not all primary shards are active for the following indices [.ml-state]", + executor.getAssignment(params, csBuilder.build()).getExplanation()); + } + public static void addJobTask(String jobId, String nodeId, JobState jobState, PersistentTasksCustomMetaData.Builder builder) { addJobTask(jobId, nodeId, jobState, builder, false); } From 887fa2c97abba907d56271a21a2c0bbd141f58dd Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 5 Feb 2019 17:10:12 +0100 Subject: [PATCH 06/23] Mute testReadRequestsReturnLatestMappingVersion (#38438) * Relates #37807 --- .../java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java index 707e44310b4b7..32f63787db908 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java @@ -234,6 +234,7 @@ public void testAddNewReplicasOnFollower() throws Exception { pauseFollow("follower-index"); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37807") public void testReadRequestsReturnLatestMappingVersion() throws Exception { InternalTestCluster leaderCluster = getLeaderCluster(); Settings nodeAttributes = Settings.builder().put("node.attr.box", "large").build(); From c9701be1e836d154a0758e609d2d0735b03a5231 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Tue, 5 Feb 2019 18:15:26 +0200 Subject: [PATCH 07/23] SQL: Implement CURRENT_DATE (#38175) Since DATE data type is now available, this implements the `CURRENT_DATE/CURRENT_DATE()/TODAY()` similar to `CURRENT_TIMESTAMP`. Closes: #38160 --- .../sql/functions/date-time.asciidoc | 79 +- .../xpack/sql/qa/cli/ShowTestCase.java | 1 + .../qa/src/main/resources/command.csv-spec | 6 +- .../sql/qa/src/main/resources/date.csv-spec | 87 +++ .../qa/src/main/resources/datetime.sql-spec | 7 + .../sql/qa/src/main/resources/docs.csv-spec | 74 +- x-pack/plugin/sql/src/main/antlr/SqlBase.g4 | 7 +- .../plugin/sql/src/main/antlr/SqlBase.tokens | 4 +- .../sql/src/main/antlr/SqlBaseLexer.tokens | 4 +- .../expression/function/FunctionRegistry.java | 4 +- .../scalar/ConfigurationFunction.java | 2 +- .../function/scalar/datetime/CurrentDate.java | 25 + .../scalar/datetime/CurrentDateTime.java | 52 +- .../scalar/datetime/CurrentFunction.java | 49 ++ .../xpack/sql/parser/ExpressionBuilder.java | 10 +- .../xpack/sql/parser/SqlBaseLexer.java | 723 +++++++++--------- .../xpack/sql/parser/SqlBaseParser.java | 656 ++++++++-------- .../scalar/datetime/CurrentDateTests.java | 46 ++ .../scalar/datetime/CurrentDateTimeTests.java | 64 +- .../xpack/sql/parser/ExpressionTests.java | 18 +- .../xpack/sql/tree/NodeSubclassTests.java | 7 +- 21 files changed, 1161 insertions(+), 764 deletions(-) create mode 100644 x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDate.java create mode 100644 x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentFunction.java create mode 100644 x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTests.java diff --git a/docs/reference/sql/functions/date-time.asciidoc b/docs/reference/sql/functions/date-time.asciidoc index 15fdba39026ef..2d2678a61704d 100644 --- a/docs/reference/sql/functions/date-time.asciidoc +++ b/docs/reference/sql/functions/date-time.asciidoc @@ -93,6 +93,48 @@ include-tagged::{sql-specs}/docs.csv-spec[dtIntervalMul] beta[] +[[sql-functions-current-date]] +==== `CURRENT_DATE/CURDATE` + +.Synopsis: +[source, sql] +-------------------------------------------------- +CURRENT_DATE +CURRENT_DATE() +-------------------------------------------------- + +*Input*: _none_ + +*Output*: date + +.Description: + +Returns the date (no time part) when the current query reached the server. +It can be used both as a keyword: `CURRENT_DATE` or as a function with no arguments: `CURRENT_DATE()`. + +[NOTE] +Unlike CURRENT_DATE, `CURDATE()` can only be used as a function with no arguments and not as a keyword. + +This method always returns the same value for its every occurrence within the same query. + +["source","sql",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{sql-specs}/docs.csv-spec[curDate] +-------------------------------------------------- + +["source","sql",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{sql-specs}/docs.csv-spec[curDateFunction] +-------------------------------------------------- + +Typically, this function (as well as its twin <> function +is used for relative date filtering: + +["source","sql",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{sql-specs}/docs.csv-spec[filterToday] +-------------------------------------------------- + [[sql-functions-current-timestamp]] ==== `CURRENT_TIMESTAMP` @@ -115,7 +157,7 @@ Returns the date/time when the current query reached the server. As a function, `CURRENT_TIMESTAMP()` accepts _precision_ as an optional parameter for rounding the second fractional digits (nanoseconds). -This method always returns the same value within a query. +This method always returns the same value for its every occurrence within the same query. ["source","sql",subs="attributes,callouts,macros"] -------------------------------------------------- @@ -422,7 +464,8 @@ NOW() .Description: This function offers the same functionality as <> function: returns -the datetime when the current query reached the server. This method always returns the same value within a query. +the datetime when the current query reached the server. This method always returns the same value for its every +occurrence within the same query. ["source","sql",subs="attributes,callouts,macros"] -------------------------------------------------- @@ -485,6 +528,38 @@ Extract the year quarter the date/datetime falls in. include-tagged::{sql-specs}/docs.csv-spec[quarter] -------------------------------------------------- +[[sql-functions-today]] +==== `TODAY` + +.Synopsis: +[source, sql] +-------------------------------------------------- +TODAY() +-------------------------------------------------- + +*Input*: _none_ + +*Output*: date + +.Description: + +This function offers the same functionality as <> function: returns +the date when the current query reached the server. This method always returns the same value for its every occurrence +within the same query. + +["source","sql",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{sql-specs}/docs.csv-spec[todayFunction] +-------------------------------------------------- + +Typically, this function (as well as its twin <> function is used +for relative date filtering: + +["source","sql",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{sql-specs}/docs.csv-spec[filterToday] +-------------------------------------------------- + [[sql-functions-datetime-week]] ==== `WEEK_OF_YEAR/WEEK` diff --git a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/cli/ShowTestCase.java b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/cli/ShowTestCase.java index 382442b5bffec..8d5e399017cf6 100644 --- a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/cli/ShowTestCase.java +++ b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/cli/ShowTestCase.java @@ -91,6 +91,7 @@ public void testShowFunctionsLikeInfix() throws IOException { assertThat(readLine(), RegexMatcher.matches("\\s*ISODAYOFWEEK\\s*\\|\\s*SCALAR\\s*")); assertThat(readLine(), RegexMatcher.matches("\\s*ISO_DAY_OF_WEEK\\s*\\|\\s*SCALAR\\s*")); assertThat(readLine(), RegexMatcher.matches("\\s*MINUTE_OF_DAY\\s*\\|\\s*SCALAR\\s*")); + assertThat(readLine(), RegexMatcher.matches("\\s*TODAY\\s*\\|\\s*SCALAR\\s*")); assertEquals("", readLine()); } } diff --git a/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec index c7ebf9420c9a0..15f9f58495deb 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec @@ -31,6 +31,8 @@ ISNULL |CONDITIONAL LEAST |CONDITIONAL NULLIF |CONDITIONAL NVL |CONDITIONAL +CURDATE |SCALAR +CURRENT_DATE |SCALAR CURRENT_TIMESTAMP|SCALAR DAY |SCALAR DAYNAME |SCALAR @@ -65,7 +67,8 @@ MONTH_OF_YEAR |SCALAR NOW |SCALAR QUARTER |SCALAR SECOND |SCALAR -SECOND_OF_MINUTE |SCALAR +SECOND_OF_MINUTE |SCALAR +TODAY |SCALAR WEEK |SCALAR WEEK_OF_YEAR |SCALAR YEAR |SCALAR @@ -175,6 +178,7 @@ HOUR_OF_DAY |SCALAR ISODAYOFWEEK |SCALAR ISO_DAY_OF_WEEK|SCALAR MINUTE_OF_DAY |SCALAR +TODAY |SCALAR ; showTables diff --git a/x-pack/plugin/sql/qa/src/main/resources/date.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/date.csv-spec index f744ea9ca6c70..35db16541babf 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/date.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/date.csv-spec @@ -2,6 +2,73 @@ // Date // +currentDateKeywordWithDivision +SELECT YEAR(CURRENT_TIMESTAMP) / 1000 AS result; + + result +--------------- +2 +; + +currentDateFunctionNoArgsWithDivision +SELECT YEAR(CURRENT_TIMESTAMP()) / 1000 AS result; + + result +--------------- +2 +; + +todayWithDivision +SELECT YEAR(TODAY()) / 1000 AS result; + + result +--------------- +2 +; + +todayIntervalSubstraction +SELECT TRUNCATE(YEAR(TODAY() - INTERVAL 50 YEARS) / 1000) AS result; + + result +--------------- +1 +; + + +currentDateFilter +SELECT first_name FROM test_emp WHERE hire_date > CURRENT_DATE() - INTERVAL 25 YEARS ORDER BY first_name ASC LIMIT 10; + + first_name +----------------- +Kazuhito +Kenroku +Lillian +Mayumi +Mingsen +Sailaja +Saniya +Shahaf +Suzette +Tuval +; + +currentDateFilterScript +SELECT first_name, TRUNCATE(YEAR(hire_date) - YEAR(TODAY()) / 1000) AS filter FROM test_emp +WHERE TRUNCATE(YEAR(hire_date) - YEAR(TODAY()) / 1000) > 1990 ORDER BY first_name ASC LIMIT 10; + + first_name | filter +Cristinel |1991 +Kazuhito |1993 +Kenroku |1992 +Lillian |1997 +Magy |1991 +Mayumi |1993 +Mingsen |1992 +Sailaja |1994 +Saniya |1992 +Shahaf |1993 +; + dateExtractDateParts SELECT DAY(CAST(birth_date AS DATE)) d, @@ -75,3 +142,23 @@ SELECT YEAR(CAST('2019-01-21' AS DATE) + INTERVAL '1-2' YEAR TO MONTH) AS y, MON y:i | m:i 2020 | 3 ; + +orderByCurrentDate +SELECT first_name FROM test_emp ORDER BY TODAY(), first_name LIMIT 5; + + first_name +--------------- +Alejandro +Amabile +Anneke +Anoosh +Arumugam +; + +groupByCurrentDate +SELECT MAX(salary) FROM test_emp GROUP BY TODAY(); + + MAX(salary) +--------------- +74999 +; diff --git a/x-pack/plugin/sql/qa/src/main/resources/datetime.sql-spec b/x-pack/plugin/sql/qa/src/main/resources/datetime.sql-spec index 1bdc090ea232f..1c21e9a7e6f9a 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/datetime.sql-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/datetime.sql-spec @@ -120,6 +120,12 @@ SELECT DAY_OF_WEEK(birth_date) day, COUNT(*) c FROM test_emp WHERE DAY_OF_WEEK(b currentTimestampYear SELECT YEAR(CURRENT_TIMESTAMP()) AS result; +orderByCurrentTimestamp +SELECT first_name FROM test_emp ORDER BY NOW(), first_name NULLS LAST LIMIT 5; + +groupByCurrentTimestamp +SELECT MAX(salary) AS max FROM test_emp GROUP BY NOW(); + // // H2 uses the local timezone instead of the specified one // @@ -131,3 +137,4 @@ SELECT HOUR(CURRENT_TIMESTAMP()) AS result; currentTimestampMinute-Ignore SELECT MINUTE(CURRENT_TIMESTAMP()) AS result; + diff --git a/x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec index 46196f79b29d0..bb572ecca9d1a 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec @@ -208,6 +208,8 @@ ISNULL |CONDITIONAL LEAST |CONDITIONAL NULLIF |CONDITIONAL NVL |CONDITIONAL +CURDATE |SCALAR +CURRENT_DATE |SCALAR CURRENT_TIMESTAMP|SCALAR DAY |SCALAR DAYNAME |SCALAR @@ -242,7 +244,8 @@ MONTH_OF_YEAR |SCALAR NOW |SCALAR QUARTER |SCALAR SECOND |SCALAR -SECOND_OF_MINUTE |SCALAR +SECOND_OF_MINUTE |SCALAR +TODAY |SCALAR WEEK |SCALAR WEEK_OF_YEAR |SCALAR YEAR |SCALAR @@ -365,6 +368,7 @@ HOUR_OF_DAY |SCALAR ISODAYOFWEEK |SCALAR ISO_DAY_OF_WEEK|SCALAR MINUTE_OF_DAY |SCALAR +TODAY |SCALAR // end::showFunctionsWithPattern ; @@ -2227,18 +2231,50 @@ SELECT WEEK(CAST('1988-01-05T09:22:10Z' AS TIMESTAMP)) AS week, ISOWEEK(CAST('19 ; -currentNow -// tag::filterNow -SELECT first_name FROM emp WHERE hire_date > NOW() - INTERVAL 100 YEARS ORDER BY first_name ASC LIMIT 5; - first_name ---------------- -Alejandro -Amabile -Anneke -Anoosh -Arumugam -// end::filterNow + +currentDate-Ignore +// tag::curDate +SELECT CURRENT_TIMESTAMP AS result; + + result +------------------------ +2018-12-12 +// end::curDate +; + +currentDateFunction-Ignore +// tag::curDateFunction +SELECT CURRENT_TIMESTAMP() AS result; + + result +------------------------ +2018-12-12 +// end::curDateFunction +; + +todayFunction-Ignore +// tag::todayFunction +SELECT TODAY() AS result; + + result +------------------------ +2018-12-12 +// end::todayFunction +; + +filterToday +// tag::filterToday +SELECT first_name FROM emp WHERE hire_date > TODAY() - INTERVAL 25 YEARS ORDER BY first_name ASC LIMIT 5; + + first_name +------------ +Kazuhito +Kenroku +Lillian +Mayumi +Mingsen +// end::filterToday ; currentTimestamp-Ignore @@ -2282,6 +2318,20 @@ SELECT NOW() AS result; // end::nowFunction ; +filterNow +// tag::filterNow +SELECT first_name FROM emp WHERE hire_date > NOW() - INTERVAL 100 YEARS ORDER BY first_name ASC LIMIT 5; + + first_name +--------------- +Alejandro +Amabile +Anneke +Anoosh +Arumugam +// end::filterNow +; + //////////// // Next two queries need to have the same output, as they should be equivalent. // They are used in the "SQL Limitations" page. diff --git a/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 b/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 index e7cb2e2b1b258..a11121feaa2bc 100644 --- a/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 +++ b/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 @@ -236,7 +236,8 @@ castTemplate ; builtinDateTimeFunction - : name=CURRENT_TIMESTAMP ('(' precision=INTEGER_VALUE? ')')? + : name=CURRENT_DATE ('(' ')')? + | name=CURRENT_TIMESTAMP ('(' precision=INTEGER_VALUE? ')')? ; convertTemplate @@ -337,7 +338,7 @@ string // http://developer.mimer.se/validator/sql-reserved-words.tml nonReserved : ANALYZE | ANALYZED - | CATALOGS | COLUMNS | CURRENT + | CATALOGS | COLUMNS | DAY | DEBUG | EXECUTABLE | EXPLAIN | FIRST | FORMAT | FULL | FUNCTIONS @@ -370,7 +371,7 @@ CATALOG: 'CATALOG'; CATALOGS: 'CATALOGS'; COLUMNS: 'COLUMNS'; CONVERT: 'CONVERT'; -CURRENT: 'CURRENT'; +CURRENT_DATE : 'CURRENT_DATE'; CURRENT_TIMESTAMP : 'CURRENT_TIMESTAMP'; DAY: 'DAY'; DAYS: 'DAYS'; diff --git a/x-pack/plugin/sql/src/main/antlr/SqlBase.tokens b/x-pack/plugin/sql/src/main/antlr/SqlBase.tokens index f2d522b2bc757..4c3cc5de06421 100644 --- a/x-pack/plugin/sql/src/main/antlr/SqlBase.tokens +++ b/x-pack/plugin/sql/src/main/antlr/SqlBase.tokens @@ -16,7 +16,7 @@ CATALOG=15 CATALOGS=16 COLUMNS=17 CONVERT=18 -CURRENT=19 +CURRENT_DATE=19 CURRENT_TIMESTAMP=20 DAY=21 DAYS=22 @@ -143,7 +143,7 @@ DELIMITER=127 'CATALOGS'=16 'COLUMNS'=17 'CONVERT'=18 -'CURRENT'=19 +'CURRENT_DATE'=19 'CURRENT_TIMESTAMP'=20 'DAY'=21 'DAYS'=22 diff --git a/x-pack/plugin/sql/src/main/antlr/SqlBaseLexer.tokens b/x-pack/plugin/sql/src/main/antlr/SqlBaseLexer.tokens index 8b586035f8520..6d4252c0e723e 100644 --- a/x-pack/plugin/sql/src/main/antlr/SqlBaseLexer.tokens +++ b/x-pack/plugin/sql/src/main/antlr/SqlBaseLexer.tokens @@ -16,7 +16,7 @@ CATALOG=15 CATALOGS=16 COLUMNS=17 CONVERT=18 -CURRENT=19 +CURRENT_DATE=19 CURRENT_TIMESTAMP=20 DAY=21 DAYS=22 @@ -142,7 +142,7 @@ UNRECOGNIZED=126 'CATALOGS'=16 'COLUMNS'=17 'CONVERT'=18 -'CURRENT'=19 +'CURRENT_DATE'=19 'CURRENT_TIMESTAMP'=20 'DAY'=21 'DAYS'=22 diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/FunctionRegistry.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/FunctionRegistry.java index 876af256294c8..ea9f8cba24b7e 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/FunctionRegistry.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/FunctionRegistry.java @@ -27,6 +27,7 @@ import org.elasticsearch.xpack.sql.expression.function.scalar.Cast; import org.elasticsearch.xpack.sql.expression.function.scalar.Database; import org.elasticsearch.xpack.sql.expression.function.scalar.User; +import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.CurrentDate; import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.CurrentDateTime; import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DayName; import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DayOfMonth; @@ -169,7 +170,8 @@ private void defineDefaultFunctions() { def(Greatest.class, Greatest::new, "GREATEST"), def(Least.class, Least::new, "LEAST")); // Date - addToMap(def(CurrentDateTime.class, CurrentDateTime::new, "CURRENT_TIMESTAMP", "NOW"), + addToMap(def(CurrentDate.class, CurrentDate::new, "CURRENT_DATE", "CURDATE", "TODAY"), + def(CurrentDateTime.class, CurrentDateTime::new, "CURRENT_TIMESTAMP", "NOW"), def(DayName.class, DayName::new, "DAY_NAME", "DAYNAME"), def(DayOfMonth.class, DayOfMonth::new, "DAY_OF_MONTH", "DAYOFMONTH", "DAY", "DOM"), def(DayOfWeek.class, DayOfWeek::new, "DAY_OF_WEEK", "DAYOFWEEK", "DOW"), diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/ConfigurationFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/ConfigurationFunction.java index a24fba1b13569..dc49d2a950a63 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/ConfigurationFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/ConfigurationFunction.java @@ -32,7 +32,7 @@ public Expression replaceChildren(List newChildren) { throw new UnsupportedOperationException("this node doesn't have any children"); } - protected Configuration configuration() { + public Configuration configuration() { return configuration; } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDate.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDate.java new file mode 100644 index 0000000000000..03b5567e9266c --- /dev/null +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDate.java @@ -0,0 +1,25 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.sql.expression.function.scalar.datetime; + +import org.elasticsearch.xpack.sql.session.Configuration; +import org.elasticsearch.xpack.sql.tree.NodeInfo; +import org.elasticsearch.xpack.sql.tree.Source; +import org.elasticsearch.xpack.sql.type.DataType; +import org.elasticsearch.xpack.sql.util.DateUtils; + +public class CurrentDate extends CurrentFunction { + + public CurrentDate(Source source, Configuration configuration) { + super(source, configuration, DateUtils.asDateOnly(configuration.now()), DataType.DATE); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, CurrentDate::new, configuration()); + } +} diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTime.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTime.java index 82556795b5961..8224ef090b78c 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTime.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTime.java @@ -7,29 +7,25 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.datetime; import org.elasticsearch.xpack.sql.expression.Expression; -import org.elasticsearch.xpack.sql.expression.function.scalar.ConfigurationFunction; +import org.elasticsearch.xpack.sql.expression.Foldables; import org.elasticsearch.xpack.sql.session.Configuration; -import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.tree.NodeInfo; +import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataType; import java.time.ZonedDateTime; -import java.util.Objects; -public class CurrentDateTime extends ConfigurationFunction { +public class CurrentDateTime extends CurrentFunction { + private final Expression precision; - private final ZonedDateTime dateTime; public CurrentDateTime(Source source, Expression precision, Configuration configuration) { - super(source, configuration, DataType.DATETIME); + super(source, configuration, nanoPrecision(configuration.now(), precision), DataType.DATETIME); this.precision = precision; - int p = precision != null ? ((Number) precision.fold()).intValue() : 0; - this.dateTime = nanoPrecision(configuration().now(), p); } - @Override - public Object fold() { - return dateTime; + Expression precision() { + return precision; } @Override @@ -37,33 +33,13 @@ protected NodeInfo info() { return NodeInfo.create(this, CurrentDateTime::new, precision, configuration()); } - @Override - public int hashCode() { - return Objects.hash(dateTime); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - - if (obj == null || getClass() != obj.getClass()) { - return false; - } - - CurrentDateTime other = (CurrentDateTime) obj; - return Objects.equals(dateTime, other.dateTime); - } - - static ZonedDateTime nanoPrecision(ZonedDateTime zdt, int precision) { - if (zdt != null) { - int nano = zdt.getNano(); - if (precision >= 0 && precision < 10) { - // remove the remainder - nano = nano - nano % (int) Math.pow(10, (9 - precision)); - return zdt.withNano(nano); - } + static ZonedDateTime nanoPrecision(ZonedDateTime zdt, Expression precisionExpression) { + int precision = precisionExpression != null ? Foldables.intValueOf(precisionExpression) : 0; + int nano = zdt.getNano(); + if (precision >= 0 && precision < 10) { + // remove the remainder + nano = nano - nano % (int) Math.pow(10, (9 - precision)); + return zdt.withNano(nano); } return zdt; } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentFunction.java new file mode 100644 index 0000000000000..3aae08903afa5 --- /dev/null +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentFunction.java @@ -0,0 +1,49 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.sql.expression.function.scalar.datetime; + +import org.elasticsearch.xpack.sql.expression.function.scalar.ConfigurationFunction; +import org.elasticsearch.xpack.sql.session.Configuration; +import org.elasticsearch.xpack.sql.tree.Source; +import org.elasticsearch.xpack.sql.type.DataType; + +import java.time.ZonedDateTime; +import java.util.Objects; + +abstract class CurrentFunction extends ConfigurationFunction { + + private final ZonedDateTime date; + + CurrentFunction(Source source, Configuration configuration, ZonedDateTime date, DataType dataType) { + super(source, configuration, dataType); + this.date = date; + } + + @Override + public Object fold() { + return date; + } + + @Override + public int hashCode() { + return Objects.hash(date); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + CurrentFunction other = (CurrentFunction) obj; + return Objects.equals(date, other.date); + } +} diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java index 432872891e5c2..fe8f5ac9925b1 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java @@ -465,7 +465,7 @@ public Function visitExtractExpression(ExtractExpressionContext ctx) { @Override public Object visitBuiltinDateTimeFunction(BuiltinDateTimeFunctionContext ctx) { - // maps current_XXX to their respective functions + // maps CURRENT_XXX to its respective function e.g: CURRENT_TIMESTAMP() // since the functions need access to the Configuration, the parser only registers the definition and not the actual function Source source = source(ctx); Literal p = null; @@ -484,13 +484,15 @@ public Object visitBuiltinDateTimeFunction(BuiltinDateTimeFunctionContext ctx) { } String functionName = ctx.name.getText(); - + switch (ctx.name.getType()) { + case SqlBaseLexer.CURRENT_DATE: + return new UnresolvedFunction(source, functionName, ResolutionType.STANDARD, emptyList()); case SqlBaseLexer.CURRENT_TIMESTAMP: return new UnresolvedFunction(source, functionName, ResolutionType.STANDARD, p != null ? singletonList(p) : emptyList()); + default: + throw new ParsingException(source, "Unknown function [{}]", functionName); } - - throw new ParsingException(source, "Unknown function [{}]", functionName); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseLexer.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseLexer.java index fc113cd58c67c..f62130b14fb07 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseLexer.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseLexer.java @@ -19,7 +19,7 @@ class SqlBaseLexer extends Lexer { public static final int T__0=1, T__1=2, T__2=3, T__3=4, ALL=5, ANALYZE=6, ANALYZED=7, AND=8, ANY=9, AS=10, ASC=11, BETWEEN=12, BY=13, CAST=14, CATALOG=15, CATALOGS=16, COLUMNS=17, - CONVERT=18, CURRENT=19, CURRENT_TIMESTAMP=20, DAY=21, DAYS=22, DEBUG=23, + CONVERT=18, CURRENT_DATE=19, CURRENT_TIMESTAMP=20, DAY=21, DAYS=22, DEBUG=23, DESC=24, DESCRIBE=25, DISTINCT=26, ESCAPE=27, EXECUTABLE=28, EXISTS=29, EXPLAIN=30, EXTRACT=31, FALSE=32, FIRST=33, FORMAT=34, FROM=35, FULL=36, FUNCTIONS=37, GRAPHVIZ=38, GROUP=39, HAVING=40, HOUR=41, HOURS=42, IN=43, @@ -43,16 +43,16 @@ class SqlBaseLexer extends Lexer { public static final String[] ruleNames = { "T__0", "T__1", "T__2", "T__3", "ALL", "ANALYZE", "ANALYZED", "AND", "ANY", "AS", "ASC", "BETWEEN", "BY", "CAST", "CATALOG", "CATALOGS", "COLUMNS", - "CONVERT", "CURRENT", "CURRENT_TIMESTAMP", "DAY", "DAYS", "DEBUG", "DESC", - "DESCRIBE", "DISTINCT", "ESCAPE", "EXECUTABLE", "EXISTS", "EXPLAIN", "EXTRACT", - "FALSE", "FIRST", "FORMAT", "FROM", "FULL", "FUNCTIONS", "GRAPHVIZ", "GROUP", - "HAVING", "HOUR", "HOURS", "IN", "INNER", "INTERVAL", "IS", "JOIN", "LAST", - "LEFT", "LIKE", "LIMIT", "MAPPED", "MATCH", "MINUTE", "MINUTES", "MONTH", - "MONTHS", "NATURAL", "NOT", "NULL", "NULLS", "ON", "OPTIMIZED", "OR", - "ORDER", "OUTER", "PARSED", "PHYSICAL", "PLAN", "RIGHT", "RLIKE", "QUERY", - "SCHEMAS", "SECOND", "SECONDS", "SELECT", "SHOW", "SYS", "TABLE", "TABLES", - "TEXT", "TRUE", "TO", "TYPE", "TYPES", "USING", "VERIFY", "WHERE", "WITH", - "YEAR", "YEARS", "ESCAPE_ESC", "FUNCTION_ESC", "LIMIT_ESC", "DATE_ESC", + "CONVERT", "CURRENT_DATE", "CURRENT_TIMESTAMP", "DAY", "DAYS", "DEBUG", + "DESC", "DESCRIBE", "DISTINCT", "ESCAPE", "EXECUTABLE", "EXISTS", "EXPLAIN", + "EXTRACT", "FALSE", "FIRST", "FORMAT", "FROM", "FULL", "FUNCTIONS", "GRAPHVIZ", + "GROUP", "HAVING", "HOUR", "HOURS", "IN", "INNER", "INTERVAL", "IS", "JOIN", + "LAST", "LEFT", "LIKE", "LIMIT", "MAPPED", "MATCH", "MINUTE", "MINUTES", + "MONTH", "MONTHS", "NATURAL", "NOT", "NULL", "NULLS", "ON", "OPTIMIZED", + "OR", "ORDER", "OUTER", "PARSED", "PHYSICAL", "PLAN", "RIGHT", "RLIKE", + "QUERY", "SCHEMAS", "SECOND", "SECONDS", "SELECT", "SHOW", "SYS", "TABLE", + "TABLES", "TEXT", "TRUE", "TO", "TYPE", "TYPES", "USING", "VERIFY", "WHERE", + "WITH", "YEAR", "YEARS", "ESCAPE_ESC", "FUNCTION_ESC", "LIMIT_ESC", "DATE_ESC", "TIME_ESC", "TIMESTAMP_ESC", "GUID_ESC", "ESC_END", "EQ", "NULLEQ", "NEQ", "LT", "LTE", "GT", "GTE", "PLUS", "MINUS", "ASTERISK", "SLASH", "PERCENT", "CONCAT", "DOT", "PARAM", "STRING", "INTEGER_VALUE", "DECIMAL_VALUE", @@ -64,7 +64,7 @@ class SqlBaseLexer extends Lexer { private static final String[] _LITERAL_NAMES = { null, "'('", "')'", "','", "':'", "'ALL'", "'ANALYZE'", "'ANALYZED'", "'AND'", "'ANY'", "'AS'", "'ASC'", "'BETWEEN'", "'BY'", "'CAST'", "'CATALOG'", - "'CATALOGS'", "'COLUMNS'", "'CONVERT'", "'CURRENT'", "'CURRENT_TIMESTAMP'", + "'CATALOGS'", "'COLUMNS'", "'CONVERT'", "'CURRENT_DATE'", "'CURRENT_TIMESTAMP'", "'DAY'", "'DAYS'", "'DEBUG'", "'DESC'", "'DESCRIBE'", "'DISTINCT'", "'ESCAPE'", "'EXECUTABLE'", "'EXISTS'", "'EXPLAIN'", "'EXTRACT'", "'FALSE'", "'FIRST'", "'FORMAT'", "'FROM'", "'FULL'", "'FUNCTIONS'", "'GRAPHVIZ'", "'GROUP'", @@ -83,16 +83,16 @@ class SqlBaseLexer extends Lexer { private static final String[] _SYMBOLIC_NAMES = { null, null, null, null, null, "ALL", "ANALYZE", "ANALYZED", "AND", "ANY", "AS", "ASC", "BETWEEN", "BY", "CAST", "CATALOG", "CATALOGS", "COLUMNS", - "CONVERT", "CURRENT", "CURRENT_TIMESTAMP", "DAY", "DAYS", "DEBUG", "DESC", - "DESCRIBE", "DISTINCT", "ESCAPE", "EXECUTABLE", "EXISTS", "EXPLAIN", "EXTRACT", - "FALSE", "FIRST", "FORMAT", "FROM", "FULL", "FUNCTIONS", "GRAPHVIZ", "GROUP", - "HAVING", "HOUR", "HOURS", "IN", "INNER", "INTERVAL", "IS", "JOIN", "LAST", - "LEFT", "LIKE", "LIMIT", "MAPPED", "MATCH", "MINUTE", "MINUTES", "MONTH", - "MONTHS", "NATURAL", "NOT", "NULL", "NULLS", "ON", "OPTIMIZED", "OR", - "ORDER", "OUTER", "PARSED", "PHYSICAL", "PLAN", "RIGHT", "RLIKE", "QUERY", - "SCHEMAS", "SECOND", "SECONDS", "SELECT", "SHOW", "SYS", "TABLE", "TABLES", - "TEXT", "TRUE", "TO", "TYPE", "TYPES", "USING", "VERIFY", "WHERE", "WITH", - "YEAR", "YEARS", "ESCAPE_ESC", "FUNCTION_ESC", "LIMIT_ESC", "DATE_ESC", + "CONVERT", "CURRENT_DATE", "CURRENT_TIMESTAMP", "DAY", "DAYS", "DEBUG", + "DESC", "DESCRIBE", "DISTINCT", "ESCAPE", "EXECUTABLE", "EXISTS", "EXPLAIN", + "EXTRACT", "FALSE", "FIRST", "FORMAT", "FROM", "FULL", "FUNCTIONS", "GRAPHVIZ", + "GROUP", "HAVING", "HOUR", "HOURS", "IN", "INNER", "INTERVAL", "IS", "JOIN", + "LAST", "LEFT", "LIKE", "LIMIT", "MAPPED", "MATCH", "MINUTE", "MINUTES", + "MONTH", "MONTHS", "NATURAL", "NOT", "NULL", "NULLS", "ON", "OPTIMIZED", + "OR", "ORDER", "OUTER", "PARSED", "PHYSICAL", "PLAN", "RIGHT", "RLIKE", + "QUERY", "SCHEMAS", "SECOND", "SECONDS", "SELECT", "SHOW", "SYS", "TABLE", + "TABLES", "TEXT", "TRUE", "TO", "TYPE", "TYPES", "USING", "VERIFY", "WHERE", + "WITH", "YEAR", "YEARS", "ESCAPE_ESC", "FUNCTION_ESC", "LIMIT_ESC", "DATE_ESC", "TIME_ESC", "TIMESTAMP_ESC", "GUID_ESC", "ESC_END", "EQ", "NULLEQ", "NEQ", "LT", "LTE", "GT", "GTE", "PLUS", "MINUS", "ASTERISK", "SLASH", "PERCENT", "CONCAT", "DOT", "PARAM", "STRING", "INTEGER_VALUE", "DECIMAL_VALUE", @@ -155,7 +155,7 @@ public SqlBaseLexer(CharStream input) { public ATN getATN() { return _ATN; } public static final String _serializedATN = - "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\2\u0080\u0423\b\1\4"+ + "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\2\u0080\u0428\b\1\4"+ "\2\t\2\4\3\t\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n"+ "\4\13\t\13\4\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\4\21\t\21\4\22"+ "\t\22\4\23\t\23\4\24\t\24\4\25\t\25\4\26\t\26\4\27\t\27\4\30\t\30\4\31"+ @@ -176,345 +176,346 @@ public SqlBaseLexer(CharStream input) { "\3\17\3\17\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\21\3\21\3\21\3\21"+ "\3\21\3\21\3\21\3\21\3\21\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\23"+ "\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\24\3\24\3\24\3\24\3\24\3\24\3\24"+ - "\3\24\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25"+ - "\3\25\3\25\3\25\3\25\3\25\3\26\3\26\3\26\3\26\3\27\3\27\3\27\3\27\3\27"+ - "\3\30\3\30\3\30\3\30\3\30\3\30\3\31\3\31\3\31\3\31\3\31\3\32\3\32\3\32"+ - "\3\32\3\32\3\32\3\32\3\32\3\32\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33"+ - "\3\33\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\35\3\35\3\35\3\35\3\35\3\35"+ - "\3\35\3\35\3\35\3\35\3\35\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\37\3\37"+ - "\3\37\3\37\3\37\3\37\3\37\3\37\3 \3 \3 \3 \3 \3 \3 \3 \3!\3!\3!\3!\3!"+ - "\3!\3\"\3\"\3\"\3\"\3\"\3\"\3#\3#\3#\3#\3#\3#\3#\3$\3$\3$\3$\3$\3%\3%"+ - "\3%\3%\3%\3&\3&\3&\3&\3&\3&\3&\3&\3&\3&\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3"+ - "\'\3\'\3(\3(\3(\3(\3(\3(\3)\3)\3)\3)\3)\3)\3)\3*\3*\3*\3*\3*\3+\3+\3+"+ - "\3+\3+\3+\3,\3,\3,\3-\3-\3-\3-\3-\3-\3.\3.\3.\3.\3.\3.\3.\3.\3.\3/\3/"+ - "\3/\3\60\3\60\3\60\3\60\3\60\3\61\3\61\3\61\3\61\3\61\3\62\3\62\3\62\3"+ - "\62\3\62\3\63\3\63\3\63\3\63\3\63\3\64\3\64\3\64\3\64\3\64\3\64\3\65\3"+ - "\65\3\65\3\65\3\65\3\65\3\65\3\66\3\66\3\66\3\66\3\66\3\66\3\67\3\67\3"+ - "\67\3\67\3\67\3\67\3\67\38\38\38\38\38\38\38\38\39\39\39\39\39\39\3:\3"+ - ":\3:\3:\3:\3:\3:\3;\3;\3;\3;\3;\3;\3;\3;\3<\3<\3<\3<\3=\3=\3=\3=\3=\3"+ - ">\3>\3>\3>\3>\3>\3?\3?\3?\3@\3@\3@\3@\3@\3@\3@\3@\3@\3@\3A\3A\3A\3B\3"+ - "B\3B\3B\3B\3B\3C\3C\3C\3C\3C\3C\3D\3D\3D\3D\3D\3D\3D\3E\3E\3E\3E\3E\3"+ - "E\3E\3E\3E\3F\3F\3F\3F\3F\3G\3G\3G\3G\3G\3G\3H\3H\3H\3H\3H\3H\3I\3I\3"+ - "I\3I\3I\3I\3J\3J\3J\3J\3J\3J\3J\3J\3K\3K\3K\3K\3K\3K\3K\3L\3L\3L\3L\3"+ - "L\3L\3L\3L\3M\3M\3M\3M\3M\3M\3M\3N\3N\3N\3N\3N\3O\3O\3O\3O\3P\3P\3P\3"+ - "P\3P\3P\3Q\3Q\3Q\3Q\3Q\3Q\3Q\3R\3R\3R\3R\3R\3S\3S\3S\3S\3S\3T\3T\3T\3"+ - "U\3U\3U\3U\3U\3V\3V\3V\3V\3V\3V\3W\3W\3W\3W\3W\3W\3X\3X\3X\3X\3X\3X\3"+ - "X\3Y\3Y\3Y\3Y\3Y\3Y\3Z\3Z\3Z\3Z\3Z\3[\3[\3[\3[\3[\3\\\3\\\3\\\3\\\3\\"+ - "\3\\\3]\3]\3]\3]\3]\3]\3]\3]\3^\3^\3^\3^\3_\3_\3_\3_\3_\3_\3_\3`\3`\3"+ - "`\3a\3a\3a\3b\3b\3b\3b\3c\3c\3c\3c\3c\3c\3d\3d\3e\3e\3f\3f\3f\3f\3g\3"+ - "g\3g\3g\5g\u0364\ng\3h\3h\3i\3i\3i\3j\3j\3k\3k\3k\3l\3l\3m\3m\3n\3n\3"+ - "o\3o\3p\3p\3q\3q\3q\3r\3r\3s\3s\3t\3t\3t\3t\7t\u0385\nt\ft\16t\u0388\13"+ - "t\3t\3t\3u\6u\u038d\nu\ru\16u\u038e\3v\6v\u0392\nv\rv\16v\u0393\3v\3v"+ - "\7v\u0398\nv\fv\16v\u039b\13v\3v\3v\6v\u039f\nv\rv\16v\u03a0\3v\6v\u03a4"+ - "\nv\rv\16v\u03a5\3v\3v\7v\u03aa\nv\fv\16v\u03ad\13v\5v\u03af\nv\3v\3v"+ - "\3v\3v\6v\u03b5\nv\rv\16v\u03b6\3v\3v\5v\u03bb\nv\3w\3w\5w\u03bf\nw\3"+ - "w\3w\3w\7w\u03c4\nw\fw\16w\u03c7\13w\3x\3x\3x\3x\6x\u03cd\nx\rx\16x\u03ce"+ - "\3y\3y\3y\6y\u03d4\ny\ry\16y\u03d5\3z\3z\3z\3z\7z\u03dc\nz\fz\16z\u03df"+ - "\13z\3z\3z\3{\3{\3{\3{\7{\u03e7\n{\f{\16{\u03ea\13{\3{\3{\3|\3|\5|\u03f0"+ - "\n|\3|\6|\u03f3\n|\r|\16|\u03f4\3}\3}\3~\3~\3\177\3\177\3\177\3\177\7"+ - "\177\u03ff\n\177\f\177\16\177\u0402\13\177\3\177\5\177\u0405\n\177\3\177"+ - "\5\177\u0408\n\177\3\177\3\177\3\u0080\3\u0080\3\u0080\3\u0080\3\u0080"+ - "\7\u0080\u0411\n\u0080\f\u0080\16\u0080\u0414\13\u0080\3\u0080\3\u0080"+ - "\3\u0080\3\u0080\3\u0080\3\u0081\6\u0081\u041c\n\u0081\r\u0081\16\u0081"+ - "\u041d\3\u0081\3\u0081\3\u0082\3\u0082\3\u0412\2\u0083\3\3\5\4\7\5\t\6"+ - "\13\7\r\b\17\t\21\n\23\13\25\f\27\r\31\16\33\17\35\20\37\21!\22#\23%\24"+ - "\'\25)\26+\27-\30/\31\61\32\63\33\65\34\67\359\36;\37= ?!A\"C#E$G%I&K"+ - "\'M(O)Q*S+U,W-Y.[/]\60_\61a\62c\63e\64g\65i\66k\67m8o9q:s;u{?}@\177"+ - "A\u0081B\u0083C\u0085D\u0087E\u0089F\u008bG\u008dH\u008fI\u0091J\u0093"+ - "K\u0095L\u0097M\u0099N\u009bO\u009dP\u009fQ\u00a1R\u00a3S\u00a5T\u00a7"+ - "U\u00a9V\u00abW\u00adX\u00afY\u00b1Z\u00b3[\u00b5\\\u00b7]\u00b9^\u00bb"+ - "_\u00bd`\u00bfa\u00c1b\u00c3c\u00c5d\u00c7e\u00c9f\u00cbg\u00cdh\u00cf"+ - "i\u00d1j\u00d3k\u00d5l\u00d7m\u00d9n\u00dbo\u00ddp\u00dfq\u00e1r\u00e3"+ - "s\u00e5t\u00e7u\u00e9v\u00ebw\u00edx\u00efy\u00f1z\u00f3{\u00f5|\u00f7"+ - "\2\u00f9\2\u00fb\2\u00fd}\u00ff~\u0101\177\u0103\u0080\3\2\f\3\2))\4\2"+ - "BBaa\5\2<\3\2\2\2\u01c3\u01c4\7G\2\2\u01c4\u01c5\7Z\2\2\u01c5\u01c6\7V\2"+ - "\2\u01c6\u01c7\7T\2\2\u01c7\u01c8\7C\2\2\u01c8\u01c9\7E\2\2\u01c9\u01ca"+ - "\7V\2\2\u01ca@\3\2\2\2\u01cb\u01cc\7H\2\2\u01cc\u01cd\7C\2\2\u01cd\u01ce"+ - "\7N\2\2\u01ce\u01cf\7U\2\2\u01cf\u01d0\7G\2\2\u01d0B\3\2\2\2\u01d1\u01d2"+ - "\7H\2\2\u01d2\u01d3\7K\2\2\u01d3\u01d4\7T\2\2\u01d4\u01d5\7U\2\2\u01d5"+ - "\u01d6\7V\2\2\u01d6D\3\2\2\2\u01d7\u01d8\7H\2\2\u01d8\u01d9\7Q\2\2\u01d9"+ - "\u01da\7T\2\2\u01da\u01db\7O\2\2\u01db\u01dc\7C\2\2\u01dc\u01dd\7V\2\2"+ - "\u01ddF\3\2\2\2\u01de\u01df\7H\2\2\u01df\u01e0\7T\2\2\u01e0\u01e1\7Q\2"+ - "\2\u01e1\u01e2\7O\2\2\u01e2H\3\2\2\2\u01e3\u01e4\7H\2\2\u01e4\u01e5\7"+ - "W\2\2\u01e5\u01e6\7N\2\2\u01e6\u01e7\7N\2\2\u01e7J\3\2\2\2\u01e8\u01e9"+ - "\7H\2\2\u01e9\u01ea\7W\2\2\u01ea\u01eb\7P\2\2\u01eb\u01ec\7E\2\2\u01ec"+ - "\u01ed\7V\2\2\u01ed\u01ee\7K\2\2\u01ee\u01ef\7Q\2\2\u01ef\u01f0\7P\2\2"+ - "\u01f0\u01f1\7U\2\2\u01f1L\3\2\2\2\u01f2\u01f3\7I\2\2\u01f3\u01f4\7T\2"+ - "\2\u01f4\u01f5\7C\2\2\u01f5\u01f6\7R\2\2\u01f6\u01f7\7J\2\2\u01f7\u01f8"+ - "\7X\2\2\u01f8\u01f9\7K\2\2\u01f9\u01fa\7\\\2\2\u01faN\3\2\2\2\u01fb\u01fc"+ - "\7I\2\2\u01fc\u01fd\7T\2\2\u01fd\u01fe\7Q\2\2\u01fe\u01ff\7W\2\2\u01ff"+ - "\u0200\7R\2\2\u0200P\3\2\2\2\u0201\u0202\7J\2\2\u0202\u0203\7C\2\2\u0203"+ - "\u0204\7X\2\2\u0204\u0205\7K\2\2\u0205\u0206\7P\2\2\u0206\u0207\7I\2\2"+ - "\u0207R\3\2\2\2\u0208\u0209\7J\2\2\u0209\u020a\7Q\2\2\u020a\u020b\7W\2"+ - "\2\u020b\u020c\7T\2\2\u020cT\3\2\2\2\u020d\u020e\7J\2\2\u020e\u020f\7"+ - "Q\2\2\u020f\u0210\7W\2\2\u0210\u0211\7T\2\2\u0211\u0212\7U\2\2\u0212V"+ - "\3\2\2\2\u0213\u0214\7K\2\2\u0214\u0215\7P\2\2\u0215X\3\2\2\2\u0216\u0217"+ - "\7K\2\2\u0217\u0218\7P\2\2\u0218\u0219\7P\2\2\u0219\u021a\7G\2\2\u021a"+ - "\u021b\7T\2\2\u021bZ\3\2\2\2\u021c\u021d\7K\2\2\u021d\u021e\7P\2\2\u021e"+ - "\u021f\7V\2\2\u021f\u0220\7G\2\2\u0220\u0221\7T\2\2\u0221\u0222\7X\2\2"+ - "\u0222\u0223\7C\2\2\u0223\u0224\7N\2\2\u0224\\\3\2\2\2\u0225\u0226\7K"+ - "\2\2\u0226\u0227\7U\2\2\u0227^\3\2\2\2\u0228\u0229\7L\2\2\u0229\u022a"+ - "\7Q\2\2\u022a\u022b\7K\2\2\u022b\u022c\7P\2\2\u022c`\3\2\2\2\u022d\u022e"+ - "\7N\2\2\u022e\u022f\7C\2\2\u022f\u0230\7U\2\2\u0230\u0231\7V\2\2\u0231"+ - "b\3\2\2\2\u0232\u0233\7N\2\2\u0233\u0234\7G\2\2\u0234\u0235\7H\2\2\u0235"+ - "\u0236\7V\2\2\u0236d\3\2\2\2\u0237\u0238\7N\2\2\u0238\u0239\7K\2\2\u0239"+ - "\u023a\7M\2\2\u023a\u023b\7G\2\2\u023bf\3\2\2\2\u023c\u023d\7N\2\2\u023d"+ - "\u023e\7K\2\2\u023e\u023f\7O\2\2\u023f\u0240\7K\2\2\u0240\u0241\7V\2\2"+ - "\u0241h\3\2\2\2\u0242\u0243\7O\2\2\u0243\u0244\7C\2\2\u0244\u0245\7R\2"+ - "\2\u0245\u0246\7R\2\2\u0246\u0247\7G\2\2\u0247\u0248\7F\2\2\u0248j\3\2"+ - "\2\2\u0249\u024a\7O\2\2\u024a\u024b\7C\2\2\u024b\u024c\7V\2\2\u024c\u024d"+ - "\7E\2\2\u024d\u024e\7J\2\2\u024el\3\2\2\2\u024f\u0250\7O\2\2\u0250\u0251"+ - "\7K\2\2\u0251\u0252\7P\2\2\u0252\u0253\7W\2\2\u0253\u0254\7V\2\2\u0254"+ - "\u0255\7G\2\2\u0255n\3\2\2\2\u0256\u0257\7O\2\2\u0257\u0258\7K\2\2\u0258"+ - "\u0259\7P\2\2\u0259\u025a\7W\2\2\u025a\u025b\7V\2\2\u025b\u025c\7G\2\2"+ - "\u025c\u025d\7U\2\2\u025dp\3\2\2\2\u025e\u025f\7O\2\2\u025f\u0260\7Q\2"+ - "\2\u0260\u0261\7P\2\2\u0261\u0262\7V\2\2\u0262\u0263\7J\2\2\u0263r\3\2"+ - "\2\2\u0264\u0265\7O\2\2\u0265\u0266\7Q\2\2\u0266\u0267\7P\2\2\u0267\u0268"+ - "\7V\2\2\u0268\u0269\7J\2\2\u0269\u026a\7U\2\2\u026at\3\2\2\2\u026b\u026c"+ - "\7P\2\2\u026c\u026d\7C\2\2\u026d\u026e\7V\2\2\u026e\u026f\7W\2\2\u026f"+ - "\u0270\7T\2\2\u0270\u0271\7C\2\2\u0271\u0272\7N\2\2\u0272v\3\2\2\2\u0273"+ - "\u0274\7P\2\2\u0274\u0275\7Q\2\2\u0275\u0276\7V\2\2\u0276x\3\2\2\2\u0277"+ - "\u0278\7P\2\2\u0278\u0279\7W\2\2\u0279\u027a\7N\2\2\u027a\u027b\7N\2\2"+ - "\u027bz\3\2\2\2\u027c\u027d\7P\2\2\u027d\u027e\7W\2\2\u027e\u027f\7N\2"+ - "\2\u027f\u0280\7N\2\2\u0280\u0281\7U\2\2\u0281|\3\2\2\2\u0282\u0283\7"+ - "Q\2\2\u0283\u0284\7P\2\2\u0284~\3\2\2\2\u0285\u0286\7Q\2\2\u0286\u0287"+ - "\7R\2\2\u0287\u0288\7V\2\2\u0288\u0289\7K\2\2\u0289\u028a\7O\2\2\u028a"+ - "\u028b\7K\2\2\u028b\u028c\7\\\2\2\u028c\u028d\7G\2\2\u028d\u028e\7F\2"+ - "\2\u028e\u0080\3\2\2\2\u028f\u0290\7Q\2\2\u0290\u0291\7T\2\2\u0291\u0082"+ - "\3\2\2\2\u0292\u0293\7Q\2\2\u0293\u0294\7T\2\2\u0294\u0295\7F\2\2\u0295"+ - "\u0296\7G\2\2\u0296\u0297\7T\2\2\u0297\u0084\3\2\2\2\u0298\u0299\7Q\2"+ - "\2\u0299\u029a\7W\2\2\u029a\u029b\7V\2\2\u029b\u029c\7G\2\2\u029c\u029d"+ - "\7T\2\2\u029d\u0086\3\2\2\2\u029e\u029f\7R\2\2\u029f\u02a0\7C\2\2\u02a0"+ - "\u02a1\7T\2\2\u02a1\u02a2\7U\2\2\u02a2\u02a3\7G\2\2\u02a3\u02a4\7F\2\2"+ - "\u02a4\u0088\3\2\2\2\u02a5\u02a6\7R\2\2\u02a6\u02a7\7J\2\2\u02a7\u02a8"+ - "\7[\2\2\u02a8\u02a9\7U\2\2\u02a9\u02aa\7K\2\2\u02aa\u02ab\7E\2\2\u02ab"+ - "\u02ac\7C\2\2\u02ac\u02ad\7N\2\2\u02ad\u008a\3\2\2\2\u02ae\u02af\7R\2"+ - "\2\u02af\u02b0\7N\2\2\u02b0\u02b1\7C\2\2\u02b1\u02b2\7P\2\2\u02b2\u008c"+ - "\3\2\2\2\u02b3\u02b4\7T\2\2\u02b4\u02b5\7K\2\2\u02b5\u02b6\7I\2\2\u02b6"+ - "\u02b7\7J\2\2\u02b7\u02b8\7V\2\2\u02b8\u008e\3\2\2\2\u02b9\u02ba\7T\2"+ - "\2\u02ba\u02bb\7N\2\2\u02bb\u02bc\7K\2\2\u02bc\u02bd\7M\2\2\u02bd\u02be"+ - "\7G\2\2\u02be\u0090\3\2\2\2\u02bf\u02c0\7S\2\2\u02c0\u02c1\7W\2\2\u02c1"+ - "\u02c2\7G\2\2\u02c2\u02c3\7T\2\2\u02c3\u02c4\7[\2\2\u02c4\u0092\3\2\2"+ - "\2\u02c5\u02c6\7U\2\2\u02c6\u02c7\7E\2\2\u02c7\u02c8\7J\2\2\u02c8\u02c9"+ - "\7G\2\2\u02c9\u02ca\7O\2\2\u02ca\u02cb\7C\2\2\u02cb\u02cc\7U\2\2\u02cc"+ - "\u0094\3\2\2\2\u02cd\u02ce\7U\2\2\u02ce\u02cf\7G\2\2\u02cf\u02d0\7E\2"+ - "\2\u02d0\u02d1\7Q\2\2\u02d1\u02d2\7P\2\2\u02d2\u02d3\7F\2\2\u02d3\u0096"+ - "\3\2\2\2\u02d4\u02d5\7U\2\2\u02d5\u02d6\7G\2\2\u02d6\u02d7\7E\2\2\u02d7"+ - "\u02d8\7Q\2\2\u02d8\u02d9\7P\2\2\u02d9\u02da\7F\2\2\u02da\u02db\7U\2\2"+ - "\u02db\u0098\3\2\2\2\u02dc\u02dd\7U\2\2\u02dd\u02de\7G\2\2\u02de\u02df"+ - "\7N\2\2\u02df\u02e0\7G\2\2\u02e0\u02e1\7E\2\2\u02e1\u02e2\7V\2\2\u02e2"+ - "\u009a\3\2\2\2\u02e3\u02e4\7U\2\2\u02e4\u02e5\7J\2\2\u02e5\u02e6\7Q\2"+ - "\2\u02e6\u02e7\7Y\2\2\u02e7\u009c\3\2\2\2\u02e8\u02e9\7U\2\2\u02e9\u02ea"+ - "\7[\2\2\u02ea\u02eb\7U\2\2\u02eb\u009e\3\2\2\2\u02ec\u02ed\7V\2\2\u02ed"+ - "\u02ee\7C\2\2\u02ee\u02ef\7D\2\2\u02ef\u02f0\7N\2\2\u02f0\u02f1\7G\2\2"+ - "\u02f1\u00a0\3\2\2\2\u02f2\u02f3\7V\2\2\u02f3\u02f4\7C\2\2\u02f4\u02f5"+ - "\7D\2\2\u02f5\u02f6\7N\2\2\u02f6\u02f7\7G\2\2\u02f7\u02f8\7U\2\2\u02f8"+ - "\u00a2\3\2\2\2\u02f9\u02fa\7V\2\2\u02fa\u02fb\7G\2\2\u02fb\u02fc\7Z\2"+ - "\2\u02fc\u02fd\7V\2\2\u02fd\u00a4\3\2\2\2\u02fe\u02ff\7V\2\2\u02ff\u0300"+ - "\7T\2\2\u0300\u0301\7W\2\2\u0301\u0302\7G\2\2\u0302\u00a6\3\2\2\2\u0303"+ - "\u0304\7V\2\2\u0304\u0305\7Q\2\2\u0305\u00a8\3\2\2\2\u0306\u0307\7V\2"+ - "\2\u0307\u0308\7[\2\2\u0308\u0309\7R\2\2\u0309\u030a\7G\2\2\u030a\u00aa"+ - "\3\2\2\2\u030b\u030c\7V\2\2\u030c\u030d\7[\2\2\u030d\u030e\7R\2\2\u030e"+ - "\u030f\7G\2\2\u030f\u0310\7U\2\2\u0310\u00ac\3\2\2\2\u0311\u0312\7W\2"+ - "\2\u0312\u0313\7U\2\2\u0313\u0314\7K\2\2\u0314\u0315\7P\2\2\u0315\u0316"+ - "\7I\2\2\u0316\u00ae\3\2\2\2\u0317\u0318\7X\2\2\u0318\u0319\7G\2\2\u0319"+ - "\u031a\7T\2\2\u031a\u031b\7K\2\2\u031b\u031c\7H\2\2\u031c\u031d\7[\2\2"+ - "\u031d\u00b0\3\2\2\2\u031e\u031f\7Y\2\2\u031f\u0320\7J\2\2\u0320\u0321"+ - "\7G\2\2\u0321\u0322\7T\2\2\u0322\u0323\7G\2\2\u0323\u00b2\3\2\2\2\u0324"+ - "\u0325\7Y\2\2\u0325\u0326\7K\2\2\u0326\u0327\7V\2\2\u0327\u0328\7J\2\2"+ - "\u0328\u00b4\3\2\2\2\u0329\u032a\7[\2\2\u032a\u032b\7G\2\2\u032b\u032c"+ - "\7C\2\2\u032c\u032d\7T\2\2\u032d\u00b6\3\2\2\2\u032e\u032f\7[\2\2\u032f"+ - "\u0330\7G\2\2\u0330\u0331\7C\2\2\u0331\u0332\7T\2\2\u0332\u0333\7U\2\2"+ - "\u0333\u00b8\3\2\2\2\u0334\u0335\7}\2\2\u0335\u0336\7G\2\2\u0336\u0337"+ - "\7U\2\2\u0337\u0338\7E\2\2\u0338\u0339\7C\2\2\u0339\u033a\7R\2\2\u033a"+ - "\u033b\7G\2\2\u033b\u00ba\3\2\2\2\u033c\u033d\7}\2\2\u033d\u033e\7H\2"+ - "\2\u033e\u033f\7P\2\2\u033f\u00bc\3\2\2\2\u0340\u0341\7}\2\2\u0341\u0342"+ - "\7N\2\2\u0342\u0343\7K\2\2\u0343\u0344\7O\2\2\u0344\u0345\7K\2\2\u0345"+ - "\u0346\7V\2\2\u0346\u00be\3\2\2\2\u0347\u0348\7}\2\2\u0348\u0349\7F\2"+ - "\2\u0349\u00c0\3\2\2\2\u034a\u034b\7}\2\2\u034b\u034c\7V\2\2\u034c\u00c2"+ - "\3\2\2\2\u034d\u034e\7}\2\2\u034e\u034f\7V\2\2\u034f\u0350\7U\2\2\u0350"+ - "\u00c4\3\2\2\2\u0351\u0352\7}\2\2\u0352\u0353\7I\2\2\u0353\u0354\7W\2"+ - "\2\u0354\u0355\7K\2\2\u0355\u0356\7F\2\2\u0356\u00c6\3\2\2\2\u0357\u0358"+ - "\7\177\2\2\u0358\u00c8\3\2\2\2\u0359\u035a\7?\2\2\u035a\u00ca\3\2\2\2"+ - "\u035b\u035c\7>\2\2\u035c\u035d\7?\2\2\u035d\u035e\7@\2\2\u035e\u00cc"+ - "\3\2\2\2\u035f\u0360\7>\2\2\u0360\u0364\7@\2\2\u0361\u0362\7#\2\2\u0362"+ - "\u0364\7?\2\2\u0363\u035f\3\2\2\2\u0363\u0361\3\2\2\2\u0364\u00ce\3\2"+ - "\2\2\u0365\u0366\7>\2\2\u0366\u00d0\3\2\2\2\u0367\u0368\7>\2\2\u0368\u0369"+ - "\7?\2\2\u0369\u00d2\3\2\2\2\u036a\u036b\7@\2\2\u036b\u00d4\3\2\2\2\u036c"+ - "\u036d\7@\2\2\u036d\u036e\7?\2\2\u036e\u00d6\3\2\2\2\u036f\u0370\7-\2"+ - "\2\u0370\u00d8\3\2\2\2\u0371\u0372\7/\2\2\u0372\u00da\3\2\2\2\u0373\u0374"+ - "\7,\2\2\u0374\u00dc\3\2\2\2\u0375\u0376\7\61\2\2\u0376\u00de\3\2\2\2\u0377"+ - "\u0378\7\'\2\2\u0378\u00e0\3\2\2\2\u0379\u037a\7~\2\2\u037a\u037b\7~\2"+ - "\2\u037b\u00e2\3\2\2\2\u037c\u037d\7\60\2\2\u037d\u00e4\3\2\2\2\u037e"+ - "\u037f\7A\2\2\u037f\u00e6\3\2\2\2\u0380\u0386\7)\2\2\u0381\u0385\n\2\2"+ - "\2\u0382\u0383\7)\2\2\u0383\u0385\7)\2\2\u0384\u0381\3\2\2\2\u0384\u0382"+ - "\3\2\2\2\u0385\u0388\3\2\2\2\u0386\u0384\3\2\2\2\u0386\u0387\3\2\2\2\u0387"+ - "\u0389\3\2\2\2\u0388\u0386\3\2\2\2\u0389\u038a\7)\2\2\u038a\u00e8\3\2"+ - "\2\2\u038b\u038d\5\u00f9}\2\u038c\u038b\3\2\2\2\u038d\u038e\3\2\2\2\u038e"+ - "\u038c\3\2\2\2\u038e\u038f\3\2\2\2\u038f\u00ea\3\2\2\2\u0390\u0392\5\u00f9"+ + "\3\24\3\24\3\24\3\24\3\24\3\24\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25"+ + "\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\26\3\26\3\26\3\26"+ + "\3\27\3\27\3\27\3\27\3\27\3\30\3\30\3\30\3\30\3\30\3\30\3\31\3\31\3\31"+ + "\3\31\3\31\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\33\3\33\3\33"+ + "\3\33\3\33\3\33\3\33\3\33\3\33\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\35"+ + "\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\36\3\36\3\36\3\36"+ + "\3\36\3\36\3\36\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3 \3 \3 \3 \3"+ + " \3 \3 \3 \3!\3!\3!\3!\3!\3!\3\"\3\"\3\"\3\"\3\"\3\"\3#\3#\3#\3#\3#\3"+ + "#\3#\3$\3$\3$\3$\3$\3%\3%\3%\3%\3%\3&\3&\3&\3&\3&\3&\3&\3&\3&\3&\3\'\3"+ + "\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3(\3(\3(\3(\3(\3(\3)\3)\3)\3)\3)\3)\3)"+ + "\3*\3*\3*\3*\3*\3+\3+\3+\3+\3+\3+\3,\3,\3,\3-\3-\3-\3-\3-\3-\3.\3.\3."+ + "\3.\3.\3.\3.\3.\3.\3/\3/\3/\3\60\3\60\3\60\3\60\3\60\3\61\3\61\3\61\3"+ + "\61\3\61\3\62\3\62\3\62\3\62\3\62\3\63\3\63\3\63\3\63\3\63\3\64\3\64\3"+ + "\64\3\64\3\64\3\64\3\65\3\65\3\65\3\65\3\65\3\65\3\65\3\66\3\66\3\66\3"+ + "\66\3\66\3\66\3\67\3\67\3\67\3\67\3\67\3\67\3\67\38\38\38\38\38\38\38"+ + "\38\39\39\39\39\39\39\3:\3:\3:\3:\3:\3:\3:\3;\3;\3;\3;\3;\3;\3;\3;\3<"+ + "\3<\3<\3<\3=\3=\3=\3=\3=\3>\3>\3>\3>\3>\3>\3?\3?\3?\3@\3@\3@\3@\3@\3@"+ + "\3@\3@\3@\3@\3A\3A\3A\3B\3B\3B\3B\3B\3B\3C\3C\3C\3C\3C\3C\3D\3D\3D\3D"+ + "\3D\3D\3D\3E\3E\3E\3E\3E\3E\3E\3E\3E\3F\3F\3F\3F\3F\3G\3G\3G\3G\3G\3G"+ + "\3H\3H\3H\3H\3H\3H\3I\3I\3I\3I\3I\3I\3J\3J\3J\3J\3J\3J\3J\3J\3K\3K\3K"+ + "\3K\3K\3K\3K\3L\3L\3L\3L\3L\3L\3L\3L\3M\3M\3M\3M\3M\3M\3M\3N\3N\3N\3N"+ + "\3N\3O\3O\3O\3O\3P\3P\3P\3P\3P\3P\3Q\3Q\3Q\3Q\3Q\3Q\3Q\3R\3R\3R\3R\3R"+ + "\3S\3S\3S\3S\3S\3T\3T\3T\3U\3U\3U\3U\3U\3V\3V\3V\3V\3V\3V\3W\3W\3W\3W"+ + "\3W\3W\3X\3X\3X\3X\3X\3X\3X\3Y\3Y\3Y\3Y\3Y\3Y\3Z\3Z\3Z\3Z\3Z\3[\3[\3["+ + "\3[\3[\3\\\3\\\3\\\3\\\3\\\3\\\3]\3]\3]\3]\3]\3]\3]\3]\3^\3^\3^\3^\3_"+ + "\3_\3_\3_\3_\3_\3_\3`\3`\3`\3a\3a\3a\3b\3b\3b\3b\3c\3c\3c\3c\3c\3c\3d"+ + "\3d\3e\3e\3f\3f\3f\3f\3g\3g\3g\3g\5g\u0369\ng\3h\3h\3i\3i\3i\3j\3j\3k"+ + "\3k\3k\3l\3l\3m\3m\3n\3n\3o\3o\3p\3p\3q\3q\3q\3r\3r\3s\3s\3t\3t\3t\3t"+ + "\7t\u038a\nt\ft\16t\u038d\13t\3t\3t\3u\6u\u0392\nu\ru\16u\u0393\3v\6v"+ + "\u0397\nv\rv\16v\u0398\3v\3v\7v\u039d\nv\fv\16v\u03a0\13v\3v\3v\6v\u03a4"+ + "\nv\rv\16v\u03a5\3v\6v\u03a9\nv\rv\16v\u03aa\3v\3v\7v\u03af\nv\fv\16v"+ + "\u03b2\13v\5v\u03b4\nv\3v\3v\3v\3v\6v\u03ba\nv\rv\16v\u03bb\3v\3v\5v\u03c0"+ + "\nv\3w\3w\5w\u03c4\nw\3w\3w\3w\7w\u03c9\nw\fw\16w\u03cc\13w\3x\3x\3x\3"+ + "x\6x\u03d2\nx\rx\16x\u03d3\3y\3y\3y\6y\u03d9\ny\ry\16y\u03da\3z\3z\3z"+ + "\3z\7z\u03e1\nz\fz\16z\u03e4\13z\3z\3z\3{\3{\3{\3{\7{\u03ec\n{\f{\16{"+ + "\u03ef\13{\3{\3{\3|\3|\5|\u03f5\n|\3|\6|\u03f8\n|\r|\16|\u03f9\3}\3}\3"+ + "~\3~\3\177\3\177\3\177\3\177\7\177\u0404\n\177\f\177\16\177\u0407\13\177"+ + "\3\177\5\177\u040a\n\177\3\177\5\177\u040d\n\177\3\177\3\177\3\u0080\3"+ + "\u0080\3\u0080\3\u0080\3\u0080\7\u0080\u0416\n\u0080\f\u0080\16\u0080"+ + "\u0419\13\u0080\3\u0080\3\u0080\3\u0080\3\u0080\3\u0080\3\u0081\6\u0081"+ + "\u0421\n\u0081\r\u0081\16\u0081\u0422\3\u0081\3\u0081\3\u0082\3\u0082"+ + "\3\u0417\2\u0083\3\3\5\4\7\5\t\6\13\7\r\b\17\t\21\n\23\13\25\f\27\r\31"+ + "\16\33\17\35\20\37\21!\22#\23%\24\'\25)\26+\27-\30/\31\61\32\63\33\65"+ + "\34\67\359\36;\37= ?!A\"C#E$G%I&K\'M(O)Q*S+U,W-Y.[/]\60_\61a\62c\63e\64"+ + "g\65i\66k\67m8o9q:s;u{?}@\177A\u0081B\u0083C\u0085D\u0087E\u0089"+ + "F\u008bG\u008dH\u008fI\u0091J\u0093K\u0095L\u0097M\u0099N\u009bO\u009d"+ + "P\u009fQ\u00a1R\u00a3S\u00a5T\u00a7U\u00a9V\u00abW\u00adX\u00afY\u00b1"+ + "Z\u00b3[\u00b5\\\u00b7]\u00b9^\u00bb_\u00bd`\u00bfa\u00c1b\u00c3c\u00c5"+ + "d\u00c7e\u00c9f\u00cbg\u00cdh\u00cfi\u00d1j\u00d3k\u00d5l\u00d7m\u00d9"+ + "n\u00dbo\u00ddp\u00dfq\u00e1r\u00e3s\u00e5t\u00e7u\u00e9v\u00ebw\u00ed"+ + "x\u00efy\u00f1z\u00f3{\u00f5|\u00f7\2\u00f9\2\u00fb\2\u00fd}\u00ff~\u0101"+ + "\177\u0103\u0080\3\2\f\3\2))\4\2BBaa\5\2<\3\2\2"+ + "\2\u01c8\u01c9\7G\2\2\u01c9\u01ca\7Z\2\2\u01ca\u01cb\7V\2\2\u01cb\u01cc"+ + "\7T\2\2\u01cc\u01cd\7C\2\2\u01cd\u01ce\7E\2\2\u01ce\u01cf\7V\2\2\u01cf"+ + "@\3\2\2\2\u01d0\u01d1\7H\2\2\u01d1\u01d2\7C\2\2\u01d2\u01d3\7N\2\2\u01d3"+ + "\u01d4\7U\2\2\u01d4\u01d5\7G\2\2\u01d5B\3\2\2\2\u01d6\u01d7\7H\2\2\u01d7"+ + "\u01d8\7K\2\2\u01d8\u01d9\7T\2\2\u01d9\u01da\7U\2\2\u01da\u01db\7V\2\2"+ + "\u01dbD\3\2\2\2\u01dc\u01dd\7H\2\2\u01dd\u01de\7Q\2\2\u01de\u01df\7T\2"+ + "\2\u01df\u01e0\7O\2\2\u01e0\u01e1\7C\2\2\u01e1\u01e2\7V\2\2\u01e2F\3\2"+ + "\2\2\u01e3\u01e4\7H\2\2\u01e4\u01e5\7T\2\2\u01e5\u01e6\7Q\2\2\u01e6\u01e7"+ + "\7O\2\2\u01e7H\3\2\2\2\u01e8\u01e9\7H\2\2\u01e9\u01ea\7W\2\2\u01ea\u01eb"+ + "\7N\2\2\u01eb\u01ec\7N\2\2\u01ecJ\3\2\2\2\u01ed\u01ee\7H\2\2\u01ee\u01ef"+ + "\7W\2\2\u01ef\u01f0\7P\2\2\u01f0\u01f1\7E\2\2\u01f1\u01f2\7V\2\2\u01f2"+ + "\u01f3\7K\2\2\u01f3\u01f4\7Q\2\2\u01f4\u01f5\7P\2\2\u01f5\u01f6\7U\2\2"+ + "\u01f6L\3\2\2\2\u01f7\u01f8\7I\2\2\u01f8\u01f9\7T\2\2\u01f9\u01fa\7C\2"+ + "\2\u01fa\u01fb\7R\2\2\u01fb\u01fc\7J\2\2\u01fc\u01fd\7X\2\2\u01fd\u01fe"+ + "\7K\2\2\u01fe\u01ff\7\\\2\2\u01ffN\3\2\2\2\u0200\u0201\7I\2\2\u0201\u0202"+ + "\7T\2\2\u0202\u0203\7Q\2\2\u0203\u0204\7W\2\2\u0204\u0205\7R\2\2\u0205"+ + "P\3\2\2\2\u0206\u0207\7J\2\2\u0207\u0208\7C\2\2\u0208\u0209\7X\2\2\u0209"+ + "\u020a\7K\2\2\u020a\u020b\7P\2\2\u020b\u020c\7I\2\2\u020cR\3\2\2\2\u020d"+ + "\u020e\7J\2\2\u020e\u020f\7Q\2\2\u020f\u0210\7W\2\2\u0210\u0211\7T\2\2"+ + "\u0211T\3\2\2\2\u0212\u0213\7J\2\2\u0213\u0214\7Q\2\2\u0214\u0215\7W\2"+ + "\2\u0215\u0216\7T\2\2\u0216\u0217\7U\2\2\u0217V\3\2\2\2\u0218\u0219\7"+ + "K\2\2\u0219\u021a\7P\2\2\u021aX\3\2\2\2\u021b\u021c\7K\2\2\u021c\u021d"+ + "\7P\2\2\u021d\u021e\7P\2\2\u021e\u021f\7G\2\2\u021f\u0220\7T\2\2\u0220"+ + "Z\3\2\2\2\u0221\u0222\7K\2\2\u0222\u0223\7P\2\2\u0223\u0224\7V\2\2\u0224"+ + "\u0225\7G\2\2\u0225\u0226\7T\2\2\u0226\u0227\7X\2\2\u0227\u0228\7C\2\2"+ + "\u0228\u0229\7N\2\2\u0229\\\3\2\2\2\u022a\u022b\7K\2\2\u022b\u022c\7U"+ + "\2\2\u022c^\3\2\2\2\u022d\u022e\7L\2\2\u022e\u022f\7Q\2\2\u022f\u0230"+ + "\7K\2\2\u0230\u0231\7P\2\2\u0231`\3\2\2\2\u0232\u0233\7N\2\2\u0233\u0234"+ + "\7C\2\2\u0234\u0235\7U\2\2\u0235\u0236\7V\2\2\u0236b\3\2\2\2\u0237\u0238"+ + "\7N\2\2\u0238\u0239\7G\2\2\u0239\u023a\7H\2\2\u023a\u023b\7V\2\2\u023b"+ + "d\3\2\2\2\u023c\u023d\7N\2\2\u023d\u023e\7K\2\2\u023e\u023f\7M\2\2\u023f"+ + "\u0240\7G\2\2\u0240f\3\2\2\2\u0241\u0242\7N\2\2\u0242\u0243\7K\2\2\u0243"+ + "\u0244\7O\2\2\u0244\u0245\7K\2\2\u0245\u0246\7V\2\2\u0246h\3\2\2\2\u0247"+ + "\u0248\7O\2\2\u0248\u0249\7C\2\2\u0249\u024a\7R\2\2\u024a\u024b\7R\2\2"+ + "\u024b\u024c\7G\2\2\u024c\u024d\7F\2\2\u024dj\3\2\2\2\u024e\u024f\7O\2"+ + "\2\u024f\u0250\7C\2\2\u0250\u0251\7V\2\2\u0251\u0252\7E\2\2\u0252\u0253"+ + "\7J\2\2\u0253l\3\2\2\2\u0254\u0255\7O\2\2\u0255\u0256\7K\2\2\u0256\u0257"+ + "\7P\2\2\u0257\u0258\7W\2\2\u0258\u0259\7V\2\2\u0259\u025a\7G\2\2\u025a"+ + "n\3\2\2\2\u025b\u025c\7O\2\2\u025c\u025d\7K\2\2\u025d\u025e\7P\2\2\u025e"+ + "\u025f\7W\2\2\u025f\u0260\7V\2\2\u0260\u0261\7G\2\2\u0261\u0262\7U\2\2"+ + "\u0262p\3\2\2\2\u0263\u0264\7O\2\2\u0264\u0265\7Q\2\2\u0265\u0266\7P\2"+ + "\2\u0266\u0267\7V\2\2\u0267\u0268\7J\2\2\u0268r\3\2\2\2\u0269\u026a\7"+ + "O\2\2\u026a\u026b\7Q\2\2\u026b\u026c\7P\2\2\u026c\u026d\7V\2\2\u026d\u026e"+ + "\7J\2\2\u026e\u026f\7U\2\2\u026ft\3\2\2\2\u0270\u0271\7P\2\2\u0271\u0272"+ + "\7C\2\2\u0272\u0273\7V\2\2\u0273\u0274\7W\2\2\u0274\u0275\7T\2\2\u0275"+ + "\u0276\7C\2\2\u0276\u0277\7N\2\2\u0277v\3\2\2\2\u0278\u0279\7P\2\2\u0279"+ + "\u027a\7Q\2\2\u027a\u027b\7V\2\2\u027bx\3\2\2\2\u027c\u027d\7P\2\2\u027d"+ + "\u027e\7W\2\2\u027e\u027f\7N\2\2\u027f\u0280\7N\2\2\u0280z\3\2\2\2\u0281"+ + "\u0282\7P\2\2\u0282\u0283\7W\2\2\u0283\u0284\7N\2\2\u0284\u0285\7N\2\2"+ + "\u0285\u0286\7U\2\2\u0286|\3\2\2\2\u0287\u0288\7Q\2\2\u0288\u0289\7P\2"+ + "\2\u0289~\3\2\2\2\u028a\u028b\7Q\2\2\u028b\u028c\7R\2\2\u028c\u028d\7"+ + "V\2\2\u028d\u028e\7K\2\2\u028e\u028f\7O\2\2\u028f\u0290\7K\2\2\u0290\u0291"+ + "\7\\\2\2\u0291\u0292\7G\2\2\u0292\u0293\7F\2\2\u0293\u0080\3\2\2\2\u0294"+ + "\u0295\7Q\2\2\u0295\u0296\7T\2\2\u0296\u0082\3\2\2\2\u0297\u0298\7Q\2"+ + "\2\u0298\u0299\7T\2\2\u0299\u029a\7F\2\2\u029a\u029b\7G\2\2\u029b\u029c"+ + "\7T\2\2\u029c\u0084\3\2\2\2\u029d\u029e\7Q\2\2\u029e\u029f\7W\2\2\u029f"+ + "\u02a0\7V\2\2\u02a0\u02a1\7G\2\2\u02a1\u02a2\7T\2\2\u02a2\u0086\3\2\2"+ + "\2\u02a3\u02a4\7R\2\2\u02a4\u02a5\7C\2\2\u02a5\u02a6\7T\2\2\u02a6\u02a7"+ + "\7U\2\2\u02a7\u02a8\7G\2\2\u02a8\u02a9\7F\2\2\u02a9\u0088\3\2\2\2\u02aa"+ + "\u02ab\7R\2\2\u02ab\u02ac\7J\2\2\u02ac\u02ad\7[\2\2\u02ad\u02ae\7U\2\2"+ + "\u02ae\u02af\7K\2\2\u02af\u02b0\7E\2\2\u02b0\u02b1\7C\2\2\u02b1\u02b2"+ + "\7N\2\2\u02b2\u008a\3\2\2\2\u02b3\u02b4\7R\2\2\u02b4\u02b5\7N\2\2\u02b5"+ + "\u02b6\7C\2\2\u02b6\u02b7\7P\2\2\u02b7\u008c\3\2\2\2\u02b8\u02b9\7T\2"+ + "\2\u02b9\u02ba\7K\2\2\u02ba\u02bb\7I\2\2\u02bb\u02bc\7J\2\2\u02bc\u02bd"+ + "\7V\2\2\u02bd\u008e\3\2\2\2\u02be\u02bf\7T\2\2\u02bf\u02c0\7N\2\2\u02c0"+ + "\u02c1\7K\2\2\u02c1\u02c2\7M\2\2\u02c2\u02c3\7G\2\2\u02c3\u0090\3\2\2"+ + "\2\u02c4\u02c5\7S\2\2\u02c5\u02c6\7W\2\2\u02c6\u02c7\7G\2\2\u02c7\u02c8"+ + "\7T\2\2\u02c8\u02c9\7[\2\2\u02c9\u0092\3\2\2\2\u02ca\u02cb\7U\2\2\u02cb"+ + "\u02cc\7E\2\2\u02cc\u02cd\7J\2\2\u02cd\u02ce\7G\2\2\u02ce\u02cf\7O\2\2"+ + "\u02cf\u02d0\7C\2\2\u02d0\u02d1\7U\2\2\u02d1\u0094\3\2\2\2\u02d2\u02d3"+ + "\7U\2\2\u02d3\u02d4\7G\2\2\u02d4\u02d5\7E\2\2\u02d5\u02d6\7Q\2\2\u02d6"+ + "\u02d7\7P\2\2\u02d7\u02d8\7F\2\2\u02d8\u0096\3\2\2\2\u02d9\u02da\7U\2"+ + "\2\u02da\u02db\7G\2\2\u02db\u02dc\7E\2\2\u02dc\u02dd\7Q\2\2\u02dd\u02de"+ + "\7P\2\2\u02de\u02df\7F\2\2\u02df\u02e0\7U\2\2\u02e0\u0098\3\2\2\2\u02e1"+ + "\u02e2\7U\2\2\u02e2\u02e3\7G\2\2\u02e3\u02e4\7N\2\2\u02e4\u02e5\7G\2\2"+ + "\u02e5\u02e6\7E\2\2\u02e6\u02e7\7V\2\2\u02e7\u009a\3\2\2\2\u02e8\u02e9"+ + "\7U\2\2\u02e9\u02ea\7J\2\2\u02ea\u02eb\7Q\2\2\u02eb\u02ec\7Y\2\2\u02ec"+ + "\u009c\3\2\2\2\u02ed\u02ee\7U\2\2\u02ee\u02ef\7[\2\2\u02ef\u02f0\7U\2"+ + "\2\u02f0\u009e\3\2\2\2\u02f1\u02f2\7V\2\2\u02f2\u02f3\7C\2\2\u02f3\u02f4"+ + "\7D\2\2\u02f4\u02f5\7N\2\2\u02f5\u02f6\7G\2\2\u02f6\u00a0\3\2\2\2\u02f7"+ + "\u02f8\7V\2\2\u02f8\u02f9\7C\2\2\u02f9\u02fa\7D\2\2\u02fa\u02fb\7N\2\2"+ + "\u02fb\u02fc\7G\2\2\u02fc\u02fd\7U\2\2\u02fd\u00a2\3\2\2\2\u02fe\u02ff"+ + "\7V\2\2\u02ff\u0300\7G\2\2\u0300\u0301\7Z\2\2\u0301\u0302\7V\2\2\u0302"+ + "\u00a4\3\2\2\2\u0303\u0304\7V\2\2\u0304\u0305\7T\2\2\u0305\u0306\7W\2"+ + "\2\u0306\u0307\7G\2\2\u0307\u00a6\3\2\2\2\u0308\u0309\7V\2\2\u0309\u030a"+ + "\7Q\2\2\u030a\u00a8\3\2\2\2\u030b\u030c\7V\2\2\u030c\u030d\7[\2\2\u030d"+ + "\u030e\7R\2\2\u030e\u030f\7G\2\2\u030f\u00aa\3\2\2\2\u0310\u0311\7V\2"+ + "\2\u0311\u0312\7[\2\2\u0312\u0313\7R\2\2\u0313\u0314\7G\2\2\u0314\u0315"+ + "\7U\2\2\u0315\u00ac\3\2\2\2\u0316\u0317\7W\2\2\u0317\u0318\7U\2\2\u0318"+ + "\u0319\7K\2\2\u0319\u031a\7P\2\2\u031a\u031b\7I\2\2\u031b\u00ae\3\2\2"+ + "\2\u031c\u031d\7X\2\2\u031d\u031e\7G\2\2\u031e\u031f\7T\2\2\u031f\u0320"+ + "\7K\2\2\u0320\u0321\7H\2\2\u0321\u0322\7[\2\2\u0322\u00b0\3\2\2\2\u0323"+ + "\u0324\7Y\2\2\u0324\u0325\7J\2\2\u0325\u0326\7G\2\2\u0326\u0327\7T\2\2"+ + "\u0327\u0328\7G\2\2\u0328\u00b2\3\2\2\2\u0329\u032a\7Y\2\2\u032a\u032b"+ + "\7K\2\2\u032b\u032c\7V\2\2\u032c\u032d\7J\2\2\u032d\u00b4\3\2\2\2\u032e"+ + "\u032f\7[\2\2\u032f\u0330\7G\2\2\u0330\u0331\7C\2\2\u0331\u0332\7T\2\2"+ + "\u0332\u00b6\3\2\2\2\u0333\u0334\7[\2\2\u0334\u0335\7G\2\2\u0335\u0336"+ + "\7C\2\2\u0336\u0337\7T\2\2\u0337\u0338\7U\2\2\u0338\u00b8\3\2\2\2\u0339"+ + "\u033a\7}\2\2\u033a\u033b\7G\2\2\u033b\u033c\7U\2\2\u033c\u033d\7E\2\2"+ + "\u033d\u033e\7C\2\2\u033e\u033f\7R\2\2\u033f\u0340\7G\2\2\u0340\u00ba"+ + "\3\2\2\2\u0341\u0342\7}\2\2\u0342\u0343\7H\2\2\u0343\u0344\7P\2\2\u0344"+ + "\u00bc\3\2\2\2\u0345\u0346\7}\2\2\u0346\u0347\7N\2\2\u0347\u0348\7K\2"+ + "\2\u0348\u0349\7O\2\2\u0349\u034a\7K\2\2\u034a\u034b\7V\2\2\u034b\u00be"+ + "\3\2\2\2\u034c\u034d\7}\2\2\u034d\u034e\7F\2\2\u034e\u00c0\3\2\2\2\u034f"+ + "\u0350\7}\2\2\u0350\u0351\7V\2\2\u0351\u00c2\3\2\2\2\u0352\u0353\7}\2"+ + "\2\u0353\u0354\7V\2\2\u0354\u0355\7U\2\2\u0355\u00c4\3\2\2\2\u0356\u0357"+ + "\7}\2\2\u0357\u0358\7I\2\2\u0358\u0359\7W\2\2\u0359\u035a\7K\2\2\u035a"+ + "\u035b\7F\2\2\u035b\u00c6\3\2\2\2\u035c\u035d\7\177\2\2\u035d\u00c8\3"+ + "\2\2\2\u035e\u035f\7?\2\2\u035f\u00ca\3\2\2\2\u0360\u0361\7>\2\2\u0361"+ + "\u0362\7?\2\2\u0362\u0363\7@\2\2\u0363\u00cc\3\2\2\2\u0364\u0365\7>\2"+ + "\2\u0365\u0369\7@\2\2\u0366\u0367\7#\2\2\u0367\u0369\7?\2\2\u0368\u0364"+ + "\3\2\2\2\u0368\u0366\3\2\2\2\u0369\u00ce\3\2\2\2\u036a\u036b\7>\2\2\u036b"+ + "\u00d0\3\2\2\2\u036c\u036d\7>\2\2\u036d\u036e\7?\2\2\u036e\u00d2\3\2\2"+ + "\2\u036f\u0370\7@\2\2\u0370\u00d4\3\2\2\2\u0371\u0372\7@\2\2\u0372\u0373"+ + "\7?\2\2\u0373\u00d6\3\2\2\2\u0374\u0375\7-\2\2\u0375\u00d8\3\2\2\2\u0376"+ + "\u0377\7/\2\2\u0377\u00da\3\2\2\2\u0378\u0379\7,\2\2\u0379\u00dc\3\2\2"+ + "\2\u037a\u037b\7\61\2\2\u037b\u00de\3\2\2\2\u037c\u037d\7\'\2\2\u037d"+ + "\u00e0\3\2\2\2\u037e\u037f\7~\2\2\u037f\u0380\7~\2\2\u0380\u00e2\3\2\2"+ + "\2\u0381\u0382\7\60\2\2\u0382\u00e4\3\2\2\2\u0383\u0384\7A\2\2\u0384\u00e6"+ + "\3\2\2\2\u0385\u038b\7)\2\2\u0386\u038a\n\2\2\2\u0387\u0388\7)\2\2\u0388"+ + "\u038a\7)\2\2\u0389\u0386\3\2\2\2\u0389\u0387\3\2\2\2\u038a\u038d\3\2"+ + "\2\2\u038b\u0389\3\2\2\2\u038b\u038c\3\2\2\2\u038c\u038e\3\2\2\2\u038d"+ + "\u038b\3\2\2\2\u038e\u038f\7)\2\2\u038f\u00e8\3\2\2\2\u0390\u0392\5\u00f9"+ "}\2\u0391\u0390\3\2\2\2\u0392\u0393\3\2\2\2\u0393\u0391\3\2\2\2\u0393"+ - "\u0394\3\2\2\2\u0394\u0395\3\2\2\2\u0395\u0399\5\u00e3r\2\u0396\u0398"+ - "\5\u00f9}\2\u0397\u0396\3\2\2\2\u0398\u039b\3\2\2\2\u0399\u0397\3\2\2"+ - "\2\u0399\u039a\3\2\2\2\u039a\u03bb\3\2\2\2\u039b\u0399\3\2\2\2\u039c\u039e"+ - "\5\u00e3r\2\u039d\u039f\5\u00f9}\2\u039e\u039d\3\2\2\2\u039f\u03a0\3\2"+ - "\2\2\u03a0\u039e\3\2\2\2\u03a0\u03a1\3\2\2\2\u03a1\u03bb\3\2\2\2\u03a2"+ - "\u03a4\5\u00f9}\2\u03a3\u03a2\3\2\2\2\u03a4\u03a5\3\2\2\2\u03a5\u03a3"+ - "\3\2\2\2\u03a5\u03a6\3\2\2\2\u03a6\u03ae\3\2\2\2\u03a7\u03ab\5\u00e3r"+ - "\2\u03a8\u03aa\5\u00f9}\2\u03a9\u03a8\3\2\2\2\u03aa\u03ad\3\2\2\2\u03ab"+ - "\u03a9\3\2\2\2\u03ab\u03ac\3\2\2\2\u03ac\u03af\3\2\2\2\u03ad\u03ab\3\2"+ - "\2\2\u03ae\u03a7\3\2\2\2\u03ae\u03af\3\2\2\2\u03af\u03b0\3\2\2\2\u03b0"+ - "\u03b1\5\u00f7|\2\u03b1\u03bb\3\2\2\2\u03b2\u03b4\5\u00e3r\2\u03b3\u03b5"+ - "\5\u00f9}\2\u03b4\u03b3\3\2\2\2\u03b5\u03b6\3\2\2\2\u03b6\u03b4\3\2\2"+ - "\2\u03b6\u03b7\3\2\2\2\u03b7\u03b8\3\2\2\2\u03b8\u03b9\5\u00f7|\2\u03b9"+ - "\u03bb\3\2\2\2\u03ba\u0391\3\2\2\2\u03ba\u039c\3\2\2\2\u03ba\u03a3\3\2"+ - "\2\2\u03ba\u03b2\3\2\2\2\u03bb\u00ec\3\2\2\2\u03bc\u03bf\5\u00fb~\2\u03bd"+ - "\u03bf\7a\2\2\u03be\u03bc\3\2\2\2\u03be\u03bd\3\2\2\2\u03bf\u03c5\3\2"+ - "\2\2\u03c0\u03c4\5\u00fb~\2\u03c1\u03c4\5\u00f9}\2\u03c2\u03c4\t\3\2\2"+ - "\u03c3\u03c0\3\2\2\2\u03c3\u03c1\3\2\2\2\u03c3\u03c2\3\2\2\2\u03c4\u03c7"+ - "\3\2\2\2\u03c5\u03c3\3\2\2\2\u03c5\u03c6\3\2\2\2\u03c6\u00ee\3\2\2\2\u03c7"+ - "\u03c5\3\2\2\2\u03c8\u03cc\5\u00f9}\2\u03c9\u03cd\5\u00fb~\2\u03ca\u03cd"+ - "\5\u00f9}\2\u03cb\u03cd\t\4\2\2\u03cc\u03c9\3\2\2\2\u03cc\u03ca\3\2\2"+ - "\2\u03cc\u03cb\3\2\2\2\u03cd\u03ce\3\2\2\2\u03ce\u03cc\3\2\2\2\u03ce\u03cf"+ - "\3\2\2\2\u03cf\u00f0\3\2\2\2\u03d0\u03d4\5\u00fb~\2\u03d1\u03d4\5\u00f9"+ - "}\2\u03d2\u03d4\7a\2\2\u03d3\u03d0\3\2\2\2\u03d3\u03d1\3\2\2\2\u03d3\u03d2"+ - "\3\2\2\2\u03d4\u03d5\3\2\2\2\u03d5\u03d3\3\2\2\2\u03d5\u03d6\3\2\2\2\u03d6"+ - "\u00f2\3\2\2\2\u03d7\u03dd\7$\2\2\u03d8\u03dc\n\5\2\2\u03d9\u03da\7$\2"+ - "\2\u03da\u03dc\7$\2\2\u03db\u03d8\3\2\2\2\u03db\u03d9\3\2\2\2\u03dc\u03df"+ - "\3\2\2\2\u03dd\u03db\3\2\2\2\u03dd\u03de\3\2\2\2\u03de\u03e0\3\2\2\2\u03df"+ - "\u03dd\3\2\2\2\u03e0\u03e1\7$\2\2\u03e1\u00f4\3\2\2\2\u03e2\u03e8\7b\2"+ - "\2\u03e3\u03e7\n\6\2\2\u03e4\u03e5\7b\2\2\u03e5\u03e7\7b\2\2\u03e6\u03e3"+ - "\3\2\2\2\u03e6\u03e4\3\2\2\2\u03e7\u03ea\3\2\2\2\u03e8\u03e6\3\2\2\2\u03e8"+ - "\u03e9\3\2\2\2\u03e9\u03eb\3\2\2\2\u03ea\u03e8\3\2\2\2\u03eb\u03ec\7b"+ - "\2\2\u03ec\u00f6\3\2\2\2\u03ed\u03ef\7G\2\2\u03ee\u03f0\t\7\2\2\u03ef"+ - "\u03ee\3\2\2\2\u03ef\u03f0\3\2\2\2\u03f0\u03f2\3\2\2\2\u03f1\u03f3\5\u00f9"+ - "}\2\u03f2\u03f1\3\2\2\2\u03f3\u03f4\3\2\2\2\u03f4\u03f2\3\2\2\2\u03f4"+ - "\u03f5\3\2\2\2\u03f5\u00f8\3\2\2\2\u03f6\u03f7\t\b\2\2\u03f7\u00fa\3\2"+ - "\2\2\u03f8\u03f9\t\t\2\2\u03f9\u00fc\3\2\2\2\u03fa\u03fb\7/\2\2\u03fb"+ - "\u03fc\7/\2\2\u03fc\u0400\3\2\2\2\u03fd\u03ff\n\n\2\2\u03fe\u03fd\3\2"+ - "\2\2\u03ff\u0402\3\2\2\2\u0400\u03fe\3\2\2\2\u0400\u0401\3\2\2\2\u0401"+ - "\u0404\3\2\2\2\u0402\u0400\3\2\2\2\u0403\u0405\7\17\2\2\u0404\u0403\3"+ - "\2\2\2\u0404\u0405\3\2\2\2\u0405\u0407\3\2\2\2\u0406\u0408\7\f\2\2\u0407"+ - "\u0406\3\2\2\2\u0407\u0408\3\2\2\2\u0408\u0409\3\2\2\2\u0409\u040a\b\177"+ - "\2\2\u040a\u00fe\3\2\2\2\u040b\u040c\7\61\2\2\u040c\u040d\7,\2\2\u040d"+ - "\u0412\3\2\2\2\u040e\u0411\5\u00ff\u0080\2\u040f\u0411\13\2\2\2\u0410"+ - "\u040e\3\2\2\2\u0410\u040f\3\2\2\2\u0411\u0414\3\2\2\2\u0412\u0413\3\2"+ - "\2\2\u0412\u0410\3\2\2\2\u0413\u0415\3\2\2\2\u0414\u0412\3\2\2\2\u0415"+ - "\u0416\7,\2\2\u0416\u0417\7\61\2\2\u0417\u0418\3\2\2\2\u0418\u0419\b\u0080"+ - "\2\2\u0419\u0100\3\2\2\2\u041a\u041c\t\13\2\2\u041b\u041a\3\2\2\2\u041c"+ - "\u041d\3\2\2\2\u041d\u041b\3\2\2\2\u041d\u041e\3\2\2\2\u041e\u041f\3\2"+ - "\2\2\u041f\u0420\b\u0081\2\2\u0420\u0102\3\2\2\2\u0421\u0422\13\2\2\2"+ - "\u0422\u0104\3\2\2\2\"\2\u0363\u0384\u0386\u038e\u0393\u0399\u03a0\u03a5"+ - "\u03ab\u03ae\u03b6\u03ba\u03be\u03c3\u03c5\u03cc\u03ce\u03d3\u03d5\u03db"+ - "\u03dd\u03e6\u03e8\u03ef\u03f4\u0400\u0404\u0407\u0410\u0412\u041d\3\2"+ - "\3\2"; + "\u0394\3\2\2\2\u0394\u00ea\3\2\2\2\u0395\u0397\5\u00f9}\2\u0396\u0395"+ + "\3\2\2\2\u0397\u0398\3\2\2\2\u0398\u0396\3\2\2\2\u0398\u0399\3\2\2\2\u0399"+ + "\u039a\3\2\2\2\u039a\u039e\5\u00e3r\2\u039b\u039d\5\u00f9}\2\u039c\u039b"+ + "\3\2\2\2\u039d\u03a0\3\2\2\2\u039e\u039c\3\2\2\2\u039e\u039f\3\2\2\2\u039f"+ + "\u03c0\3\2\2\2\u03a0\u039e\3\2\2\2\u03a1\u03a3\5\u00e3r\2\u03a2\u03a4"+ + "\5\u00f9}\2\u03a3\u03a2\3\2\2\2\u03a4\u03a5\3\2\2\2\u03a5\u03a3\3\2\2"+ + "\2\u03a5\u03a6\3\2\2\2\u03a6\u03c0\3\2\2\2\u03a7\u03a9\5\u00f9}\2\u03a8"+ + "\u03a7\3\2\2\2\u03a9\u03aa\3\2\2\2\u03aa\u03a8\3\2\2\2\u03aa\u03ab\3\2"+ + "\2\2\u03ab\u03b3\3\2\2\2\u03ac\u03b0\5\u00e3r\2\u03ad\u03af\5\u00f9}\2"+ + "\u03ae\u03ad\3\2\2\2\u03af\u03b2\3\2\2\2\u03b0\u03ae\3\2\2\2\u03b0\u03b1"+ + "\3\2\2\2\u03b1\u03b4\3\2\2\2\u03b2\u03b0\3\2\2\2\u03b3\u03ac\3\2\2\2\u03b3"+ + "\u03b4\3\2\2\2\u03b4\u03b5\3\2\2\2\u03b5\u03b6\5\u00f7|\2\u03b6\u03c0"+ + "\3\2\2\2\u03b7\u03b9\5\u00e3r\2\u03b8\u03ba\5\u00f9}\2\u03b9\u03b8\3\2"+ + "\2\2\u03ba\u03bb\3\2\2\2\u03bb\u03b9\3\2\2\2\u03bb\u03bc\3\2\2\2\u03bc"+ + "\u03bd\3\2\2\2\u03bd\u03be\5\u00f7|\2\u03be\u03c0\3\2\2\2\u03bf\u0396"+ + "\3\2\2\2\u03bf\u03a1\3\2\2\2\u03bf\u03a8\3\2\2\2\u03bf\u03b7\3\2\2\2\u03c0"+ + "\u00ec\3\2\2\2\u03c1\u03c4\5\u00fb~\2\u03c2\u03c4\7a\2\2\u03c3\u03c1\3"+ + "\2\2\2\u03c3\u03c2\3\2\2\2\u03c4\u03ca\3\2\2\2\u03c5\u03c9\5\u00fb~\2"+ + "\u03c6\u03c9\5\u00f9}\2\u03c7\u03c9\t\3\2\2\u03c8\u03c5\3\2\2\2\u03c8"+ + "\u03c6\3\2\2\2\u03c8\u03c7\3\2\2\2\u03c9\u03cc\3\2\2\2\u03ca\u03c8\3\2"+ + "\2\2\u03ca\u03cb\3\2\2\2\u03cb\u00ee\3\2\2\2\u03cc\u03ca\3\2\2\2\u03cd"+ + "\u03d1\5\u00f9}\2\u03ce\u03d2\5\u00fb~\2\u03cf\u03d2\5\u00f9}\2\u03d0"+ + "\u03d2\t\4\2\2\u03d1\u03ce\3\2\2\2\u03d1\u03cf\3\2\2\2\u03d1\u03d0\3\2"+ + "\2\2\u03d2\u03d3\3\2\2\2\u03d3\u03d1\3\2\2\2\u03d3\u03d4\3\2\2\2\u03d4"+ + "\u00f0\3\2\2\2\u03d5\u03d9\5\u00fb~\2\u03d6\u03d9\5\u00f9}\2\u03d7\u03d9"+ + "\7a\2\2\u03d8\u03d5\3\2\2\2\u03d8\u03d6\3\2\2\2\u03d8\u03d7\3\2\2\2\u03d9"+ + "\u03da\3\2\2\2\u03da\u03d8\3\2\2\2\u03da\u03db\3\2\2\2\u03db\u00f2\3\2"+ + "\2\2\u03dc\u03e2\7$\2\2\u03dd\u03e1\n\5\2\2\u03de\u03df\7$\2\2\u03df\u03e1"+ + "\7$\2\2\u03e0\u03dd\3\2\2\2\u03e0\u03de\3\2\2\2\u03e1\u03e4\3\2\2\2\u03e2"+ + "\u03e0\3\2\2\2\u03e2\u03e3\3\2\2\2\u03e3\u03e5\3\2\2\2\u03e4\u03e2\3\2"+ + "\2\2\u03e5\u03e6\7$\2\2\u03e6\u00f4\3\2\2\2\u03e7\u03ed\7b\2\2\u03e8\u03ec"+ + "\n\6\2\2\u03e9\u03ea\7b\2\2\u03ea\u03ec\7b\2\2\u03eb\u03e8\3\2\2\2\u03eb"+ + "\u03e9\3\2\2\2\u03ec\u03ef\3\2\2\2\u03ed\u03eb\3\2\2\2\u03ed\u03ee\3\2"+ + "\2\2\u03ee\u03f0\3\2\2\2\u03ef\u03ed\3\2\2\2\u03f0\u03f1\7b\2\2\u03f1"+ + "\u00f6\3\2\2\2\u03f2\u03f4\7G\2\2\u03f3\u03f5\t\7\2\2\u03f4\u03f3\3\2"+ + "\2\2\u03f4\u03f5\3\2\2\2\u03f5\u03f7\3\2\2\2\u03f6\u03f8\5\u00f9}\2\u03f7"+ + "\u03f6\3\2\2\2\u03f8\u03f9\3\2\2\2\u03f9\u03f7\3\2\2\2\u03f9\u03fa\3\2"+ + "\2\2\u03fa\u00f8\3\2\2\2\u03fb\u03fc\t\b\2\2\u03fc\u00fa\3\2\2\2\u03fd"+ + "\u03fe\t\t\2\2\u03fe\u00fc\3\2\2\2\u03ff\u0400\7/\2\2\u0400\u0401\7/\2"+ + "\2\u0401\u0405\3\2\2\2\u0402\u0404\n\n\2\2\u0403\u0402\3\2\2\2\u0404\u0407"+ + "\3\2\2\2\u0405\u0403\3\2\2\2\u0405\u0406\3\2\2\2\u0406\u0409\3\2\2\2\u0407"+ + "\u0405\3\2\2\2\u0408\u040a\7\17\2\2\u0409\u0408\3\2\2\2\u0409\u040a\3"+ + "\2\2\2\u040a\u040c\3\2\2\2\u040b\u040d\7\f\2\2\u040c\u040b\3\2\2\2\u040c"+ + "\u040d\3\2\2\2\u040d\u040e\3\2\2\2\u040e\u040f\b\177\2\2\u040f\u00fe\3"+ + "\2\2\2\u0410\u0411\7\61\2\2\u0411\u0412\7,\2\2\u0412\u0417\3\2\2\2\u0413"+ + "\u0416\5\u00ff\u0080\2\u0414\u0416\13\2\2\2\u0415\u0413\3\2\2\2\u0415"+ + "\u0414\3\2\2\2\u0416\u0419\3\2\2\2\u0417\u0418\3\2\2\2\u0417\u0415\3\2"+ + "\2\2\u0418\u041a\3\2\2\2\u0419\u0417\3\2\2\2\u041a\u041b\7,\2\2\u041b"+ + "\u041c\7\61\2\2\u041c\u041d\3\2\2\2\u041d\u041e\b\u0080\2\2\u041e\u0100"+ + "\3\2\2\2\u041f\u0421\t\13\2\2\u0420\u041f\3\2\2\2\u0421\u0422\3\2\2\2"+ + "\u0422\u0420\3\2\2\2\u0422\u0423\3\2\2\2\u0423\u0424\3\2\2\2\u0424\u0425"+ + "\b\u0081\2\2\u0425\u0102\3\2\2\2\u0426\u0427\13\2\2\2\u0427\u0104\3\2"+ + "\2\2\"\2\u0368\u0389\u038b\u0393\u0398\u039e\u03a5\u03aa\u03b0\u03b3\u03bb"+ + "\u03bf\u03c3\u03c8\u03ca\u03d1\u03d3\u03d8\u03da\u03e0\u03e2\u03eb\u03ed"+ + "\u03f4\u03f9\u0405\u0409\u040c\u0415\u0417\u0422\3\2\3\2"; public static final ATN _ATN = new ATNDeserializer().deserialize(_serializedATN.toCharArray()); static { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java index 7549bfab8320a..a690169409e81 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java @@ -19,7 +19,7 @@ class SqlBaseParser extends Parser { public static final int T__0=1, T__1=2, T__2=3, T__3=4, ALL=5, ANALYZE=6, ANALYZED=7, AND=8, ANY=9, AS=10, ASC=11, BETWEEN=12, BY=13, CAST=14, CATALOG=15, CATALOGS=16, COLUMNS=17, - CONVERT=18, CURRENT=19, CURRENT_TIMESTAMP=20, DAY=21, DAYS=22, DEBUG=23, + CONVERT=18, CURRENT_DATE=19, CURRENT_TIMESTAMP=20, DAY=21, DAYS=22, DEBUG=23, DESC=24, DESCRIBE=25, DISTINCT=26, ESCAPE=27, EXECUTABLE=28, EXISTS=29, EXPLAIN=30, EXTRACT=31, FALSE=32, FIRST=33, FORMAT=34, FROM=35, FULL=36, FUNCTIONS=37, GRAPHVIZ=38, GROUP=39, HAVING=40, HOUR=41, HOURS=42, IN=43, @@ -71,7 +71,7 @@ class SqlBaseParser extends Parser { private static final String[] _LITERAL_NAMES = { null, "'('", "')'", "','", "':'", "'ALL'", "'ANALYZE'", "'ANALYZED'", "'AND'", "'ANY'", "'AS'", "'ASC'", "'BETWEEN'", "'BY'", "'CAST'", "'CATALOG'", - "'CATALOGS'", "'COLUMNS'", "'CONVERT'", "'CURRENT'", "'CURRENT_TIMESTAMP'", + "'CATALOGS'", "'COLUMNS'", "'CONVERT'", "'CURRENT_DATE'", "'CURRENT_TIMESTAMP'", "'DAY'", "'DAYS'", "'DEBUG'", "'DESC'", "'DESCRIBE'", "'DISTINCT'", "'ESCAPE'", "'EXECUTABLE'", "'EXISTS'", "'EXPLAIN'", "'EXTRACT'", "'FALSE'", "'FIRST'", "'FORMAT'", "'FROM'", "'FULL'", "'FUNCTIONS'", "'GRAPHVIZ'", "'GROUP'", @@ -90,16 +90,16 @@ class SqlBaseParser extends Parser { private static final String[] _SYMBOLIC_NAMES = { null, null, null, null, null, "ALL", "ANALYZE", "ANALYZED", "AND", "ANY", "AS", "ASC", "BETWEEN", "BY", "CAST", "CATALOG", "CATALOGS", "COLUMNS", - "CONVERT", "CURRENT", "CURRENT_TIMESTAMP", "DAY", "DAYS", "DEBUG", "DESC", - "DESCRIBE", "DISTINCT", "ESCAPE", "EXECUTABLE", "EXISTS", "EXPLAIN", "EXTRACT", - "FALSE", "FIRST", "FORMAT", "FROM", "FULL", "FUNCTIONS", "GRAPHVIZ", "GROUP", - "HAVING", "HOUR", "HOURS", "IN", "INNER", "INTERVAL", "IS", "JOIN", "LAST", - "LEFT", "LIKE", "LIMIT", "MAPPED", "MATCH", "MINUTE", "MINUTES", "MONTH", - "MONTHS", "NATURAL", "NOT", "NULL", "NULLS", "ON", "OPTIMIZED", "OR", - "ORDER", "OUTER", "PARSED", "PHYSICAL", "PLAN", "RIGHT", "RLIKE", "QUERY", - "SCHEMAS", "SECOND", "SECONDS", "SELECT", "SHOW", "SYS", "TABLE", "TABLES", - "TEXT", "TRUE", "TO", "TYPE", "TYPES", "USING", "VERIFY", "WHERE", "WITH", - "YEAR", "YEARS", "ESCAPE_ESC", "FUNCTION_ESC", "LIMIT_ESC", "DATE_ESC", + "CONVERT", "CURRENT_DATE", "CURRENT_TIMESTAMP", "DAY", "DAYS", "DEBUG", + "DESC", "DESCRIBE", "DISTINCT", "ESCAPE", "EXECUTABLE", "EXISTS", "EXPLAIN", + "EXTRACT", "FALSE", "FIRST", "FORMAT", "FROM", "FULL", "FUNCTIONS", "GRAPHVIZ", + "GROUP", "HAVING", "HOUR", "HOURS", "IN", "INNER", "INTERVAL", "IS", "JOIN", + "LAST", "LEFT", "LIKE", "LIMIT", "MAPPED", "MATCH", "MINUTE", "MINUTES", + "MONTH", "MONTHS", "NATURAL", "NOT", "NULL", "NULLS", "ON", "OPTIMIZED", + "OR", "ORDER", "OUTER", "PARSED", "PHYSICAL", "PLAN", "RIGHT", "RLIKE", + "QUERY", "SCHEMAS", "SECOND", "SECONDS", "SELECT", "SHOW", "SYS", "TABLE", + "TABLES", "TEXT", "TRUE", "TO", "TYPE", "TYPES", "USING", "VERIFY", "WHERE", + "WITH", "YEAR", "YEARS", "ESCAPE_ESC", "FUNCTION_ESC", "LIMIT_ESC", "DATE_ESC", "TIME_ESC", "TIMESTAMP_ESC", "GUID_ESC", "ESC_END", "EQ", "NULLEQ", "NEQ", "LT", "LTE", "GT", "GTE", "PLUS", "MINUS", "ASTERISK", "SLASH", "PERCENT", "CONCAT", "DOT", "PARAM", "STRING", "INTEGER_VALUE", "DECIMAL_VALUE", @@ -767,7 +767,6 @@ public final StatementContext statement() throws RecognitionException { case ANALYZED: case CATALOGS: case COLUMNS: - case CURRENT: case DAY: case DEBUG: case EXECUTABLE: @@ -844,7 +843,6 @@ public final StatementContext statement() throws RecognitionException { case ANALYZED: case CATALOGS: case COLUMNS: - case CURRENT: case DAY: case DEBUG: case EXECUTABLE: @@ -915,7 +913,6 @@ public final StatementContext statement() throws RecognitionException { case ANALYZED: case CATALOGS: case COLUMNS: - case CURRENT: case DAY: case DEBUG: case EXECUTABLE: @@ -1089,7 +1086,6 @@ public final StatementContext statement() throws RecognitionException { case ANALYZED: case CATALOGS: case COLUMNS: - case CURRENT: case DAY: case DEBUG: case EXECUTABLE: @@ -2001,7 +1997,7 @@ public final GroupingExpressionsContext groupingExpressions() throws Recognition match(T__0); setState(327); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << T__0) | (1L << ANALYZE) | (1L << ANALYZED) | (1L << CAST) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CONVERT) | (1L << CURRENT) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXISTS) | (1L << EXPLAIN) | (1L << EXTRACT) | (1L << FALSE) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LEFT) | (1L << LIMIT) | (1L << MAPPED) | (1L << MATCH) | (1L << MINUTE) | (1L << MONTH) | (1L << NOT) | (1L << NULL) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RIGHT - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TRUE - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (FUNCTION_ESC - 67)) | (1L << (DATE_ESC - 67)) | (1L << (TIME_ESC - 67)) | (1L << (TIMESTAMP_ESC - 67)) | (1L << (GUID_ESC - 67)) | (1L << (PLUS - 67)) | (1L << (MINUS - 67)) | (1L << (ASTERISK - 67)) | (1L << (PARAM - 67)) | (1L << (STRING - 67)) | (1L << (INTEGER_VALUE - 67)) | (1L << (DECIMAL_VALUE - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { + if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << T__0) | (1L << ANALYZE) | (1L << ANALYZED) | (1L << CAST) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CONVERT) | (1L << CURRENT_DATE) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXISTS) | (1L << EXPLAIN) | (1L << EXTRACT) | (1L << FALSE) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LEFT) | (1L << LIMIT) | (1L << MAPPED) | (1L << MATCH) | (1L << MINUTE) | (1L << MONTH) | (1L << NOT) | (1L << NULL) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RIGHT - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TRUE - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (FUNCTION_ESC - 67)) | (1L << (DATE_ESC - 67)) | (1L << (TIME_ESC - 67)) | (1L << (TIMESTAMP_ESC - 67)) | (1L << (GUID_ESC - 67)) | (1L << (PLUS - 67)) | (1L << (MINUS - 67)) | (1L << (ASTERISK - 67)) | (1L << (PARAM - 67)) | (1L << (STRING - 67)) | (1L << (INTEGER_VALUE - 67)) | (1L << (DECIMAL_VALUE - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { { setState(319); expression(); @@ -3805,7 +3801,7 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti case CATALOGS: case COLUMNS: case CONVERT: - case CURRENT: + case CURRENT_DATE: case CURRENT_TIMESTAMP: case DAY: case DEBUG: @@ -4203,7 +4199,7 @@ public final PrimaryExpressionContext primaryExpression() throws RecognitionExce { setState(576); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { + if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { { setState(573); qualifiedName(); @@ -4415,6 +4411,7 @@ public final CastTemplateContext castTemplate() throws RecognitionException { public static class BuiltinDateTimeFunctionContext extends ParserRuleContext { public Token name; public Token precision; + public TerminalNode CURRENT_DATE() { return getToken(SqlBaseParser.CURRENT_DATE, 0); } public TerminalNode CURRENT_TIMESTAMP() { return getToken(SqlBaseParser.CURRENT_TIMESTAMP, 0); } public TerminalNode INTEGER_VALUE() { return getToken(SqlBaseParser.INTEGER_VALUE, 0); } public BuiltinDateTimeFunctionContext(ParserRuleContext parent, int invokingState) { @@ -4441,31 +4438,57 @@ public final BuiltinDateTimeFunctionContext builtinDateTimeFunction() throws Rec enterRule(_localctx, 66, RULE_builtinDateTimeFunction); int _la; try { - enterOuterAlt(_localctx, 1); - { - setState(610); - ((BuiltinDateTimeFunctionContext)_localctx).name = match(CURRENT_TIMESTAMP); - setState(616); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,82,_ctx) ) { - case 1: + setState(623); + switch (_input.LA(1)) { + case CURRENT_DATE: + enterOuterAlt(_localctx, 1); { - setState(611); - match(T__0); + setState(610); + ((BuiltinDateTimeFunctionContext)_localctx).name = match(CURRENT_DATE); setState(613); - _la = _input.LA(1); - if (_la==INTEGER_VALUE) { + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,81,_ctx) ) { + case 1: { + setState(611); + match(T__0); setState(612); - ((BuiltinDateTimeFunctionContext)_localctx).precision = match(INTEGER_VALUE); + match(T__1); } + break; } - + } + break; + case CURRENT_TIMESTAMP: + enterOuterAlt(_localctx, 2); + { setState(615); - match(T__1); + ((BuiltinDateTimeFunctionContext)_localctx).name = match(CURRENT_TIMESTAMP); + setState(621); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,83,_ctx) ) { + case 1: + { + setState(616); + match(T__0); + setState(618); + _la = _input.LA(1); + if (_la==INTEGER_VALUE) { + { + setState(617); + ((BuiltinDateTimeFunctionContext)_localctx).precision = match(INTEGER_VALUE); + } + } + + setState(620); + match(T__1); + } + break; + } } break; - } + default: + throw new NoViableAltException(this); } } catch (RecognitionException re) { @@ -4512,17 +4535,17 @@ public final ConvertTemplateContext convertTemplate() throws RecognitionExceptio try { enterOuterAlt(_localctx, 1); { - setState(618); + setState(625); match(CONVERT); - setState(619); + setState(626); match(T__0); - setState(620); + setState(627); expression(); - setState(621); + setState(628); match(T__2); - setState(622); + setState(629); dataType(); - setState(623); + setState(630); match(T__1); } } @@ -4566,23 +4589,23 @@ public final ExtractExpressionContext extractExpression() throws RecognitionExce ExtractExpressionContext _localctx = new ExtractExpressionContext(_ctx, getState()); enterRule(_localctx, 70, RULE_extractExpression); try { - setState(630); + setState(637); switch (_input.LA(1)) { case EXTRACT: enterOuterAlt(_localctx, 1); { - setState(625); + setState(632); extractTemplate(); } break; case FUNCTION_ESC: enterOuterAlt(_localctx, 2); { - setState(626); + setState(633); match(FUNCTION_ESC); - setState(627); + setState(634); extractTemplate(); - setState(628); + setState(635); match(ESC_END); } break; @@ -4636,17 +4659,17 @@ public final ExtractTemplateContext extractTemplate() throws RecognitionExceptio try { enterOuterAlt(_localctx, 1); { - setState(632); + setState(639); match(EXTRACT); - setState(633); + setState(640); match(T__0); - setState(634); + setState(641); ((ExtractTemplateContext)_localctx).field = identifier(); - setState(635); + setState(642); match(FROM); - setState(636); + setState(643); valueExpression(0); - setState(637); + setState(644); match(T__1); } } @@ -4689,13 +4712,12 @@ public final FunctionExpressionContext functionExpression() throws RecognitionEx FunctionExpressionContext _localctx = new FunctionExpressionContext(_ctx, getState()); enterRule(_localctx, 74, RULE_functionExpression); try { - setState(644); + setState(651); switch (_input.LA(1)) { case ANALYZE: case ANALYZED: case CATALOGS: case COLUMNS: - case CURRENT: case DAY: case DEBUG: case EXECUTABLE: @@ -4736,18 +4758,18 @@ public final FunctionExpressionContext functionExpression() throws RecognitionEx case BACKQUOTED_IDENTIFIER: enterOuterAlt(_localctx, 1); { - setState(639); + setState(646); functionTemplate(); } break; case FUNCTION_ESC: enterOuterAlt(_localctx, 2); { - setState(640); + setState(647); match(FUNCTION_ESC); - setState(641); + setState(648); functionTemplate(); - setState(642); + setState(649); match(ESC_END); } break; @@ -4805,45 +4827,45 @@ public final FunctionTemplateContext functionTemplate() throws RecognitionExcept try { enterOuterAlt(_localctx, 1); { - setState(646); + setState(653); functionName(); - setState(647); + setState(654); match(T__0); - setState(659); + setState(666); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << T__0) | (1L << ALL) | (1L << ANALYZE) | (1L << ANALYZED) | (1L << CAST) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CONVERT) | (1L << CURRENT) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << DISTINCT) | (1L << EXECUTABLE) | (1L << EXISTS) | (1L << EXPLAIN) | (1L << EXTRACT) | (1L << FALSE) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LEFT) | (1L << LIMIT) | (1L << MAPPED) | (1L << MATCH) | (1L << MINUTE) | (1L << MONTH) | (1L << NOT) | (1L << NULL) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RIGHT - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TRUE - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (FUNCTION_ESC - 67)) | (1L << (DATE_ESC - 67)) | (1L << (TIME_ESC - 67)) | (1L << (TIMESTAMP_ESC - 67)) | (1L << (GUID_ESC - 67)) | (1L << (PLUS - 67)) | (1L << (MINUS - 67)) | (1L << (ASTERISK - 67)) | (1L << (PARAM - 67)) | (1L << (STRING - 67)) | (1L << (INTEGER_VALUE - 67)) | (1L << (DECIMAL_VALUE - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { + if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << T__0) | (1L << ALL) | (1L << ANALYZE) | (1L << ANALYZED) | (1L << CAST) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CONVERT) | (1L << CURRENT_DATE) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << DISTINCT) | (1L << EXECUTABLE) | (1L << EXISTS) | (1L << EXPLAIN) | (1L << EXTRACT) | (1L << FALSE) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LEFT) | (1L << LIMIT) | (1L << MAPPED) | (1L << MATCH) | (1L << MINUTE) | (1L << MONTH) | (1L << NOT) | (1L << NULL) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RIGHT - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TRUE - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (FUNCTION_ESC - 67)) | (1L << (DATE_ESC - 67)) | (1L << (TIME_ESC - 67)) | (1L << (TIMESTAMP_ESC - 67)) | (1L << (GUID_ESC - 67)) | (1L << (PLUS - 67)) | (1L << (MINUS - 67)) | (1L << (ASTERISK - 67)) | (1L << (PARAM - 67)) | (1L << (STRING - 67)) | (1L << (INTEGER_VALUE - 67)) | (1L << (DECIMAL_VALUE - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { { - setState(649); + setState(656); _la = _input.LA(1); if (_la==ALL || _la==DISTINCT) { { - setState(648); + setState(655); setQuantifier(); } } - setState(651); + setState(658); expression(); - setState(656); + setState(663); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(652); + setState(659); match(T__2); - setState(653); + setState(660); expression(); } } - setState(658); + setState(665); _errHandler.sync(this); _la = _input.LA(1); } } } - setState(661); + setState(668); match(T__1); } } @@ -4887,19 +4909,19 @@ public final FunctionNameContext functionName() throws RecognitionException { FunctionNameContext _localctx = new FunctionNameContext(_ctx, getState()); enterRule(_localctx, 78, RULE_functionName); try { - setState(666); + setState(673); switch (_input.LA(1)) { case LEFT: enterOuterAlt(_localctx, 1); { - setState(663); + setState(670); match(LEFT); } break; case RIGHT: enterOuterAlt(_localctx, 2); { - setState(664); + setState(671); match(RIGHT); } break; @@ -4907,7 +4929,6 @@ public final FunctionNameContext functionName() throws RecognitionException { case ANALYZED: case CATALOGS: case COLUMNS: - case CURRENT: case DAY: case DEBUG: case EXECUTABLE: @@ -4946,7 +4967,7 @@ public final FunctionNameContext functionName() throws RecognitionException { case BACKQUOTED_IDENTIFIER: enterOuterAlt(_localctx, 3); { - setState(665); + setState(672); identifier(); } break; @@ -5177,13 +5198,13 @@ public final ConstantContext constant() throws RecognitionException { enterRule(_localctx, 80, RULE_constant); try { int _alt; - setState(694); + setState(701); switch (_input.LA(1)) { case NULL: _localctx = new NullLiteralContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(668); + setState(675); match(NULL); } break; @@ -5191,7 +5212,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new IntervalLiteralContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(669); + setState(676); interval(); } break; @@ -5200,7 +5221,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new NumericLiteralContext(_localctx); enterOuterAlt(_localctx, 3); { - setState(670); + setState(677); number(); } break; @@ -5209,7 +5230,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new BooleanLiteralContext(_localctx); enterOuterAlt(_localctx, 4); { - setState(671); + setState(678); booleanValue(); } break; @@ -5217,7 +5238,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new StringLiteralContext(_localctx); enterOuterAlt(_localctx, 5); { - setState(673); + setState(680); _errHandler.sync(this); _alt = 1; do { @@ -5225,7 +5246,7 @@ public final ConstantContext constant() throws RecognitionException { case 1: { { - setState(672); + setState(679); match(STRING); } } @@ -5233,9 +5254,9 @@ public final ConstantContext constant() throws RecognitionException { default: throw new NoViableAltException(this); } - setState(675); + setState(682); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,89,_ctx); + _alt = getInterpreter().adaptivePredict(_input,91,_ctx); } while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ); } break; @@ -5243,7 +5264,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new ParamLiteralContext(_localctx); enterOuterAlt(_localctx, 6); { - setState(677); + setState(684); match(PARAM); } break; @@ -5251,11 +5272,11 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new DateEscapedLiteralContext(_localctx); enterOuterAlt(_localctx, 7); { - setState(678); + setState(685); match(DATE_ESC); - setState(679); + setState(686); string(); - setState(680); + setState(687); match(ESC_END); } break; @@ -5263,11 +5284,11 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new TimeEscapedLiteralContext(_localctx); enterOuterAlt(_localctx, 8); { - setState(682); + setState(689); match(TIME_ESC); - setState(683); + setState(690); string(); - setState(684); + setState(691); match(ESC_END); } break; @@ -5275,11 +5296,11 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new TimestampEscapedLiteralContext(_localctx); enterOuterAlt(_localctx, 9); { - setState(686); + setState(693); match(TIMESTAMP_ESC); - setState(687); + setState(694); string(); - setState(688); + setState(695); match(ESC_END); } break; @@ -5287,11 +5308,11 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new GuidEscapedLiteralContext(_localctx); enterOuterAlt(_localctx, 10); { - setState(690); + setState(697); match(GUID_ESC); - setState(691); + setState(698); string(); - setState(692); + setState(699); match(ESC_END); } break; @@ -5344,7 +5365,7 @@ public final ComparisonOperatorContext comparisonOperator() throws RecognitionEx try { enterOuterAlt(_localctx, 1); { - setState(696); + setState(703); _la = _input.LA(1); if ( !(((((_la - 100)) & ~0x3f) == 0 && ((1L << (_la - 100)) & ((1L << (EQ - 100)) | (1L << (NULLEQ - 100)) | (1L << (NEQ - 100)) | (1L << (LT - 100)) | (1L << (LTE - 100)) | (1L << (GT - 100)) | (1L << (GTE - 100)))) != 0)) ) { _errHandler.recoverInline(this); @@ -5393,7 +5414,7 @@ public final BooleanValueContext booleanValue() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(698); + setState(705); _la = _input.LA(1); if ( !(_la==FALSE || _la==TRUE) ) { _errHandler.recoverInline(this); @@ -5461,13 +5482,13 @@ public final IntervalContext interval() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(700); + setState(707); match(INTERVAL); - setState(702); + setState(709); _la = _input.LA(1); if (_la==PLUS || _la==MINUS) { { - setState(701); + setState(708); ((IntervalContext)_localctx).sign = _input.LT(1); _la = _input.LA(1); if ( !(_la==PLUS || _la==MINUS) ) { @@ -5478,35 +5499,35 @@ public final IntervalContext interval() throws RecognitionException { } } - setState(706); + setState(713); switch (_input.LA(1)) { case INTEGER_VALUE: case DECIMAL_VALUE: { - setState(704); + setState(711); ((IntervalContext)_localctx).valueNumeric = number(); } break; case PARAM: case STRING: { - setState(705); + setState(712); ((IntervalContext)_localctx).valuePattern = string(); } break; default: throw new NoViableAltException(this); } - setState(708); + setState(715); ((IntervalContext)_localctx).leading = intervalField(); - setState(711); + setState(718); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,93,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,95,_ctx) ) { case 1: { - setState(709); + setState(716); match(TO); - setState(710); + setState(717); ((IntervalContext)_localctx).trailing = intervalField(); } break; @@ -5563,7 +5584,7 @@ public final IntervalFieldContext intervalField() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(713); + setState(720); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << DAY) | (1L << DAYS) | (1L << HOUR) | (1L << HOURS) | (1L << MINUTE) | (1L << MINUTES) | (1L << MONTH) | (1L << MONTHS))) != 0) || ((((_la - 74)) & ~0x3f) == 0 && ((1L << (_la - 74)) & ((1L << (SECOND - 74)) | (1L << (SECONDS - 74)) | (1L << (YEAR - 74)) | (1L << (YEARS - 74)))) != 0)) ) { _errHandler.recoverInline(this); @@ -5621,7 +5642,7 @@ public final DataTypeContext dataType() throws RecognitionException { _localctx = new PrimitiveDataTypeContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(715); + setState(722); identifier(); } } @@ -5673,25 +5694,25 @@ public final QualifiedNameContext qualifiedName() throws RecognitionException { int _alt; enterOuterAlt(_localctx, 1); { - setState(722); + setState(729); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,94,_ctx); + _alt = getInterpreter().adaptivePredict(_input,96,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { if ( _alt==1 ) { { { - setState(717); + setState(724); identifier(); - setState(718); + setState(725); match(DOT); } } } - setState(724); + setState(731); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,94,_ctx); + _alt = getInterpreter().adaptivePredict(_input,96,_ctx); } - setState(725); + setState(732); identifier(); } } @@ -5736,13 +5757,13 @@ public final IdentifierContext identifier() throws RecognitionException { IdentifierContext _localctx = new IdentifierContext(_ctx, getState()); enterRule(_localctx, 94, RULE_identifier); try { - setState(729); + setState(736); switch (_input.LA(1)) { case QUOTED_IDENTIFIER: case BACKQUOTED_IDENTIFIER: enterOuterAlt(_localctx, 1); { - setState(727); + setState(734); quoteIdentifier(); } break; @@ -5750,7 +5771,6 @@ public final IdentifierContext identifier() throws RecognitionException { case ANALYZED: case CATALOGS: case COLUMNS: - case CURRENT: case DAY: case DEBUG: case EXECUTABLE: @@ -5787,7 +5807,7 @@ public final IdentifierContext identifier() throws RecognitionException { case DIGIT_IDENTIFIER: enterOuterAlt(_localctx, 2); { - setState(728); + setState(735); unquoteIdentifier(); } break; @@ -5840,43 +5860,43 @@ public final TableIdentifierContext tableIdentifier() throws RecognitionExceptio enterRule(_localctx, 96, RULE_tableIdentifier); int _la; try { - setState(743); + setState(750); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,98,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,100,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(734); + setState(741); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { + if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { { - setState(731); + setState(738); ((TableIdentifierContext)_localctx).catalog = identifier(); - setState(732); + setState(739); match(T__3); } } - setState(736); + setState(743); match(TABLE_IDENTIFIER); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(740); + setState(747); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,97,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,99,_ctx) ) { case 1: { - setState(737); + setState(744); ((TableIdentifierContext)_localctx).catalog = identifier(); - setState(738); + setState(745); match(T__3); } break; } - setState(742); + setState(749); ((TableIdentifierContext)_localctx).name = identifier(); } break; @@ -5943,13 +5963,13 @@ public final QuoteIdentifierContext quoteIdentifier() throws RecognitionExceptio QuoteIdentifierContext _localctx = new QuoteIdentifierContext(_ctx, getState()); enterRule(_localctx, 98, RULE_quoteIdentifier); try { - setState(747); + setState(754); switch (_input.LA(1)) { case QUOTED_IDENTIFIER: _localctx = new QuotedIdentifierContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(745); + setState(752); match(QUOTED_IDENTIFIER); } break; @@ -5957,7 +5977,7 @@ public final QuoteIdentifierContext quoteIdentifier() throws RecognitionExceptio _localctx = new BackQuotedIdentifierContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(746); + setState(753); match(BACKQUOTED_IDENTIFIER); } break; @@ -6029,13 +6049,13 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce UnquoteIdentifierContext _localctx = new UnquoteIdentifierContext(_ctx, getState()); enterRule(_localctx, 100, RULE_unquoteIdentifier); try { - setState(752); + setState(759); switch (_input.LA(1)) { case IDENTIFIER: _localctx = new UnquotedIdentifierContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(749); + setState(756); match(IDENTIFIER); } break; @@ -6043,7 +6063,6 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce case ANALYZED: case CATALOGS: case COLUMNS: - case CURRENT: case DAY: case DEBUG: case EXECUTABLE: @@ -6079,7 +6098,7 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce _localctx = new UnquotedIdentifierContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(750); + setState(757); nonReserved(); } break; @@ -6087,7 +6106,7 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce _localctx = new DigitIdentifierContext(_localctx); enterOuterAlt(_localctx, 3); { - setState(751); + setState(758); match(DIGIT_IDENTIFIER); } break; @@ -6156,13 +6175,13 @@ public final NumberContext number() throws RecognitionException { NumberContext _localctx = new NumberContext(_ctx, getState()); enterRule(_localctx, 102, RULE_number); try { - setState(756); + setState(763); switch (_input.LA(1)) { case DECIMAL_VALUE: _localctx = new DecimalLiteralContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(754); + setState(761); match(DECIMAL_VALUE); } break; @@ -6170,7 +6189,7 @@ public final NumberContext number() throws RecognitionException { _localctx = new IntegerLiteralContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(755); + setState(762); match(INTEGER_VALUE); } break; @@ -6218,7 +6237,7 @@ public final StringContext string() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(758); + setState(765); _la = _input.LA(1); if ( !(_la==PARAM || _la==STRING) ) { _errHandler.recoverInline(this); @@ -6243,7 +6262,6 @@ public static class NonReservedContext extends ParserRuleContext { public TerminalNode ANALYZED() { return getToken(SqlBaseParser.ANALYZED, 0); } public TerminalNode CATALOGS() { return getToken(SqlBaseParser.CATALOGS, 0); } public TerminalNode COLUMNS() { return getToken(SqlBaseParser.COLUMNS, 0); } - public TerminalNode CURRENT() { return getToken(SqlBaseParser.CURRENT, 0); } public TerminalNode DAY() { return getToken(SqlBaseParser.DAY, 0); } public TerminalNode DEBUG() { return getToken(SqlBaseParser.DEBUG, 0); } public TerminalNode EXECUTABLE() { return getToken(SqlBaseParser.EXECUTABLE, 0); } @@ -6302,9 +6320,9 @@ public final NonReservedContext nonReserved() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(760); + setState(767); _la = _input.LA(1); - if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)))) != 0)) ) { + if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); @@ -6353,7 +6371,7 @@ private boolean valueExpression_sempred(ValueExpressionContext _localctx, int pr } public static final String _serializedATN = - "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\3\u0081\u02fd\4\2\t"+ + "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\3\u0081\u0304\4\2\t"+ "\2\4\3\t\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n\4\13"+ "\t\13\4\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\4\21\t\21\4\22\t\22"+ "\4\23\t\23\4\24\t\24\4\25\t\25\4\26\t\26\4\27\t\27\4\30\t\30\4\31\t\31"+ @@ -6400,114 +6418,114 @@ private boolean valueExpression_sempred(ValueExpressionContext _localctx, int pr "\n\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\7\37\u0237\n\37"+ "\f\37\16\37\u023a\13\37\3 \3 \3 \3 \3 \3 \3 \5 \u0243\n \3 \3 \3 \3 \3"+ " \3 \3 \3 \3 \3 \3 \5 \u0250\n \3!\3!\3!\3!\3!\3!\3!\3!\3!\3!\5!\u025c"+ - "\n!\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3#\3#\3#\5#\u0268\n#\3#\5#\u026b\n#\3"+ - "$\3$\3$\3$\3$\3$\3$\3%\3%\3%\3%\3%\5%\u0279\n%\3&\3&\3&\3&\3&\3&\3&\3"+ - "\'\3\'\3\'\3\'\3\'\5\'\u0287\n\'\3(\3(\3(\5(\u028c\n(\3(\3(\3(\7(\u0291"+ - "\n(\f(\16(\u0294\13(\5(\u0296\n(\3(\3(\3)\3)\3)\5)\u029d\n)\3*\3*\3*\3"+ - "*\3*\6*\u02a4\n*\r*\16*\u02a5\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3"+ - "*\3*\3*\3*\5*\u02b9\n*\3+\3+\3,\3,\3-\3-\5-\u02c1\n-\3-\3-\5-\u02c5\n"+ - "-\3-\3-\3-\5-\u02ca\n-\3.\3.\3/\3/\3\60\3\60\3\60\7\60\u02d3\n\60\f\60"+ - "\16\60\u02d6\13\60\3\60\3\60\3\61\3\61\5\61\u02dc\n\61\3\62\3\62\3\62"+ - "\5\62\u02e1\n\62\3\62\3\62\3\62\3\62\5\62\u02e7\n\62\3\62\5\62\u02ea\n"+ - "\62\3\63\3\63\5\63\u02ee\n\63\3\64\3\64\3\64\5\64\u02f3\n\64\3\65\3\65"+ - "\5\65\u02f7\n\65\3\66\3\66\3\67\3\67\3\67\2\4.<8\2\4\6\b\n\f\16\20\22"+ - "\24\26\30\32\34\36 \"$&(*,.\60\62\64\668:<>@BDFHJLNPRTVXZ\\^`bdfhjl\2"+ - "\22\b\2\7\7\t\t\36\36\66\66AAEE\4\2((SS\4\2\t\tAA\4\2%%--\3\2\32\33\3"+ - "\2mn\4\2\7\7vv\4\2\r\r\32\32\4\2##\62\62\4\2\7\7\34\34\3\2oq\3\2fl\4\2"+ - "\"\"TT\7\2\27\30+,8;LM\\]\3\2tu\31\2\b\t\22\23\25\25\27\27\31\31\36\36"+ - " #$&(++//\62\62\65\6688::AAEGILOPRSVWYY\\\\\u0358\2n\3\2\2\2\4q\3\2\2"+ - "\2\6\u00d9\3\2\2\2\b\u00e4\3\2\2\2\n\u00e8\3\2\2\2\f\u00fd\3\2\2\2\16"+ - "\u0104\3\2\2\2\20\u0106\3\2\2\2\22\u010e\3\2\2\2\24\u012a\3\2\2\2\26\u0134"+ - "\3\2\2\2\30\u013e\3\2\2\2\32\u014d\3\2\2\2\34\u014f\3\2\2\2\36\u0155\3"+ - "\2\2\2 \u0157\3\2\2\2\"\u015e\3\2\2\2$\u0170\3\2\2\2&\u0181\3\2\2\2(\u0191"+ - "\3\2\2\2*\u01ac\3\2\2\2,\u01ae\3\2\2\2.\u01cf\3\2\2\2\60\u01e0\3\2\2\2"+ - "\62\u01e3\3\2\2\2\64\u0215\3\2\2\2\66\u0217\3\2\2\28\u021a\3\2\2\2:\u0224"+ - "\3\2\2\2<\u022a\3\2\2\2>\u024f\3\2\2\2@\u025b\3\2\2\2B\u025d\3\2\2\2D"+ - "\u0264\3\2\2\2F\u026c\3\2\2\2H\u0278\3\2\2\2J\u027a\3\2\2\2L\u0286\3\2"+ - "\2\2N\u0288\3\2\2\2P\u029c\3\2\2\2R\u02b8\3\2\2\2T\u02ba\3\2\2\2V\u02bc"+ - "\3\2\2\2X\u02be\3\2\2\2Z\u02cb\3\2\2\2\\\u02cd\3\2\2\2^\u02d4\3\2\2\2"+ - "`\u02db\3\2\2\2b\u02e9\3\2\2\2d\u02ed\3\2\2\2f\u02f2\3\2\2\2h\u02f6\3"+ - "\2\2\2j\u02f8\3\2\2\2l\u02fa\3\2\2\2no\5\6\4\2op\7\2\2\3p\3\3\2\2\2qr"+ - "\5,\27\2rs\7\2\2\3s\5\3\2\2\2t\u00da\5\b\5\2u\u0083\7 \2\2v\177\7\3\2"+ - "\2wx\7G\2\2x~\t\2\2\2yz\7$\2\2z~\t\3\2\2{|\7Y\2\2|~\5V,\2}w\3\2\2\2}y"+ - "\3\2\2\2}{\3\2\2\2~\u0081\3\2\2\2\177}\3\2\2\2\177\u0080\3\2\2\2\u0080"+ - "\u0082\3\2\2\2\u0081\177\3\2\2\2\u0082\u0084\7\4\2\2\u0083v\3\2\2\2\u0083"+ - "\u0084\3\2\2\2\u0084\u0085\3\2\2\2\u0085\u00da\5\6\4\2\u0086\u0092\7\31"+ - "\2\2\u0087\u008e\7\3\2\2\u0088\u0089\7G\2\2\u0089\u008d\t\4\2\2\u008a"+ - "\u008b\7$\2\2\u008b\u008d\t\3\2\2\u008c\u0088\3\2\2\2\u008c\u008a\3\2"+ - "\2\2\u008d\u0090\3\2\2\2\u008e\u008c\3\2\2\2\u008e\u008f\3\2\2\2\u008f"+ - "\u0091\3\2\2\2\u0090\u008e\3\2\2\2\u0091\u0093\7\4\2\2\u0092\u0087\3\2"+ - "\2\2\u0092\u0093\3\2\2\2\u0093\u0094\3\2\2\2\u0094\u00da\5\6\4\2\u0095"+ - "\u0096\7O\2\2\u0096\u0099\7R\2\2\u0097\u009a\5\66\34\2\u0098\u009a\5b"+ - "\62\2\u0099\u0097\3\2\2\2\u0099\u0098\3\2\2\2\u0099\u009a\3\2\2\2\u009a"+ - "\u00da\3\2\2\2\u009b\u009c\7O\2\2\u009c\u009d\7\23\2\2\u009d\u00a0\t\5"+ - "\2\2\u009e\u00a1\5\66\34\2\u009f\u00a1\5b\62\2\u00a0\u009e\3\2\2\2\u00a0"+ - "\u009f\3\2\2\2\u00a1\u00da\3\2\2\2\u00a2\u00a5\t\6\2\2\u00a3\u00a6\5\66"+ - "\34\2\u00a4\u00a6\5b\62\2\u00a5\u00a3\3\2\2\2\u00a5\u00a4\3\2\2\2\u00a6"+ - "\u00da\3\2\2\2\u00a7\u00a8\7O\2\2\u00a8\u00aa\7\'\2\2\u00a9\u00ab\5\66"+ - "\34\2\u00aa\u00a9\3\2\2\2\u00aa\u00ab\3\2\2\2\u00ab\u00da\3\2\2\2\u00ac"+ - "\u00ad\7O\2\2\u00ad\u00da\7K\2\2\u00ae\u00af\7P\2\2\u00af\u00b2\7R\2\2"+ - "\u00b0\u00b1\7\21\2\2\u00b1\u00b3\5\66\34\2\u00b2\u00b0\3\2\2\2\u00b2"+ - "\u00b3\3\2\2\2\u00b3\u00b6\3\2\2\2\u00b4\u00b7\5\66\34\2\u00b5\u00b7\5"+ - "b\62\2\u00b6\u00b4\3\2\2\2\u00b6\u00b5\3\2\2\2\u00b6\u00b7\3\2\2\2\u00b7"+ - "\u00c1\3\2\2\2\u00b8\u00b9\7V\2\2\u00b9\u00be\5j\66\2\u00ba\u00bb\7\5"+ - "\2\2\u00bb\u00bd\5j\66\2\u00bc\u00ba\3\2\2\2\u00bd\u00c0\3\2\2\2\u00be"+ - "\u00bc\3\2\2\2\u00be\u00bf\3\2\2\2\u00bf\u00c2\3\2\2\2\u00c0\u00be\3\2"+ - "\2\2\u00c1\u00b8\3\2\2\2\u00c1\u00c2\3\2\2\2\u00c2\u00da\3\2\2\2\u00c3"+ - "\u00c4\7P\2\2\u00c4\u00c7\7\23\2\2\u00c5\u00c6\7\21\2\2\u00c6\u00c8\5"+ - "j\66\2\u00c7\u00c5\3\2\2\2\u00c7\u00c8\3\2\2\2\u00c8\u00cc\3\2\2\2\u00c9"+ - "\u00ca\7Q\2\2\u00ca\u00cd\5\66\34\2\u00cb\u00cd\5b\62\2\u00cc\u00c9\3"+ - "\2\2\2\u00cc\u00cb\3\2\2\2\u00cc\u00cd\3\2\2\2\u00cd\u00cf\3\2\2\2\u00ce"+ - "\u00d0\5\66\34\2\u00cf\u00ce\3\2\2\2\u00cf\u00d0\3\2\2\2\u00d0\u00da\3"+ - "\2\2\2\u00d1\u00d2\7P\2\2\u00d2\u00d7\7W\2\2\u00d3\u00d5\t\7\2\2\u00d4"+ - "\u00d3\3\2\2\2\u00d4\u00d5\3\2\2\2\u00d5\u00d6\3\2\2\2\u00d6\u00d8\5h"+ - "\65\2\u00d7\u00d4\3\2\2\2\u00d7\u00d8\3\2\2\2\u00d8\u00da\3\2\2\2\u00d9"+ - "t\3\2\2\2\u00d9u\3\2\2\2\u00d9\u0086\3\2\2\2\u00d9\u0095\3\2\2\2\u00d9"+ - "\u009b\3\2\2\2\u00d9\u00a2\3\2\2\2\u00d9\u00a7\3\2\2\2\u00d9\u00ac\3\2"+ - "\2\2\u00d9\u00ae\3\2\2\2\u00d9\u00c3\3\2\2\2\u00d9\u00d1\3\2\2\2\u00da"+ - "\7\3\2\2\2\u00db\u00dc\7[\2\2\u00dc\u00e1\5\34\17\2\u00dd\u00de\7\5\2"+ - "\2\u00de\u00e0\5\34\17\2\u00df\u00dd\3\2\2\2\u00e0\u00e3\3\2\2\2\u00e1"+ - "\u00df\3\2\2\2\u00e1\u00e2\3\2\2\2\u00e2\u00e5\3\2\2\2\u00e3\u00e1\3\2"+ - "\2\2\u00e4\u00db\3\2\2\2\u00e4\u00e5\3\2\2\2\u00e5\u00e6\3\2\2\2\u00e6"+ - "\u00e7\5\n\6\2\u00e7\t\3\2\2\2\u00e8\u00f3\5\16\b\2\u00e9\u00ea\7C\2\2"+ - "\u00ea\u00eb\7\17\2\2\u00eb\u00f0\5\20\t\2\u00ec\u00ed\7\5\2\2\u00ed\u00ef"+ - "\5\20\t\2\u00ee\u00ec\3\2\2\2\u00ef\u00f2\3\2\2\2\u00f0\u00ee\3\2\2\2"+ - "\u00f0\u00f1\3\2\2\2\u00f1\u00f4\3\2\2\2\u00f2\u00f0\3\2\2\2\u00f3\u00e9"+ - "\3\2\2\2\u00f3\u00f4\3\2\2\2\u00f4\u00f6\3\2\2\2\u00f5\u00f7\5\f\7\2\u00f6"+ - "\u00f5\3\2\2\2\u00f6\u00f7\3\2\2\2\u00f7\13\3\2\2\2\u00f8\u00f9\7\65\2"+ - "\2\u00f9\u00fe\t\b\2\2\u00fa\u00fb\7`\2\2\u00fb\u00fc\t\b\2\2\u00fc\u00fe"+ - "\7e\2\2\u00fd\u00f8\3\2\2\2\u00fd\u00fa\3\2\2\2\u00fe\r\3\2\2\2\u00ff"+ - "\u0105\5\22\n\2\u0100\u0101\7\3\2\2\u0101\u0102\5\n\6\2\u0102\u0103\7"+ - "\4\2\2\u0103\u0105\3\2\2\2\u0104\u00ff\3\2\2\2\u0104\u0100\3\2\2\2\u0105"+ - "\17\3\2\2\2\u0106\u0108\5,\27\2\u0107\u0109\t\t\2\2\u0108\u0107\3\2\2"+ - "\2\u0108\u0109\3\2\2\2\u0109\u010c\3\2\2\2\u010a\u010b\7?\2\2\u010b\u010d"+ - "\t\n\2\2\u010c\u010a\3\2\2\2\u010c\u010d\3\2\2\2\u010d\21\3\2\2\2\u010e"+ - "\u0110\7N\2\2\u010f\u0111\5\36\20\2\u0110\u010f\3\2\2\2\u0110\u0111\3"+ - "\2\2\2\u0111\u0112\3\2\2\2\u0112\u0117\5 \21\2\u0113\u0114\7\5\2\2\u0114"+ - "\u0116\5 \21\2\u0115\u0113\3\2\2\2\u0116\u0119\3\2\2\2\u0117\u0115\3\2"+ - "\2\2\u0117\u0118\3\2\2\2\u0118\u011b\3\2\2\2\u0119\u0117\3\2\2\2\u011a"+ - "\u011c\5\24\13\2\u011b\u011a\3\2\2\2\u011b\u011c\3\2\2\2\u011c\u011f\3"+ - "\2\2\2\u011d\u011e\7Z\2\2\u011e\u0120\5.\30\2\u011f\u011d\3\2\2\2\u011f"+ - "\u0120\3\2\2\2\u0120\u0124\3\2\2\2\u0121\u0122\7)\2\2\u0122\u0123\7\17"+ - "\2\2\u0123\u0125\5\26\f\2\u0124\u0121\3\2\2\2\u0124\u0125\3\2\2\2\u0125"+ - "\u0128\3\2\2\2\u0126\u0127\7*\2\2\u0127\u0129\5.\30\2\u0128\u0126\3\2"+ - "\2\2\u0128\u0129\3\2\2\2\u0129\23\3\2\2\2\u012a\u012b\7%\2\2\u012b\u0130"+ - "\5\"\22\2\u012c\u012d\7\5\2\2\u012d\u012f\5\"\22\2\u012e\u012c\3\2\2\2"+ - "\u012f\u0132\3\2\2\2\u0130\u012e\3\2\2\2\u0130\u0131\3\2\2\2\u0131\25"+ - "\3\2\2\2\u0132\u0130\3\2\2\2\u0133\u0135\5\36\20\2\u0134\u0133\3\2\2\2"+ - "\u0134\u0135\3\2\2\2\u0135\u0136\3\2\2\2\u0136\u013b\5\30\r\2\u0137\u0138"+ - "\7\5\2\2\u0138\u013a\5\30\r\2\u0139\u0137\3\2\2\2\u013a\u013d\3\2\2\2"+ - "\u013b\u0139\3\2\2\2\u013b\u013c\3\2\2\2\u013c\27\3\2\2\2\u013d\u013b"+ - "\3\2\2\2\u013e\u013f\5\32\16\2\u013f\31\3\2\2\2\u0140\u0149\7\3\2\2\u0141"+ - "\u0146\5,\27\2\u0142\u0143\7\5\2\2\u0143\u0145\5,\27\2\u0144\u0142\3\2"+ - "\2\2\u0145\u0148\3\2\2\2\u0146\u0144\3\2\2\2\u0146\u0147\3\2\2\2\u0147"+ - "\u014a\3\2\2\2\u0148\u0146\3\2\2\2\u0149\u0141\3\2\2\2\u0149\u014a\3\2"+ - "\2\2\u014a\u014b\3\2\2\2\u014b\u014e\7\4\2\2\u014c\u014e\5,\27\2\u014d"+ - "\u0140\3\2\2\2\u014d\u014c\3\2\2\2\u014e\33\3\2\2\2\u014f\u0150\5`\61"+ - "\2\u0150\u0151\7\f\2\2\u0151\u0152\7\3\2\2\u0152\u0153\5\n\6\2\u0153\u0154"+ - "\7\4\2\2\u0154\35\3\2\2\2\u0155\u0156\t\13\2\2\u0156\37\3\2\2\2\u0157"+ - "\u015c\5,\27\2\u0158\u015a\7\f\2\2\u0159\u0158\3\2\2\2\u0159\u015a\3\2"+ - "\2\2\u015a\u015b\3\2\2\2\u015b\u015d\5`\61\2\u015c\u0159\3\2\2\2\u015c"+ + "\n!\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3#\3#\3#\5#\u0268\n#\3#\3#\3#\5#\u026d"+ + "\n#\3#\5#\u0270\n#\5#\u0272\n#\3$\3$\3$\3$\3$\3$\3$\3%\3%\3%\3%\3%\5%"+ + "\u0280\n%\3&\3&\3&\3&\3&\3&\3&\3\'\3\'\3\'\3\'\3\'\5\'\u028e\n\'\3(\3"+ + "(\3(\5(\u0293\n(\3(\3(\3(\7(\u0298\n(\f(\16(\u029b\13(\5(\u029d\n(\3("+ + "\3(\3)\3)\3)\5)\u02a4\n)\3*\3*\3*\3*\3*\6*\u02ab\n*\r*\16*\u02ac\3*\3"+ + "*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\5*\u02c0\n*\3+\3+\3,\3"+ + ",\3-\3-\5-\u02c8\n-\3-\3-\5-\u02cc\n-\3-\3-\3-\5-\u02d1\n-\3.\3.\3/\3"+ + "/\3\60\3\60\3\60\7\60\u02da\n\60\f\60\16\60\u02dd\13\60\3\60\3\60\3\61"+ + "\3\61\5\61\u02e3\n\61\3\62\3\62\3\62\5\62\u02e8\n\62\3\62\3\62\3\62\3"+ + "\62\5\62\u02ee\n\62\3\62\5\62\u02f1\n\62\3\63\3\63\5\63\u02f5\n\63\3\64"+ + "\3\64\3\64\5\64\u02fa\n\64\3\65\3\65\5\65\u02fe\n\65\3\66\3\66\3\67\3"+ + "\67\3\67\2\4.<8\2\4\6\b\n\f\16\20\22\24\26\30\32\34\36 \"$&(*,.\60\62"+ + "\64\668:<>@BDFHJLNPRTVXZ\\^`bdfhjl\2\22\b\2\7\7\t\t\36\36\66\66AAEE\4"+ + "\2((SS\4\2\t\tAA\4\2%%--\3\2\32\33\3\2mn\4\2\7\7vv\4\2\r\r\32\32\4\2#"+ + "#\62\62\4\2\7\7\34\34\3\2oq\3\2fl\4\2\"\"TT\7\2\27\30+,8;LM\\]\3\2tu\30"+ + "\2\b\t\22\23\27\27\31\31\36\36 #$&(++//\62\62\65\6688::AAEGILOPRSVWY"+ + "Y\\\\\u0361\2n\3\2\2\2\4q\3\2\2\2\6\u00d9\3\2\2\2\b\u00e4\3\2\2\2\n\u00e8"+ + "\3\2\2\2\f\u00fd\3\2\2\2\16\u0104\3\2\2\2\20\u0106\3\2\2\2\22\u010e\3"+ + "\2\2\2\24\u012a\3\2\2\2\26\u0134\3\2\2\2\30\u013e\3\2\2\2\32\u014d\3\2"+ + "\2\2\34\u014f\3\2\2\2\36\u0155\3\2\2\2 \u0157\3\2\2\2\"\u015e\3\2\2\2"+ + "$\u0170\3\2\2\2&\u0181\3\2\2\2(\u0191\3\2\2\2*\u01ac\3\2\2\2,\u01ae\3"+ + "\2\2\2.\u01cf\3\2\2\2\60\u01e0\3\2\2\2\62\u01e3\3\2\2\2\64\u0215\3\2\2"+ + "\2\66\u0217\3\2\2\28\u021a\3\2\2\2:\u0224\3\2\2\2<\u022a\3\2\2\2>\u024f"+ + "\3\2\2\2@\u025b\3\2\2\2B\u025d\3\2\2\2D\u0271\3\2\2\2F\u0273\3\2\2\2H"+ + "\u027f\3\2\2\2J\u0281\3\2\2\2L\u028d\3\2\2\2N\u028f\3\2\2\2P\u02a3\3\2"+ + "\2\2R\u02bf\3\2\2\2T\u02c1\3\2\2\2V\u02c3\3\2\2\2X\u02c5\3\2\2\2Z\u02d2"+ + "\3\2\2\2\\\u02d4\3\2\2\2^\u02db\3\2\2\2`\u02e2\3\2\2\2b\u02f0\3\2\2\2"+ + "d\u02f4\3\2\2\2f\u02f9\3\2\2\2h\u02fd\3\2\2\2j\u02ff\3\2\2\2l\u0301\3"+ + "\2\2\2no\5\6\4\2op\7\2\2\3p\3\3\2\2\2qr\5,\27\2rs\7\2\2\3s\5\3\2\2\2t"+ + "\u00da\5\b\5\2u\u0083\7 \2\2v\177\7\3\2\2wx\7G\2\2x~\t\2\2\2yz\7$\2\2"+ + "z~\t\3\2\2{|\7Y\2\2|~\5V,\2}w\3\2\2\2}y\3\2\2\2}{\3\2\2\2~\u0081\3\2\2"+ + "\2\177}\3\2\2\2\177\u0080\3\2\2\2\u0080\u0082\3\2\2\2\u0081\177\3\2\2"+ + "\2\u0082\u0084\7\4\2\2\u0083v\3\2\2\2\u0083\u0084\3\2\2\2\u0084\u0085"+ + "\3\2\2\2\u0085\u00da\5\6\4\2\u0086\u0092\7\31\2\2\u0087\u008e\7\3\2\2"+ + "\u0088\u0089\7G\2\2\u0089\u008d\t\4\2\2\u008a\u008b\7$\2\2\u008b\u008d"+ + "\t\3\2\2\u008c\u0088\3\2\2\2\u008c\u008a\3\2\2\2\u008d\u0090\3\2\2\2\u008e"+ + "\u008c\3\2\2\2\u008e\u008f\3\2\2\2\u008f\u0091\3\2\2\2\u0090\u008e\3\2"+ + "\2\2\u0091\u0093\7\4\2\2\u0092\u0087\3\2\2\2\u0092\u0093\3\2\2\2\u0093"+ + "\u0094\3\2\2\2\u0094\u00da\5\6\4\2\u0095\u0096\7O\2\2\u0096\u0099\7R\2"+ + "\2\u0097\u009a\5\66\34\2\u0098\u009a\5b\62\2\u0099\u0097\3\2\2\2\u0099"+ + "\u0098\3\2\2\2\u0099\u009a\3\2\2\2\u009a\u00da\3\2\2\2\u009b\u009c\7O"+ + "\2\2\u009c\u009d\7\23\2\2\u009d\u00a0\t\5\2\2\u009e\u00a1\5\66\34\2\u009f"+ + "\u00a1\5b\62\2\u00a0\u009e\3\2\2\2\u00a0\u009f\3\2\2\2\u00a1\u00da\3\2"+ + "\2\2\u00a2\u00a5\t\6\2\2\u00a3\u00a6\5\66\34\2\u00a4\u00a6\5b\62\2\u00a5"+ + "\u00a3\3\2\2\2\u00a5\u00a4\3\2\2\2\u00a6\u00da\3\2\2\2\u00a7\u00a8\7O"+ + "\2\2\u00a8\u00aa\7\'\2\2\u00a9\u00ab\5\66\34\2\u00aa\u00a9\3\2\2\2\u00aa"+ + "\u00ab\3\2\2\2\u00ab\u00da\3\2\2\2\u00ac\u00ad\7O\2\2\u00ad\u00da\7K\2"+ + "\2\u00ae\u00af\7P\2\2\u00af\u00b2\7R\2\2\u00b0\u00b1\7\21\2\2\u00b1\u00b3"+ + "\5\66\34\2\u00b2\u00b0\3\2\2\2\u00b2\u00b3\3\2\2\2\u00b3\u00b6\3\2\2\2"+ + "\u00b4\u00b7\5\66\34\2\u00b5\u00b7\5b\62\2\u00b6\u00b4\3\2\2\2\u00b6\u00b5"+ + "\3\2\2\2\u00b6\u00b7\3\2\2\2\u00b7\u00c1\3\2\2\2\u00b8\u00b9\7V\2\2\u00b9"+ + "\u00be\5j\66\2\u00ba\u00bb\7\5\2\2\u00bb\u00bd\5j\66\2\u00bc\u00ba\3\2"+ + "\2\2\u00bd\u00c0\3\2\2\2\u00be\u00bc\3\2\2\2\u00be\u00bf\3\2\2\2\u00bf"+ + "\u00c2\3\2\2\2\u00c0\u00be\3\2\2\2\u00c1\u00b8\3\2\2\2\u00c1\u00c2\3\2"+ + "\2\2\u00c2\u00da\3\2\2\2\u00c3\u00c4\7P\2\2\u00c4\u00c7\7\23\2\2\u00c5"+ + "\u00c6\7\21\2\2\u00c6\u00c8\5j\66\2\u00c7\u00c5\3\2\2\2\u00c7\u00c8\3"+ + "\2\2\2\u00c8\u00cc\3\2\2\2\u00c9\u00ca\7Q\2\2\u00ca\u00cd\5\66\34\2\u00cb"+ + "\u00cd\5b\62\2\u00cc\u00c9\3\2\2\2\u00cc\u00cb\3\2\2\2\u00cc\u00cd\3\2"+ + "\2\2\u00cd\u00cf\3\2\2\2\u00ce\u00d0\5\66\34\2\u00cf\u00ce\3\2\2\2\u00cf"+ + "\u00d0\3\2\2\2\u00d0\u00da\3\2\2\2\u00d1\u00d2\7P\2\2\u00d2\u00d7\7W\2"+ + "\2\u00d3\u00d5\t\7\2\2\u00d4\u00d3\3\2\2\2\u00d4\u00d5\3\2\2\2\u00d5\u00d6"+ + "\3\2\2\2\u00d6\u00d8\5h\65\2\u00d7\u00d4\3\2\2\2\u00d7\u00d8\3\2\2\2\u00d8"+ + "\u00da\3\2\2\2\u00d9t\3\2\2\2\u00d9u\3\2\2\2\u00d9\u0086\3\2\2\2\u00d9"+ + "\u0095\3\2\2\2\u00d9\u009b\3\2\2\2\u00d9\u00a2\3\2\2\2\u00d9\u00a7\3\2"+ + "\2\2\u00d9\u00ac\3\2\2\2\u00d9\u00ae\3\2\2\2\u00d9\u00c3\3\2\2\2\u00d9"+ + "\u00d1\3\2\2\2\u00da\7\3\2\2\2\u00db\u00dc\7[\2\2\u00dc\u00e1\5\34\17"+ + "\2\u00dd\u00de\7\5\2\2\u00de\u00e0\5\34\17\2\u00df\u00dd\3\2\2\2\u00e0"+ + "\u00e3\3\2\2\2\u00e1\u00df\3\2\2\2\u00e1\u00e2\3\2\2\2\u00e2\u00e5\3\2"+ + "\2\2\u00e3\u00e1\3\2\2\2\u00e4\u00db\3\2\2\2\u00e4\u00e5\3\2\2\2\u00e5"+ + "\u00e6\3\2\2\2\u00e6\u00e7\5\n\6\2\u00e7\t\3\2\2\2\u00e8\u00f3\5\16\b"+ + "\2\u00e9\u00ea\7C\2\2\u00ea\u00eb\7\17\2\2\u00eb\u00f0\5\20\t\2\u00ec"+ + "\u00ed\7\5\2\2\u00ed\u00ef\5\20\t\2\u00ee\u00ec\3\2\2\2\u00ef\u00f2\3"+ + "\2\2\2\u00f0\u00ee\3\2\2\2\u00f0\u00f1\3\2\2\2\u00f1\u00f4\3\2\2\2\u00f2"+ + "\u00f0\3\2\2\2\u00f3\u00e9\3\2\2\2\u00f3\u00f4\3\2\2\2\u00f4\u00f6\3\2"+ + "\2\2\u00f5\u00f7\5\f\7\2\u00f6\u00f5\3\2\2\2\u00f6\u00f7\3\2\2\2\u00f7"+ + "\13\3\2\2\2\u00f8\u00f9\7\65\2\2\u00f9\u00fe\t\b\2\2\u00fa\u00fb\7`\2"+ + "\2\u00fb\u00fc\t\b\2\2\u00fc\u00fe\7e\2\2\u00fd\u00f8\3\2\2\2\u00fd\u00fa"+ + "\3\2\2\2\u00fe\r\3\2\2\2\u00ff\u0105\5\22\n\2\u0100\u0101\7\3\2\2\u0101"+ + "\u0102\5\n\6\2\u0102\u0103\7\4\2\2\u0103\u0105\3\2\2\2\u0104\u00ff\3\2"+ + "\2\2\u0104\u0100\3\2\2\2\u0105\17\3\2\2\2\u0106\u0108\5,\27\2\u0107\u0109"+ + "\t\t\2\2\u0108\u0107\3\2\2\2\u0108\u0109\3\2\2\2\u0109\u010c\3\2\2\2\u010a"+ + "\u010b\7?\2\2\u010b\u010d\t\n\2\2\u010c\u010a\3\2\2\2\u010c\u010d\3\2"+ + "\2\2\u010d\21\3\2\2\2\u010e\u0110\7N\2\2\u010f\u0111\5\36\20\2\u0110\u010f"+ + "\3\2\2\2\u0110\u0111\3\2\2\2\u0111\u0112\3\2\2\2\u0112\u0117\5 \21\2\u0113"+ + "\u0114\7\5\2\2\u0114\u0116\5 \21\2\u0115\u0113\3\2\2\2\u0116\u0119\3\2"+ + "\2\2\u0117\u0115\3\2\2\2\u0117\u0118\3\2\2\2\u0118\u011b\3\2\2\2\u0119"+ + "\u0117\3\2\2\2\u011a\u011c\5\24\13\2\u011b\u011a\3\2\2\2\u011b\u011c\3"+ + "\2\2\2\u011c\u011f\3\2\2\2\u011d\u011e\7Z\2\2\u011e\u0120\5.\30\2\u011f"+ + "\u011d\3\2\2\2\u011f\u0120\3\2\2\2\u0120\u0124\3\2\2\2\u0121\u0122\7)"+ + "\2\2\u0122\u0123\7\17\2\2\u0123\u0125\5\26\f\2\u0124\u0121\3\2\2\2\u0124"+ + "\u0125\3\2\2\2\u0125\u0128\3\2\2\2\u0126\u0127\7*\2\2\u0127\u0129\5.\30"+ + "\2\u0128\u0126\3\2\2\2\u0128\u0129\3\2\2\2\u0129\23\3\2\2\2\u012a\u012b"+ + "\7%\2\2\u012b\u0130\5\"\22\2\u012c\u012d\7\5\2\2\u012d\u012f\5\"\22\2"+ + "\u012e\u012c\3\2\2\2\u012f\u0132\3\2\2\2\u0130\u012e\3\2\2\2\u0130\u0131"+ + "\3\2\2\2\u0131\25\3\2\2\2\u0132\u0130\3\2\2\2\u0133\u0135\5\36\20\2\u0134"+ + "\u0133\3\2\2\2\u0134\u0135\3\2\2\2\u0135\u0136\3\2\2\2\u0136\u013b\5\30"+ + "\r\2\u0137\u0138\7\5\2\2\u0138\u013a\5\30\r\2\u0139\u0137\3\2\2\2\u013a"+ + "\u013d\3\2\2\2\u013b\u0139\3\2\2\2\u013b\u013c\3\2\2\2\u013c\27\3\2\2"+ + "\2\u013d\u013b\3\2\2\2\u013e\u013f\5\32\16\2\u013f\31\3\2\2\2\u0140\u0149"+ + "\7\3\2\2\u0141\u0146\5,\27\2\u0142\u0143\7\5\2\2\u0143\u0145\5,\27\2\u0144"+ + "\u0142\3\2\2\2\u0145\u0148\3\2\2\2\u0146\u0144\3\2\2\2\u0146\u0147\3\2"+ + "\2\2\u0147\u014a\3\2\2\2\u0148\u0146\3\2\2\2\u0149\u0141\3\2\2\2\u0149"+ + "\u014a\3\2\2\2\u014a\u014b\3\2\2\2\u014b\u014e\7\4\2\2\u014c\u014e\5,"+ + "\27\2\u014d\u0140\3\2\2\2\u014d\u014c\3\2\2\2\u014e\33\3\2\2\2\u014f\u0150"+ + "\5`\61\2\u0150\u0151\7\f\2\2\u0151\u0152\7\3\2\2\u0152\u0153\5\n\6\2\u0153"+ + "\u0154\7\4\2\2\u0154\35\3\2\2\2\u0155\u0156\t\13\2\2\u0156\37\3\2\2\2"+ + "\u0157\u015c\5,\27\2\u0158\u015a\7\f\2\2\u0159\u0158\3\2\2\2\u0159\u015a"+ + "\3\2\2\2\u015a\u015b\3\2\2\2\u015b\u015d\5`\61\2\u015c\u0159\3\2\2\2\u015c"+ "\u015d\3\2\2\2\u015d!\3\2\2\2\u015e\u0162\5*\26\2\u015f\u0161\5$\23\2"+ "\u0160\u015f\3\2\2\2\u0161\u0164\3\2\2\2\u0162\u0160\3\2\2\2\u0162\u0163"+ "\3\2\2\2\u0163#\3\2\2\2\u0164\u0162\3\2\2\2\u0165\u0166\5&\24\2\u0166"+ @@ -6598,65 +6616,67 @@ private boolean valueExpression_sempred(ValueExpressionContext _localctx, int pr "\2\u025b\u0252\3\2\2\2\u025b\u0256\3\2\2\2\u025b\u0257\3\2\2\2\u025cA"+ "\3\2\2\2\u025d\u025e\7\20\2\2\u025e\u025f\7\3\2\2\u025f\u0260\5,\27\2"+ "\u0260\u0261\7\f\2\2\u0261\u0262\5\\/\2\u0262\u0263\7\4\2\2\u0263C\3\2"+ - "\2\2\u0264\u026a\7\26\2\2\u0265\u0267\7\3\2\2\u0266\u0268\7v\2\2\u0267"+ - "\u0266\3\2\2\2\u0267\u0268\3\2\2\2\u0268\u0269\3\2\2\2\u0269\u026b\7\4"+ - "\2\2\u026a\u0265\3\2\2\2\u026a\u026b\3\2\2\2\u026bE\3\2\2\2\u026c\u026d"+ - "\7\24\2\2\u026d\u026e\7\3\2\2\u026e\u026f\5,\27\2\u026f\u0270\7\5\2\2"+ - "\u0270\u0271\5\\/\2\u0271\u0272\7\4\2\2\u0272G\3\2\2\2\u0273\u0279\5J"+ - "&\2\u0274\u0275\7_\2\2\u0275\u0276\5J&\2\u0276\u0277\7e\2\2\u0277\u0279"+ - "\3\2\2\2\u0278\u0273\3\2\2\2\u0278\u0274\3\2\2\2\u0279I\3\2\2\2\u027a"+ - "\u027b\7!\2\2\u027b\u027c\7\3\2\2\u027c\u027d\5`\61\2\u027d\u027e\7%\2"+ - "\2\u027e\u027f\5<\37\2\u027f\u0280\7\4\2\2\u0280K\3\2\2\2\u0281\u0287"+ - "\5N(\2\u0282\u0283\7_\2\2\u0283\u0284\5N(\2\u0284\u0285\7e\2\2\u0285\u0287"+ - "\3\2\2\2\u0286\u0281\3\2\2\2\u0286\u0282\3\2\2\2\u0287M\3\2\2\2\u0288"+ - "\u0289\5P)\2\u0289\u0295\7\3\2\2\u028a\u028c\5\36\20\2\u028b\u028a\3\2"+ - "\2\2\u028b\u028c\3\2\2\2\u028c\u028d\3\2\2\2\u028d\u0292\5,\27\2\u028e"+ - "\u028f\7\5\2\2\u028f\u0291\5,\27\2\u0290\u028e\3\2\2\2\u0291\u0294\3\2"+ - "\2\2\u0292\u0290\3\2\2\2\u0292\u0293\3\2\2\2\u0293\u0296\3\2\2\2\u0294"+ - "\u0292\3\2\2\2\u0295\u028b\3\2\2\2\u0295\u0296\3\2\2\2\u0296\u0297\3\2"+ - "\2\2\u0297\u0298\7\4\2\2\u0298O\3\2\2\2\u0299\u029d\7\63\2\2\u029a\u029d"+ - "\7H\2\2\u029b\u029d\5`\61\2\u029c\u0299\3\2\2\2\u029c\u029a\3\2\2\2\u029c"+ - "\u029b\3\2\2\2\u029dQ\3\2\2\2\u029e\u02b9\7>\2\2\u029f\u02b9\5X-\2\u02a0"+ - "\u02b9\5h\65\2\u02a1\u02b9\5V,\2\u02a2\u02a4\7u\2\2\u02a3\u02a2\3\2\2"+ - "\2\u02a4\u02a5\3\2\2\2\u02a5\u02a3\3\2\2\2\u02a5\u02a6\3\2\2\2\u02a6\u02b9"+ - "\3\2\2\2\u02a7\u02b9\7t\2\2\u02a8\u02a9\7a\2\2\u02a9\u02aa\5j\66\2\u02aa"+ - "\u02ab\7e\2\2\u02ab\u02b9\3\2\2\2\u02ac\u02ad\7b\2\2\u02ad\u02ae\5j\66"+ - "\2\u02ae\u02af\7e\2\2\u02af\u02b9\3\2\2\2\u02b0\u02b1\7c\2\2\u02b1\u02b2"+ - "\5j\66\2\u02b2\u02b3\7e\2\2\u02b3\u02b9\3\2\2\2\u02b4\u02b5\7d\2\2\u02b5"+ - "\u02b6\5j\66\2\u02b6\u02b7\7e\2\2\u02b7\u02b9\3\2\2\2\u02b8\u029e\3\2"+ - "\2\2\u02b8\u029f\3\2\2\2\u02b8\u02a0\3\2\2\2\u02b8\u02a1\3\2\2\2\u02b8"+ - "\u02a3\3\2\2\2\u02b8\u02a7\3\2\2\2\u02b8\u02a8\3\2\2\2\u02b8\u02ac\3\2"+ - "\2\2\u02b8\u02b0\3\2\2\2\u02b8\u02b4\3\2\2\2\u02b9S\3\2\2\2\u02ba\u02bb"+ - "\t\r\2\2\u02bbU\3\2\2\2\u02bc\u02bd\t\16\2\2\u02bdW\3\2\2\2\u02be\u02c0"+ - "\7/\2\2\u02bf\u02c1\t\7\2\2\u02c0\u02bf\3\2\2\2\u02c0\u02c1\3\2\2\2\u02c1"+ - "\u02c4\3\2\2\2\u02c2\u02c5\5h\65\2\u02c3\u02c5\5j\66\2\u02c4\u02c2\3\2"+ - "\2\2\u02c4\u02c3\3\2\2\2\u02c5\u02c6\3\2\2\2\u02c6\u02c9\5Z.\2\u02c7\u02c8"+ - "\7U\2\2\u02c8\u02ca\5Z.\2\u02c9\u02c7\3\2\2\2\u02c9\u02ca\3\2\2\2\u02ca"+ - "Y\3\2\2\2\u02cb\u02cc\t\17\2\2\u02cc[\3\2\2\2\u02cd\u02ce\5`\61\2\u02ce"+ - "]\3\2\2\2\u02cf\u02d0\5`\61\2\u02d0\u02d1\7s\2\2\u02d1\u02d3\3\2\2\2\u02d2"+ - "\u02cf\3\2\2\2\u02d3\u02d6\3\2\2\2\u02d4\u02d2\3\2\2\2\u02d4\u02d5\3\2"+ - "\2\2\u02d5\u02d7\3\2\2\2\u02d6\u02d4\3\2\2\2\u02d7\u02d8\5`\61\2\u02d8"+ - "_\3\2\2\2\u02d9\u02dc\5d\63\2\u02da\u02dc\5f\64\2\u02db\u02d9\3\2\2\2"+ - "\u02db\u02da\3\2\2\2\u02dca\3\2\2\2\u02dd\u02de\5`\61\2\u02de\u02df\7"+ - "\6\2\2\u02df\u02e1\3\2\2\2\u02e0\u02dd\3\2\2\2\u02e0\u02e1\3\2\2\2\u02e1"+ - "\u02e2\3\2\2\2\u02e2\u02ea\7z\2\2\u02e3\u02e4\5`\61\2\u02e4\u02e5\7\6"+ - "\2\2\u02e5\u02e7\3\2\2\2\u02e6\u02e3\3\2\2\2\u02e6\u02e7\3\2\2\2\u02e7"+ - "\u02e8\3\2\2\2\u02e8\u02ea\5`\61\2\u02e9\u02e0\3\2\2\2\u02e9\u02e6\3\2"+ - "\2\2\u02eac\3\2\2\2\u02eb\u02ee\7{\2\2\u02ec\u02ee\7|\2\2\u02ed\u02eb"+ - "\3\2\2\2\u02ed\u02ec\3\2\2\2\u02eee\3\2\2\2\u02ef\u02f3\7x\2\2\u02f0\u02f3"+ - "\5l\67\2\u02f1\u02f3\7y\2\2\u02f2\u02ef\3\2\2\2\u02f2\u02f0\3\2\2\2\u02f2"+ - "\u02f1\3\2\2\2\u02f3g\3\2\2\2\u02f4\u02f7\7w\2\2\u02f5\u02f7\7v\2\2\u02f6"+ - "\u02f4\3\2\2\2\u02f6\u02f5\3\2\2\2\u02f7i\3\2\2\2\u02f8\u02f9\t\20\2\2"+ - "\u02f9k\3\2\2\2\u02fa\u02fb\t\21\2\2\u02fbm\3\2\2\2h}\177\u0083\u008c"+ - "\u008e\u0092\u0099\u00a0\u00a5\u00aa\u00b2\u00b6\u00be\u00c1\u00c7\u00cc"+ - "\u00cf\u00d4\u00d7\u00d9\u00e1\u00e4\u00f0\u00f3\u00f6\u00fd\u0104\u0108"+ - "\u010c\u0110\u0117\u011b\u011f\u0124\u0128\u0130\u0134\u013b\u0146\u0149"+ - "\u014d\u0159\u015c\u0162\u0169\u0170\u0173\u0177\u017b\u017f\u0181\u018c"+ - "\u0191\u0195\u0198\u019e\u01a1\u01a7\u01aa\u01ac\u01cf\u01d7\u01d9\u01e0"+ - "\u01e5\u01e8\u01f0\u01f9\u01ff\u0207\u020c\u0212\u0215\u021c\u0224\u022a"+ - "\u0236\u0238\u0242\u024f\u025b\u0267\u026a\u0278\u0286\u028b\u0292\u0295"+ - "\u029c\u02a5\u02b8\u02c0\u02c4\u02c9\u02d4\u02db\u02e0\u02e6\u02e9\u02ed"+ - "\u02f2\u02f6"; + "\2\2\u0264\u0267\7\25\2\2\u0265\u0266\7\3\2\2\u0266\u0268\7\4\2\2\u0267"+ + "\u0265\3\2\2\2\u0267\u0268\3\2\2\2\u0268\u0272\3\2\2\2\u0269\u026f\7\26"+ + "\2\2\u026a\u026c\7\3\2\2\u026b\u026d\7v\2\2\u026c\u026b\3\2\2\2\u026c"+ + "\u026d\3\2\2\2\u026d\u026e\3\2\2\2\u026e\u0270\7\4\2\2\u026f\u026a\3\2"+ + "\2\2\u026f\u0270\3\2\2\2\u0270\u0272\3\2\2\2\u0271\u0264\3\2\2\2\u0271"+ + "\u0269\3\2\2\2\u0272E\3\2\2\2\u0273\u0274\7\24\2\2\u0274\u0275\7\3\2\2"+ + "\u0275\u0276\5,\27\2\u0276\u0277\7\5\2\2\u0277\u0278\5\\/\2\u0278\u0279"+ + "\7\4\2\2\u0279G\3\2\2\2\u027a\u0280\5J&\2\u027b\u027c\7_\2\2\u027c\u027d"+ + "\5J&\2\u027d\u027e\7e\2\2\u027e\u0280\3\2\2\2\u027f\u027a\3\2\2\2\u027f"+ + "\u027b\3\2\2\2\u0280I\3\2\2\2\u0281\u0282\7!\2\2\u0282\u0283\7\3\2\2\u0283"+ + "\u0284\5`\61\2\u0284\u0285\7%\2\2\u0285\u0286\5<\37\2\u0286\u0287\7\4"+ + "\2\2\u0287K\3\2\2\2\u0288\u028e\5N(\2\u0289\u028a\7_\2\2\u028a\u028b\5"+ + "N(\2\u028b\u028c\7e\2\2\u028c\u028e\3\2\2\2\u028d\u0288\3\2\2\2\u028d"+ + "\u0289\3\2\2\2\u028eM\3\2\2\2\u028f\u0290\5P)\2\u0290\u029c\7\3\2\2\u0291"+ + "\u0293\5\36\20\2\u0292\u0291\3\2\2\2\u0292\u0293\3\2\2\2\u0293\u0294\3"+ + "\2\2\2\u0294\u0299\5,\27\2\u0295\u0296\7\5\2\2\u0296\u0298\5,\27\2\u0297"+ + "\u0295\3\2\2\2\u0298\u029b\3\2\2\2\u0299\u0297\3\2\2\2\u0299\u029a\3\2"+ + "\2\2\u029a\u029d\3\2\2\2\u029b\u0299\3\2\2\2\u029c\u0292\3\2\2\2\u029c"+ + "\u029d\3\2\2\2\u029d\u029e\3\2\2\2\u029e\u029f\7\4\2\2\u029fO\3\2\2\2"+ + "\u02a0\u02a4\7\63\2\2\u02a1\u02a4\7H\2\2\u02a2\u02a4\5`\61\2\u02a3\u02a0"+ + "\3\2\2\2\u02a3\u02a1\3\2\2\2\u02a3\u02a2\3\2\2\2\u02a4Q\3\2\2\2\u02a5"+ + "\u02c0\7>\2\2\u02a6\u02c0\5X-\2\u02a7\u02c0\5h\65\2\u02a8\u02c0\5V,\2"+ + "\u02a9\u02ab\7u\2\2\u02aa\u02a9\3\2\2\2\u02ab\u02ac\3\2\2\2\u02ac\u02aa"+ + "\3\2\2\2\u02ac\u02ad\3\2\2\2\u02ad\u02c0\3\2\2\2\u02ae\u02c0\7t\2\2\u02af"+ + "\u02b0\7a\2\2\u02b0\u02b1\5j\66\2\u02b1\u02b2\7e\2\2\u02b2\u02c0\3\2\2"+ + "\2\u02b3\u02b4\7b\2\2\u02b4\u02b5\5j\66\2\u02b5\u02b6\7e\2\2\u02b6\u02c0"+ + "\3\2\2\2\u02b7\u02b8\7c\2\2\u02b8\u02b9\5j\66\2\u02b9\u02ba\7e\2\2\u02ba"+ + "\u02c0\3\2\2\2\u02bb\u02bc\7d\2\2\u02bc\u02bd\5j\66\2\u02bd\u02be\7e\2"+ + "\2\u02be\u02c0\3\2\2\2\u02bf\u02a5\3\2\2\2\u02bf\u02a6\3\2\2\2\u02bf\u02a7"+ + "\3\2\2\2\u02bf\u02a8\3\2\2\2\u02bf\u02aa\3\2\2\2\u02bf\u02ae\3\2\2\2\u02bf"+ + "\u02af\3\2\2\2\u02bf\u02b3\3\2\2\2\u02bf\u02b7\3\2\2\2\u02bf\u02bb\3\2"+ + "\2\2\u02c0S\3\2\2\2\u02c1\u02c2\t\r\2\2\u02c2U\3\2\2\2\u02c3\u02c4\t\16"+ + "\2\2\u02c4W\3\2\2\2\u02c5\u02c7\7/\2\2\u02c6\u02c8\t\7\2\2\u02c7\u02c6"+ + "\3\2\2\2\u02c7\u02c8\3\2\2\2\u02c8\u02cb\3\2\2\2\u02c9\u02cc\5h\65\2\u02ca"+ + "\u02cc\5j\66\2\u02cb\u02c9\3\2\2\2\u02cb\u02ca\3\2\2\2\u02cc\u02cd\3\2"+ + "\2\2\u02cd\u02d0\5Z.\2\u02ce\u02cf\7U\2\2\u02cf\u02d1\5Z.\2\u02d0\u02ce"+ + "\3\2\2\2\u02d0\u02d1\3\2\2\2\u02d1Y\3\2\2\2\u02d2\u02d3\t\17\2\2\u02d3"+ + "[\3\2\2\2\u02d4\u02d5\5`\61\2\u02d5]\3\2\2\2\u02d6\u02d7\5`\61\2\u02d7"+ + "\u02d8\7s\2\2\u02d8\u02da\3\2\2\2\u02d9\u02d6\3\2\2\2\u02da\u02dd\3\2"+ + "\2\2\u02db\u02d9\3\2\2\2\u02db\u02dc\3\2\2\2\u02dc\u02de\3\2\2\2\u02dd"+ + "\u02db\3\2\2\2\u02de\u02df\5`\61\2\u02df_\3\2\2\2\u02e0\u02e3\5d\63\2"+ + "\u02e1\u02e3\5f\64\2\u02e2\u02e0\3\2\2\2\u02e2\u02e1\3\2\2\2\u02e3a\3"+ + "\2\2\2\u02e4\u02e5\5`\61\2\u02e5\u02e6\7\6\2\2\u02e6\u02e8\3\2\2\2\u02e7"+ + "\u02e4\3\2\2\2\u02e7\u02e8\3\2\2\2\u02e8\u02e9\3\2\2\2\u02e9\u02f1\7z"+ + "\2\2\u02ea\u02eb\5`\61\2\u02eb\u02ec\7\6\2\2\u02ec\u02ee\3\2\2\2\u02ed"+ + "\u02ea\3\2\2\2\u02ed\u02ee\3\2\2\2\u02ee\u02ef\3\2\2\2\u02ef\u02f1\5`"+ + "\61\2\u02f0\u02e7\3\2\2\2\u02f0\u02ed\3\2\2\2\u02f1c\3\2\2\2\u02f2\u02f5"+ + "\7{\2\2\u02f3\u02f5\7|\2\2\u02f4\u02f2\3\2\2\2\u02f4\u02f3\3\2\2\2\u02f5"+ + "e\3\2\2\2\u02f6\u02fa\7x\2\2\u02f7\u02fa\5l\67\2\u02f8\u02fa\7y\2\2\u02f9"+ + "\u02f6\3\2\2\2\u02f9\u02f7\3\2\2\2\u02f9\u02f8\3\2\2\2\u02fag\3\2\2\2"+ + "\u02fb\u02fe\7w\2\2\u02fc\u02fe\7v\2\2\u02fd\u02fb\3\2\2\2\u02fd\u02fc"+ + "\3\2\2\2\u02fei\3\2\2\2\u02ff\u0300\t\20\2\2\u0300k\3\2\2\2\u0301\u0302"+ + "\t\21\2\2\u0302m\3\2\2\2j}\177\u0083\u008c\u008e\u0092\u0099\u00a0\u00a5"+ + "\u00aa\u00b2\u00b6\u00be\u00c1\u00c7\u00cc\u00cf\u00d4\u00d7\u00d9\u00e1"+ + "\u00e4\u00f0\u00f3\u00f6\u00fd\u0104\u0108\u010c\u0110\u0117\u011b\u011f"+ + "\u0124\u0128\u0130\u0134\u013b\u0146\u0149\u014d\u0159\u015c\u0162\u0169"+ + "\u0170\u0173\u0177\u017b\u017f\u0181\u018c\u0191\u0195\u0198\u019e\u01a1"+ + "\u01a7\u01aa\u01ac\u01cf\u01d7\u01d9\u01e0\u01e5\u01e8\u01f0\u01f9\u01ff"+ + "\u0207\u020c\u0212\u0215\u021c\u0224\u022a\u0236\u0238\u0242\u024f\u025b"+ + "\u0267\u026c\u026f\u0271\u027f\u028d\u0292\u0299\u029c\u02a3\u02ac\u02bf"+ + "\u02c7\u02cb\u02d0\u02db\u02e2\u02e7\u02ed\u02f0\u02f4\u02f9\u02fd"; public static final ATN _ATN = new ATNDeserializer().deserialize(_serializedATN.toCharArray()); static { diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTests.java new file mode 100644 index 0000000000000..5eaa9ccd6c268 --- /dev/null +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTests.java @@ -0,0 +1,46 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.sql.expression.function.scalar.datetime; + +import org.elasticsearch.xpack.sql.expression.Expression; +import org.elasticsearch.xpack.sql.proto.Mode; +import org.elasticsearch.xpack.sql.proto.Protocol; +import org.elasticsearch.xpack.sql.session.Configuration; +import org.elasticsearch.xpack.sql.tree.AbstractNodeTestCase; +import org.elasticsearch.xpack.sql.tree.Source; + +public class CurrentDateTests extends AbstractNodeTestCase { + + public static CurrentDate randomCurrentDate() { + return new CurrentDate(Source.EMPTY, new Configuration(randomZone(), Protocol.FETCH_SIZE, + Protocol.REQUEST_TIMEOUT, Protocol.PAGE_TIMEOUT, null, Mode.PLAIN, null, null, null)); + } + + @Override + protected CurrentDate randomInstance() { + return randomCurrentDate(); + } + + @Override + protected CurrentDate copy(CurrentDate instance) { + return new CurrentDate(instance.source(), instance.configuration()); + } + + @Override + protected CurrentDate mutate(CurrentDate instance) { + return new CurrentDate(instance.source(), new Configuration(randomZone(), Protocol.FETCH_SIZE, + Protocol.REQUEST_TIMEOUT, Protocol.PAGE_TIMEOUT, null, Mode.PLAIN, null, null, null)); + } + + @Override + public void testTransform() { + } + + @Override + public void testReplaceChildren() { + } +} diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTimeTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTimeTests.java index c5cdb06724bf8..28b99c1863486 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTimeTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTimeTests.java @@ -6,23 +6,61 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.datetime; -import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.sql.expression.Expression; +import org.elasticsearch.xpack.sql.expression.Literal; +import org.elasticsearch.xpack.sql.proto.Mode; +import org.elasticsearch.xpack.sql.proto.Protocol; +import org.elasticsearch.xpack.sql.session.Configuration; +import org.elasticsearch.xpack.sql.tree.AbstractNodeTestCase; import java.time.ZonedDateTime; -public class CurrentDateTimeTests extends ESTestCase { +import static org.elasticsearch.xpack.sql.tree.Source.EMPTY; - public void testNanoPrecision() throws Exception { +public class CurrentDateTimeTests extends AbstractNodeTestCase { + + public static CurrentDateTime randomCurrentDateTime() { + return new CurrentDateTime(EMPTY, Literal.of(EMPTY, randomInt(10)), + new Configuration(randomZone(), Protocol.FETCH_SIZE, + Protocol.REQUEST_TIMEOUT, Protocol.PAGE_TIMEOUT, null, Mode.PLAIN, null, null, null)); + } + + @Override + protected CurrentDateTime randomInstance() { + return randomCurrentDateTime(); + } + + @Override + protected CurrentDateTime copy(CurrentDateTime instance) { + return new CurrentDateTime(instance.source(), instance.precision(), instance.configuration()); + } + + @Override + protected CurrentDateTime mutate(CurrentDateTime instance) { + return new CurrentDateTime(instance.source(), Literal.of(EMPTY, randomInt(10)), + new Configuration(randomZone(), Protocol.FETCH_SIZE, + Protocol.REQUEST_TIMEOUT, Protocol.PAGE_TIMEOUT, null, Mode.PLAIN, null, null, null)); + } + + @Override + public void testTransform() { + } + + @Override + public void testReplaceChildren() { + } + + public void testNanoPrecision() { ZonedDateTime zdt = ZonedDateTime.parse("2018-01-23T12:34:45.123456789Z"); - assertEquals(000_000_000, CurrentDateTime.nanoPrecision(zdt, 0).getNano()); - assertEquals(100_000_000, CurrentDateTime.nanoPrecision(zdt, 1).getNano()); - assertEquals(120_000_000, CurrentDateTime.nanoPrecision(zdt, 2).getNano()); - assertEquals(123_000_000, CurrentDateTime.nanoPrecision(zdt, 3).getNano()); - assertEquals(123_400_000, CurrentDateTime.nanoPrecision(zdt, 4).getNano()); - assertEquals(123_450_000, CurrentDateTime.nanoPrecision(zdt, 5).getNano()); - assertEquals(123_456_000, CurrentDateTime.nanoPrecision(zdt, 6).getNano()); - assertEquals(123_456_700, CurrentDateTime.nanoPrecision(zdt, 7).getNano()); - assertEquals(123_456_780, CurrentDateTime.nanoPrecision(zdt, 8).getNano()); - assertEquals(123_456_789, CurrentDateTime.nanoPrecision(zdt, 9).getNano()); + assertEquals(000_000_000, CurrentDateTime.nanoPrecision(zdt, Literal.of(EMPTY, 0)).getNano()); + assertEquals(100_000_000, CurrentDateTime.nanoPrecision(zdt, Literal.of(EMPTY, 1)).getNano()); + assertEquals(120_000_000, CurrentDateTime.nanoPrecision(zdt, Literal.of(EMPTY, 2)).getNano()); + assertEquals(123_000_000, CurrentDateTime.nanoPrecision(zdt, Literal.of(EMPTY, 3)).getNano()); + assertEquals(123_400_000, CurrentDateTime.nanoPrecision(zdt, Literal.of(EMPTY, 4)).getNano()); + assertEquals(123_450_000, CurrentDateTime.nanoPrecision(zdt, Literal.of(EMPTY, 5)).getNano()); + assertEquals(123_456_000, CurrentDateTime.nanoPrecision(zdt, Literal.of(EMPTY, 6)).getNano()); + assertEquals(123_456_700, CurrentDateTime.nanoPrecision(zdt, Literal.of(EMPTY, 7)).getNano()); + assertEquals(123_456_780, CurrentDateTime.nanoPrecision(zdt, Literal.of(EMPTY, 8)).getNano()); + assertEquals(123_456_789, CurrentDateTime.nanoPrecision(zdt, Literal.of(EMPTY, 9)).getNano()); } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java index 2926f380ec237..229d39705cd82 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java @@ -338,6 +338,22 @@ public void testConvertWithInvalidESDataType() { assertEquals("line 1:13: Invalid data type [INVALID] provided", ex.getMessage()); } + public void testCurrentDate() { + Expression expr = parser.createExpression("CURRENT_DATE"); + assertEquals(UnresolvedFunction.class, expr.getClass()); + UnresolvedFunction ur = (UnresolvedFunction) expr; + assertEquals("CURRENT_DATE", ur.sourceText()); + assertEquals(0, ur.children().size()); + } + + public void testCurrentDateWithParentheses() { + Expression expr = parser.createExpression("CURRENT_DATE( )"); + assertEquals(UnresolvedFunction.class, expr.getClass()); + UnresolvedFunction ur = (UnresolvedFunction) expr; + assertEquals("CURRENT_DATE( )", ur.sourceText()); + assertEquals(0, ur.children().size()); + } + public void testCurrentTimestamp() { Expression expr = parser.createExpression("CURRENT_TIMESTAMP"); assertEquals(UnresolvedFunction.class, expr.getClass()); @@ -373,4 +389,4 @@ public void testSourceFunction() throws Exception { Expression expr = parser.createExpression(s); assertEquals(s, expr.sourceText()); } -} \ No newline at end of file +} diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/tree/NodeSubclassTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/tree/NodeSubclassTests.java index 03f1a032cdab2..f11519f0f973f 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/tree/NodeSubclassTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/tree/NodeSubclassTests.java @@ -92,8 +92,7 @@ */ public class NodeSubclassTests> extends ESTestCase { - - private static final List> CLASSES_WITH_MIN_TWO_CHILDREN = Arrays.> asList(IfNull.class, In.class, InPipe.class, + private static final List> CLASSES_WITH_MIN_TWO_CHILDREN = Arrays.asList(IfNull.class, In.class, InPipe.class, Percentile.class, Percentiles.class, PercentileRanks.class); private final Class subclass; @@ -138,9 +137,7 @@ public void testTransform() throws Exception { Type changedArgType = argTypes[changedArgOffset]; Object changedArgValue = randomValueOtherThan(nodeCtorArgs[changedArgOffset], () -> makeArg(changedArgType)); - B transformed = node.transformNodeProps(prop -> { - return Objects.equals(prop, originalArgValue) ? changedArgValue : prop; - }, Object.class); + B transformed = node.transformNodeProps(prop -> Objects.equals(prop, originalArgValue) ? changedArgValue : prop, Object.class); if (node.children().contains(originalArgValue) || node.children().equals(originalArgValue)) { if (node.children().equals(emptyList()) && originalArgValue.equals(emptyList())) { From 440d1eda8a51f94aa6effaff7944d958b293d1ca Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Tue, 5 Feb 2019 08:42:37 -0800 Subject: [PATCH 08/23] Fix failures in BulkProcessorIT#testGlobalParametersAndBulkProcessor. (#38129) This PR fixes a couple test issues: * It narrows an assertWarnings call that was too broad, and wasn't always applicable with certain random sequences. * Previously, we could send a typeless bulk request containing '_type: 'null'. Now we omit the _type key altogether for typeless requests. --- .../elasticsearch/client/BulkProcessorIT.java | 62 +++++++++++-------- 1 file changed, 35 insertions(+), 27 deletions(-) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorIT.java index 72ffcd7c5062e..52110989e1715 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorIT.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.rest.action.document.RestBulkAction; @@ -75,12 +76,12 @@ private static BulkProcessor.Builder initBulkProcessorBuilder(BulkProcessor.List (request, bulkListener) -> highLevelClient().bulkAsync(request, RequestOptions.DEFAULT, bulkListener), listener); } - + private static BulkProcessor.Builder initBulkProcessorBuilderUsingTypes(BulkProcessor.Listener listener) { return BulkProcessor.builder( (request, bulkListener) -> highLevelClient().bulkAsync(request, expectWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE), bulkListener), listener); - } + } public void testThatBulkProcessorCountIsCorrect() throws Exception { final CountDownLatch latch = new CountDownLatch(1); @@ -383,14 +384,14 @@ public void testGlobalParametersAndBulkProcessor() throws Exception { .build()) { indexDocs(processor, numDocs, null, localType, "test", globalType, "pipeline_id"); latch.await(); - + assertThat(listener.beforeCounts.get(), equalTo(1)); assertThat(listener.afterCounts.get(), equalTo(1)); assertThat(listener.bulkFailures.size(), equalTo(0)); assertResponseItems(listener.bulkItems, numDocs, localType); - + Iterable hits = searchAll(new SearchRequest("test").routing("routing")); - + assertThat(hits, everyItem(hasProperty(fieldFromSource("fieldNameXYZ"), equalTo("valueXYZ")))); assertThat(hits, everyItem(Matchers.allOf(hasIndex("test"), hasType(localType)))); assertThat(hits, containsInAnyOrder(expectedIds(numDocs))); @@ -398,7 +399,7 @@ public void testGlobalParametersAndBulkProcessor() throws Exception { } { //Check that untyped document additions and untyped global inherit the established custom type - // (the custom document type introduced to the mapping by the earlier code in this test) + // (the custom document type introduced to the mapping by the earlier code in this test) String globalType = null; String localType = null; final CountDownLatch latch = new CountDownLatch(1); @@ -414,20 +415,19 @@ public void testGlobalParametersAndBulkProcessor() throws Exception { .build()) { indexDocs(processor, numDocs, null, localType, "test", globalType, "pipeline_id"); latch.await(); - + assertThat(listener.beforeCounts.get(), equalTo(1)); assertThat(listener.afterCounts.get(), equalTo(1)); assertThat(listener.bulkFailures.size(), equalTo(0)); assertResponseItems(listener.bulkItems, numDocs, MapperService.SINGLE_MAPPING_NAME); - + Iterable hits = searchAll(new SearchRequest("test").routing("routing")); - + assertThat(hits, everyItem(hasProperty(fieldFromSource("fieldNameXYZ"), equalTo("valueXYZ")))); assertThat(hits, everyItem(Matchers.allOf(hasIndex("test"), hasType(customType)))); assertThat(hits, containsInAnyOrder(expectedIds(numDocs))); } - } - assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); + } } @SuppressWarnings("unchecked") @@ -438,8 +438,8 @@ private Matcher[] expectedIds(int numDocs) { .>toArray(Matcher[]::new); } - private static MultiGetRequest indexDocs(BulkProcessor processor, int numDocs, String localIndex, String localType, - String globalIndex, String globalType, String globalPipeline) throws Exception { + private MultiGetRequest indexDocs(BulkProcessor processor, int numDocs, String localIndex, String localType, + String globalIndex, String globalType, String globalPipeline) throws Exception { MultiGetRequest multiGetRequest = new MultiGetRequest(); for (int i = 1; i <= numDocs; i++) { if (randomBoolean()) { @@ -448,6 +448,11 @@ private static MultiGetRequest indexDocs(BulkProcessor processor, int numDocs, S } else { BytesArray data = bytesBulkRequest(localIndex, localType, i); processor.add(data, globalIndex, globalType, globalPipeline, null, XContentType.JSON); + + if (localType != null) { + // If the payload contains types, parsing it into a bulk request results in a warning. + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); + } } multiGetRequest.add(localIndex, Integer.toString(i)); } @@ -455,26 +460,29 @@ private static MultiGetRequest indexDocs(BulkProcessor processor, int numDocs, S } private static BytesArray bytesBulkRequest(String localIndex, String localType, int id) throws IOException { - String action = Strings.toString(jsonBuilder() - .startObject() - .startObject("index") - .field("_index", localIndex) - .field("_type", localType) - .field("_id", Integer.toString(id)) - .endObject() - .endObject() - ); - String source = Strings.toString(jsonBuilder() + XContentBuilder action = jsonBuilder().startObject().startObject("index"); + + if (localIndex != null) { + action.field("_index", localIndex); + } + + if (localType != null) { + action.field("_type", localType); + } + + action.field("_id", Integer.toString(id)); + action.endObject().endObject(); + + XContentBuilder source = jsonBuilder() .startObject() .field("field", randomRealisticUnicodeOfLengthBetween(1, 30)) - .endObject() - ); + .endObject(); - String request = action + "\n" + source + "\n"; + String request = Strings.toString(action) + "\n" + Strings.toString(source) + "\n"; return new BytesArray(request); } - private static MultiGetRequest indexDocs(BulkProcessor processor, int numDocs) throws Exception { + private MultiGetRequest indexDocs(BulkProcessor processor, int numDocs) throws Exception { return indexDocs(processor, numDocs, "test", null, null, null, null); } From df4eb0485ddd52631830e86985373a3799d3df12 Mon Sep 17 00:00:00 2001 From: Przemyslaw Gomulka Date: Tue, 5 Feb 2019 17:48:47 +0100 Subject: [PATCH 09/23] Enable CronEvalToolTest.testEnsureDateIsShownInRootLocale (#38394) The test is now expected to be always passing no matter what the random locale is. This is fixed with using jdk ZoneId.systemDefault() in both the test and CronEvalTool closes #35687 --- .../xpack/watcher/trigger/schedule/tool/CronEvalToolTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/trigger/schedule/tool/CronEvalToolTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/trigger/schedule/tool/CronEvalToolTests.java index 0c898d1363b0d..74fabf2b4ac01 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/trigger/schedule/tool/CronEvalToolTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/trigger/schedule/tool/CronEvalToolTests.java @@ -56,7 +56,6 @@ public void testGetNextValidTimes() throws Exception { // randomized testing sets arbitrary locales and timezones, and we do not care // we always have to output in standard locale and independent from timezone - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/35687") public void testEnsureDateIsShownInRootLocale() throws Exception { String output = execute("-c","1", "0 0 11 ? * MON-SAT 2040"); if (ZoneId.systemDefault().equals(ZoneOffset.UTC)) { From 12657fda4498d11e3a0102d52af469d0b9907c20 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 5 Feb 2019 18:05:56 +0100 Subject: [PATCH 10/23] `if_seq_no` and `if_primary_term` parameters aren't wired correctly in REST Client's CRUD API (#38411) --- .../client/RequestConverters.java | 9 ++++ .../java/org/elasticsearch/client/CrudIT.java | 22 +++++---- .../client/RequestConvertersTests.java | 35 +++++++++++++-- .../rest-api-spec/test/bulk/80_cas.yml | 42 +++++++++++++++++ .../test/bulk/81_cas_with_types.yml | 45 +++++++++++++++++++ .../rest-api-spec/test/index/30_cas.yml | 2 +- 6 files changed, 142 insertions(+), 13 deletions(-) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/bulk/80_cas.yml create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/bulk/81_cas_with_types.yml diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java index a30fec41b0bf3..860788e0157a5 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java @@ -108,6 +108,8 @@ static Request delete(DeleteRequest deleteRequest) { parameters.withTimeout(deleteRequest.timeout()); parameters.withVersion(deleteRequest.version()); parameters.withVersionType(deleteRequest.versionType()); + parameters.withIfSeqNo(deleteRequest.ifSeqNo()); + parameters.withIfPrimaryTerm(deleteRequest.ifPrimaryTerm()); parameters.withRefreshPolicy(deleteRequest.getRefreshPolicy()); parameters.withWaitForActiveShards(deleteRequest.waitForActiveShards()); return request; @@ -191,6 +193,11 @@ static Request bulk(BulkRequest bulkRequest) throws IOException { } } + if (action.ifSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { + metadata.field("if_seq_no", action.ifSeqNo()); + metadata.field("if_primary_term", action.ifPrimaryTerm()); + } + if (opType == DocWriteRequest.OpType.INDEX || opType == DocWriteRequest.OpType.CREATE) { IndexRequest indexRequest = (IndexRequest) action; if (Strings.hasLength(indexRequest.getPipeline())) { @@ -319,6 +326,8 @@ static Request index(IndexRequest indexRequest) { parameters.withTimeout(indexRequest.timeout()); parameters.withVersion(indexRequest.version()); parameters.withVersionType(indexRequest.versionType()); + parameters.withIfSeqNo(indexRequest.ifSeqNo()); + parameters.withIfPrimaryTerm(indexRequest.ifPrimaryTerm()); parameters.withPipeline(indexRequest.getPipeline()); parameters.withRefreshPolicy(indexRequest.getRefreshPolicy()); parameters.withWaitForActiveShards(indexRequest.waitForActiveShards()); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/CrudIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/CrudIT.java index 3bd3c79072dc9..e2102236cc422 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/CrudIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/CrudIT.java @@ -104,11 +104,13 @@ public void testDelete() throws IOException { { // Testing deletion String docId = "id"; - highLevelClient().index( + IndexResponse indexResponse = highLevelClient().index( new IndexRequest("index").id(docId).source(Collections.singletonMap("foo", "bar")), RequestOptions.DEFAULT); + assertThat(indexResponse.getSeqNo(), greaterThanOrEqualTo(0L)); DeleteRequest deleteRequest = new DeleteRequest("index", docId); if (randomBoolean()) { - deleteRequest.version(1L); + deleteRequest.setIfSeqNo(indexResponse.getSeqNo()); + deleteRequest.setIfPrimaryTerm(indexResponse.getPrimaryTerm()); } DeleteResponse deleteResponse = execute(deleteRequest, highLevelClient()::delete, highLevelClient()::deleteAsync); assertEquals("index", deleteResponse.getIndex()); @@ -131,12 +133,13 @@ public void testDelete() throws IOException { String docId = "version_conflict"; highLevelClient().index( new IndexRequest("index").id( docId).source(Collections.singletonMap("foo", "bar")), RequestOptions.DEFAULT); - DeleteRequest deleteRequest = new DeleteRequest("index", docId).version(2); + DeleteRequest deleteRequest = new DeleteRequest("index", docId).setIfSeqNo(2).setIfPrimaryTerm(2); ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> execute(deleteRequest, highLevelClient()::delete, highLevelClient()::deleteAsync)); assertEquals(RestStatus.CONFLICT, exception.status()); assertEquals("Elasticsearch exception [type=version_conflict_engine_exception, reason=[_doc][" + docId + "]: " + - "version conflict, current version [1] is different than the one provided [2]]", exception.getMessage()); + "version conflict, required seqNo [2], primary term [2]. current document has seqNo [3] and primary term [1]]", + exception.getMessage()); assertEquals("index", exception.getMetadata("es.index").get(0)); } { @@ -519,13 +522,14 @@ public void testIndex() throws IOException { ElasticsearchStatusException exception = expectThrows(ElasticsearchStatusException.class, () -> { IndexRequest wrongRequest = new IndexRequest("index").id("id"); wrongRequest.source(XContentBuilder.builder(xContentType.xContent()).startObject().field("field", "test").endObject()); - wrongRequest.version(5L); + wrongRequest.setIfSeqNo(1L).setIfPrimaryTerm(5L); execute(wrongRequest, highLevelClient()::index, highLevelClient()::indexAsync); }); assertEquals(RestStatus.CONFLICT, exception.status()); assertEquals("Elasticsearch exception [type=version_conflict_engine_exception, reason=[_doc][id]: " + - "version conflict, current version [2] is different than the one provided [5]]", exception.getMessage()); + "version conflict, required seqNo [1], primary term [5]. current document has seqNo [2] and primary term [1]]", + exception.getMessage()); assertEquals("index", exception.getMetadata("es.index").get(0)); } { @@ -820,7 +824,8 @@ public void testBulk() throws IOException { if (opType == DocWriteRequest.OpType.INDEX) { IndexRequest indexRequest = new IndexRequest("index").id(id).source(source, xContentType); if (erroneous) { - indexRequest.version(12L); + indexRequest.setIfSeqNo(12L); + indexRequest.setIfPrimaryTerm(12L); } bulkRequest.add(indexRequest); @@ -1130,7 +1135,8 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) if (opType == DocWriteRequest.OpType.INDEX) { IndexRequest indexRequest = new IndexRequest("index").id(id).source(xContentType, "id", i); if (erroneous) { - indexRequest.version(12L); + indexRequest.setIfSeqNo(12L); + indexRequest.setIfPrimaryTerm(12L); } processor.add(indexRequest); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java index b58e5ae8852d3..9364e2ce2d57c 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java @@ -281,6 +281,7 @@ public void testDelete() { setRandomRefreshPolicy(deleteRequest::setRefreshPolicy, expectedParams); setRandomVersion(deleteRequest, expectedParams); setRandomVersionType(deleteRequest::versionType, expectedParams); + setRandomIfSeqNoAndTerm(deleteRequest, expectedParams); if (frequently()) { if (randomBoolean()) { @@ -631,6 +632,7 @@ public void testIndex() throws IOException { } else { setRandomVersion(indexRequest, expectedParams); setRandomVersionType(indexRequest::versionType, expectedParams); + setRandomIfSeqNoAndTerm(indexRequest, expectedParams); } if (frequently()) { @@ -768,6 +770,7 @@ public void testUpdate() throws IOException { setRandomWaitForActiveShards(updateRequest::waitForActiveShards, expectedParams); setRandomVersion(updateRequest, expectedParams); setRandomVersionType(updateRequest::versionType, expectedParams); + setRandomIfSeqNoAndTerm(updateRequest, new HashMap<>()); // if* params are passed in the body if (randomBoolean()) { int retryOnConflict = randomIntBetween(0, 5); updateRequest.retryOnConflict(retryOnConflict); @@ -798,6 +801,7 @@ public void testUpdate() throws IOException { assertEquals(updateRequest.docAsUpsert(), parsedUpdateRequest.docAsUpsert()); assertEquals(updateRequest.detectNoop(), parsedUpdateRequest.detectNoop()); assertEquals(updateRequest.fetchSource(), parsedUpdateRequest.fetchSource()); + assertIfSeqNoAndTerm(updateRequest, parsedUpdateRequest); assertEquals(updateRequest.script(), parsedUpdateRequest.script()); if (updateRequest.doc() != null) { assertToXContentEquivalent(updateRequest.doc().source(), parsedUpdateRequest.doc().source(), xContentType); @@ -811,6 +815,22 @@ public void testUpdate() throws IOException { } } + private static void assertIfSeqNoAndTerm(DocWriteRequestrequest, DocWriteRequest parsedRequest) { + assertEquals(request.ifSeqNo(), parsedRequest.ifSeqNo()); + assertEquals(request.ifPrimaryTerm(), parsedRequest.ifPrimaryTerm()); + } + + private static void setRandomIfSeqNoAndTerm(DocWriteRequest request, Map expectedParams) { + if (randomBoolean()) { + final long seqNo = randomNonNegativeLong(); + request.setIfSeqNo(seqNo); + expectedParams.put("if_seq_no", Long.toString(seqNo)); + final long primaryTerm = randomLongBetween(1, 200); + request.setIfPrimaryTerm(primaryTerm); + expectedParams.put("if_primary_term", Long.toString(primaryTerm)); + } + } + public void testUpdateWithType() throws IOException { String index = randomAlphaOfLengthBetween(3, 10); String type = randomAlphaOfLengthBetween(3, 10); @@ -892,10 +912,15 @@ public void testBulk() throws IOException { docWriteRequest.routing(randomAlphaOfLength(10)); } if (randomBoolean()) { - docWriteRequest.version(randomNonNegativeLong()); - } - if (randomBoolean()) { - docWriteRequest.versionType(randomFrom(VersionType.values())); + if (randomBoolean()) { + docWriteRequest.version(randomNonNegativeLong()); + } + if (randomBoolean()) { + docWriteRequest.versionType(randomFrom(VersionType.values())); + } + } else if (randomBoolean()) { + docWriteRequest.setIfSeqNo(randomNonNegativeLong()); + docWriteRequest.setIfPrimaryTerm(randomLongBetween(1, 200)); } bulkRequest.add(docWriteRequest); } @@ -925,6 +950,8 @@ public void testBulk() throws IOException { assertEquals(originalRequest.routing(), parsedRequest.routing()); assertEquals(originalRequest.version(), parsedRequest.version()); assertEquals(originalRequest.versionType(), parsedRequest.versionType()); + assertEquals(originalRequest.ifSeqNo(), parsedRequest.ifSeqNo()); + assertEquals(originalRequest.ifPrimaryTerm(), parsedRequest.ifPrimaryTerm()); DocWriteRequest.OpType opType = originalRequest.opType(); if (opType == DocWriteRequest.OpType.INDEX) { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/80_cas.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/80_cas.yml new file mode 100644 index 0000000000000..902621cfba578 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/80_cas.yml @@ -0,0 +1,42 @@ +--- +"Compare And Swap Sequence Numbers": + + - skip: + version: " - 6.99.99" + reason: typeless API are add in 7.0.0 + + - do: + index: + index: test_1 + id: 1 + body: { foo: bar } + - match: { _version: 1} + - set: { _seq_no: seqno } + - set: { _primary_term: primary_term } + + - do: + bulk: + body: + - index: + _index: test_1 + _id: 1 + if_seq_no: 10000 + if_primary_term: $primary_term + - foo: bar2 + + - match: { errors: true } + - match: { items.0.index.status: 409 } + - match: { items.0.index.error.type: version_conflict_engine_exception } + + - do: + bulk: + body: + - index: + _index: test_1 + _id: 1 + if_seq_no: $seqno + if_primary_term: $primary_term + - foo: bar2 + + - match: { errors: false} + - match: { items.0.index.status: 200 } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/81_cas_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/81_cas_with_types.yml new file mode 100644 index 0000000000000..101316e7bf504 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/81_cas_with_types.yml @@ -0,0 +1,45 @@ +--- +"Compare And Swap Sequence Numbers": + + - skip: + version: " - 6.6.99" + reason: cas operations with sequence numbers was added in 6.7 + + - do: + index: + index: test_1 + type: _doc + id: 1 + body: { foo: bar } + - match: { _version: 1} + - set: { _seq_no: seqno } + - set: { _primary_term: primary_term } + + - do: + bulk: + body: + - index: + _index: test_1 + _type: _doc + _id: 1 + if_seq_no: 10000 + if_primary_term: $primary_term + - foo: bar2 + + - match: { errors: true } + - match: { items.0.index.status: 409 } + - match: { items.0.index.error.type: version_conflict_engine_exception } + + - do: + bulk: + body: + - index: + _index: test_1 + _type: _doc + _id: 1 + if_seq_no: $seqno + if_primary_term: $primary_term + - foo: bar2 + + - match: { errors: false} + - match: { items.0.index.status: 200 } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml index a43ec1437a50b..550582e9816eb 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml @@ -3,7 +3,7 @@ - skip: version: " - 6.99.99" - reason: cas ops are introduced in 7.0.0 + reason: typesless api was introduces in 7.0 - do: index: From b7ab521eb1e78e7bbc84738c9c6c5cda85cbbfb0 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 5 Feb 2019 17:11:20 +0000 Subject: [PATCH 11/23] Throw AssertionError when no master (#38432) Today we throw a fatal `RuntimeException` if an exception occurs in `getMasterName()`, and this includes the case where there is currently no master. However, sometimes we call this method inside an `assertBusy()` in order to allow for a cluster that is in the process of stabilising and electing a master. The trouble is that `assertBusy()` only retries on an `AssertionError` and not on a general `RuntimeException`, so the lack of a master is immediately fatal. This commit fixes the issue by asserting there is a master, triggering a retry if there is not. Fixes #38331 --- .../org/elasticsearch/cluster/SpecificMasterNodesIT.java | 1 - .../java/org/elasticsearch/test/InternalTestCluster.java | 6 ++++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/SpecificMasterNodesIT.java b/server/src/test/java/org/elasticsearch/cluster/SpecificMasterNodesIT.java index fb1a7b834d8af..071c8a0195531 100644 --- a/server/src/test/java/org/elasticsearch/cluster/SpecificMasterNodesIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/SpecificMasterNodesIT.java @@ -84,7 +84,6 @@ public void testSimpleOnlyMasterNodeElection() throws IOException { .execute().actionGet().getState().nodes().getMasterNode().getName(), equalTo(nextMasterEligibleNodeName)); } - @AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/38331") public void testElectOnlyBetweenMasterNodes() throws Exception { internalCluster().setBootstrapMasterNodeIndex(0); logger.info("--> start data node / non master node"); diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 06f3878c1e5c2..12902b7017767 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -167,6 +167,7 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -1909,8 +1910,9 @@ public String getMasterName() { public String getMasterName(@Nullable String viaNode) { try { Client client = viaNode != null ? client(viaNode) : client(); - ClusterState state = client.admin().cluster().prepareState().execute().actionGet().getState(); - return state.nodes().getMasterNode().getName(); + final DiscoveryNode masterNode = client.admin().cluster().prepareState().get().getState().nodes().getMasterNode(); + assertNotNull(masterNode); + return masterNode.getName(); } catch (Exception e) { logger.warn("Can't fetch cluster state", e); throw new RuntimeException("Can't get master node " + e.getMessage(), e); From 963b474f2f2ab2edfbbb22c1eedebe7c841cd5ae Mon Sep 17 00:00:00 2001 From: Przemyslaw Gomulka Date: Tue, 5 Feb 2019 18:27:24 +0100 Subject: [PATCH 12/23] Fix the clock resolution to millis in GetWatchResponseTests (#38405) the clock resolution changed from jdk8->jdk10, hence the test is passing in jdk8 but failing in jdk10. The Watcher's objects are serialised and deserialised with milliseconds precision, making test to fail in jdk 10 and higher closes #38400 --- .../xpack/watcher/GetWatchResponseTests.java | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/protocol/xpack/watcher/GetWatchResponseTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/protocol/xpack/watcher/GetWatchResponseTests.java index 52ac74acc309b..9b71079a6e5e7 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/protocol/xpack/watcher/GetWatchResponseTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/protocol/xpack/watcher/GetWatchResponseTests.java @@ -23,6 +23,8 @@ import java.io.IOException; import java.io.InputStream; +import java.time.Clock; +import java.time.Instant; import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.util.Collections; @@ -124,15 +126,15 @@ private static BytesReference simpleWatch() { private static WatchStatus randomWatchStatus() { long version = randomLongBetween(-1, Long.MAX_VALUE); - WatchStatus.State state = new WatchStatus.State(randomBoolean(), ZonedDateTime.now(ZoneOffset.UTC)); + WatchStatus.State state = new WatchStatus.State(randomBoolean(), nowWithMillisResolution()); ExecutionState executionState = randomFrom(ExecutionState.values()); - ZonedDateTime lastChecked = rarely() ? null : ZonedDateTime.now(ZoneOffset.UTC); - ZonedDateTime lastMetCondition = rarely() ? null : ZonedDateTime.now(ZoneOffset.UTC); + ZonedDateTime lastChecked = rarely() ? null : nowWithMillisResolution(); + ZonedDateTime lastMetCondition = rarely() ? null : nowWithMillisResolution(); int size = randomIntBetween(0, 5); Map actionMap = new HashMap<>(); for (int i = 0; i < size; i++) { ActionStatus.AckStatus ack = new ActionStatus.AckStatus( - ZonedDateTime.now(ZoneOffset.UTC), + nowWithMillisResolution(), randomFrom(ActionStatus.AckStatus.State.values()) ); ActionStatus actionStatus = new ActionStatus( @@ -152,16 +154,16 @@ private static WatchStatus randomWatchStatus() { } private static ActionStatus.Throttle randomThrottle() { - return new ActionStatus.Throttle(ZonedDateTime.now(ZoneOffset.UTC), randomAlphaOfLengthBetween(10, 20)); + return new ActionStatus.Throttle(nowWithMillisResolution(), randomAlphaOfLengthBetween(10, 20)); } private static ActionStatus.Execution randomExecution() { if (randomBoolean()) { return null; } else if (randomBoolean()) { - return ActionStatus.Execution.failure(ZonedDateTime.now(ZoneOffset.UTC), randomAlphaOfLengthBetween(10, 20)); + return ActionStatus.Execution.failure(nowWithMillisResolution(), randomAlphaOfLengthBetween(10, 20)); } else { - return ActionStatus.Execution.successful(ZonedDateTime.now(ZoneOffset.UTC)); + return ActionStatus.Execution.successful(nowWithMillisResolution()); } } @@ -227,4 +229,8 @@ private static ActionStatus.Execution convertHlrcToInternal(org.elasticsearch.cl private static ActionStatus.Throttle convertHlrcToInternal(org.elasticsearch.client.watcher.ActionStatus.Throttle throttle) { return new ActionStatus.Throttle(throttle.timestamp(), throttle.reason()); } + + private static ZonedDateTime nowWithMillisResolution() { + return Instant.ofEpochMilli(Clock.systemUTC().millis()).atZone(ZoneOffset.UTC); + } } From f2dd5dd6eb6cbaffe86967875faaf8a0148b9e4e Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 5 Feb 2019 17:42:24 +0000 Subject: [PATCH 13/23] Remove DiscoveryPlugin#getDiscoveryTypes (#38414) With this change we no longer support pluggable discovery implementations. No known implementations of `DiscoveryPlugin` actually override this method, so in practice this should have no effect on the wider world. However, we were using this rather extensively in tests to provide the `test-zen` discovery type. We no longer need a separate discovery type for tests as we no longer need to customise its behaviour. Relates #38410 --- .../migration/migrate_7_0/plugins.asciidoc | 5 + .../rest/discovery/Zen2RestApiIT.java | 6 - .../discovery/DiscoveryModule.java | 15 +- .../plugins/DiscoveryPlugin.java | 44 +----- .../master/IndexingMasterFailoverIT.java | 7 - .../client/transport/TransportClientIT.java | 5 +- .../coordination/RareClusterStateIT.java | 7 - .../cluster/coordination/Zen1IT.java | 8 +- .../cluster/routing/PrimaryAllocationIT.java | 7 - .../discovery/AbstractDisruptionTestCase.java | 19 +-- .../discovery/DiscoveryModuleTests.java | 35 ----- .../discovery/SnapshotDisruptionIT.java | 8 +- .../elasticsearch/test/ESIntegTestCase.java | 15 -- .../test/ESSingleNodeTestCase.java | 6 - .../org/elasticsearch/test/ESTestCase.java | 11 -- .../test/InternalTestCluster.java | 22 +-- .../test/discovery/TestZenDiscovery.java | 133 ------------------ .../test/test/InternalTestClusterTests.java | 3 +- .../elasticsearch/xpack/CcrIntegTestCase.java | 3 +- .../ml/integration/NetworkDisruptionIT.java | 9 -- .../xpack/ml/support/BaseMlIntegTestCase.java | 3 +- .../elasticsearch/license/LicensingTests.java | 19 +-- ...ServerTransportFilterIntegrationTests.java | 27 +--- 23 files changed, 44 insertions(+), 373 deletions(-) delete mode 100644 test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java diff --git a/docs/reference/migration/migrate_7_0/plugins.asciidoc b/docs/reference/migration/migrate_7_0/plugins.asciidoc index 7e65a9fb41cf2..5641d412a1407 100644 --- a/docs/reference/migration/migrate_7_0/plugins.asciidoc +++ b/docs/reference/migration/migrate_7_0/plugins.asciidoc @@ -70,3 +70,8 @@ The `RealmSettings.simpleString` method can be used as a convenience for the abo Tribe node functionality has been removed in favor of <>. +[float] +==== Discovery implementations are no longer pluggable + +* The method `DiscoveryPlugin#getDiscoveryTypes()` was removed, so that plugins + can no longer provide their own discovery implementations. diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java index f26b02696e7e5..5cf96419f55bc 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java @@ -36,7 +36,6 @@ import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalTestCluster; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.hamcrest.Matchers; import java.io.IOException; @@ -51,11 +50,6 @@ @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, transportClientRatio = 0, autoMinMasterNodes = false) public class Zen2RestApiIT extends ESNetty4IntegTestCase { - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(TestZenDiscovery.USE_ZEN2.getKey(), true).build(); - } - @Override protected boolean addMockHttpTransport() { return false; // enable http diff --git a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java index 5db6cd3ee0fc3..b6639a9c25ea9 100644 --- a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java @@ -55,6 +55,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Random; import java.util.Set; import java.util.function.BiConsumer; import java.util.function.Function; @@ -69,8 +70,8 @@ public class DiscoveryModule { private static final Logger logger = LogManager.getLogger(DiscoveryModule.class); - public static final String ZEN_DISCOVERY_TYPE = "zen"; - public static final String ZEN2_DISCOVERY_TYPE = "zen2"; + public static final String ZEN_DISCOVERY_TYPE = "legacy-zen"; + public static final String ZEN2_DISCOVERY_TYPE = "zen"; public static final Setting DISCOVERY_TYPE_SETTING = new Setting<>("discovery.type", ZEN2_DISCOVERY_TYPE, Function.identity(), Property.NodeScope); @@ -136,17 +137,9 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic discoveryTypes.put(ZEN2_DISCOVERY_TYPE, () -> new Coordinator(NODE_NAME_SETTING.get(settings), settings, clusterSettings, transportService, namedWriteableRegistry, allocationService, masterService, () -> gatewayMetaState.getPersistedState(settings, (ClusterApplierService) clusterApplier), hostsProvider, clusterApplier, - joinValidators, Randomness.get())); + joinValidators, new Random(Randomness.get().nextLong()))); discoveryTypes.put("single-node", () -> new SingleNodeDiscovery(settings, transportService, masterService, clusterApplier, gatewayMetaState)); - for (DiscoveryPlugin plugin : plugins) { - plugin.getDiscoveryTypes(threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier, clusterSettings, - hostsProvider, allocationService, gatewayMetaState).forEach((key, value) -> { - if (discoveryTypes.put(key, value) != null) { - throw new IllegalArgumentException("Cannot register discovery type [" + key + "] twice"); - } - }); - } String discoveryType = DISCOVERY_TYPE_SETTING.get(settings); Supplier discoverySupplier = discoveryTypes.get(discoveryType); if (discoverySupplier == null) { diff --git a/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java b/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java index 994607a9c97f7..5886a9d773e84 100644 --- a/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java +++ b/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java @@ -19,26 +19,18 @@ package org.elasticsearch.plugins; -import java.util.Collections; -import java.util.Map; -import java.util.function.BiConsumer; -import java.util.function.Supplier; - import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.service.ClusterApplier; -import org.elasticsearch.cluster.service.MasterService; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkService; -import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.zen.UnicastHostsProvider; -import org.elasticsearch.gateway.GatewayMetaState; -import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.util.Collections; +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.Supplier; + /** * An additional extension point for {@link Plugin}s that extends Elasticsearch's discovery functionality. To add an additional * {@link NetworkService.CustomNameResolver} just implement the interface and implement the {@link #getCustomNameResolver(Settings)} method: @@ -53,32 +45,6 @@ * } */ public interface DiscoveryPlugin { - - /** - * Returns custom discovery implementations added by this plugin. - * - * The key of the returned map is the name of the discovery implementation - * (see {@link org.elasticsearch.discovery.DiscoveryModule#DISCOVERY_TYPE_SETTING}, and - * the value is a supplier to construct the {@link Discovery}. - * - * @param threadPool Use to schedule ping actions - * @param transportService Use to communicate with other nodes - * @param masterService Use to submit cluster state update tasks - * @param clusterApplier Use to locally apply cluster state updates - * @param clusterSettings Use to get cluster settings - * @param hostsProvider Use to find configured hosts which should be pinged for initial discovery - */ - default Map> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService, - NamedWriteableRegistry namedWriteableRegistry, - MasterService masterService, - ClusterApplier clusterApplier, - ClusterSettings clusterSettings, - UnicastHostsProvider hostsProvider, - AllocationService allocationService, - GatewayMetaState gatewayMetaState) { - return Collections.emptyMap(); - } - /** * Override to add additional {@link NetworkService.CustomNameResolver}s. * This can be handy if you want to provide your own Network interface name like _mycard_ diff --git a/server/src/test/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java b/server/src/test/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java index 4068d36ec40d9..1317183f286b3 100644 --- a/server/src/test/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java +++ b/server/src/test/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java @@ -23,7 +23,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.disruption.NetworkDisruption; import org.elasticsearch.test.disruption.NetworkDisruption.NetworkDisconnect; import org.elasticsearch.test.disruption.NetworkDisruption.TwoPartitions; @@ -50,12 +49,6 @@ protected Collection> nodePlugins() { return classes; } - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false).build(); - } - /** * Indexing operations which entail mapping changes require a blocking request to the master node to update the mapping. * If the master node is being disrupted or if it cannot commit cluster state changes, it needs to retry within timeout limits. diff --git a/server/src/test/java/org/elasticsearch/client/transport/TransportClientIT.java b/server/src/test/java/org/elasticsearch/client/transport/TransportClientIT.java index 709af1dfe3ca4..dab44b37a3ee9 100644 --- a/server/src/test/java/org/elasticsearch/client/transport/TransportClientIT.java +++ b/server/src/test/java/org/elasticsearch/client/transport/TransportClientIT.java @@ -34,7 +34,6 @@ import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.MockHttpTransport; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.transport.MockTransportClient; import org.elasticsearch.transport.TransportService; @@ -66,10 +65,8 @@ public void testNodeVersionIsUpdated() throws IOException, NodeValidationExcepti .put("transport.type", getTestTransportType()) .put(Node.NODE_DATA_SETTING.getKey(), false) .put("cluster.name", "foobar") - .put(TestZenDiscovery.USE_ZEN2.getKey(), getUseZen2()) .putList(ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING.getKey(), "testNodeVersionIsUpdated") - .build(), Arrays.asList(getTestTransportPlugin(), TestZenDiscovery.TestPlugin.class, - MockHttpTransport.TestPlugin.class)).start()) { + .build(), Arrays.asList(getTestTransportPlugin(), MockHttpTransport.TestPlugin.class)).start()) { TransportAddress transportAddress = node.injector().getInstance(TransportService.class).boundAddress().publishAddress(); client.addTransportAddress(transportAddress); // since we force transport clients there has to be one node started that we connect to. diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/RareClusterStateIT.java b/server/src/test/java/org/elasticsearch/cluster/coordination/RareClusterStateIT.java index f072fd4fb9c63..ccde363fdc59a 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/RareClusterStateIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/RareClusterStateIT.java @@ -49,7 +49,6 @@ import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.disruption.BlockClusterStateProcessing; import org.elasticsearch.test.junit.annotations.TestLogging; @@ -71,12 +70,6 @@ @TestLogging("_root:DEBUG") public class RareClusterStateIT extends ESIntegTestCase { - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false).build(); - } - @Override protected int numberOfShards() { return 1; diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/Zen1IT.java b/server/src/test/java/org/elasticsearch/cluster/coordination/Zen1IT.java index 6ac753b5bc6d5..3ba9f58523d16 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/Zen1IT.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/Zen1IT.java @@ -37,13 +37,13 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.discovery.Discovery; +import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.gateway.MetaStateService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalTestCluster.RestartCallback; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.TransportService; @@ -71,12 +71,10 @@ public class Zen1IT extends ESIntegTestCase { private static Settings ZEN1_SETTINGS = Coordinator.addZen1Attribute(true, Settings.builder() - .put(TestZenDiscovery.USE_ZEN2.getKey(), false) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false)) // Zen2 does not know about mock pings - .build(); + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), DiscoveryModule.ZEN_DISCOVERY_TYPE)).build(); private static Settings ZEN2_SETTINGS = Settings.builder() - .put(TestZenDiscovery.USE_ZEN2.getKey(), true) + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), DiscoveryModule.ZEN2_DISCOVERY_TYPE) .build(); protected Collection> nodePlugins() { diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index a64f509363854..9a7e25d29bb08 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -47,7 +47,6 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalTestCluster; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.disruption.NetworkDisruption; import org.elasticsearch.test.disruption.NetworkDisruption.NetworkDisconnect; import org.elasticsearch.test.disruption.NetworkDisruption.TwoPartitions; @@ -88,12 +87,6 @@ protected Collection> nodePlugins() { return Arrays.asList(MockTransportService.TestPlugin.class); } - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false).build(); - } - public void testBulkWeirdScenario() throws Exception { String master = internalCluster().startMasterOnlyNode(Settings.EMPTY); internalCluster().startDataOnlyNodes(2); diff --git a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java index 825a4203ef9a0..97ba76b822020 100644 --- a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java +++ b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java @@ -30,12 +30,9 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.discovery.zen.UnicastZenPing; -import org.elasticsearch.discovery.zen.ZenPing; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalTestCluster; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.disruption.NetworkDisruption; import org.elasticsearch.test.disruption.NetworkDisruption.Bridge; import org.elasticsearch.test.disruption.NetworkDisruption.DisruptedLinks; @@ -65,8 +62,7 @@ public abstract class AbstractDisruptionTestCase extends ESIntegTestCase { @Override protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(DEFAULT_SETTINGS) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false).build(); + return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(DEFAULT_SETTINGS).build(); } @Override @@ -114,22 +110,9 @@ List startCluster(int numberOfNodes) { InternalTestCluster internalCluster = internalCluster(); List nodes = internalCluster.startNodes(numberOfNodes); ensureStableCluster(numberOfNodes); - - // TODO: this is a temporary solution so that nodes will not base their reaction to a partition based on previous successful results - clearTemporalResponses(); return nodes; } - protected void clearTemporalResponses() { - final Discovery discovery = internalCluster().getInstance(Discovery.class); - if (discovery instanceof TestZenDiscovery) { - ZenPing zenPing = ((TestZenDiscovery) discovery).getZenPing(); - if (zenPing instanceof UnicastZenPing) { - ((UnicastZenPing) zenPing).clearTemporalResponses(); - } - } - } - static final Settings DEFAULT_SETTINGS = Settings.builder() .put(LeaderChecker.LEADER_CHECK_TIMEOUT_SETTING.getKey(), "1s") // for hitting simulated network failures quickly .put(LeaderChecker.LEADER_CHECK_RETRY_COUNT_SETTING.getKey(), 1) // for hitting simulated network failures quickly diff --git a/server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java b/server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java index f44f33e71ee06..285bb7b1fd109 100644 --- a/server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java @@ -22,7 +22,6 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.coordination.Coordinator; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.ClusterApplier; import org.elasticsearch.cluster.service.MasterService; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; @@ -36,7 +35,6 @@ import org.elasticsearch.gateway.GatewayMetaState; import org.elasticsearch.plugins.DiscoveryPlugin; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.NoopDiscovery; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -75,18 +73,6 @@ default Map> getZenHostsProviders(Transpo } } - public interface DummyDiscoveryPlugin extends DiscoveryPlugin { - Map> impl(); - @Override - default Map> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService, - NamedWriteableRegistry namedWriteableRegistry, - MasterService masterService, ClusterApplier clusterApplier, - ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider, - AllocationService allocationService, GatewayMetaState gatewayMetaState) { - return impl(); - } - } - @Before public void setupDummyServices() { threadPool = mock(ThreadPool.class); @@ -114,19 +100,6 @@ public void testDefaults() { assertTrue(module.getDiscovery() instanceof Coordinator); } - public void testLazyConstructionDiscovery() { - DummyDiscoveryPlugin plugin = () -> Collections.singletonMap("custom", - () -> { throw new AssertionError("created discovery type which was not selected"); }); - newModule(Settings.EMPTY, Collections.singletonList(plugin)); - } - - public void testRegisterDiscovery() { - Settings settings = Settings.builder().put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "custom").build(); - DummyDiscoveryPlugin plugin = () -> Collections.singletonMap("custom", NoopDiscovery::new); - DiscoveryModule module = newModule(settings, Collections.singletonList(plugin)); - assertTrue(module.getDiscovery() instanceof NoopDiscovery); - } - public void testUnknownDiscovery() { Settings settings = Settings.builder().put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "dne").build(); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> @@ -134,14 +107,6 @@ public void testUnknownDiscovery() { assertEquals("Unknown discovery type [dne]", e.getMessage()); } - public void testDuplicateDiscovery() { - DummyDiscoveryPlugin plugin1 = () -> Collections.singletonMap("dup", () -> null); - DummyDiscoveryPlugin plugin2 = () -> Collections.singletonMap("dup", () -> null); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> - newModule(Settings.EMPTY, Arrays.asList(plugin1, plugin2))); - assertEquals("Cannot register discovery type [dup] twice", e.getMessage()); - } - public void testHostsProvider() { Settings settings = Settings.builder().put(DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING.getKey(), "custom").build(); AtomicBoolean created = new AtomicBoolean(false); diff --git a/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java index 8548b332c1ad1..b9ac5f33dd911 100644 --- a/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java @@ -18,8 +18,6 @@ */ package org.elasticsearch.discovery; -import java.util.Arrays; -import java.util.Collection; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse; @@ -38,17 +36,18 @@ import org.elasticsearch.snapshots.SnapshotMissingException; import org.elasticsearch.snapshots.SnapshotState; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.disruption.NetworkDisruption; import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.test.transport.MockTransportService; +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import org.elasticsearch.test.transport.MockTransportService; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -68,7 +67,6 @@ protected Collection> nodePlugins() { protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder().put(super.nodeSettings(nodeOrdinal)) .put(AbstractDisruptionTestCase.DEFAULT_SETTINGS) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false) .build(); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 363050fde26cb..4aeebaf05051c 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -140,7 +140,6 @@ import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchService; import org.elasticsearch.test.client.RandomizingClient; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.disruption.NetworkDisruption; import org.elasticsearch.test.disruption.ServiceDisruptionScheme; import org.elasticsearch.test.store.MockFSIndexStore; @@ -1931,9 +1930,6 @@ protected NodeConfigurationSource getNodeConfigSource() { initialNodeSettings.put(NetworkModule.TRANSPORT_TYPE_KEY, getTestTransportType()); initialTransportClientSettings.put(NetworkModule.TRANSPORT_TYPE_KEY, getTestTransportType()); } - if (addTestZenDiscovery() && getUseZen2() == false) { - initialNodeSettings.put(TestZenDiscovery.USE_ZEN2.getKey(), false); - } return new NodeConfigurationSource() { @Override public Settings nodeSettings(int nodeOrdinal) { @@ -1979,14 +1975,6 @@ protected boolean addMockTransportService() { return true; } - /** - * Iff this returns true test zen discovery implementations is used for the test runs. - * The default is {@code true}. - */ - protected boolean addTestZenDiscovery() { - return true; - } - /** Returns {@code true} iff this test cluster should use a dummy http transport */ protected boolean addMockHttpTransport() { return true; @@ -2028,9 +2016,6 @@ protected Collection> getMockPlugins() { if (addMockTransportService()) { mocks.add(getTestTransportPlugin()); } - if (addTestZenDiscovery()) { - mocks.add(TestZenDiscovery.TestPlugin.class); - } if (addMockHttpTransport()) { mocks.add(MockHttpTransport.TestPlugin.class); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java index e2d21043282ae..376264954d731 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java @@ -50,7 +50,6 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.threadpool.ThreadPool; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -193,7 +192,6 @@ private Node newNode() { .put(EsExecutors.PROCESSORS_SETTING.getKey(), 1) // limit the number of threads created .put("transport.type", getTestTransportType()) .put(Node.NODE_DATA_SETTING.getKey(), true) - .put(TestZenDiscovery.USE_ZEN2.getKey(), getUseZen2()) .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) // default the watermarks low values to prevent tests from failing on nodes without enough disk space .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "1b") @@ -212,10 +210,6 @@ private Node newNode() { plugins = new ArrayList<>(plugins); plugins.add(getTestTransportPlugin()); } - if (plugins.contains(TestZenDiscovery.TestPlugin.class) == false) { - plugins = new ArrayList<>(plugins); - plugins.add(TestZenDiscovery.TestPlugin.class); - } if (addMockHttpTransport()) { plugins.add(MockHttpTransport.TestPlugin.class); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index 235cf2ad24fb9..a36018921e9f4 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -1006,17 +1006,6 @@ public static String randomGeohash(int minPrecision, int maxPrecision) { return geohashGenerator.ofStringLength(random(), minPrecision, maxPrecision); } - private static boolean useZen2; - - @BeforeClass - public static void setUseZen2() { - useZen2 = true; - } - - protected static boolean getUseZen2() { - return useZen2; - } - public static String getTestTransportType() { return MockNioTransportPlugin.MOCK_NIO_TRANSPORT_NAME; } diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 12902b7017767..a1a3865dcdebe 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -76,7 +76,6 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.env.Environment; @@ -106,7 +105,6 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.SearchService; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.disruption.ServiceDisruptionScheme; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.MockTransportClient; @@ -151,6 +149,9 @@ import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING; import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; +import static org.elasticsearch.discovery.DiscoveryModule.DISCOVERY_TYPE_SETTING; +import static org.elasticsearch.discovery.DiscoveryModule.ZEN2_DISCOVERY_TYPE; +import static org.elasticsearch.discovery.DiscoveryModule.ZEN_DISCOVERY_TYPE; import static org.elasticsearch.discovery.DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING; import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING; import static org.elasticsearch.discovery.zen.FileBasedUnicastHostsProvider.UNICAST_HOSTS_FILE; @@ -158,8 +159,6 @@ import static org.elasticsearch.test.ESTestCase.awaitBusy; import static org.elasticsearch.test.ESTestCase.getTestTransportType; import static org.elasticsearch.test.ESTestCase.randomFrom; -import static org.elasticsearch.test.discovery.TestZenDiscovery.USE_ZEN2; -import static org.elasticsearch.test.discovery.TestZenDiscovery.usingZen1; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -401,6 +400,10 @@ public InternalTestCluster( EsExecutors.daemonThreadFactory("test_" + clusterName), new ThreadContext(Settings.EMPTY)); } + private static boolean usingZen1(Settings settings) { + return ZEN_DISCOVERY_TYPE.equals(DISCOVERY_TYPE_SETTING.get(settings)); + } + public int getBootstrapMasterNodeIndex() { return bootstrapMasterNodeIndex; } @@ -636,9 +639,9 @@ private Settings getNodeSettings(final int nodeId, final long seed, final Settin .put("node.name", name) .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), seed); - final String discoveryType = DiscoveryModule.DISCOVERY_TYPE_SETTING.get(updatedSettings.build()); + final String discoveryType = DISCOVERY_TYPE_SETTING.get(updatedSettings.build()); final boolean usingSingleNodeDiscovery = discoveryType.equals("single-node"); - final boolean usingZen1 = TestZenDiscovery.usingZen1(updatedSettings.build()); + final boolean usingZen1 = usingZen1(updatedSettings.build()); if (usingSingleNodeDiscovery == false) { if (autoManageMinMasterNodes) { assertThat("min master nodes may not be set when auto managed", @@ -1009,8 +1012,8 @@ private void recreateNode(final Settings newSettings, final Runnable onTransport if (DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(finalSettings)) { // simulating an upgrade from Zen1 to Zen2, but there's no way to remove a setting when restarting a node, so // you have to set it to REMOVED_MINIMUM_MASTER_NODES (== Integer.MAX_VALUE) to indicate its removal: - assertTrue(USE_ZEN2.exists(finalSettings)); - assertTrue(USE_ZEN2.get(finalSettings)); + assertTrue(DISCOVERY_TYPE_SETTING.exists(finalSettings)); + assertThat(DISCOVERY_TYPE_SETTING.get(finalSettings), equalTo(ZEN2_DISCOVERY_TYPE)); assertThat(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(finalSettings), equalTo(REMOVED_MINIMUM_MASTER_NODES)); final Builder builder = Settings.builder().put(finalSettings); @@ -2057,7 +2060,8 @@ public synchronized List startNodes(Settings... extraSettings) { final int prevMasterCount = getMasterNodesCount(); int autoBootstrapMasterNodeIndex = prevMasterCount == 0 && autoManageMinMasterNodes && newMasterCount > 0 && Arrays.stream(extraSettings) - .allMatch(s -> Node.NODE_MASTER_SETTING.get(s) == false || TestZenDiscovery.USE_ZEN2.get(s) == true) + .allMatch(s -> Node.NODE_MASTER_SETTING.get(s) == false + || ZEN2_DISCOVERY_TYPE.equals(DISCOVERY_TYPE_SETTING.get(s))) ? RandomNumbers.randomIntBetween(random, 0, newMasterCount - 1) : -1; final int numOfNodes = extraSettings.length; diff --git a/test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java b/test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java deleted file mode 100644 index 43976b9dfc76f..0000000000000 --- a/test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java +++ /dev/null @@ -1,133 +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.test.discovery; - -import org.elasticsearch.cluster.coordination.Coordinator; -import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.service.ClusterApplier; -import org.elasticsearch.cluster.service.ClusterApplierService; -import org.elasticsearch.cluster.service.MasterService; -import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.settings.ClusterSettings; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.Discovery; -import org.elasticsearch.discovery.DiscoveryModule; -import org.elasticsearch.discovery.zen.UnicastHostsProvider; -import org.elasticsearch.discovery.zen.ZenDiscovery; -import org.elasticsearch.discovery.zen.ZenPing; -import org.elasticsearch.gateway.GatewayMetaState; -import org.elasticsearch.plugins.DiscoveryPlugin; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportService; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.function.Supplier; - -import static org.elasticsearch.discovery.zen.SettingsBasedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING; - -/** - * A alternative zen discovery which allows using mocks for things like pings, as well as - * giving access to internals. - */ -public class TestZenDiscovery extends ZenDiscovery { - - public static final Setting USE_MOCK_PINGS = - Setting.boolSetting("discovery.zen.use_mock_pings", true, Setting.Property.NodeScope); - - public static final Setting USE_ZEN2 = - Setting.boolSetting("discovery.zen.use_zen2", true, Setting.Property.NodeScope); - private static final String TEST_ZEN_DISCOVERY_TYPE = "test-zen"; - - /** A plugin which installs mock discovery and configures it to be used. */ - public static class TestPlugin extends Plugin implements DiscoveryPlugin { - protected final Settings settings; - public TestPlugin(Settings settings) { - this.settings = settings; - } - - @Override - public Map> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService, - NamedWriteableRegistry namedWriteableRegistry, - MasterService masterService, ClusterApplier clusterApplier, - ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider, - AllocationService allocationService, GatewayMetaState gatewayMetaState) { - // we don't get the latest setting which were updated by the extra settings for the plugin. TODO: fix. - Settings fixedSettings = Settings.builder().put(settings).putList(DISCOVERY_SEED_HOSTS_SETTING.getKey()).build(); - return Collections.singletonMap("test-zen", () -> { - if (USE_ZEN2.get(settings)) { - return new Coordinator("test_node", fixedSettings, clusterSettings, transportService, namedWriteableRegistry, - allocationService, masterService, - () -> gatewayMetaState.getPersistedState(settings, (ClusterApplierService) clusterApplier), hostsProvider, - clusterApplier, Collections.emptyList(), new Random(Randomness.get().nextLong())); - } else { - return new TestZenDiscovery(fixedSettings, threadPool, transportService, namedWriteableRegistry, masterService, - clusterApplier, clusterSettings, hostsProvider, allocationService, gatewayMetaState); - } - }); - } - - @Override - public List> getSettings() { - return Arrays.asList(USE_MOCK_PINGS, USE_ZEN2); - } - - @Override - public Settings additionalSettings() { - return Settings.builder() - .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), TEST_ZEN_DISCOVERY_TYPE) - .putList(DISCOVERY_SEED_HOSTS_SETTING.getKey()) - .build(); - } - } - - private TestZenDiscovery(Settings settings, ThreadPool threadPool, TransportService transportService, - NamedWriteableRegistry namedWriteableRegistry, MasterService masterService, - ClusterApplier clusterApplier, ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider, - AllocationService allocationService, GatewayMetaState gatewayMetaState) { - super(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier, clusterSettings, - hostsProvider, allocationService, Collections.emptyList(), gatewayMetaState); - } - - @Override - protected ZenPing newZenPing(Settings settings, ThreadPool threadPool, TransportService transportService, - UnicastHostsProvider hostsProvider) { - if (USE_MOCK_PINGS.get(settings) && USE_ZEN2.get(settings) == false) { - return new MockZenPing(this); - } else { - return super.newZenPing(settings, threadPool, transportService, hostsProvider); - } - } - - public ZenPing getZenPing() { - return zenPing; - } - - public static boolean usingZen1(Settings settings) { - return DiscoveryModule.ZEN_DISCOVERY_TYPE.equals(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings)) - || USE_ZEN2.get(settings) == false; - } -} diff --git a/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java b/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java index f1028322754a5..1aaaf6aa90a59 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java @@ -35,7 +35,6 @@ import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.MockHttpTransport; import org.elasticsearch.test.NodeConfigurationSource; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.transport.TransportSettings; import java.io.IOException; @@ -72,7 +71,7 @@ public class InternalTestClusterTests extends ESTestCase { private static Collection> mockPlugins() { - return Arrays.asList(getTestTransportPlugin(), TestZenDiscovery.TestPlugin.class, MockHttpTransport.TestPlugin.class); + return Arrays.asList(getTestTransportPlugin(), MockHttpTransport.TestPlugin.class); } public void testInitializiationIsConsistent() { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java index 48ea50af9990b..8dd70a23cf8da 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java @@ -64,7 +64,6 @@ import org.elasticsearch.test.MockHttpTransport; import org.elasticsearch.test.NodeConfigurationSource; import org.elasticsearch.test.TestCluster; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.nio.MockNioTransportPlugin; @@ -121,7 +120,7 @@ public final void startClusters() throws Exception { stopClusters(); Collection> mockPlugins = Arrays.asList(ESIntegTestCase.TestSeedPlugin.class, - TestZenDiscovery.TestPlugin.class, MockHttpTransport.TestPlugin.class, MockTransportService.TestPlugin.class, + MockHttpTransport.TestPlugin.class, MockTransportService.TestPlugin.class, MockNioTransportPlugin.class); InternalTestCluster leaderCluster = new InternalTestCluster(randomLong(), createTempDir(), true, true, numberOfNodesPerCluster(), diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/NetworkDisruptionIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/NetworkDisruptionIT.java index 5429bddc437c4..8a257baa3d628 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/NetworkDisruptionIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/NetworkDisruptionIT.java @@ -8,12 +8,10 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.AcknowledgedResponse; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.disruption.NetworkDisruption; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.xpack.core.ml.action.CloseJobAction; @@ -33,13 +31,6 @@ public class NetworkDisruptionIT extends BaseMlIntegTestCase { - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false) - .build(); - } - @Override protected Collection> nodePlugins() { Collection> plugins = new ArrayList<>(super.nodePlugins()); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/support/BaseMlIntegTestCase.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/support/BaseMlIntegTestCase.java index e5627fdd41b81..a95f341ed1512 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/support/BaseMlIntegTestCase.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/support/BaseMlIntegTestCase.java @@ -26,7 +26,6 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.MockHttpTransport; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.ml.MachineLearningField; import org.elasticsearch.xpack.core.ml.action.CloseJobAction; @@ -114,7 +113,7 @@ protected Collection> transportClientPlugins() { @Override protected Collection> getMockPlugins() { - return Arrays.asList(TestZenDiscovery.TestPlugin.class, TestSeedPlugin.class, MockHttpTransport.TestPlugin.class); + return Arrays.asList(TestSeedPlugin.class, MockHttpTransport.TestPlugin.class); } @Before diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java index 3d19568772128..bd78867312348 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.discovery.DiscoveryModule; -import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.node.MockNode; import org.elasticsearch.node.Node; import org.elasticsearch.plugins.Plugin; @@ -35,7 +34,6 @@ import org.elasticsearch.test.SecurityIntegTestCase; import org.elasticsearch.test.SecuritySettingsSource; import org.elasticsearch.test.SecuritySettingsSourceField; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.transport.Netty4Plugin; import org.elasticsearch.transport.Transport; @@ -132,13 +130,6 @@ protected int maxNumberOfNodes() { return super.maxNumberOfNodes() + 1; } - @Override - public Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false) - .build(); - } - @Before public void resetLicensing() { enableLicensing(); @@ -297,18 +288,10 @@ public void testNodeJoinWithoutSecurityExplicitlyEnabled() throws Exception { .put("network.host", "localhost") .put("cluster.name", internalCluster().getClusterName()) .put("path.home", home) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false) - .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "test-zen") - .put(TestZenDiscovery.USE_ZEN2.getKey(), getUseZen2()) .putList(DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING.getKey()) .putList(DISCOVERY_SEED_HOSTS_SETTING.getKey(), unicastHostsList); - if (getUseZen2() == false) { - nodeSettings.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), - ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(internalCluster().getInstance(Settings.class))); - } - Collection> mockPlugins = Arrays.asList(LocalStateSecurity.class, TestZenDiscovery.TestPlugin.class, - MockHttpTransport.TestPlugin.class); + Collection> mockPlugins = Arrays.asList(LocalStateSecurity.class, MockHttpTransport.TestPlugin.class); try (Node node = new MockNode(nodeSettings.build(), mockPlugins)) { node.start(); ensureStableCluster(cluster().size() + 1); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterIntegrationTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterIntegrationTests.java index 5dcb48173969b..53dd508c542a5 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterIntegrationTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterIntegrationTests.java @@ -13,7 +13,6 @@ import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.node.MockNode; import org.elasticsearch.node.Node; import org.elasticsearch.node.NodeValidationException; @@ -22,7 +21,6 @@ import org.elasticsearch.test.SecurityIntegTestCase; import org.elasticsearch.test.SecuritySettingsSource; import org.elasticsearch.test.SecuritySettingsSourceField; -import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ConnectionProfile; import org.elasticsearch.transport.Transport; @@ -77,8 +75,7 @@ protected Settings nodeSettings(int nodeOrdinal) { // make sure this is "localhost", no matter if ipv4 or ipv6, but be consistent .put("transport.profiles.client.bind_host", "localhost") .put("xpack.security.audit.enabled", false) - .put(XPackSettings.WATCHER_ENABLED.getKey(), false) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false); + .put(XPackSettings.WATCHER_ENABLED.getKey(), false); if (randomBoolean()) { settingsBuilder.put("transport.profiles.default.xpack.security.type", "node"); // this is default lets set it randomly } @@ -108,15 +105,8 @@ public void testThatConnectionToServerTypeConnectionWorks() throws IOException, .put("xpack.security.transport.ssl.enabled", true) .put(XPackSettings.WATCHER_ENABLED.getKey(), false) .put("path.home", home) - .put(Node.NODE_MASTER_SETTING.getKey(), false) - .put(TestZenDiscovery.USE_ZEN2.getKey(), getUseZen2()) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false); - if (getUseZen2() == false) { - nodeSettings.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), - ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(internalCluster().getInstance(Settings.class))); - } - Collection> mockPlugins = Arrays.asList( - LocalStateSecurity.class, TestZenDiscovery.TestPlugin.class, MockHttpTransport.TestPlugin.class); + .put(Node.NODE_MASTER_SETTING.getKey(), false); + Collection> mockPlugins = Arrays.asList(LocalStateSecurity.class, MockHttpTransport.TestPlugin.class); addSSLSettingsForPEMFiles( nodeSettings, "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem", @@ -154,15 +144,8 @@ public void testThatConnectionToClientTypeConnectionIsRejected() throws IOExcept .put(XPackSettings.WATCHER_ENABLED.getKey(), false) .put("discovery.initial_state_timeout", "0s") .put("path.home", home) - .put(Node.NODE_MASTER_SETTING.getKey(), false) - .put(TestZenDiscovery.USE_ZEN2.getKey(), getUseZen2()) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false); - if (getUseZen2() == false) { - nodeSettings.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), - ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(internalCluster().getInstance(Settings.class))); - } - Collection> mockPlugins = Arrays.asList( - LocalStateSecurity.class, TestZenDiscovery.TestPlugin.class, MockHttpTransport.TestPlugin.class); + .put(Node.NODE_MASTER_SETTING.getKey(), false); + Collection> mockPlugins = Arrays.asList(LocalStateSecurity.class, MockHttpTransport.TestPlugin.class); addSSLSettingsForPEMFiles( nodeSettings, "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem", From ae47c025e225eec2a75d63c65ff18378124c4722 Mon Sep 17 00:00:00 2001 From: Tal Levy Date: Tue, 5 Feb 2019 09:44:47 -0800 Subject: [PATCH 14/23] add basic REST test for geohash_grid (#37996) --- .../search.aggregation/280_geohash_grid.yml | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/280_geohash_grid.yml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/280_geohash_grid.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/280_geohash_grid.yml new file mode 100644 index 0000000000000..534e552fc0ea2 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/280_geohash_grid.yml @@ -0,0 +1,62 @@ +setup: + - do: + indices.create: + include_type_name: false + index: test_1 + body: + settings: + number_of_replicas: 0 + mappings: + properties: + location: + type: geo_point + +--- +"Basic test": + - do: + bulk: + refresh: true + body: + - index: + _index: test_1 + _id: 1 + - location: "52.374081,4.912350" + - index: + _index: test_1 + _id: 2 + - location: "52.369219,4.901618" + - index: + _index: test_1 + _id: 3 + - location: "52.371667,4.914722" + - index: + _index: test_1 + _id: 4 + - location: "51.222900,4.405200" + - index: + _index: test_1 + _id: 5 + - location: "48.861111,2.336389" + - index: + _index: test_1 + _id: 6 + - location: "48.860000,2.327000" + + - do: + search: + rest_total_hits_as_int: true + body: + aggregations: + grid: + geohash_grid: + field: location + precision: 3 + + + - match: { hits.total: 6 } + - match: { aggregations.grid.buckets.0.key: u17 } + - match: { aggregations.grid.buckets.0.doc_count: 3 } + - match: { aggregations.grid.buckets.1.key: u09 } + - match: { aggregations.grid.buckets.1.doc_count: 2 } + - match: { aggregations.grid.buckets.2.key: u15 } + - match: { aggregations.grid.buckets.2.doc_count: 1 } From 54e684bedde2d863247ffe68a536e525fb706ee7 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Tue, 5 Feb 2019 12:59:05 -0500 Subject: [PATCH 15/23] testHlrcFromXContent() should respect assertToXContentEquivalence() (#38232) Tests can override assertToXContentEquivalence() in case their xcontent cannot be directly compared (e.g. due to insertion order in maps affecting the xcontent ordering). But the `testHlrcFromXContent` test hardcoded the equivalence test to `true` instead of consulting `assertToXContentEquivalence()` Fixes #36034 --- .../elasticsearch/protocol/AbstractHlrcXContentTestCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/protocol/AbstractHlrcXContentTestCase.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/protocol/AbstractHlrcXContentTestCase.java index d6d8f9afe3659..dfe81ab79ce45 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/protocol/AbstractHlrcXContentTestCase.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/protocol/AbstractHlrcXContentTestCase.java @@ -21,7 +21,7 @@ public final void testHlrcFromXContent() throws IOException { AbstractXContentTestCase.testFromXContent(NUMBER_OF_TEST_RUNS, this::createTestInstance, supportsUnknownFields(), getShuffleFieldsExceptions(), getRandomFieldsExcludeFilter(), this::createParser, p -> convertHlrcToInternal(doHlrcParseInstance(p)), - this::assertEqualInstances, true, getToXContentParams()); + this::assertEqualInstances, assertToXContentEquivalence(), getToXContentParams()); } /** From ca47f680912ddf3d67465ad0722b180a8e70d20d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Tue, 5 Feb 2019 19:07:53 +0100 Subject: [PATCH 16/23] Ignore type-removal warnings in XPackRestTestHelper (#38431) The backport of #38022 introduced types-deprecation warning for get/put template requests that cause problems on tests master in mixed cluster scenarios. While these warnings are caught and ignored in regular Rest tests, the get template requests in XPackRestTestHelper were missed. Closes #38412 --- .../elasticsearch/xpack/test/rest/XPackRestTestHelper.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/test/rest/XPackRestTestHelper.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/test/rest/XPackRestTestHelper.java index 082992d95ff87..863908ab7fa72 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/test/rest/XPackRestTestHelper.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/test/rest/XPackRestTestHelper.java @@ -14,6 +14,7 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.xpack.core.ml.MlMetaIndex; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndexFields; @@ -76,7 +77,9 @@ public static void waitForTemplates(RestClient client, List templateName ESTestCase.awaitBusy(() -> { Map response; try { - String string = EntityUtils.toString(client.performRequest(new Request("GET", "/_template/" + template)).getEntity()); + final Request getRequest = new Request("GET", "_template/" + template); + getRequest.setOptions(ESRestTestCase.allowTypeRemovalWarnings()); + String string = EntityUtils.toString(client.performRequest(getRequest).getEntity()); response = XContentHelper.convertToMap(JsonXContent.jsonXContent, string, false); } catch (ResponseException e) { if (e.getResponse().getStatusLine().getStatusCode() == 404) { From 8ebff0512b7a42f67f196f2bfb62ab00a66ea765 Mon Sep 17 00:00:00 2001 From: austintp Date: Tue, 5 Feb 2019 12:37:02 -0600 Subject: [PATCH 17/23] Updates the grok patterns to be consistent with logstash (#27181) --- libs/grok/src/main/resources/patterns/grok-patterns | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/libs/grok/src/main/resources/patterns/grok-patterns b/libs/grok/src/main/resources/patterns/grok-patterns index 6351a7710164e..27bf6732790d6 100644 --- a/libs/grok/src/main/resources/patterns/grok-patterns +++ b/libs/grok/src/main/resources/patterns/grok-patterns @@ -33,21 +33,21 @@ HOSTPORT %{IPORHOST}:%{POSINT} # paths PATH (?:%{UNIXPATH}|%{WINPATH}) -UNIXPATH (/([\w_%!$@:.,~-]+|\\.)*)+ +UNIXPATH (/([\w_%!$@:.,+~-]+|\\.)*)+ TTY (?:/dev/(pts|tty([pq])?)(\w+)?/?(?:[0-9]+)) WINPATH (?>[A-Za-z]+:|\\)(?:\\[^\\?*]*)+ -URIPROTO [A-Za-z]+(\+[A-Za-z+]+)? +URIPROTO [A-Za-z]([A-Za-z0-9+\-.]+)+ URIHOST %{IPORHOST}(?::%{POSINT:port})? # uripath comes loosely from RFC1738, but mostly from what Firefox # doesn't turn into %XX -URIPATH (?:/[A-Za-z0-9$.+!*'(){},~:;=@#%_\-]*)+ +URIPATH (?:/[A-Za-z0-9$.+!*'(){},~:;=@#%&_\-]*)+ #URIPARAM \?(?:[A-Za-z0-9]+(?:=(?:[^&]*))?(?:&(?:[A-Za-z0-9]+(?:=(?:[^&]*))?)?)*)? URIPARAM \?[A-Za-z0-9$.+!*'|(){},~@#%&/=:;_?\-\[\]<>]* URIPATHPARAM %{URIPATH}(?:%{URIPARAM})? URI %{URIPROTO}://(?:%{USER}(?::[^@]*)?@)?(?:%{URIHOST})?(?:%{URIPATHPARAM})? # Months: January, Feb, 3, 03, 12, December -MONTH \b(?:Jan(?:uary|uar)?|Feb(?:ruary|ruar)?|M(?:a|ä)?r(?:ch|z)?|Apr(?:il)?|Ma(?:y|i)?|Jun(?:e|i)?|Jul(?:y)?|Aug(?:ust)?|Sep(?:tember)?|O(?:c|k)?t(?:ober)?|Nov(?:ember)?|De(?:c|z)(?:ember)?)\b +MONTH \b(?:[Jj]an(?:uary|uar)?|[Ff]eb(?:ruary|ruar)?|[Mm](?:a|ä)?r(?:ch|z)?|[Aa]pr(?:il)?|[Mm]a(?:y|i)?|[Jj]un(?:e|i)?|[Jj]ul(?:y)?|[Aa]ug(?:ust)?|[Ss]ep(?:tember)?|[Oo](?:c|k)?t(?:ober)?|[Nn]ov(?:ember)?|[Dd]e(?:c|z)(?:ember)?)\b MONTHNUM (?:0?[1-9]|1[0-2]) MONTHNUM2 (?:0[1-9]|1[0-2]) MONTHDAY (?:(?:0[1-9])|(?:[12][0-9])|(?:3[01])|[1-9]) @@ -71,7 +71,7 @@ ISO8601_HOUR (?:2[0123]|[01][0-9]) TIMESTAMP_ISO8601 %{YEAR}-%{MONTHNUM}-%{MONTHDAY}[T ]%{ISO8601_HOUR}:?%{MINUTE}(?::?%{SECOND})?%{ISO8601_TIMEZONE}? DATE %{DATE_US}|%{DATE_EU} DATESTAMP %{DATE}[- ]%{TIME} -TZ (?:[PMCE][SD]T|UTC) +TZ (?:[APMCE][SD]T|UTC) DATESTAMP_RFC822 %{DAY} %{MONTH} %{MONTHDAY} %{YEAR} %{TIME} %{TZ} DATESTAMP_RFC2822 %{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{TIME} %{ISO8601_TIMEZONE} DATESTAMP_OTHER %{DAY} %{MONTH} %{MONTHDAY} %{TIME} %{TZ} %{YEAR} From 3ce7d2c9b621e796ace43008134413cea7648a06 Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Tue, 5 Feb 2019 10:52:32 -0800 Subject: [PATCH 18/23] Make sure to reject mappings with type _doc when include_type_name is false. (#38270) `CreateIndexRequest#source(Map, ... )`, which is used when deserializing index creation requests, accidentally accepts mappings that are nested twice under the type key (as described in the bug report #38266). This in turn causes us to be too lenient in parsing typeless mappings. In particular, we accept the following index creation request, even though it should not contain the type key `_doc`: ``` PUT index?include_type_name=false { "mappings": { "_doc": { "properties": { ... } } } } ``` There is a similar issue for both 'put templates' and 'put mappings' requests as well. This PR makes the minimal changes to detect and reject these typed mappings in requests. It does not address #38266 generally, or attempt a larger refactor around types in these server-side requests, as I think this should be done at a later time. --- .../documentation/CRUDDocumentationIT.java | 20 +- .../IndicesClientDocumentationIT.java | 20 +- docs/build.gradle | 333 ++++++++---------- .../test/painless/80_script_score.yml | 36 +- .../test/update_by_query/20_validation.yml | 5 +- .../test/indices.create/10_basic.yml | 19 + .../test/indices.put_mapping/10_basic.yml | 23 ++ .../test/indices.put_template/10_basic.yml | 21 ++ .../test/indices.rollover/40_mapping.yml | 29 ++ .../indices/rollover/RolloverRequest.java | 9 +- .../metadata/MetaDataMappingService.java | 20 +- .../index/mapper/MapperService.java | 15 + .../admin/indices/RestCreateIndexAction.java | 32 +- .../indices/RestPutIndexTemplateAction.java | 21 +- .../admin/indices/RestPutMappingAction.java | 16 +- .../indices/RestCreateIndexActionTests.java | 80 +++++ .../RestPutIndexTemplateActionTests.java | 68 +--- x-pack/docs/build.gradle | 201 +++++------ .../indexlifecycle/CCRIndexLifecycleIT.java | 2 +- .../xpack/sql/qa/jdbc/DataLoader.java | 112 +++--- .../xpack/sql/qa/jdbc/FetchSizeTestCase.java | 12 +- .../rest-api-spec/test/rollup/delete_job.yml | 11 +- .../rest-api-spec/test/rollup/get_jobs.yml | 11 +- .../rest-api-spec/test/rollup/put_job.yml | 11 +- .../test/rollup/rollup_search.yml | 65 ++-- .../test/rollup/security_tests.yml | 41 +-- .../rest-api-spec/test/rollup/start_job.yml | 11 +- .../rest-api-spec/test/rollup/stop_job.yml | 11 +- .../elasticsearch/multi_node/RollupIT.java | 3 +- 29 files changed, 642 insertions(+), 616 deletions(-) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java index 1cd76e48ffff5..53524a6a5c215 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java @@ -1235,12 +1235,10 @@ public void testGet() throws Exception { createIndex.setJsonEntity( "{\n" + " \"mappings\" : {\n" + - " \"_doc\" : {\n" + - " \"properties\" : {\n" + - " \"message\" : {\n" + - " \"type\": \"text\",\n" + - " \"store\": true\n" + - " }\n" + + " \"properties\" : {\n" + + " \"message\" : {\n" + + " \"type\": \"text\",\n" + + " \"store\": true\n" + " }\n" + " }\n" + " }\n" + @@ -1764,12 +1762,10 @@ public void testMultiGet() throws Exception { createIndex.setJsonEntity( "{\n" + " \"mappings\" : {\n" + - " \"_doc\" : {\n" + - " \"properties\" : {\n" + - " \"foo\" : {\n" + - " \"type\": \"text\",\n" + - " \"store\": true\n" + - " }\n" + + " \"properties\" : {\n" + + " \"foo\" : {\n" + + " \"type\": \"text\",\n" + + " \"store\": true\n" + " }\n" + " }\n" + " }\n" + diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java index 02b7d597ce24e..14def60b277e8 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java @@ -313,15 +313,13 @@ public void testCreateIndex() throws IOException { { request = new CreateIndexRequest("twitter2"); //tag::create-index-mappings-map - Map jsonMap = new HashMap<>(); Map message = new HashMap<>(); message.put("type", "text"); Map properties = new HashMap<>(); properties.put("message", message); Map mapping = new HashMap<>(); mapping.put("properties", properties); - jsonMap.put("_doc", mapping); - request.mapping(jsonMap); // <1> + request.mapping(mapping); // <1> //end::create-index-mappings-map CreateIndexResponse createIndexResponse = client.indices().create(request, RequestOptions.DEFAULT); assertTrue(createIndexResponse.isAcknowledged()); @@ -332,15 +330,11 @@ public void testCreateIndex() throws IOException { XContentBuilder builder = XContentFactory.jsonBuilder(); builder.startObject(); { - builder.startObject("_doc"); + builder.startObject("properties"); { - builder.startObject("properties"); + builder.startObject("message"); { - builder.startObject("message"); - { - builder.field("type", "text"); - } - builder.endObject(); + builder.field("type", "text"); } builder.endObject(); } @@ -381,10 +375,8 @@ public void testCreateIndex() throws IOException { " \"number_of_replicas\" : 0\n" + " },\n" + " \"mappings\" : {\n" + - " \"_doc\" : {\n" + - " \"properties\" : {\n" + - " \"message\" : { \"type\" : \"text\" }\n" + - " }\n" + + " \"properties\" : {\n" + + " \"message\" : { \"type\" : \"text\" }\n" + " }\n" + " },\n" + " \"aliases\" : {\n" + diff --git a/docs/build.gradle b/docs/build.gradle index e7112c08ac41e..1083d07b94f46 100644 --- a/docs/build.gradle +++ b/docs/build.gradle @@ -98,19 +98,17 @@ Closure setupTwitter = { String name, int count -> number_of_shards: 1 number_of_replicas: 1 mappings: - _doc: - properties: - user: - type: keyword - doc_values: true - date: - type: date - likes: - type: long + properties: + user: + type: keyword + doc_values: true + date: + type: date + likes: + type: long - do: bulk: index: twitter - type: _doc refresh: true body: |''' for (int i = 0; i < count; i++) { @@ -161,16 +159,14 @@ buildRestTests.setups['ledger'] = ''' number_of_shards: 2 number_of_replicas: 1 mappings: - _doc: - properties: - type: - type: keyword - amount: - type: double + properties: + type: + type: keyword + amount: + type: double - do: bulk: index: ledger - type: _doc refresh: true body: | {"index":{}} @@ -194,14 +190,12 @@ buildRestTests.setups['sales'] = ''' number_of_shards: 2 number_of_replicas: 1 mappings: - _doc: - properties: - type: - type: keyword + properties: + type: + type: keyword - do: bulk: index: sales - type: _doc refresh: true body: | {"index":{}} @@ -231,7 +225,6 @@ buildRestTests.setups['bank'] = ''' - do: bulk: index: bank - type: _doc refresh: true body: | #bank_data# @@ -273,16 +266,14 @@ buildRestTests.setups['stackoverflow'] = ''' number_of_shards: 1 number_of_replicas: 1 mappings: - _doc: - properties: - author: - type: keyword - tags: - type: keyword + properties: + author: + type: keyword + tags: + type: keyword - do: bulk: index: stackoverflow - type: _doc refresh: true body: |''' @@ -328,16 +319,14 @@ buildRestTests.setups['news'] = ''' number_of_shards: 1 number_of_replicas: 1 mappings: - _doc: - properties: - source: - type: keyword - content: - type: text + properties: + source: + type: keyword + content: + type: text - do: bulk: index: news - type: _doc refresh: true body: |''' @@ -381,14 +370,12 @@ buildRestTests.setups['exams'] = ''' number_of_shards: 1 number_of_replicas: 1 mappings: - _doc: - properties: - grade: - type: byte + properties: + grade: + type: byte - do: bulk: index: exams - type: _doc refresh: true body: | {"index":{}} @@ -446,10 +433,9 @@ buildRestTests.setups['analyze_sample'] = ''' type: custom filter: [lowercase] mappings: - _doc: - properties: - obj1.field1: - type: text''' + properties: + obj1.field1: + type: text''' // Used by percentile/percentile-rank aggregations buildRestTests.setups['latency'] = ''' @@ -461,14 +447,12 @@ buildRestTests.setups['latency'] = ''' number_of_shards: 1 number_of_replicas: 1 mappings: - _doc: - properties: - load_time: - type: long + properties: + load_time: + type: long - do: bulk: index: latency - type: _doc refresh: true body: |''' @@ -493,14 +477,12 @@ buildRestTests.setups['iprange'] = ''' number_of_shards: 1 number_of_replicas: 1 mappings: - _doc: - properties: - ip: - type: ip + properties: + ip: + type: ip - do: bulk: index: ip_addresses - type: _doc refresh: true body: |''' @@ -613,16 +595,15 @@ buildRestTests.setups['sensor_rollup_job'] = ''' number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - timestamp: - type: date - temperature: - type: long - voltage: - type: float - node: - type: keyword + properties: + timestamp: + type: date + temperature: + type: long + voltage: + type: float + node: + type: keyword - do: raw: method: PUT @@ -664,21 +645,19 @@ buildRestTests.setups['sensor_started_rollup_job'] = ''' number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - timestamp: - type: date - temperature: - type: long - voltage: - type: float - node: - type: keyword + properties: + timestamp: + type: date + temperature: + type: long + voltage: + type: float + node: + type: keyword - do: bulk: index: sensor-1 - type: _doc refresh: true body: | {"index":{}} @@ -740,26 +719,25 @@ buildRestTests.setups['sensor_index'] = ''' number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - timestamp: - type: date - temperature: - type: long - voltage: - type: float - node: - type: keyword - load: - type: double - net_in: - type: long - net_out: - type: long - hostname: - type: keyword - datacenter: - type: keyword + properties: + timestamp: + type: date + temperature: + type: long + voltage: + type: float + node: + type: keyword + load: + type: double + net_in: + type: long + net_out: + type: long + hostname: + type: keyword + datacenter: + type: keyword ''' buildRestTests.setups['sensor_prefab_data'] = ''' @@ -771,16 +749,15 @@ buildRestTests.setups['sensor_prefab_data'] = ''' number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - timestamp: - type: date - temperature: - type: long - voltage: - type: float - node: - type: keyword + properties: + timestamp: + type: date + temperature: + type: long + voltage: + type: float + node: + type: keyword - do: indices.create: index: sensor_rollup @@ -789,64 +766,62 @@ buildRestTests.setups['sensor_prefab_data'] = ''' number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - node.terms.value: - type: keyword - temperature.sum.value: - type: double - temperature.max.value: - type: double - temperature.min.value: - type: double - timestamp.date_histogram.time_zone: - type: keyword - timestamp.date_histogram.interval: - type: keyword - timestamp.date_histogram.timestamp: - type: date - timestamp.date_histogram._count: - type: long - voltage.avg.value: - type: double - voltage.avg._count: - type: long - _rollup.id: - type: keyword - _rollup.version: - type: long - _meta: - _rollup: - sensor: - cron: "* * * * * ?" - rollup_index: "sensor_rollup" - index_pattern: "sensor-*" - timeout: "20s" - page_size: 1000 - groups: - date_histogram: - delay: "7d" - field: "timestamp" - interval: "60m" - time_zone: "UTC" - terms: - fields: - - "node" - id: sensor - metrics: - - field: "temperature" - metrics: - - min - - max - - sum - - field: "voltage" - metrics: - - avg + properties: + node.terms.value: + type: keyword + temperature.sum.value: + type: double + temperature.max.value: + type: double + temperature.min.value: + type: double + timestamp.date_histogram.time_zone: + type: keyword + timestamp.date_histogram.interval: + type: keyword + timestamp.date_histogram.timestamp: + type: date + timestamp.date_histogram._count: + type: long + voltage.avg.value: + type: double + voltage.avg._count: + type: long + _rollup.id: + type: keyword + _rollup.version: + type: long + _meta: + _rollup: + sensor: + cron: "* * * * * ?" + rollup_index: "sensor_rollup" + index_pattern: "sensor-*" + timeout: "20s" + page_size: 1000 + groups: + date_histogram: + delay: "7d" + field: "timestamp" + interval: "60m" + time_zone: "UTC" + terms: + fields: + - "node" + id: sensor + metrics: + - field: "temperature" + metrics: + - min + - max + - sum + - field: "voltage" + metrics: + - avg - do: bulk: index: sensor_rollup - type: _doc refresh: true body: | {"index":{}} @@ -1093,16 +1068,14 @@ buildRestTests.setups['reviews'] = ''' number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - product: - type: keyword - rating: - type: long + properties: + product: + type: keyword + rating: + type: long - do: bulk: index: reviews - type: _doc refresh: true body: | {"index": {"_id": "1"}} @@ -1139,22 +1112,20 @@ buildRestTests.setups['seats'] = ''' number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - theatre: - type: keyword - cost: - type: long - row: - type: long - number: - type: long - sold: - type: boolean + properties: + theatre: + type: keyword + cost: + type: long + row: + type: long + number: + type: long + sold: + type: boolean - do: bulk: index: seats - type: _doc refresh: true body: | {"index":{"_id": "1"}} diff --git a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/80_script_score.yml b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/80_script_score.yml index c9b73bca029b1..a3135777c952c 100644 --- a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/80_script_score.yml +++ b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/80_script_score.yml @@ -14,26 +14,22 @@ setup: settings: number_of_shards: 2 mappings: - _doc: - properties: - dval: - type: double + properties: + dval: + type: double - do: index: index: test - type: _doc id: d1 body: {"dval": 10} - do: index: index: test - type: _doc id: d2 body: {"dval": 100} - do: index: index: test - type: _doc id: d3 body: {"dval": 1000} @@ -225,10 +221,9 @@ setup: settings: number_of_shards: 1 mappings: - _doc: - properties: - date: - type: date + properties: + date: + type: date - do: index: index: test @@ -309,16 +304,15 @@ setup: settings: number_of_shards: 1 mappings: - _doc: - properties: - ival: - type: integer - lval: - type: long - fval: - type: float - dval: - type: double + properties: + ival: + type: integer + lval: + type: long + fval: + type: float + dval: + type: double - do: index: diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/20_validation.yml b/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/20_validation.yml index 5e750f89f12ad..b6823f88c15fc 100644 --- a/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/20_validation.yml +++ b/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/20_validation.yml @@ -44,9 +44,8 @@ index: test body: mappings: - _doc: - _source: - enabled: false + _source: + enabled: false - do: index: index: test diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/10_basic.yml index 78e67541a1f36..965083421cbaf 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/10_basic.yml @@ -135,3 +135,22 @@ properties: "": type: keyword + +--- +"Create index with explicit _doc type": + - skip: + version: " - 6.99.99" + reason: include_type_name defaults to true before 7.0 + - do: + catch: bad_request + indices.create: + index: test_index + body: + mappings: + _doc: + properties: + field: + type: keyword + + - match: { error.type: "illegal_argument_exception" } + - match: { error.reason: "The mapping definition cannot be nested under a type [_doc] unless include_type_name is set to true." } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/10_basic.yml index 420b12398d267..2c4658afad2db 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/10_basic.yml @@ -68,3 +68,26 @@ properties: "": type: keyword + +--- +"Put mappings with explicit _doc type": + - skip: + version: " - 6.99.99" + reason: include_type_name defaults to true before 7.0 + + - do: + indices.create: + index: test_index + + - do: + catch: bad_request + indices.put_mapping: + index: test_index + body: + _doc: + properties: + field: + type: keyword + + - match: { error.type: "illegal_argument_exception" } + - match: { error.reason: "Types cannot be provided in put mapping requests, unless the include_type_name parameter is set to true." } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/10_basic.yml index c0b0bbe5d30b7..3850ba4150b4f 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/10_basic.yml @@ -238,3 +238,24 @@ indices.put_template: name: test body: {} + +--- +"Put template with explicit _doc type": + - skip: + version: " - 6.99.99" + reason: include_type_name defaults to true before 7.0 + + - do: + catch: bad_request + indices.put_template: + name: test + body: + index_patterns: test-* + mappings: + _doc: + properties: + field: + type: keyword + + - match: { error.type: "illegal_argument_exception" } + - match: { error.reason: "The mapping definition cannot be nested under a type [_doc] unless include_type_name is set to true." } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/40_mapping.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/40_mapping.yml index 59e027fb98457..47b004326a457 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/40_mapping.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/40_mapping.yml @@ -41,3 +41,32 @@ - match: { conditions: { "[max_docs: 2]": true } } - match: { rolled_over: true } + +--- +"Mappings with explicit _doc type": + - skip: + version: " - 6.99.99" + reason: include_type_name defaults to true before 7.0 + + - do: + indices.create: + index: logs-1 + body: + aliases: + logs_search: {} + + - do: + catch: bad_request + indices.rollover: + alias: "logs_search" + body: + conditions: + max_docs: 2 + mappings: + _doc: + properties: + field: + type: keyword + + - match: { error.caused_by.type: "illegal_argument_exception" } + - match: { error.caused_by.reason: "The mapping definition cannot be nested under a type [_doc] unless include_type_name is set to true." } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverRequest.java index 3bd3153d83180..1150a29b07766 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverRequest.java @@ -70,13 +70,18 @@ public class RolloverRequest extends AcknowledgedRequest implem CONDITIONS, ObjectParser.ValueType.OBJECT); PARSER.declareField((parser, request, context) -> request.createIndexRequest.settings(parser.map()), CreateIndexRequest.SETTINGS, ObjectParser.ValueType.OBJECT); - PARSER.declareField((parser, request, isTypeIncluded) -> { - if (isTypeIncluded) { + PARSER.declareField((parser, request, includeTypeName) -> { + if (includeTypeName) { for (Map.Entry mappingsEntry : parser.map().entrySet()) { request.createIndexRequest.mapping(mappingsEntry.getKey(), (Map) mappingsEntry.getValue()); } } else { // a type is not included, add a dummy _doc type + Map mappings = parser.map(); + if (MapperService.isMappingSourceTyped(MapperService.SINGLE_MAPPING_NAME, mappings)) { + throw new IllegalArgumentException("The mapping definition cannot be nested under a type " + + "[" + MapperService.SINGLE_MAPPING_NAME + "] unless include_type_name is set to true."); + } request.createIndexRequest.mapping(MapperService.SINGLE_MAPPING_NAME, parser.map()); } }, CreateIndexRequest.MAPPINGS, ObjectParser.ValueType.OBJECT); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java index cf31401983a62..feec1833443a2 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java @@ -22,13 +22,12 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.mapping.put.PutMappingClusterStateUpdateRequest; import org.elasticsearch.cluster.AckedClusterStateTaskListener; -import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateTaskConfig; +import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ack.ClusterStateUpdateResponse; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; @@ -37,8 +36,7 @@ import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.xcontent.XContentHelper; -import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.mapper.DocumentMapper; @@ -55,6 +53,7 @@ import java.util.List; import java.util.Map; +import static org.elasticsearch.index.mapper.MapperService.isMappingSourceTyped; import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.NO_LONGER_ASSIGNED; /** @@ -279,7 +278,7 @@ private ClusterState applyRequest(ClusterState currentState, PutMappingClusterSt if (mappingType == null) { mappingType = newMapper.type(); } else if (mappingType.equals(newMapper.type()) == false - && (isMappingSourceTyped(mapperService, mappingUpdateSource, request.type()) + && (isMappingSourceTyped(request.type(), mappingUpdateSource) || mapperService.resolveDocumentType(mappingType).equals(newMapper.type()) == false)) { throw new InvalidTypeNameException("Type name provided does not match type name within mapping definition."); } @@ -304,7 +303,7 @@ private ClusterState applyRequest(ClusterState currentState, PutMappingClusterSt // are handling a typeless call. In such a case, we override _doc with the actual type // name in the mappings. This allows to use typeless APIs on typed indices. String typeForUpdate = mappingType; // the type to use to apply the mapping update - if (isMappingSourceTyped(mapperService, mappingUpdateSource, request.type()) == false) { + if (isMappingSourceTyped(request.type(), mappingUpdateSource) == false) { typeForUpdate = mapperService.resolveDocumentType(mappingType); } @@ -371,15 +370,6 @@ public String describeTasks(List tasks) { } } - /** - * Returns {@code true} if the given {@code mappingSource} includes a type - * as a top-level object. - */ - private static boolean isMappingSourceTyped(MapperService mapperService, CompressedXContent mappingSource, String type) { - Map root = XContentHelper.convertToMap(mappingSource.compressedReference(), true, XContentType.JSON).v2(); - return root.size() == 1 && root.keySet().iterator().next().equals(type); - } - public void putMapping(final PutMappingClusterStateUpdateRequest request, final ActionListener listener) { clusterService.submitStateUpdateTask("put-mapping", request, diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index add313d7aa87b..398ce4cdd17ce 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -39,6 +39,7 @@ import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.AbstractIndexComponent; @@ -665,6 +666,20 @@ public DocumentMapper documentMapper(String type) { return null; } + /** + * Returns {@code true} if the given {@code mappingSource} includes a type + * as a top-level object. + */ + public static boolean isMappingSourceTyped(String type, Map mapping) { + return mapping.size() == 1 && mapping.keySet().iterator().next().equals(type); + } + + + public static boolean isMappingSourceTyped(String type, CompressedXContent mappingSource) { + Map root = XContentHelper.convertToMap(mappingSource.compressedReference(), true, XContentType.JSON).v2(); + return isMappingSourceTyped(type, root); + } + /** * Resolves a type from a mapping-related request into the type that should be used when * merging and updating mappings. diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexAction.java index adcdc0b281c2e..527b0830c8022 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexAction.java @@ -64,18 +64,38 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC } CreateIndexRequest createIndexRequest = new CreateIndexRequest(request.param("index")); + if (request.hasContent()) { - Map sourceAsMap = XContentHelper.convertToMap(request.content(), false, request.getXContentType()).v2(); - if (includeTypeName == false && sourceAsMap.containsKey("mappings")) { - Map newSourceAsMap = new HashMap<>(sourceAsMap); - newSourceAsMap.put("mappings", Collections.singletonMap(MapperService.SINGLE_MAPPING_NAME, sourceAsMap.get("mappings"))); - sourceAsMap = newSourceAsMap; - } + Map sourceAsMap = XContentHelper.convertToMap(request.requiredContent(), false, + request.getXContentType()).v2(); + sourceAsMap = prepareMappings(sourceAsMap, includeTypeName); createIndexRequest.source(sourceAsMap, LoggingDeprecationHandler.INSTANCE); } + createIndexRequest.timeout(request.paramAsTime("timeout", createIndexRequest.timeout())); createIndexRequest.masterNodeTimeout(request.paramAsTime("master_timeout", createIndexRequest.masterNodeTimeout())); createIndexRequest.waitForActiveShards(ActiveShardCount.parseString(request.param("wait_for_active_shards"))); return channel -> client.admin().indices().create(createIndexRequest, new RestToXContentListener<>(channel)); } + + + static Map prepareMappings(Map source, boolean includeTypeName) { + if (includeTypeName + || source.containsKey("mappings") == false + || (source.get("mappings") instanceof Map) == false) { + return source; + } + + Map newSource = new HashMap<>(source); + + @SuppressWarnings("unchecked") + Map mappings = (Map) source.get("mappings"); + if (MapperService.isMappingSourceTyped(MapperService.SINGLE_MAPPING_NAME, mappings)) { + throw new IllegalArgumentException("The mapping definition cannot be nested under a type " + + "[" + MapperService.SINGLE_MAPPING_NAME + "] unless include_type_name is set to true."); + } + + newSource.put("mappings", Collections.singletonMap(MapperService.SINGLE_MAPPING_NAME, mappings)); + return newSource; + } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateAction.java index 445d393fef82b..bb0f282639bd4 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateAction.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentHelper; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; @@ -35,7 +34,6 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.Map; public class RestPutIndexTemplateAction extends BaseRestHandler { @@ -59,6 +57,8 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { + boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, DEFAULT_INCLUDE_TYPE_NAME_POLICY); + PutIndexTemplateRequest putRequest = new PutIndexTemplateRequest(request.param("name")); if (request.hasParam(INCLUDE_TYPE_NAME_PARAMETER)) { deprecationLogger.deprecatedAndMaybeLog("put_index_template_with_types", TYPES_DEPRECATION_MESSAGE); @@ -74,22 +74,11 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC putRequest.create(request.paramAsBoolean("create", false)); putRequest.cause(request.param("cause", "")); - boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, DEFAULT_INCLUDE_TYPE_NAME_POLICY); - Map sourceAsMap = prepareRequestSource(request, includeTypeName); + Map sourceAsMap = XContentHelper.convertToMap(request.requiredContent(), false, + request.getXContentType()).v2(); + sourceAsMap = RestCreateIndexAction.prepareMappings(sourceAsMap, includeTypeName); putRequest.source(sourceAsMap); return channel -> client.admin().indices().putTemplate(putRequest, new RestToXContentListener<>(channel)); } - - Map prepareRequestSource(RestRequest request, boolean includeTypeName) { - Map sourceAsMap = XContentHelper.convertToMap(request.requiredContent(), false, - request.getXContentType()).v2(); - if (includeTypeName == false && sourceAsMap.containsKey("mappings")) { - Map newSourceAsMap = new HashMap<>(sourceAsMap); - newSourceAsMap.put("mappings", Collections.singletonMap(MapperService.SINGLE_MAPPING_NAME, sourceAsMap.get("mappings"))); - return newSourceAsMap; - } else { - return sourceAsMap; - } - } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutMappingAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutMappingAction.java index bd99c26a155a0..f5e760d54ed45 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutMappingAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutMappingAction.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; @@ -33,8 +34,10 @@ import org.elasticsearch.rest.action.RestToXContentListener; import java.io.IOException; +import java.util.Map; import static org.elasticsearch.client.Requests.putMappingRequest; +import static org.elasticsearch.index.mapper.MapperService.isMappingSourceTyped; import static org.elasticsearch.rest.RestRequest.Method.POST; import static org.elasticsearch.rest.RestRequest.Method.PUT; @@ -81,15 +84,20 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC deprecationLogger.deprecatedAndMaybeLog("put_mapping_with_types", TYPES_DEPRECATION_MESSAGE); } + PutMappingRequest putMappingRequest = putMappingRequest(Strings.splitStringByCommaToArray(request.param("index"))); + final String type = request.param("type"); - if (type != null && includeTypeName == false) { + putMappingRequest.type(includeTypeName ? type : MapperService.SINGLE_MAPPING_NAME); + + Map sourceAsMap = XContentHelper.convertToMap(request.requiredContent(), false, + request.getXContentType()).v2(); + if (includeTypeName == false && + (type != null || isMappingSourceTyped(MapperService.SINGLE_MAPPING_NAME, sourceAsMap))) { throw new IllegalArgumentException("Types cannot be provided in put mapping requests, unless " + "the include_type_name parameter is set to true."); } - PutMappingRequest putMappingRequest = putMappingRequest(Strings.splitStringByCommaToArray(request.param("index"))); - putMappingRequest.type(includeTypeName ? type : MapperService.SINGLE_MAPPING_NAME); - putMappingRequest.source(request.requiredContent(), request.getXContentType()); + putMappingRequest.source(sourceAsMap); putMappingRequest.timeout(request.paramAsTime("timeout", putMappingRequest.timeout())); putMappingRequest.masterNodeTimeout(request.paramAsTime("master_timeout", putMappingRequest.masterNodeTimeout())); putMappingRequest.indicesOptions(IndicesOptions.fromRequest(request, putMappingRequest.indicesOptions())); diff --git a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexActionTests.java index 1ec0a0f949965..f37f36a8d0645 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexActionTests.java @@ -20,7 +20,11 @@ package org.elasticsearch.rest.action.admin.indices; import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.test.rest.FakeRestRequest; import org.elasticsearch.test.rest.RestActionTestCase; @@ -59,4 +63,80 @@ public void testIncludeTypeName() throws IOException { .build(); action.prepareRequest(validRequest, mock(NodeClient.class)); } + + public void testPrepareTypelessRequest() throws IOException { + XContentBuilder content = XContentFactory.jsonBuilder().startObject() + .startObject("mappings") + .startObject("properties") + .startObject("field1").field("type", "keyword").endObject() + .startObject("field2").field("type", "text").endObject() + .endObject() + .endObject() + .startObject("aliases") + .startObject("read_alias").endObject() + .endObject() + .endObject(); + + Map contentAsMap = XContentHelper.convertToMap( + BytesReference.bytes(content), true, content.contentType()).v2(); + boolean includeTypeName = false; + Map source = RestCreateIndexAction.prepareMappings(contentAsMap, includeTypeName); + + XContentBuilder expectedContent = XContentFactory.jsonBuilder().startObject() + .startObject("mappings") + .startObject("_doc") + .startObject("properties") + .startObject("field1").field("type", "keyword").endObject() + .startObject("field2").field("type", "text").endObject() + .endObject() + .endObject() + .endObject() + .startObject("aliases") + .startObject("read_alias").endObject() + .endObject() + .endObject(); + Map expectedContentAsMap = XContentHelper.convertToMap( + BytesReference.bytes(expectedContent), true, expectedContent.contentType()).v2(); + + assertEquals(expectedContentAsMap, source); + } + + public void testPrepareTypedRequest() throws IOException { + XContentBuilder content = XContentFactory.jsonBuilder().startObject() + .startObject("mappings") + .startObject("type") + .startObject("properties") + .startObject("field1").field("type", "keyword").endObject() + .startObject("field2").field("type", "text").endObject() + .endObject() + .endObject() + .endObject() + .startObject("aliases") + .startObject("read_alias").endObject() + .endObject() + .endObject(); + + Map contentAsMap = XContentHelper.convertToMap( + BytesReference.bytes(content), true, content.contentType()).v2(); + boolean includeTypeName = true; + Map source = RestCreateIndexAction.prepareMappings(contentAsMap, includeTypeName); + + assertEquals(contentAsMap, source); + } + + public void testMalformedMappings() throws IOException { + XContentBuilder content = XContentFactory.jsonBuilder().startObject() + .field("mappings", "some string") + .startObject("aliases") + .startObject("read_alias").endObject() + .endObject() + .endObject(); + + Map contentAsMap = XContentHelper.convertToMap( + BytesReference.bytes(content), true, content.contentType()).v2(); + + boolean includeTypeName = false; + Map source = RestCreateIndexAction.prepareMappings(contentAsMap, includeTypeName); + assertEquals(contentAsMap, source); + } } diff --git a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateActionTests.java index d1900aaee84d5..2a5e5db92d40e 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateActionTests.java @@ -24,7 +24,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.test.rest.FakeRestRequest; @@ -46,51 +45,6 @@ public void setUpAction() { action = new RestPutIndexTemplateAction(Settings.EMPTY, controller()); } - public void testPrepareTypelessRequest() throws IOException { - XContentBuilder content = XContentFactory.jsonBuilder().startObject() - .startObject("mappings") - .startObject("properties") - .startObject("field1").field("type", "keyword").endObject() - .startObject("field2").field("type", "text").endObject() - .endObject() - .endObject() - .startObject("aliases") - .startObject("read_alias").endObject() - .endObject() - .endObject(); - - RestRequest request = new FakeRestRequest.Builder(xContentRegistry()) - .withMethod(RestRequest.Method.PUT) - .withPath("/_template/_some_template") - .withContent(BytesReference.bytes(content), XContentType.JSON) - .build(); - action.prepareRequest(request, mock(NodeClient.class)); - - // Internally the above prepareRequest method calls prepareRequestSource to inject a - // default type into the mapping. Here we test that this does what is expected by - // explicitly calling that same helper function - boolean includeTypeName = false; - Map source = action.prepareRequestSource(request, includeTypeName); - - XContentBuilder expectedContent = XContentFactory.jsonBuilder().startObject() - .startObject("mappings") - .startObject("_doc") - .startObject("properties") - .startObject("field1").field("type", "keyword").endObject() - .startObject("field2").field("type", "text").endObject() - .endObject() - .endObject() - .endObject() - .startObject("aliases") - .startObject("read_alias").endObject() - .endObject() - .endObject(); - Map expectedContentAsMap = XContentHelper.convertToMap( - BytesReference.bytes(expectedContent), true, expectedContent.contentType()).v2(); - - assertEquals(expectedContentAsMap, source); - } - public void testIncludeTypeName() throws IOException { XContentBuilder typedContent = XContentFactory.jsonBuilder().startObject() .startObject("mappings") @@ -116,25 +70,5 @@ public void testIncludeTypeName() throws IOException { .build(); action.prepareRequest(request, mock(NodeClient.class)); assertWarnings(RestPutIndexTemplateAction.TYPES_DEPRECATION_MESSAGE); - boolean includeTypeName = true; - Map source = action.prepareRequestSource(request, includeTypeName); - - XContentBuilder expectedContent = XContentFactory.jsonBuilder().startObject() - .startObject("mappings") - .startObject("my_doc") - .startObject("properties") - .startObject("field1").field("type", "keyword").endObject() - .startObject("field2").field("type", "text").endObject() - .endObject() - .endObject() - .endObject() - .startObject("aliases") - .startObject("read_alias").endObject() - .endObject() - .endObject(); - Map expectedContentAsMap = XContentHelper.convertToMap( - BytesReference.bytes(expectedContent), true, expectedContent.contentType()).v2(); - - assertEquals(expectedContentAsMap, source); - } + } } diff --git a/x-pack/docs/build.gradle b/x-pack/docs/build.gradle index ecfd30bb7469b..0e5cd633a9061 100644 --- a/x-pack/docs/build.gradle +++ b/x-pack/docs/build.gradle @@ -464,16 +464,15 @@ setups['sensor_rollup_job'] = ''' number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - timestamp: - type: date - temperature: - type: long - voltage: - type: float - node: - type: keyword + properties: + timestamp: + type: date + temperature: + type: long + voltage: + type: float + node: + type: keyword - do: xpack.rollup.put_job: id: "sensor" @@ -514,21 +513,19 @@ setups['sensor_started_rollup_job'] = ''' number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - timestamp: - type: date - temperature: - type: long - voltage: - type: float - node: - type: keyword + properties: + timestamp: + type: date + temperature: + type: long + voltage: + type: float + node: + type: keyword - do: bulk: index: sensor-1 - type: _doc refresh: true body: | {"index":{}} @@ -588,26 +585,25 @@ setups['sensor_index'] = ''' number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - timestamp: - type: date - temperature: - type: long - voltage: - type: float - node: - type: keyword - load: - type: double - net_in: - type: long - net_out: - type: long - hostname: - type: keyword - datacenter: - type: keyword + properties: + timestamp: + type: date + temperature: + type: long + voltage: + type: float + node: + type: keyword + load: + type: double + net_in: + type: long + net_out: + type: long + hostname: + type: keyword + datacenter: + type: keyword ''' setups['sensor_prefab_data'] = ''' @@ -619,16 +615,15 @@ setups['sensor_prefab_data'] = ''' number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - timestamp: - type: date - temperature: - type: long - voltage: - type: float - node: - type: keyword + properties: + timestamp: + type: date + temperature: + type: long + voltage: + type: float + node: + type: keyword - do: indices.create: index: sensor_rollup @@ -637,63 +632,61 @@ setups['sensor_prefab_data'] = ''' number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - node.terms.value: - type: keyword - temperature.sum.value: - type: double - temperature.max.value: - type: double - temperature.min.value: - type: double - timestamp.date_histogram.time_zone: - type: keyword - timestamp.date_histogram.interval: - type: keyword - timestamp.date_histogram.timestamp: - type: date - timestamp.date_histogram._count: - type: long - voltage.avg.value: - type: double - voltage.avg._count: - type: long - _rollup.id: - type: keyword - _rollup.version: - type: long - _meta: - _rollup: - sensor: - cron: "* * * * * ?" - rollup_index: "sensor_rollup" - index_pattern: "sensor-*" - timeout: "20s" - page_size: 1000 - groups: - date_histogram: - field: "timestamp" - interval: "7d" - time_zone: "UTC" - terms: - fields: - - "node" - id: sensor - metrics: - - field: "temperature" - metrics: - - min - - max - - sum - - field: "voltage" - metrics: - - avg + properties: + node.terms.value: + type: keyword + temperature.sum.value: + type: double + temperature.max.value: + type: double + temperature.min.value: + type: double + timestamp.date_histogram.time_zone: + type: keyword + timestamp.date_histogram.interval: + type: keyword + timestamp.date_histogram.timestamp: + type: date + timestamp.date_histogram._count: + type: long + voltage.avg.value: + type: double + voltage.avg._count: + type: long + _rollup.id: + type: keyword + _rollup.version: + type: long + _meta: + _rollup: + sensor: + cron: "* * * * * ?" + rollup_index: "sensor_rollup" + index_pattern: "sensor-*" + timeout: "20s" + page_size: 1000 + groups: + date_histogram: + field: "timestamp" + interval: "7d" + time_zone: "UTC" + terms: + fields: + - "node" + id: sensor + metrics: + - field: "temperature" + metrics: + - min + - max + - sum + - field: "voltage" + metrics: + - avg - do: bulk: index: sensor_rollup - type: _doc refresh: true body: | {"index":{}} diff --git a/x-pack/plugin/ilm/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java b/x-pack/plugin/ilm/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java index 01f0eb4c7d0d4..b3c93acb97b99 100644 --- a/x-pack/plugin/ilm/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java +++ b/x-pack/plugin/ilm/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java @@ -210,7 +210,7 @@ public void testCcrAndIlmWithRollover() throws Exception { // Create an index on the leader using the template set up above Request createIndexRequest = new Request("PUT", "/" + indexName); createIndexRequest.setJsonEntity("{" + - "\"mappings\": {\"_doc\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}}, " + + "\"mappings\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}, " + "\"aliases\": {\"" + alias + "\": {\"is_write_index\": true}} }"); assertOK(leaderClient.performRequest(createIndexRequest)); // Check that the new index is created diff --git a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DataLoader.java b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DataLoader.java index efa39ec517eb2..b12203294c158 100644 --- a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DataLoader.java +++ b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DataLoader.java @@ -83,50 +83,46 @@ private static void loadEmpDatasetIntoEs(RestClient client, String index, String createIndex.endObject(); createIndex.startObject("mappings"); { - createIndex.startObject("_doc"); + createIndex.startObject("properties"); { - createIndex.startObject("properties"); + createIndex.startObject("emp_no").field("type", "integer"); + if (extraFields) { + createIndex.field("copy_to", "extra_no"); + } + createIndex.endObject(); + if (extraFields) { + createIndex.startObject("extra_no").field("type", "integer").endObject(); + } + createString("first_name", createIndex); + createString("last_name", createIndex); + createIndex.startObject("gender").field("type", "keyword"); + if (extraFields) { + createIndex.field("copy_to", "extra_gender"); + } + createIndex.endObject(); + + if (extraFields) { + createIndex.startObject("extra_gender").field("type", "keyword").endObject(); + createIndex.startObject("extra.info.gender") + .field("type", "alias") + .field("path", "gender") + .endObject(); + } + + createIndex.startObject("birth_date").field("type", "date").endObject(); + createIndex.startObject("hire_date").field("type", "date").endObject(); + createIndex.startObject("salary").field("type", "integer").endObject(); + createIndex.startObject("languages").field("type", "byte").endObject(); { - createIndex.startObject("emp_no").field("type", "integer"); - if (extraFields) { - createIndex.field("copy_to", "extra_no"); - } + createIndex.startObject("dep").field("type", "nested"); + createIndex.startObject("properties"); + createIndex.startObject("dep_id").field("type", "keyword").endObject(); + createString("dep_name", createIndex); + createIndex.startObject("from_date").field("type", "date").endObject(); + createIndex.startObject("to_date").field("type", "date").endObject(); createIndex.endObject(); - if (extraFields) { - createIndex.startObject("extra_no").field("type", "integer").endObject(); - } - createString("first_name", createIndex); - createString("last_name", createIndex); - createIndex.startObject("gender").field("type", "keyword"); - if (extraFields) { - createIndex.field("copy_to", "extra_gender"); - } createIndex.endObject(); - - if (extraFields) { - createIndex.startObject("extra_gender").field("type", "keyword").endObject(); - createIndex.startObject("extra.info.gender") - .field("type", "alias") - .field("path", "gender") - .endObject(); - } - - createIndex.startObject("birth_date").field("type", "date").endObject(); - createIndex.startObject("hire_date").field("type", "date").endObject(); - createIndex.startObject("salary").field("type", "integer").endObject(); - createIndex.startObject("languages").field("type", "byte").endObject(); - { - createIndex.startObject("dep").field("type", "nested"); - createIndex.startObject("properties"); - createIndex.startObject("dep_id").field("type", "keyword").endObject(); - createString("dep_name", createIndex); - createIndex.startObject("from_date").field("type", "date").endObject(); - createIndex.startObject("to_date").field("type", "date").endObject(); - createIndex.endObject(); - createIndex.endObject(); - } } - createIndex.endObject(); } createIndex.endObject(); } @@ -211,20 +207,16 @@ protected static void loadLogsDatasetIntoEs(RestClient client, String index, Str createIndex.endObject(); createIndex.startObject("mappings"); { - createIndex.startObject("_doc"); + createIndex.startObject("properties"); { - createIndex.startObject("properties"); - { - createIndex.startObject("id").field("type", "integer").endObject(); - createIndex.startObject("@timestamp").field("type", "date").endObject(); - createIndex.startObject("bytes_in").field("type", "integer").endObject(); - createIndex.startObject("bytes_out").field("type", "integer").endObject(); - createIndex.startObject("client_ip").field("type", "ip").endObject(); - createIndex.startObject("client_port").field("type", "integer").endObject(); - createIndex.startObject("dest_ip").field("type", "ip").endObject(); - createIndex.startObject("status").field("type", "keyword").endObject(); - } - createIndex.endObject(); + createIndex.startObject("id").field("type", "integer").endObject(); + createIndex.startObject("@timestamp").field("type", "date").endObject(); + createIndex.startObject("bytes_in").field("type", "integer").endObject(); + createIndex.startObject("bytes_out").field("type", "integer").endObject(); + createIndex.startObject("client_ip").field("type", "ip").endObject(); + createIndex.startObject("client_port").field("type", "integer").endObject(); + createIndex.startObject("dest_ip").field("type", "ip").endObject(); + createIndex.startObject("status").field("type", "keyword").endObject(); } createIndex.endObject(); } @@ -263,16 +255,12 @@ protected static void loadLibDatasetIntoEs(RestClient client, String index) thro createIndex.endObject(); createIndex.startObject("mappings"); { - createIndex.startObject("_doc"); + createIndex.startObject("properties"); { - createIndex.startObject("properties"); - { - createString("name", createIndex); - createString("author", createIndex); - createIndex.startObject("release_date").field("type", "date").endObject(); - createIndex.startObject("page_count").field("type", "short").endObject(); - } - createIndex.endObject(); + createString("name", createIndex); + createString("author", createIndex); + createIndex.startObject("release_date").field("type", "date").endObject(); + createIndex.startObject("page_count").field("type", "short").endObject(); } createIndex.endObject(); } @@ -329,4 +317,4 @@ private static void csvToLines(String name, CheckedBiConsumer, List public static InputStream readFromJarUrl(URL source) throws IOException { return source.openStream(); } -} \ No newline at end of file +} diff --git a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/FetchSizeTestCase.java b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/FetchSizeTestCase.java index 86fa4805ba6ea..3da3c0ba73bc2 100644 --- a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/FetchSizeTestCase.java +++ b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/FetchSizeTestCase.java @@ -30,16 +30,12 @@ public void createTestIndex() throws IOException { XContentBuilder createIndex = JsonXContent.contentBuilder().startObject(); createIndex.startObject("mappings"); { - createIndex.startObject("_doc"); + createIndex.startObject("properties"); { + createIndex.startObject("nested").field("type", "nested"); createIndex.startObject("properties"); - { - createIndex.startObject("nested").field("type", "nested"); - createIndex.startObject("properties"); - createIndex.startObject("inner_field").field("type", "integer").endObject(); - createIndex.endObject(); - createIndex.endObject(); - } + createIndex.startObject("inner_field").field("type", "integer").endObject(); + createIndex.endObject(); createIndex.endObject(); } createIndex.endObject(); diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml index 2208502b3e4cf..40fa404f36147 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml @@ -6,12 +6,11 @@ setup: index: foo body: mappings: - _doc: - properties: - the_field: - type: date - value_field: - type: integer + properties: + the_field: + type: date + value_field: + type: integer - do: headers: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml index 3e03ac924ec89..6332302e67418 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml @@ -6,12 +6,11 @@ setup: index: foo body: mappings: - _doc: - properties: - the_field: - type: date - value_field: - type: integer + properties: + the_field: + type: date + value_field: + type: integer --- "Test basic get_jobs": diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml index 7f3f0347ec0df..f8cfe85cc2e6a 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml @@ -6,12 +6,11 @@ setup: index: foo body: mappings: - _doc: - properties: - the_field: - type: date - value_field: - type: integer + properties: + the_field: + type: date + value_field: + type: integer --- "Test basic put_job": diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/rollup_search.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/rollup_search.yml index 0865cf084eb61..9af896f4c9fab 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/rollup_search.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/rollup_search.yml @@ -6,14 +6,13 @@ setup: index: foo body: mappings: - _doc: - properties: - timestamp: - type: date - partition: - type: keyword - price: - type: integer + properties: + timestamp: + type: date + partition: + type: keyword + price: + type: integer - do: headers: @@ -51,7 +50,6 @@ setup: body: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T05:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -67,7 +65,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T06:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -83,7 +80,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T07:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -99,7 +95,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T08:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -115,7 +110,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T08:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -319,7 +313,6 @@ setup: body: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T05:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -335,7 +328,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T06:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -351,7 +343,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T07:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -367,7 +358,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T08:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -383,7 +373,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T08:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -467,7 +456,6 @@ setup: body: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T05:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -483,7 +471,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T06:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -499,7 +486,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T07:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -515,7 +501,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T08:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -531,7 +516,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T08:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -617,7 +601,6 @@ setup: body: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T05:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -633,7 +616,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T06:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -649,7 +631,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T07:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -665,7 +646,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T08:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -681,7 +661,6 @@ setup: - index: _index: "foo_rollup" - _type: "_doc" - timestamp.date_histogram.timestamp: "2017-01-01T08:00:00Z" timestamp.date_histogram.interval: "1h" timestamp.date_histogram.time_zone: "UTC" @@ -760,14 +739,13 @@ setup: index: bar body: mappings: - _doc: - properties: - timestamp: - type: date - partition: - type: keyword - price: - type: integer + properties: + timestamp: + type: date + partition: + type: keyword + price: + type: integer - do: headers: @@ -849,14 +827,13 @@ setup: index: bar body: mappings: - _doc: - properties: - timestamp: - type: date - partition: - type: keyword - price: - type: integer + properties: + timestamp: + type: date + partition: + type: keyword + price: + type: integer - do: headers: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/security_tests.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/security_tests.yml index c42ab9d06f6ce..5a53847187484 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/security_tests.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/security_tests.yml @@ -48,18 +48,16 @@ teardown: index: foo body: mappings: - _doc: - properties: - timestamp: - type: date - value_field: - type: integer + properties: + timestamp: + type: date + value_field: + type: integer - do: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser index: index: foo - type: _doc body: timestamp: 123 value_field: 1232 @@ -69,18 +67,16 @@ teardown: index: foobar body: mappings: - _doc: - properties: - timestamp: - type: date - value_field: - type: integer + properties: + timestamp: + type: date + value_field: + type: integer - do: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser index: index: foobar - type: _doc body: timestamp: 123 value_field: 456 @@ -220,20 +216,18 @@ teardown: index: foo body: mappings: - _doc: - properties: - timestamp: - type: date - value_field: - type: integer - visibility: - type: keyword + properties: + timestamp: + type: date + value_field: + type: integer + visibility: + type: keyword - do: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser index: index: foo - type: _doc body: timestamp: 123 value_field: 1232 @@ -243,7 +237,6 @@ teardown: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser index: index: foobar - type: _doc body: timestamp: 123 value_field: 456 diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/start_job.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/start_job.yml index 0ed11ab1b04a0..88619e0dfc8ee 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/start_job.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/start_job.yml @@ -6,12 +6,11 @@ setup: index: foo body: mappings: - _doc: - properties: - the_field: - type: date - value_field: - type: integer + properties: + the_field: + type: date + value_field: + type: integer - do: headers: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/stop_job.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/stop_job.yml index c19325bf707de..bcb592be7a074 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/stop_job.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/stop_job.yml @@ -6,12 +6,11 @@ setup: index: foo body: mappings: - _doc: - properties: - the_field: - type: date - value_field: - type: integer + properties: + the_field: + type: date + value_field: + type: integer - do: headers: diff --git a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java index 1feb05b9c6468..feddc57f6e014 100644 --- a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java +++ b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java @@ -69,7 +69,7 @@ public void testBigRollup() throws Exception { try (XContentBuilder builder = jsonBuilder()) { builder.startObject(); { - builder.startObject("mappings").startObject("_doc") + builder.startObject("mappings") .startObject("properties") .startObject("timestamp") .field("type", "date") @@ -78,7 +78,6 @@ public void testBigRollup() throws Exception { .startObject("value") .field("type", "integer") .endObject() - .endObject() .endObject().endObject(); } builder.endObject(); From e4fa32470bfe19776715400f5d983f3f877afc8f Mon Sep 17 00:00:00 2001 From: Mayya Sharipova Date: Tue, 5 Feb 2019 14:15:43 -0500 Subject: [PATCH 19/23] Types removal fix FullClusterRestartIT warnings (#38445) Backport PR #38389 for 6.7 produces warnings for rollover test. This fixes FullClusterRestartIT warning expectations for rollover request Relates to #38389 --- .../java/org/elasticsearch/upgrades/FullClusterRestartIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index a5d939af5e405..9740ff4222d7e 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -518,6 +518,7 @@ public void testRollover() throws IOException { if (isRunningAgainstOldCluster()) { Request rolloverRequest = new Request("POST", "/" + index + "_write/_rollover"); + rolloverRequest.setOptions(allowTypeRemovalWarnings()); rolloverRequest.setJsonEntity("{" + " \"conditions\": {" + " \"max_docs\": 5" From aef5775561deaf64d3f877ec4b8a4cc36f40f62e Mon Sep 17 00:00:00 2001 From: Tal Levy Date: Tue, 5 Feb 2019 11:20:40 -0800 Subject: [PATCH 20/23] re-enables awaitsfixed datemath tests (#38376) Previously, date formats of `YYYY.MM.dd` would hit an issue where the year would jump towards the end of the calendar year. This was an issue that had since been resolved in tests by using `yyyy` to be the more accurate representation of the year. Closes #37037. --- .../admin/indices/rollover/RolloverIT.java | 19 +++++++++---------- .../DateMathExpressionResolverTests.java | 3 --- 2 files changed, 9 insertions(+), 13 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/RolloverIT.java b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/RolloverIT.java index 9f6f19596d080..4926c90c0a691 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/RolloverIT.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/RolloverIT.java @@ -272,11 +272,10 @@ public void testRolloverOnExistingIndex() throws Exception { } } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37037") public void testRolloverWithDateMath() { ZonedDateTime now = ZonedDateTime.now(ZoneOffset.UTC); assumeTrue("only works on the same day", now.plusMinutes(5).getDayOfYear() == now.getDayOfYear()); - String index = "test-" + DateFormatter.forPattern("YYYY.MM.dd").format(now) + "-1"; + String index = "test-" + DateFormatter.forPattern("yyyy.MM.dd").format(now) + "-1"; String dateMathExp = ""; assertAcked(prepareCreate(dateMathExp).addAlias(new Alias("test_alias")).get()); ensureGreen(index); @@ -284,34 +283,34 @@ public void testRolloverWithDateMath() { client().admin().indices().prepareClose(index).get(); client().admin().indices().prepareUpdateSettings(index).setSettings(Settings.builder() .put(IndexMetaData.SETTING_INDEX_PROVIDED_NAME, - "")).get(); + "")).get(); client().admin().indices().prepareOpen(index).get(); ensureGreen(index); RolloverResponse response = client().admin().indices().prepareRolloverIndex("test_alias").get(); assertThat(response.getOldIndex(), equalTo(index)); - assertThat(response.getNewIndex(), equalTo("test-" + DateFormatter.forPattern("YYYY.MM").format(now) + "-000002")); + assertThat(response.getNewIndex(), equalTo("test-" + DateFormatter.forPattern("yyyy.MM").format(now) + "-000002")); assertThat(response.isDryRun(), equalTo(false)); assertThat(response.isRolledOver(), equalTo(true)); assertThat(response.getConditionStatus().size(), equalTo(0)); response = client().admin().indices().prepareRolloverIndex("test_alias").get(); - assertThat(response.getOldIndex(), equalTo("test-" + DateFormatter.forPattern("YYYY.MM").format(now) + "-000002")); - assertThat(response.getNewIndex(), equalTo("test-" + DateFormatter.forPattern("YYYY.MM").format(now) + "-000003")); + assertThat(response.getOldIndex(), equalTo("test-" + DateFormatter.forPattern("yyyy.MM").format(now) + "-000002")); + assertThat(response.getNewIndex(), equalTo("test-" + DateFormatter.forPattern("yyyy.MM").format(now) + "-000003")); assertThat(response.isDryRun(), equalTo(false)); assertThat(response.isRolledOver(), equalTo(true)); assertThat(response.getConditionStatus().size(), equalTo(0)); GetSettingsResponse getSettingsResponse = client().admin().indices().prepareGetSettings(response.getOldIndex(), response.getNewIndex()).get(); - assertEquals("", getSettingsResponse.getSetting(response.getOldIndex(), + assertEquals("", getSettingsResponse.getSetting(response.getOldIndex(), IndexMetaData.SETTING_INDEX_PROVIDED_NAME)); - assertEquals("", getSettingsResponse.getSetting(response.getNewIndex(), + assertEquals("", getSettingsResponse.getSetting(response.getNewIndex(), IndexMetaData.SETTING_INDEX_PROVIDED_NAME)); response = client().admin().indices().prepareRolloverIndex("test_alias").setNewIndexName("").get(); - assertThat(response.getOldIndex(), equalTo("test-" + DateFormatter.forPattern("YYYY.MM").format(now) + "-000003")); - assertThat(response.getNewIndex(), equalTo("test-" + DateFormatter.forPattern("YYYY.MM.dd").format(now) + "-000004")); + assertThat(response.getOldIndex(), equalTo("test-" + DateFormatter.forPattern("yyyy.MM").format(now) + "-000003")); + assertThat(response.getNewIndex(), equalTo("test-" + DateFormatter.forPattern("yyyy.MM.dd").format(now) + "-000004")); assertThat(response.isDryRun(), equalTo(false)); assertThat(response.isRolledOver(), equalTo(true)); assertThat(response.getConditionStatus().size(), equalTo(0)); diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java index 2f52bd0d40aae..ee1a93646cfe3 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java @@ -91,7 +91,6 @@ public void testExpression_MultiParts() throws Exception { + DateTimeFormat.forPattern("YYYY.MM.dd").print(new DateTime(context.getStartTime(), UTC).withDayOfMonth(1)))); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37037") public void testExpression_CustomFormat() throws Exception { List results = expressionResolver.resolve(context, Arrays.asList("<.marvel-{now/d{yyyy.MM.dd}}>")); assertThat(results.size(), equalTo(1)); @@ -106,7 +105,6 @@ public void testExpression_EscapeStatic() throws Exception { equalTo(".mar{v}el-" + DateTimeFormat.forPattern("yyyy.MM.dd").print(new DateTime(context.getStartTime(), UTC)))); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37037") public void testExpression_EscapeDateFormat() throws Exception { List result = expressionResolver.resolve(context, Arrays.asList("<.marvel-{now/d{'\\{year\\}'yyyy}}>")); assertThat(result.size(), equalTo(1)); @@ -127,7 +125,6 @@ public void testExpression_MixedArray() throws Exception { DateTimeFormat.forPattern("YYYY.MM").print(new DateTime(context.getStartTime(), UTC).withDayOfMonth(1)))); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37037") public void testExpression_CustomTimeZoneInIndexName() throws Exception { DateTimeZone timeZone; int hoursOffset; From c2a8fe1f91e2cec7c930463f65987df4e4c094bf Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 5 Feb 2019 13:32:41 -0600 Subject: [PATCH 21/23] Prevent CCR recovery from missing documents (#38237) Currently the snapshot/restore process manually sets the global checkpoint to the max sequence number from the restored segements. This does not work for Ccr as this will lead to documents that would be recovered in the normal followering operation from being recovered. This commit fixes this issue by setting the initial global checkpoint to the existing local checkpoint. --- .../index/shard/StoreRecovery.java | 8 +- .../org/elasticsearch/index/store/Store.java | 11 +- .../index/shard/IndexShardTests.java | 131 ++-------------- .../index/shard/RestoreOnlyRepository.java | 146 ++++++++++++++++++ .../elasticsearch/test/BackgroundIndexer.java | 6 + .../ccr/action/TransportPutFollowAction.java | 26 ++-- .../elasticsearch/xpack/CcrIntegTestCase.java | 68 ++++++++ .../xpack/ccr/IndexFollowingIT.java | 136 ++++++++-------- .../engine/FollowEngineIndexShardTests.java | 78 ++++++++++ .../SourceOnlySnapshotRepository.java | 3 +- 10 files changed, 397 insertions(+), 216 deletions(-) create mode 100644 test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java diff --git a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index feb48ef85d1ba..d15de54c54e99 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -397,9 +397,9 @@ private void internalRecoverFromStore(IndexShard indexShard) throws IndexShardRe assert indexShouldExists; store.bootstrapNewHistory(); final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); - final long maxSeqNo = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.MAX_SEQ_NO)); + final long localCheckpoint = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); final String translogUUID = Translog.createEmptyTranslog( - indexShard.shardPath().resolveTranslog(), maxSeqNo, shardId, indexShard.getPendingPrimaryTerm()); + indexShard.shardPath().resolveTranslog(), localCheckpoint, shardId, indexShard.getPendingPrimaryTerm()); store.associateIndexWithNewTranslog(translogUUID); } else if (indexShouldExists) { if (recoveryState.getRecoverySource().shouldBootstrapNewHistoryUUID()) { @@ -466,9 +466,9 @@ private void restore(final IndexShard indexShard, final Repository repository, f final Store store = indexShard.store(); store.bootstrapNewHistory(); final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); - final long maxSeqNo = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.MAX_SEQ_NO)); + final long localCheckpoint = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); final String translogUUID = Translog.createEmptyTranslog( - indexShard.shardPath().resolveTranslog(), maxSeqNo, shardId, indexShard.getPendingPrimaryTerm()); + indexShard.shardPath().resolveTranslog(), localCheckpoint, shardId, indexShard.getPendingPrimaryTerm()); store.associateIndexWithNewTranslog(translogUUID); assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; indexShard.openEngineAndRecoverFromTranslog(); diff --git a/server/src/main/java/org/elasticsearch/index/store/Store.java b/server/src/main/java/org/elasticsearch/index/store/Store.java index 66e3e4d5558d8..73ac8a65d3007 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -1428,26 +1428,27 @@ public void bootstrapNewHistory() throws IOException { try { Map userData = readLastCommittedSegmentsInfo().getUserData(); final long maxSeqNo = Long.parseLong(userData.get(SequenceNumbers.MAX_SEQ_NO)); - bootstrapNewHistory(maxSeqNo); + final long localCheckpoint = Long.parseLong(userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); + bootstrapNewHistory(localCheckpoint, maxSeqNo); } finally { metadataLock.writeLock().unlock(); } } /** - * Marks an existing lucene index with a new history uuid and sets the given maxSeqNo as the local checkpoint + * Marks an existing lucene index with a new history uuid and sets the given local checkpoint * as well as the maximum sequence number. - * This is used to make sure no existing shard will recovery from this index using ops based recovery. + * This is used to make sure no existing shard will recover from this index using ops based recovery. * @see SequenceNumbers#LOCAL_CHECKPOINT_KEY * @see SequenceNumbers#MAX_SEQ_NO */ - public void bootstrapNewHistory(long maxSeqNo) throws IOException { + public void bootstrapNewHistory(long localCheckpoint, long maxSeqNo) throws IOException { metadataLock.writeLock().lock(); try (IndexWriter writer = newAppendingIndexWriter(directory, null)) { final Map map = new HashMap<>(); map.put(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID()); + map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(localCheckpoint)); map.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); - map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(maxSeqNo)); updateCommitData(writer, map); } finally { metadataLock.writeLock().unlock(); diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 12a7fad466e29..2851f43b1b990 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -21,7 +21,6 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; @@ -46,8 +45,6 @@ import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; -import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; @@ -62,7 +59,6 @@ import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.lease.Releasable; @@ -107,7 +103,6 @@ import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; -import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreStats; import org.elasticsearch.index.store.StoreUtils; @@ -121,12 +116,8 @@ import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryTarget; import org.elasticsearch.repositories.IndexId; -import org.elasticsearch.repositories.Repository; -import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; -import org.elasticsearch.snapshots.SnapshotInfo; -import org.elasticsearch.snapshots.SnapshotShardFailure; import org.elasticsearch.test.CorruptionUtils; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.FieldMaskingReader; @@ -143,7 +134,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -174,7 +164,6 @@ import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; -import static org.elasticsearch.repositories.RepositoryData.EMPTY_REPO_GEN; import static org.elasticsearch.test.hamcrest.RegexMatcher.matches; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; @@ -2159,9 +2148,12 @@ public void testRecoveryFailsAfterMovingToRelocatedState() throws InterruptedExc public void testRestoreShard() throws IOException { final IndexShard source = newStartedShard(true); - IndexShard target = newStartedShard(true); + IndexShard target = newStartedShard(true, Settings.builder() + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), source.indexSettings().isSoftDeleteEnabled()).build()); indexDoc(source, "_doc", "0"); + EngineTestCase.generateNewSeqNo(source.getEngine()); // create a gap in the history + indexDoc(source, "_doc", "2"); if (randomBoolean()) { source.refresh("test"); } @@ -2197,16 +2189,18 @@ public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version versio } } })); - assertThat(target.getLocalCheckpoint(), equalTo(0L)); - assertThat(target.seqNoStats().getMaxSeqNo(), equalTo(0L)); - assertThat(target.getReplicationTracker().getGlobalCheckpoint(), equalTo(0L)); + assertThat(target.getLocalCheckpoint(), equalTo(2L)); + assertThat(target.seqNoStats().getMaxSeqNo(), equalTo(2L)); + assertThat(target.seqNoStats().getGlobalCheckpoint(), equalTo(0L)); IndexShardTestCase.updateRoutingEntry(target, routing.moveToStarted()); assertThat(target.getReplicationTracker().getTrackedLocalCheckpointForShard( - target.routingEntry().allocationId().getId()).getLocalCheckpoint(), equalTo(0L)); + target.routingEntry().allocationId().getId()).getLocalCheckpoint(), equalTo(2L)); + assertThat(target.seqNoStats().getGlobalCheckpoint(), equalTo(2L)); - assertDocs(target, "0"); + assertDocs(target, "0", "2"); - closeShards(source, target); + closeShard(source, false); + closeShards(target); } public void testSearcherWrapperIsUsed() throws IOException { @@ -3131,107 +3125,6 @@ private Result indexOnReplicaWithGaps( return new Result(localCheckpoint, max); } - /** A dummy repository for testing which just needs restore overridden */ - private abstract static class RestoreOnlyRepository extends AbstractLifecycleComponent implements Repository { - private final String indexName; - - RestoreOnlyRepository(String indexName) { - this.indexName = indexName; - } - - @Override - protected void doStart() { - } - - @Override - protected void doStop() { - } - - @Override - protected void doClose() { - } - - @Override - public RepositoryMetaData getMetadata() { - return null; - } - - @Override - public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { - return null; - } - - @Override - public MetaData getSnapshotGlobalMetaData(SnapshotId snapshotId) { - return null; - } - - @Override - public IndexMetaData getSnapshotIndexMetaData(SnapshotId snapshotId, IndexId index) throws IOException { - return null; - } - - @Override - public RepositoryData getRepositoryData() { - Map> map = new HashMap<>(); - map.put(new IndexId(indexName, "blah"), emptySet()); - return new RepositoryData(EMPTY_REPO_GEN, Collections.emptyMap(), Collections.emptyMap(), map, Collections.emptyList()); - } - - @Override - public void initializeSnapshot(SnapshotId snapshotId, List indices, MetaData metaData) { - } - - @Override - public SnapshotInfo finalizeSnapshot(SnapshotId snapshotId, List indices, long startTime, String failure, - int totalShards, List shardFailures, long repositoryStateId, - boolean includeGlobalState) { - return null; - } - - @Override - public void deleteSnapshot(SnapshotId snapshotId, long repositoryStateId) { - } - - @Override - public long getSnapshotThrottleTimeInNanos() { - return 0; - } - - @Override - public long getRestoreThrottleTimeInNanos() { - return 0; - } - - @Override - public String startVerification() { - return null; - } - - @Override - public void endVerification(String verificationToken) { - } - - @Override - public boolean isReadOnly() { - return false; - } - - @Override - public void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, - IndexShardSnapshotStatus snapshotStatus) { - } - - @Override - public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId) { - return null; - } - - @Override - public void verify(String verificationToken, DiscoveryNode localNode) { - } - } - public void testIsSearchIdle() throws Exception { Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java new file mode 100644 index 0000000000000..11bdfb7bcc741 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java @@ -0,0 +1,146 @@ +/* + * 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.shard; + +import org.apache.lucene.index.IndexCommit; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.metadata.RepositoryMetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.Repository; +import org.elasticsearch.repositories.RepositoryData; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.snapshots.SnapshotInfo; +import org.elasticsearch.snapshots.SnapshotShardFailure; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static java.util.Collections.emptySet; +import static org.elasticsearch.repositories.RepositoryData.EMPTY_REPO_GEN; + +/** A dummy repository for testing which just needs restore overridden */ +public abstract class RestoreOnlyRepository extends AbstractLifecycleComponent implements Repository { + private final String indexName; + + public RestoreOnlyRepository(String indexName) { + this.indexName = indexName; + } + + @Override + protected void doStart() { + } + + @Override + protected void doStop() { + } + + @Override + protected void doClose() { + } + + @Override + public RepositoryMetaData getMetadata() { + return null; + } + + @Override + public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { + return null; + } + + @Override + public MetaData getSnapshotGlobalMetaData(SnapshotId snapshotId) { + return null; + } + + @Override + public IndexMetaData getSnapshotIndexMetaData(SnapshotId snapshotId, IndexId index) throws IOException { + return null; + } + + @Override + public RepositoryData getRepositoryData() { + Map> map = new HashMap<>(); + map.put(new IndexId(indexName, "blah"), emptySet()); + return new RepositoryData(EMPTY_REPO_GEN, Collections.emptyMap(), Collections.emptyMap(), map, Collections.emptyList()); + } + + @Override + public void initializeSnapshot(SnapshotId snapshotId, List indices, MetaData metaData) { + } + + @Override + public SnapshotInfo finalizeSnapshot(SnapshotId snapshotId, List indices, long startTime, String failure, + int totalShards, List shardFailures, long repositoryStateId, + boolean includeGlobalState) { + return null; + } + + @Override + public void deleteSnapshot(SnapshotId snapshotId, long repositoryStateId) { + } + + @Override + public long getSnapshotThrottleTimeInNanos() { + return 0; + } + + @Override + public long getRestoreThrottleTimeInNanos() { + return 0; + } + + @Override + public String startVerification() { + return null; + } + + @Override + public void endVerification(String verificationToken) { + } + + @Override + public boolean isReadOnly() { + return false; + } + + @Override + public void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, + IndexShardSnapshotStatus snapshotStatus) { + } + + @Override + public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId) { + return null; + } + + @Override + public void verify(String verificationToken, DiscoveryNode localNode) { + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java b/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java index eabb05a537ca7..ed3a836d2c506 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java +++ b/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java @@ -52,6 +52,7 @@ public class BackgroundIndexer implements AutoCloseable { private final Logger logger = LogManager.getLogger(getClass()); final Thread[] writers; + final Client client; final CountDownLatch stopLatch; final CopyOnWriteArrayList failures; final AtomicBoolean stop = new AtomicBoolean(false); @@ -122,6 +123,7 @@ public BackgroundIndexer(final String index, final String type, final Client cli if (random == null) { random = RandomizedTest.getRandom(); } + this.client = client; useAutoGeneratedIDs = random.nextBoolean(); failures = new CopyOnWriteArrayList<>(); writers = new Thread[writerCount]; @@ -316,6 +318,10 @@ public void close() throws Exception { stop(); } + public Client getClient() { + return client; + } + /** * Returns the ID set of all documents indexed by this indexer run */ diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutFollowAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutFollowAction.java index 84250baaeaa21..aa94071ac1d35 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutFollowAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutFollowAction.java @@ -218,21 +218,17 @@ private void initiateFollowing( final PutFollowAction.Request request, final ActionListener listener) { assert request.waitForActiveShards() != ActiveShardCount.DEFAULT : "PutFollowAction does not support DEFAULT."; - activeShardsObserver.waitForActiveShards(new String[]{request.getFollowerIndex()}, - request.waitForActiveShards(), request.timeout(), result -> { - if (result) { - FollowParameters parameters = request.getParameters(); - ResumeFollowAction.Request resumeFollowRequest = new ResumeFollowAction.Request(); - resumeFollowRequest.setFollowerIndex(request.getFollowerIndex()); - resumeFollowRequest.setParameters(new FollowParameters(parameters)); - client.execute(ResumeFollowAction.INSTANCE, resumeFollowRequest, ActionListener.wrap( - r -> listener.onResponse(new PutFollowAction.Response(true, true, r.isAcknowledged())), - listener::onFailure - )); - } else { - listener.onResponse(new PutFollowAction.Response(true, false, false)); - } - }, listener::onFailure); + FollowParameters parameters = request.getParameters(); + ResumeFollowAction.Request resumeFollowRequest = new ResumeFollowAction.Request(); + resumeFollowRequest.setFollowerIndex(request.getFollowerIndex()); + resumeFollowRequest.setParameters(new FollowParameters(parameters)); + client.execute(ResumeFollowAction.INSTANCE, resumeFollowRequest, ActionListener.wrap( + r -> activeShardsObserver.waitForActiveShards(new String[]{request.getFollowerIndex()}, + request.waitForActiveShards(), request.timeout(), result -> + listener.onResponse(new PutFollowAction.Response(true, result, r.isAcknowledged())), + listener::onFailure), + listener::onFailure + )); } @Override diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java index 8dd70a23cf8da..300bca4c7038c 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java @@ -44,6 +44,7 @@ import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; import org.elasticsearch.index.engine.DocIdSeqNoAndTerm; +import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -58,6 +59,7 @@ import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.tasks.TaskInfo; +import org.elasticsearch.test.BackgroundIndexer; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.InternalTestCluster; @@ -93,6 +95,8 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BooleanSupplier; import java.util.function.Function; import java.util.stream.Collectors; @@ -554,6 +558,70 @@ protected void assertMaxSeqNoOfUpdatesIsTransferred(Index leaderIndex, Index fol }); } + /** + * Waits until at least a give number of document is visible for searchers + * + * @param numDocs number of documents to wait for + * @param indexer a {@link org.elasticsearch.test.BackgroundIndexer}. Will be first checked for documents indexed. + * This saves on unneeded searches. + * @return the actual number of docs seen. + */ + public long waitForDocs(final long numDocs, final BackgroundIndexer indexer) throws InterruptedException { + // indexing threads can wait for up to ~1m before retrying when they first try to index into a shard which is not STARTED. + return waitForDocs(numDocs, 90, TimeUnit.SECONDS, indexer); + } + + /** + * Waits until at least a give number of document is visible for searchers + * + * @param numDocs number of documents to wait for + * @param maxWaitTime if not progress have been made during this time, fail the test + * @param maxWaitTimeUnit the unit in which maxWaitTime is specified + * @param indexer Will be first checked for documents indexed. + * This saves on unneeded searches. + * @return the actual number of docs seen. + */ + public long waitForDocs(final long numDocs, int maxWaitTime, TimeUnit maxWaitTimeUnit, final BackgroundIndexer indexer) + throws InterruptedException { + final AtomicLong lastKnownCount = new AtomicLong(-1); + long lastStartCount = -1; + BooleanSupplier testDocs = () -> { + lastKnownCount.set(indexer.totalIndexedDocs()); + if (lastKnownCount.get() >= numDocs) { + try { + long count = indexer.getClient().prepareSearch() + .setTrackTotalHits(true) + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .get() + .getHits().getTotalHits().value; + + if (count == lastKnownCount.get()) { + // no progress - try to refresh for the next time + indexer.getClient().admin().indices().prepareRefresh().get(); + } + lastKnownCount.set(count); + } catch (Exception e) { // count now acts like search and barfs if all shards failed... + logger.debug("failed to executed count", e); + return false; + } + logger.debug("[{}] docs visible for search. waiting for [{}]", lastKnownCount.get(), numDocs); + } else { + logger.debug("[{}] docs indexed. waiting for [{}]", lastKnownCount.get(), numDocs); + } + return lastKnownCount.get() >= numDocs; + }; + + while (!awaitBusy(testDocs, maxWaitTime, maxWaitTimeUnit)) { + if (lastStartCount == lastKnownCount.get()) { + // we didn't make any progress + fail("failed to reach " + numDocs + "docs"); + } + lastStartCount = lastKnownCount.get(); + } + return lastKnownCount.get(); + } + static void removeCCRRelatedMetadataFromClusterState(ClusterService clusterService) throws Exception { CountDownLatch latch = new CountDownLatch(1); clusterService.submitStateUpdateTask("remove-ccr-related-metadata", new ClusterStateUpdateTask() { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index 28f845fe7d463..80bded6a5d1d3 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -32,7 +32,6 @@ import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; @@ -61,6 +60,7 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.snapshots.SnapshotRestoreException; import org.elasticsearch.tasks.TaskInfo; +import org.elasticsearch.test.BackgroundIndexer; import org.elasticsearch.transport.NoSuchRemoteClusterException; import org.elasticsearch.xpack.CcrIntegTestCase; import org.elasticsearch.xpack.ccr.action.ShardFollowTask; @@ -118,93 +118,85 @@ public void testFollowIndex() throws Exception { } else { firstBatchNumDocs = randomIntBetween(10, 64); } - final int flushPoint = (int) (firstBatchNumDocs * 0.75); logger.info("Indexing [{}] docs as first batch", firstBatchNumDocs); - BulkRequestBuilder bulkRequestBuilder = leaderClient().prepareBulk(); - for (int i = 0; i < flushPoint; i++) { - final String source = String.format(Locale.ROOT, "{\"f\":%d}", i); - IndexRequest indexRequest = new IndexRequest("index1", "doc", Integer.toString(i)) - .source(source, XContentType.JSON) - .timeout(TimeValue.timeValueSeconds(1)); - bulkRequestBuilder.add(indexRequest); - } - bulkRequestBuilder.get(); - - leaderClient().admin().indices().prepareFlush("index1").setWaitIfOngoing(true).get(); - - // Index some docs after the flush that might be recovered in the normal index following operations - for (int i = flushPoint; i < firstBatchNumDocs; i++) { - final String source = String.format(Locale.ROOT, "{\"f\":%d}", i); - leaderClient().prepareIndex("index1", "doc", Integer.toString(i)).setSource(source, XContentType.JSON).get(); - } + try (BackgroundIndexer indexer = new BackgroundIndexer("index1", "_doc", leaderClient(), firstBatchNumDocs, + randomIntBetween(1, 5))) { + waitForDocs(randomInt(firstBatchNumDocs), indexer); + leaderClient().admin().indices().prepareFlush("index1").setWaitIfOngoing(true).get(); + waitForDocs(firstBatchNumDocs, indexer); + indexer.assertNoFailures(); - boolean waitOnAll = randomBoolean(); + boolean waitOnAll = randomBoolean(); - final PutFollowAction.Request followRequest; - if (waitOnAll) { - followRequest = putFollow("index1", "index2", ActiveShardCount.ALL); - } else { - followRequest = putFollow("index1", "index2", ActiveShardCount.ONE); - } - PutFollowAction.Response response = followerClient().execute(PutFollowAction.INSTANCE, followRequest).get(); - assertTrue(response.isFollowIndexCreated()); - assertTrue(response.isFollowIndexShardsAcked()); - assertTrue(response.isIndexFollowingStarted()); - - ClusterHealthRequest healthRequest = Requests.clusterHealthRequest("index2").waitForNoRelocatingShards(true); - ClusterIndexHealth indexHealth = followerClient().admin().cluster().health(healthRequest).actionGet().getIndices().get("index2"); - for (ClusterShardHealth shardHealth : indexHealth.getShards().values()) { + final PutFollowAction.Request followRequest; if (waitOnAll) { - assertTrue(shardHealth.isPrimaryActive()); - assertEquals(1 + numberOfReplicas, shardHealth.getActiveShards()); + followRequest = putFollow("index1", "index2", ActiveShardCount.ALL); } else { - assertTrue(shardHealth.isPrimaryActive()); + followRequest = putFollow("index1", "index2", ActiveShardCount.ONE); } - } - - final Map firstBatchNumDocsPerShard = new HashMap<>(); - final ShardStats[] firstBatchShardStats = - leaderClient().admin().indices().prepareStats("index1").get().getIndex("index1").getShards(); - for (final ShardStats shardStats : firstBatchShardStats) { - if (shardStats.getShardRouting().primary()) { - long value = shardStats.getStats().getIndexing().getTotal().getIndexCount() - 1; - firstBatchNumDocsPerShard.put(shardStats.getShardRouting().shardId(), value); + PutFollowAction.Response response = followerClient().execute(PutFollowAction.INSTANCE, followRequest).get(); + assertTrue(response.isFollowIndexCreated()); + assertTrue(response.isFollowIndexShardsAcked()); + assertTrue(response.isIndexFollowingStarted()); + + ClusterHealthRequest healthRequest = Requests.clusterHealthRequest("index2").waitForNoRelocatingShards(true); + ClusterIndexHealth indexHealth = followerClient().admin().cluster().health(healthRequest).get().getIndices().get("index2"); + for (ClusterShardHealth shardHealth : indexHealth.getShards().values()) { + if (waitOnAll) { + assertTrue(shardHealth.isPrimaryActive()); + assertEquals(1 + numberOfReplicas, shardHealth.getActiveShards()); + } else { + assertTrue(shardHealth.isPrimaryActive()); + } } - } - assertBusy(assertTask(numberOfPrimaryShards, firstBatchNumDocsPerShard)); + final Map firstBatchNumDocsPerShard = new HashMap<>(); + final ShardStats[] firstBatchShardStats = + leaderClient().admin().indices().prepareStats("index1").get().getIndex("index1").getShards(); + for (final ShardStats shardStats : firstBatchShardStats) { + if (shardStats.getShardRouting().primary()) { + long value = shardStats.getStats().getIndexing().getTotal().getIndexCount() - 1; + firstBatchNumDocsPerShard.put(shardStats.getShardRouting().shardId(), value); + } + } - for (int i = 0; i < firstBatchNumDocs; i++) { - assertBusy(assertExpectedDocumentRunnable(i)); - } + assertBusy(assertTask(numberOfPrimaryShards, firstBatchNumDocsPerShard)); - pauseFollow("index2"); - followerClient().execute(ResumeFollowAction.INSTANCE, resumeFollow("index2")).get(); - final int secondBatchNumDocs = randomIntBetween(2, 64); - logger.info("Indexing [{}] docs as second batch", secondBatchNumDocs); - for (int i = firstBatchNumDocs; i < firstBatchNumDocs + secondBatchNumDocs; i++) { - final String source = String.format(Locale.ROOT, "{\"f\":%d}", i); - leaderClient().prepareIndex("index1", "doc", Integer.toString(i)).setSource(source, XContentType.JSON).get(); - } + for (String docId : indexer.getIds()) { + assertBusy(() -> { + final GetResponse getResponse = followerClient().prepareGet("index2", "_doc", docId).get(); + assertTrue("Doc with id [" + docId + "] is missing", getResponse.isExists()); + }); + } - final Map secondBatchNumDocsPerShard = new HashMap<>(); - final ShardStats[] secondBatchShardStats = - leaderClient().admin().indices().prepareStats("index1").get().getIndex("index1").getShards(); - for (final ShardStats shardStats : secondBatchShardStats) { - if (shardStats.getShardRouting().primary()) { - final long value = shardStats.getStats().getIndexing().getTotal().getIndexCount() - 1; - secondBatchNumDocsPerShard.put(shardStats.getShardRouting().shardId(), value); + pauseFollow("index2"); + followerClient().execute(ResumeFollowAction.INSTANCE, resumeFollow("index2")).get(); + final int secondBatchNumDocs = randomIntBetween(2, 64); + logger.info("Indexing [{}] docs as second batch", secondBatchNumDocs); + indexer.continueIndexing(secondBatchNumDocs); + + final Map secondBatchNumDocsPerShard = new HashMap<>(); + final ShardStats[] secondBatchShardStats = + leaderClient().admin().indices().prepareStats("index1").get().getIndex("index1").getShards(); + for (final ShardStats shardStats : secondBatchShardStats) { + if (shardStats.getShardRouting().primary()) { + final long value = shardStats.getStats().getIndexing().getTotal().getIndexCount() - 1; + secondBatchNumDocsPerShard.put(shardStats.getShardRouting().shardId(), value); + } } - } - assertBusy(assertTask(numberOfPrimaryShards, secondBatchNumDocsPerShard)); + assertBusy(assertTask(numberOfPrimaryShards, secondBatchNumDocsPerShard)); - for (int i = firstBatchNumDocs; i < firstBatchNumDocs + secondBatchNumDocs; i++) { - assertBusy(assertExpectedDocumentRunnable(i)); + for (String docId : indexer.getIds()) { + assertBusy(() -> { + final GetResponse getResponse = followerClient().prepareGet("index2", "_doc", docId).get(); + assertTrue("Doc with id [" + docId + "] is missing", getResponse.isExists()); + }); + } + pauseFollow("index2"); + assertMaxSeqNoOfUpdatesIsTransferred(resolveLeaderIndex("index1"), resolveFollowerIndex("index2"), numberOfPrimaryShards); } - pauseFollow("index2"); - assertMaxSeqNoOfUpdatesIsTransferred(resolveLeaderIndex("index1"), resolveFollowerIndex("index2"), numberOfPrimaryShards); } public void testFollowIndexWithConcurrentMappingChanges() throws Exception { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowEngineIndexShardTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowEngineIndexShardTests.java index 8d3c0c3b472aa..1326f0ebc79bb 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowEngineIndexShardTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowEngineIndexShardTests.java @@ -5,27 +5,46 @@ */ package org.elasticsearch.xpack.ccr.index.engine; +import org.apache.lucene.store.IOContext; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; +import org.elasticsearch.index.shard.RestoreOnlyRepository; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.indices.recovery.RecoveryState; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.snapshots.Snapshot; +import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.ccr.CcrSettings; +import java.io.IOException; import java.util.Collections; import java.util.concurrent.CountDownLatch; +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; import static org.elasticsearch.cluster.routing.TestShardRouting.newShardRouting; +import static org.elasticsearch.common.lucene.Lucene.cleanLuceneIndex; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; public class FollowEngineIndexShardTests extends IndexShardTestCase { @@ -76,4 +95,63 @@ public void testDoNotFillGaps() throws Exception { closeShards(indexShard); } + public void testRestoreShard() throws IOException { + final Settings sourceSettings = Settings.builder() + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .build(); + final IndexShard source = newStartedShard(true, sourceSettings); + final Settings targetSettings = Settings.builder() + .put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .build(); + IndexShard target = newStartedShard(true, targetSettings, new FollowingEngineFactory()); + assertThat(IndexShardTestCase.getEngine(target), instanceOf(FollowingEngine.class)); + + indexDoc(source, "_doc", "0"); + EngineTestCase.generateNewSeqNo(IndexShardTestCase.getEngine(source)); + indexDoc(source, "_doc", "2"); + if (randomBoolean()) { + source.refresh("test"); + } + flushShard(source); // only flush source + ShardRouting routing = ShardRoutingHelper.initWithSameId(target.routingEntry(), + RecoverySource.ExistingStoreRecoverySource.INSTANCE); + final Snapshot snapshot = new Snapshot("foo", new SnapshotId("bar", UUIDs.randomBase64UUID())); + routing = ShardRoutingHelper.newWithRestoreSource(routing, + new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT, "test")); + target = reinitShard(target, routing); + Store sourceStore = source.store(); + Store targetStore = target.store(); + + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); + target.markAsRecovering("store", new RecoveryState(routing, localNode, null)); + assertTrue(target.restoreFromRepository(new RestoreOnlyRepository("test") { + @Override + public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, + RecoveryState recoveryState) { + try { + cleanLuceneIndex(targetStore.directory()); + for (String file : sourceStore.directory().listAll()) { + if (file.equals("write.lock") || file.startsWith("extra")) { + continue; + } + targetStore.directory().copyFrom(sourceStore.directory(), file, file, IOContext.DEFAULT); + } + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + })); + assertThat(target.getLocalCheckpoint(), equalTo(0L)); + assertThat(target.seqNoStats().getMaxSeqNo(), equalTo(2L)); + assertThat(target.seqNoStats().getGlobalCheckpoint(), equalTo(0L)); + IndexShardTestCase.updateRoutingEntry(target, routing.moveToStarted()); + assertThat(target.seqNoStats().getGlobalCheckpoint(), equalTo(0L)); + + assertDocs(target, "0", "2"); + + closeShard(source, false); + closeShards(target); + } + } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 18e96619ec822..3e36fc5977491 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -128,7 +128,8 @@ protected void closeInternal() { snapshot.syncSnapshot(snapshotIndexCommit); // we will use the lucene doc ID as the seq ID so we set the local checkpoint to maxDoc with a new index UUID SegmentInfos segmentInfos = tempStore.readLastCommittedSegmentsInfo(); - tempStore.bootstrapNewHistory(segmentInfos.totalMaxDoc()); + final long maxDoc = segmentInfos.totalMaxDoc(); + tempStore.bootstrapNewHistory(maxDoc, maxDoc); store.incRef(); try (DirectoryReader reader = DirectoryReader.open(tempStore.directory())) { IndexCommit indexCommit = reader.getIndexCommit(); From 4a15e2b29e9fdca39bb5e224415f653eeae5fdf4 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 5 Feb 2019 13:34:00 -0600 Subject: [PATCH 22/23] Make Ccr recovery file chunk size configurable (#38370) This commit adds a byte setting `ccr.indices.recovery.chunk_size`. This setting configs the size of file chunk requested while recovering from remote. --- .../elasticsearch/xpack/ccr/CcrSettings.java | 20 +++++++++++++++++++ .../xpack/ccr/repository/CcrRepository.java | 4 +--- .../xpack/ccr/CcrRepositoryIT.java | 11 +++++++++- .../CcrRestoreSourceServiceTests.java | 3 ++- 4 files changed, 33 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java index 625429dc0abc6..0e147f66d6ebc 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java @@ -49,6 +49,14 @@ public final class CcrSettings { Setting.byteSizeSetting("ccr.indices.recovery.max_bytes_per_sec", new ByteSizeValue(40, ByteSizeUnit.MB), Setting.Property.Dynamic, Setting.Property.NodeScope); + /** + * File chunk size to send during recovery + */ + public static final Setting RECOVERY_CHUNK_SIZE = + Setting.byteSizeSetting("ccr.indices.recovery.chunk_size", new ByteSizeValue(1, ByteSizeUnit.MB), + new ByteSizeValue(1, ByteSizeUnit.KB), new ByteSizeValue(1, ByteSizeUnit.GB), Setting.Property.Dynamic, + Setting.Property.NodeScope); + /** * The leader must open resources for a ccr recovery. If there is no activity for this interval of time, * the leader will close the restore session. @@ -77,22 +85,30 @@ static List> getSettings() { INDICES_RECOVERY_ACTION_TIMEOUT_SETTING, INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING, CCR_AUTO_FOLLOW_WAIT_FOR_METADATA_TIMEOUT, + RECOVERY_CHUNK_SIZE, CCR_WAIT_FOR_METADATA_TIMEOUT); } private final CombinedRateLimiter ccrRateLimiter; private volatile TimeValue recoveryActivityTimeout; private volatile TimeValue recoveryActionTimeout; + private volatile ByteSizeValue chunkSize; public CcrSettings(Settings settings, ClusterSettings clusterSettings) { this.recoveryActivityTimeout = INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING.get(settings); this.recoveryActionTimeout = INDICES_RECOVERY_ACTION_TIMEOUT_SETTING.get(settings); this.ccrRateLimiter = new CombinedRateLimiter(RECOVERY_MAX_BYTES_PER_SECOND.get(settings)); + this.chunkSize = RECOVERY_MAX_BYTES_PER_SECOND.get(settings); clusterSettings.addSettingsUpdateConsumer(RECOVERY_MAX_BYTES_PER_SECOND, this::setMaxBytesPerSec); + clusterSettings.addSettingsUpdateConsumer(RECOVERY_CHUNK_SIZE, this::setChunkSize); clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING, this::setRecoveryActivityTimeout); clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_ACTION_TIMEOUT_SETTING, this::setRecoveryActionTimeout); } + private void setChunkSize(ByteSizeValue chunkSize) { + this.chunkSize = chunkSize; + } + private void setMaxBytesPerSec(ByteSizeValue maxBytesPerSec) { ccrRateLimiter.setMBPerSec(maxBytesPerSec); } @@ -105,6 +121,10 @@ private void setRecoveryActionTimeout(TimeValue recoveryActionTimeout) { this.recoveryActionTimeout = recoveryActionTimeout; } + public ByteSizeValue getChunkSize() { + return chunkSize; + } + public CombinedRateLimiter getRateLimiter() { return ccrRateLimiter; } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index baad95d5a94df..7ceaeb903ec0e 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -329,8 +329,6 @@ private RestoreSession openSession(String repositoryName, Client remoteClient, S private static class RestoreSession extends FileRestoreContext implements Closeable { - private static final int BUFFER_SIZE = 1 << 16; - private final Client remoteClient; private final String sessionUUID; private final DiscoveryNode node; @@ -342,7 +340,7 @@ private static class RestoreSession extends FileRestoreContext implements Closea RestoreSession(String repositoryName, Client remoteClient, String sessionUUID, DiscoveryNode node, IndexShard indexShard, RecoveryState recoveryState, Store.MetadataSnapshot sourceMetaData, long mappingVersion, CcrSettings ccrSettings, LongConsumer throttleListener) { - super(repositoryName, indexShard, SNAPSHOT_ID, recoveryState, BUFFER_SIZE); + super(repositoryName, indexShard, SNAPSHOT_ID, recoveryState, Math.toIntExact(ccrSettings.getChunkSize().getBytes())); this.remoteClient = remoteClient; this.sessionUUID = sessionUUID; this.node = node; diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java index d4d6d13f7a292..9f061b9c33099 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java @@ -169,8 +169,12 @@ public void testThatRepositoryRecoversEmptyIndexBasedOnLeaderSettings() throws I assertNotEquals(leaderMetadata.getIndexUUID(), followerMetadata.getIndexUUID()); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/38100") public void testDocsAreRecovered() throws Exception { + ClusterUpdateSettingsRequest settingsRequest = new ClusterUpdateSettingsRequest(); + String chunkSize = randomFrom("4KB", "128KB", "1MB"); + settingsRequest.persistentSettings(Settings.builder().put(CcrSettings.RECOVERY_CHUNK_SIZE.getKey(), chunkSize)); + assertAcked(followerClient().admin().cluster().updateSettings(settingsRequest).actionGet()); + String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster"; String leaderIndex = "index1"; String followerIndex = "index2"; @@ -243,6 +247,11 @@ public void testDocsAreRecovered() throws Exception { isRunning.set(false); thread.join(); + + settingsRequest = new ClusterUpdateSettingsRequest(); + ByteSizeValue defaultValue = CcrSettings.RECOVERY_CHUNK_SIZE.getDefault(Settings.EMPTY); + settingsRequest.persistentSettings(Settings.builder().put(CcrSettings.RECOVERY_CHUNK_SIZE.getKey(), defaultValue)); + assertAcked(followerClient().admin().cluster().updateSettings(settingsRequest).actionGet()); } public void testRateLimitingIsEmployed() throws Exception { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java index 3035b96b5bcac..1c3c0da3d3c8a 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java @@ -40,7 +40,8 @@ public void setUp() throws Exception { Settings settings = Settings.builder().put(NODE_NAME_SETTING.getKey(), "node").build(); taskQueue = new DeterministicTaskQueue(settings, random()); Set> registeredSettings = Sets.newHashSet(CcrSettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING, - CcrSettings.RECOVERY_MAX_BYTES_PER_SECOND, CcrSettings.INDICES_RECOVERY_ACTION_TIMEOUT_SETTING); + CcrSettings.RECOVERY_MAX_BYTES_PER_SECOND, CcrSettings.INDICES_RECOVERY_ACTION_TIMEOUT_SETTING, + CcrSettings.RECOVERY_CHUNK_SIZE); ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, registeredSettings); restoreSourceService = new CcrRestoreSourceService(taskQueue.getThreadPool(), new CcrSettings(Settings.EMPTY, clusterSettings)); } From b03d138122d61c9ee074b747ce040975e9e300b3 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 5 Feb 2019 14:42:17 -0500 Subject: [PATCH 23/23] Lift retention lease expiration to index shard (#38380) This commit lifts the control of when retention leases are expired to index shard. In this case, we move expiration to an explicit action rather than a side-effect of calling ReplicationTracker#getRetentionLeases. This explicit action is invoked on a timer. If any retention leases expire, then we hard sync the retention leases to the replicas. Otherwise, we proceed with a background sync. --- .../org/elasticsearch/index/IndexService.java | 22 +-- .../elasticsearch/index/IndexSettings.java | 5 + .../index/seqno/ReplicationTracker.java | 76 +++++----- .../elasticsearch/index/shard/IndexShard.java | 26 +++- ...ReplicationTrackerRetentionLeaseTests.java | 134 ++++-------------- .../index/seqno/RetentionLeaseSyncIT.java | 72 +++++++--- .../shard/IndexShardRetentionLeaseTests.java | 69 +++++---- 7 files changed, 192 insertions(+), 212 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index 57d8cc0b32641..1b1784495e685 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -121,7 +121,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private volatile AsyncRefreshTask refreshTask; private volatile AsyncTranslogFSync fsyncTask; private volatile AsyncGlobalCheckpointTask globalCheckpointTask; - private volatile AsyncRetentionLeaseBackgroundSyncTask retentionLeaseBackgroundSyncTask; + private volatile AsyncRetentionLeaseSyncTask retentionLeaseSyncTask; // don't convert to Setting<> and register... we only set this in tests and register via a plugin private final String INDEX_TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING = "index.translog.retention.check_interval"; @@ -198,7 +198,7 @@ public IndexService( this.refreshTask = new AsyncRefreshTask(this); this.trimTranslogTask = new AsyncTrimTranslogTask(this); this.globalCheckpointTask = new AsyncGlobalCheckpointTask(this); - this.retentionLeaseBackgroundSyncTask = new AsyncRetentionLeaseBackgroundSyncTask(this); + this.retentionLeaseSyncTask = new AsyncRetentionLeaseSyncTask(this); rescheduleFsyncTask(indexSettings.getTranslogDurability()); } @@ -289,7 +289,7 @@ public synchronized void close(final String reason, boolean delete) throws IOExc fsyncTask, trimTranslogTask, globalCheckpointTask, - retentionLeaseBackgroundSyncTask); + retentionLeaseSyncTask); } } } @@ -788,8 +788,8 @@ private void maybeSyncGlobalCheckpoints() { sync(is -> is.maybeSyncGlobalCheckpoint("background"), "global checkpoint"); } - private void backgroundSyncRetentionLeases() { - sync(IndexShard::backgroundSyncRetentionLeases, "retention lease"); + private void syncRetentionLeases() { + sync(IndexShard::syncRetentionLeases, "retention lease"); } private void sync(final Consumer sync, final String source) { @@ -812,11 +812,11 @@ private void sync(final Consumer sync, final String source) { && e instanceof IndexShardClosedException == false) { logger.warn( new ParameterizedMessage( - "{} failed to execute background {} sync", shard.shardId(), source), e); + "{} failed to execute {} sync", shard.shardId(), source), e); } }, ThreadPool.Names.SAME, - "background " + source + " sync"); + source + " sync"); } catch (final AlreadyClosedException | IndexShardClosedException e) { // the shard was closed concurrently, continue } @@ -957,15 +957,15 @@ public String toString() { } } - final class AsyncRetentionLeaseBackgroundSyncTask extends BaseAsyncTask { + final class AsyncRetentionLeaseSyncTask extends BaseAsyncTask { - AsyncRetentionLeaseBackgroundSyncTask(final IndexService indexService) { + AsyncRetentionLeaseSyncTask(final IndexService indexService) { super(indexService, RETENTION_LEASE_SYNC_INTERVAL_SETTING.get(indexService.getIndexSettings().getSettings())); } @Override protected void runInternal() { - indexService.backgroundSyncRetentionLeases(); + indexService.syncRetentionLeases(); } @Override @@ -975,7 +975,7 @@ protected String getThreadPool() { @Override public String toString() { - return "retention_lease_background_sync"; + return "retention_lease_sync"; } } diff --git a/server/src/main/java/org/elasticsearch/index/IndexSettings.java b/server/src/main/java/org/elasticsearch/index/IndexSettings.java index 4d9a8f7d37b70..97b499f9bd309 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexSettings.java +++ b/server/src/main/java/org/elasticsearch/index/IndexSettings.java @@ -339,6 +339,10 @@ public long getRetentionLeaseMillis() { return retentionLeaseMillis; } + private void setRetentionLeaseMillis(final TimeValue retentionLease) { + this.retentionLeaseMillis = retentionLease.millis(); + } + private volatile boolean warmerEnabled; private volatile int maxResultWindow; private volatile int maxInnerResultWindow; @@ -523,6 +527,7 @@ public IndexSettings(final IndexMetaData indexMetaData, final Settings nodeSetti scopedSettings.addSettingsUpdateConsumer(DEFAULT_PIPELINE, this::setDefaultPipeline); scopedSettings.addSettingsUpdateConsumer(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, this::setSoftDeleteRetentionOperations); scopedSettings.addSettingsUpdateConsumer(INDEX_SEARCH_THROTTLED, this::setSearchThrottled); + scopedSettings.addSettingsUpdateConsumer(INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING, this::setRetentionLeaseMillis); } private void setSearchIdleAfter(TimeValue searchIdleAfter) { this.searchIdleAfter = searchIdleAfter; } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java index 3b68dfa6addae..31f491d24cf9d 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -28,6 +28,7 @@ import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -155,10 +156,10 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L private final LongSupplier currentTimeMillisSupplier; /** - * A callback when a new retention lease is created or an existing retention lease expires. In practice, this callback invokes the - * retention lease sync action, to sync retention leases to replicas. + * A callback when a new retention lease is created. In practice, this callback invokes the retention lease sync action, to sync + * retention leases to replicas. */ - private final BiConsumer> onSyncRetentionLeases; + private final BiConsumer> onAddRetentionLease; /** * This set contains allocation IDs for which there is a thread actively waiting for the local checkpoint to advance to at least the @@ -177,43 +178,42 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L private RetentionLeases retentionLeases = RetentionLeases.EMPTY; /** - * Get all non-expired retention leases tracked on this shard. Note that only the primary shard calculates which leases are expired, - * and if any have expired, syncs the retention leases to any replicas. + * Get all retention leases tracked on this shard. * * @return the retention leases */ public RetentionLeases getRetentionLeases() { - final boolean wasPrimaryMode; - final RetentionLeases nonExpiredRetentionLeases; - synchronized (this) { - if (primaryMode) { - // the primary calculates the non-expired retention leases and syncs them to replicas - final long currentTimeMillis = currentTimeMillisSupplier.getAsLong(); - final long retentionLeaseMillis = indexSettings.getRetentionLeaseMillis(); - final Map> partitionByExpiration = retentionLeases - .leases() - .stream() - .collect(Collectors.groupingBy(lease -> currentTimeMillis - lease.timestamp() > retentionLeaseMillis)); - if (partitionByExpiration.get(true) == null) { - // early out as no retention leases have expired - return retentionLeases; - } - final Collection nonExpiredLeases = - partitionByExpiration.get(false) != null ? partitionByExpiration.get(false) : Collections.emptyList(); - retentionLeases = new RetentionLeases(operationPrimaryTerm, retentionLeases.version() + 1, nonExpiredLeases); - } - /* - * At this point, we were either in primary mode and have updated the non-expired retention leases into the tracking map, or - * we were in replica mode and merely need to copy the existing retention leases since a replica does not calculate the - * non-expired retention leases, instead receiving them on syncs from the primary. - */ - wasPrimaryMode = primaryMode; - nonExpiredRetentionLeases = retentionLeases; + return getRetentionLeases(false).v2(); + } + + /** + * If the expire leases parameter is false, gets all retention leases tracked on this shard and otherwise first calculates + * expiration of existing retention leases, and then gets all non-expired retention leases tracked on this shard. Note that only the + * primary shard calculates which leases are expired, and if any have expired, syncs the retention leases to any replicas. If the + * expire leases parameter is true, this replication tracker must be in primary mode. + * + * @return a tuple indicating whether or not any retention leases were expired, and the non-expired retention leases + */ + public synchronized Tuple getRetentionLeases(final boolean expireLeases) { + if (expireLeases == false) { + return Tuple.tuple(false, retentionLeases); } - if (wasPrimaryMode) { - onSyncRetentionLeases.accept(nonExpiredRetentionLeases, ActionListener.wrap(() -> {})); + assert primaryMode; + // the primary calculates the non-expired retention leases and syncs them to replicas + final long currentTimeMillis = currentTimeMillisSupplier.getAsLong(); + final long retentionLeaseMillis = indexSettings.getRetentionLeaseMillis(); + final Map> partitionByExpiration = retentionLeases + .leases() + .stream() + .collect(Collectors.groupingBy(lease -> currentTimeMillis - lease.timestamp() > retentionLeaseMillis)); + if (partitionByExpiration.get(true) == null) { + // early out as no retention leases have expired + return Tuple.tuple(false, retentionLeases); } - return nonExpiredRetentionLeases; + final Collection nonExpiredLeases = + partitionByExpiration.get(false) != null ? partitionByExpiration.get(false) : Collections.emptyList(); + retentionLeases = new RetentionLeases(operationPrimaryTerm, retentionLeases.version() + 1, nonExpiredLeases); + return Tuple.tuple(true, retentionLeases); } /** @@ -246,7 +246,7 @@ public RetentionLease addRetentionLease( Stream.concat(retentionLeases.leases().stream(), Stream.of(retentionLease)).collect(Collectors.toList())); currentRetentionLeases = retentionLeases; } - onSyncRetentionLeases.accept(currentRetentionLeases, listener); + onAddRetentionLease.accept(currentRetentionLeases, listener); return retentionLease; } @@ -563,7 +563,7 @@ private static long inSyncCheckpointStates( * @param indexSettings the index settings * @param operationPrimaryTerm the current primary term * @param globalCheckpoint the last known global checkpoint for this shard, or {@link SequenceNumbers#UNASSIGNED_SEQ_NO} - * @param onSyncRetentionLeases a callback when a new retention lease is created or an existing retention lease expires + * @param onAddRetentionLease a callback when a new retention lease is created or an existing retention lease expires */ public ReplicationTracker( final ShardId shardId, @@ -573,7 +573,7 @@ public ReplicationTracker( final long globalCheckpoint, final LongConsumer onGlobalCheckpointUpdated, final LongSupplier currentTimeMillisSupplier, - final BiConsumer> onSyncRetentionLeases) { + final BiConsumer> onAddRetentionLease) { super(shardId, indexSettings); assert globalCheckpoint >= SequenceNumbers.UNASSIGNED_SEQ_NO : "illegal initial global checkpoint: " + globalCheckpoint; this.shardAllocationId = allocationId; @@ -585,7 +585,7 @@ public ReplicationTracker( checkpoints.put(allocationId, new CheckpointState(SequenceNumbers.UNASSIGNED_SEQ_NO, globalCheckpoint, false, false)); this.onGlobalCheckpointUpdated = Objects.requireNonNull(onGlobalCheckpointUpdated); this.currentTimeMillisSupplier = Objects.requireNonNull(currentTimeMillisSupplier); - this.onSyncRetentionLeases = Objects.requireNonNull(onSyncRetentionLeases); + this.onAddRetentionLease = Objects.requireNonNull(onAddRetentionLease); this.pendingInSync = new HashSet<>(); this.routingTable = null; this.replicationGroup = null; diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index c3d653e2fde06..dfecdc173951e 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1892,13 +1892,26 @@ public void addGlobalCheckpointListener( } /** - * Get all non-expired retention leases tracked on this shard. + * Get all retention leases tracked on this shard. * * @return the retention leases */ public RetentionLeases getRetentionLeases() { + return getRetentionLeases(false).v2(); + } + + /** + * If the expire leases parameter is false, gets all retention leases tracked on this shard and otherwise first calculates + * expiration of existing retention leases, and then gets all non-expired retention leases tracked on this shard. Note that only the + * primary shard calculates which leases are expired, and if any have expired, syncs the retention leases to any replicas. If the + * expire leases parameter is true, this replication tracker must be in primary mode. + * + * @return a tuple indicating whether or not any retention leases were expired, and the non-expired retention leases + */ + public Tuple getRetentionLeases(final boolean expireLeases) { + assert expireLeases == false || assertPrimaryMode(); verifyNotClosed(); - return replicationTracker.getRetentionLeases(); + return replicationTracker.getRetentionLeases(expireLeases); } public RetentionLeaseStats getRetentionLeaseStats() { @@ -1956,10 +1969,15 @@ public void updateRetentionLeasesOnReplica(final RetentionLeases retentionLeases /** * Syncs the current retention leases to all replicas. */ - public void backgroundSyncRetentionLeases() { + public void syncRetentionLeases() { assert assertPrimaryMode(); verifyNotClosed(); - retentionLeaseSyncer.backgroundSync(shardId, getRetentionLeases()); + final Tuple retentionLeases = getRetentionLeases(true); + if (retentionLeases.v1()) { + retentionLeaseSyncer.sync(shardId, retentionLeases.v2(), ActionListener.wrap(() -> {})); + } else { + retentionLeaseSyncer.backgroundSync(shardId, retentionLeases.v2()); + } } /** diff --git a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java index 520344489adf9..bb526a3470873 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java @@ -37,7 +37,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.LongSupplier; import java.util.stream.Collectors; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; @@ -46,7 +45,6 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.lessThanOrEqualTo; public class ReplicationTrackerRetentionLeaseTests extends ReplicationTrackerTestCase { @@ -78,7 +76,7 @@ public void testAddOrRenewRetentionLease() { minimumRetainingSequenceNumbers[i] = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE); replicationTracker.addRetentionLease( Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i, ActionListener.wrap(() -> {})); - assertRetentionLeases(replicationTracker, i + 1, minimumRetainingSequenceNumbers, () -> 0L, primaryTerm, 1 + i, true); + assertRetentionLeases(replicationTracker, i + 1, minimumRetainingSequenceNumbers, primaryTerm, 1 + i, true, false); } for (int i = 0; i < length; i++) { @@ -88,7 +86,7 @@ public void testAddOrRenewRetentionLease() { } minimumRetainingSequenceNumbers[i] = randomLongBetween(minimumRetainingSequenceNumbers[i], Long.MAX_VALUE); replicationTracker.renewRetentionLease(Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i); - assertRetentionLeases(replicationTracker, length, minimumRetainingSequenceNumbers, () -> 0L, primaryTerm, 1 + length + i, true); + assertRetentionLeases(replicationTracker, length, minimumRetainingSequenceNumbers, primaryTerm, 1 + length + i, true, false); } } @@ -193,7 +191,7 @@ private void runExpirationTest(final boolean primaryMode) { assertThat(retentionLeases.leases(), hasSize(1)); final RetentionLease retentionLease = retentionLeases.leases().iterator().next(); assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get())); - assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, currentTimeMillis::get, primaryTerm, 1, primaryMode); + assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 1, primaryMode, false); } // renew the lease @@ -215,108 +213,20 @@ private void runExpirationTest(final boolean primaryMode) { assertThat(retentionLeases.leases(), hasSize(1)); final RetentionLease retentionLease = retentionLeases.leases().iterator().next(); assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get())); - assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, currentTimeMillis::get, primaryTerm, 2, primaryMode); + assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 2, primaryMode, false); } // now force the lease to expire currentTimeMillis.set(currentTimeMillis.get() + randomLongBetween(retentionLeaseMillis, Long.MAX_VALUE - currentTimeMillis.get())); if (primaryMode) { - assertRetentionLeases(replicationTracker, 0, retainingSequenceNumbers, currentTimeMillis::get, primaryTerm, 3, true); + assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 2, true, false); + assertRetentionLeases(replicationTracker, 0, new long[0], primaryTerm, 3, true, true); } else { // leases do not expire on replicas until synced from the primary - assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, currentTimeMillis::get, primaryTerm, 2, false); + assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 2, false, false); } } - public void testRetentionLeaseExpirationCausesRetentionLeaseSync() { - final AllocationId allocationId = AllocationId.newInitializing(); - final AtomicLong currentTimeMillis = new AtomicLong(randomLongBetween(0, 1024)); - final long retentionLeaseMillis = randomLongBetween(1, TimeValue.timeValueHours(12).millis()); - final Settings settings = Settings - .builder() - .put( - IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING.getKey(), - TimeValue.timeValueMillis(retentionLeaseMillis)) - .build(); - final Map> retentionLeases = new HashMap<>(); - final AtomicBoolean invoked = new AtomicBoolean(); - final AtomicReference reference = new AtomicReference<>(); - final ReplicationTracker replicationTracker = new ReplicationTracker( - new ShardId("test", "_na", 0), - allocationId.getId(), - IndexSettingsModule.newIndexSettings("test", settings), - randomNonNegativeLong(), - UNASSIGNED_SEQ_NO, - value -> {}, - currentTimeMillis::get, - (leases, listener) -> { - // we do not want to hold a lock on the replication tracker in the callback! - assertFalse(Thread.holdsLock(reference.get())); - invoked.set(true); - assertThat( - leases.leases() - .stream() - .collect(Collectors.toMap(RetentionLease::id, ReplicationTrackerRetentionLeaseTests::toTuple)), - equalTo(retentionLeases)); - }); - reference.set(replicationTracker); - replicationTracker.updateFromMaster( - randomNonNegativeLong(), - Collections.singleton(allocationId.getId()), - routingTable(Collections.emptySet(), allocationId), - Collections.emptySet()); - replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); - - final int length = randomIntBetween(0, 8); - long version = 0; - for (int i = 0; i < length; i++) { - final String id = randomAlphaOfLength(8); - final long retainingSequenceNumber = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE); - retentionLeases.put(id, Tuple.tuple(retainingSequenceNumber, currentTimeMillis.get())); - replicationTracker.addRetentionLease(id, retainingSequenceNumber, "test", ActionListener.wrap(() -> {})); - version++; - assertThat(replicationTracker.getRetentionLeases().version(), equalTo(version)); - // assert that the new retention lease callback was invoked - assertTrue(invoked.get()); - - // reset the invocation marker so that we can assert the callback was not invoked when renewing the lease - invoked.set(false); - currentTimeMillis.set(1 + currentTimeMillis.get()); - retentionLeases.put(id, Tuple.tuple(retainingSequenceNumber, currentTimeMillis.get())); - replicationTracker.renewRetentionLease(id, retainingSequenceNumber, "test"); - version++; - assertThat(replicationTracker.getRetentionLeases().version(), equalTo(version)); - - // reset the invocation marker so that we can assert the callback was invoked if any leases are expired - assertFalse(invoked.get()); - // randomly expire some leases - final long currentTimeMillisIncrement = randomLongBetween(0, Long.MAX_VALUE - currentTimeMillis.get()); - // calculate the expired leases and update our tracking map - final List expiredIds = retentionLeases.entrySet() - .stream() - .filter(r -> currentTimeMillis.get() + currentTimeMillisIncrement > r.getValue().v2() + retentionLeaseMillis) - .map(Map.Entry::getKey) - .collect(Collectors.toList()); - expiredIds.forEach(retentionLeases::remove); - if (expiredIds.isEmpty() == false) { - version++; - } - currentTimeMillis.set(currentTimeMillis.get() + currentTimeMillisIncrement); - // getting the leases has the side effect of calculating which leases are expired and invoking the sync callback - final RetentionLeases current = replicationTracker.getRetentionLeases(); - assertThat(current.version(), equalTo(version)); - // the current leases should equal our tracking map - assertThat( - current.leases() - .stream() - .collect(Collectors.toMap(RetentionLease::id, ReplicationTrackerRetentionLeaseTests::toTuple)), - equalTo(retentionLeases)); - // the callback should only be invoked if there were expired leases - assertThat(invoked.get(), equalTo(expiredIds.isEmpty() == false)); - } - assertThat(replicationTracker.getRetentionLeases().version(), equalTo(version)); - } - public void testReplicaIgnoresOlderRetentionLeasesVersion() { final AllocationId allocationId = AllocationId.newInitializing(); final ReplicationTracker replicationTracker = new ReplicationTracker( @@ -370,19 +280,29 @@ public void testReplicaIgnoresOlderRetentionLeasesVersion() { } } - private static Tuple toTuple(final RetentionLease retentionLease) { - return Tuple.tuple(retentionLease.retainingSequenceNumber(), retentionLease.timestamp()); - } - private void assertRetentionLeases( final ReplicationTracker replicationTracker, final int size, final long[] minimumRetainingSequenceNumbers, - final LongSupplier currentTimeMillisSupplier, final long primaryTerm, final long version, - final boolean primaryMode) { - final RetentionLeases retentionLeases = replicationTracker.getRetentionLeases(); + final boolean primaryMode, + final boolean expireLeases) { + assertTrue(expireLeases == false || primaryMode); + final RetentionLeases retentionLeases; + if (expireLeases == false) { + if (randomBoolean()) { + retentionLeases = replicationTracker.getRetentionLeases(); + } else { + final Tuple tuple = replicationTracker.getRetentionLeases(false); + assertFalse(tuple.v1()); + retentionLeases = tuple.v2(); + } + } else { + final Tuple tuple = replicationTracker.getRetentionLeases(true); + assertTrue(tuple.v1()); + retentionLeases = tuple.v2(); + } assertThat(retentionLeases.primaryTerm(), equalTo(primaryTerm)); assertThat(retentionLeases.version(), equalTo(version)); final Map idToRetentionLease = new HashMap<>(); @@ -395,12 +315,6 @@ private void assertRetentionLeases( assertThat(idToRetentionLease.keySet(), hasItem(Integer.toString(i))); final RetentionLease retentionLease = idToRetentionLease.get(Integer.toString(i)); assertThat(retentionLease.retainingSequenceNumber(), equalTo(minimumRetainingSequenceNumbers[i])); - if (primaryMode) { - // retention leases can be expired on replicas, so we can only assert on primaries here - assertThat( - currentTimeMillisSupplier.getAsLong() - retentionLease.timestamp(), - lessThanOrEqualTo(replicationTracker.indexSettings().getRetentionLeaseMillis())); - } assertThat(retentionLease.source(), equalTo("test-" + i)); } } diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncIT.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncIT.java index 3e69c84e3cde3..2eb0b54f36127 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncIT.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncIT.java @@ -20,33 +20,58 @@ package org.elasticsearch.index.seqno; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.threadpool.ThreadPool; import java.io.Closeable; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasItem; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST) public class RetentionLeaseSyncIT extends ESIntegTestCase { + public static final class RetentionLeaseSyncIntervalSettingPlugin extends Plugin { + + @Override + public List> getSettings() { + return Collections.singletonList(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING); + } + + } + + @Override + protected Collection> nodePlugins() { + return Stream.concat( + super.nodePlugins().stream(), + Stream.of(RetentionLeaseBackgroundSyncIT.RetentionLeaseSyncIntervalSettingPlugin.class)) + .collect(Collectors.toList()); + } + public void testRetentionLeasesSyncedOnAdd() throws Exception { final int numberOfReplicas = 2 - scaledRandomIntBetween(0, 2); internalCluster().ensureAtLeastNumDataNodes(1 + numberOfReplicas); @@ -99,7 +124,6 @@ public void testRetentionLeasesSyncedOnAdd() throws Exception { } } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37963") public void testRetentionLeasesSyncOnExpiration() throws Exception { final int numberOfReplicas = 2 - scaledRandomIntBetween(0, 2); internalCluster().ensureAtLeastNumDataNodes(1 + numberOfReplicas); @@ -109,7 +133,7 @@ public void testRetentionLeasesSyncOnExpiration() throws Exception { final Settings settings = Settings.builder() .put("index.number_of_shards", 1) .put("index.number_of_replicas", numberOfReplicas) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING.getKey(), retentionLeaseTimeToLive) + .put(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING.getKey(), TimeValue.timeValueSeconds(1)) .build(); createIndex("index", settings); ensureGreen("index"); @@ -121,6 +145,17 @@ public void testRetentionLeasesSyncOnExpiration() throws Exception { // we will add multiple retention leases, wait for some to expire, and assert a consistent view between the primary and the replicas final int length = randomIntBetween(1, 8); for (int i = 0; i < length; i++) { + // update the index for retention leases to live a long time + final AcknowledgedResponse longTtlResponse = client().admin() + .indices() + .prepareUpdateSettings("index") + .setSettings( + Settings.builder() + .putNull(IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING.getKey()) + .build()) + .get(); + assertTrue(longTtlResponse.isAcknowledged()); + final String id = randomAlphaOfLength(8); final long retainingSequenceNumber = randomLongBetween(0, Long.MAX_VALUE); final String source = randomAlphaOfLength(8); @@ -137,19 +172,26 @@ public void testRetentionLeasesSyncOnExpiration() throws Exception { final IndexShard replica = internalCluster() .getInstance(IndicesService.class, replicaShardNodeName) .getShardOrNull(new ShardId(resolveIndex("index"), 0)); - assertThat(replica.getRetentionLeases().leases(), hasItem(currentRetentionLease)); + assertThat(replica.getRetentionLeases().leases(), anyOf(empty(), contains(currentRetentionLease))); } - // sleep long enough that *possibly* the current retention lease has expired, and certainly that any previous have + // update the index for retention leases to short a long time, to force expiration + final AcknowledgedResponse shortTtlResponse = client().admin() + .indices() + .prepareUpdateSettings("index") + .setSettings( + Settings.builder() + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING.getKey(), retentionLeaseTimeToLive) + .build()) + .get(); + assertTrue(shortTtlResponse.isAcknowledged()); + + // sleep long enough that the current retention lease has expired final long later = System.nanoTime(); Thread.sleep(Math.max(0, retentionLeaseTimeToLive.millis() - TimeUnit.NANOSECONDS.toMillis(later - now))); - final RetentionLeases currentRetentionLeases = primary.getRetentionLeases(); - assertThat(currentRetentionLeases.leases(), anyOf(empty(), contains(currentRetentionLease))); + assertBusy(() -> assertThat(primary.getRetentionLeases().leases(), empty())); - /* - * Check that expiration of retention leases has been synced to all replicas. We have to assert busy since syncing happens in - * the background. - */ + // now that all retention leases are expired should have been synced to all replicas assertBusy(() -> { for (final ShardRouting replicaShard : clusterService().state().routingTable().index("index").shard(0).replicaShards()) { final String replicaShardNodeId = replicaShard.currentNodeId(); @@ -157,13 +199,7 @@ public void testRetentionLeasesSyncOnExpiration() throws Exception { final IndexShard replica = internalCluster() .getInstance(IndicesService.class, replicaShardNodeName) .getShardOrNull(new ShardId(resolveIndex("index"), 0)); - if (currentRetentionLeases.leases().isEmpty()) { - assertThat(replica.getRetentionLeases().leases(), empty()); - } else { - assertThat( - replica.getRetentionLeases().leases(), - contains(currentRetentionLeases.leases().toArray(new RetentionLease[0]))); - } + assertThat(replica.getRetentionLeases().leases(), empty()); } }); } diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java index 75d8d7e8e2679..cc64fc6f8b2de 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.ShardRoutingHelper; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexSettings; @@ -43,14 +44,12 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -import java.util.function.LongSupplier; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; @@ -85,13 +84,20 @@ public void testAddOrRenewRetentionLease() throws IOException { indexShard.addRetentionLease( Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i, ActionListener.wrap(() -> {})); assertRetentionLeases( - indexShard, i + 1, minimumRetainingSequenceNumbers, () -> 0L, primaryTerm, 1 + i, true); + indexShard, i + 1, minimumRetainingSequenceNumbers, primaryTerm, 1 + i, true, false); } for (int i = 0; i < length; i++) { minimumRetainingSequenceNumbers[i] = randomLongBetween(minimumRetainingSequenceNumbers[i], Long.MAX_VALUE); indexShard.renewRetentionLease(Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i); - assertRetentionLeases(indexShard, length, minimumRetainingSequenceNumbers, () -> 0L, primaryTerm, 1 + length + i, true); + assertRetentionLeases( + indexShard, + length, + minimumRetainingSequenceNumbers, + primaryTerm, + 1 + length + i, + true, + false); } } finally { closeShards(indexShard); @@ -121,8 +127,7 @@ private void runExpirationTest(final boolean primary) throws IOException { final long[] retainingSequenceNumbers = new long[1]; retainingSequenceNumbers[0] = randomLongBetween(0, Long.MAX_VALUE); if (primary) { - indexShard.addRetentionLease("0", retainingSequenceNumbers[0], "test-0", ActionListener.wrap(() -> { - })); + indexShard.addRetentionLease("0", retainingSequenceNumbers[0], "test-0", ActionListener.wrap(() -> {})); } else { final RetentionLeases retentionLeases = new RetentionLeases( primaryTerm, @@ -137,7 +142,7 @@ private void runExpirationTest(final boolean primary) throws IOException { assertThat(retentionLeases.leases(), hasSize(1)); final RetentionLease retentionLease = retentionLeases.leases().iterator().next(); assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get())); - assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, currentTimeMillis::get, primaryTerm, 1, primary); + assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, 1, primary, false); } // renew the lease @@ -159,16 +164,17 @@ private void runExpirationTest(final boolean primary) throws IOException { assertThat(retentionLeases.leases(), hasSize(1)); final RetentionLease retentionLease = retentionLeases.leases().iterator().next(); assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get())); - assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, currentTimeMillis::get, primaryTerm, 2, primary); + assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, 2, primary, false); } // now force the lease to expire currentTimeMillis.set( currentTimeMillis.get() + randomLongBetween(retentionLeaseMillis, Long.MAX_VALUE - currentTimeMillis.get())); if (primary) { - assertRetentionLeases(indexShard, 0, retainingSequenceNumbers, currentTimeMillis::get, primaryTerm, 3, true); + assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, 2, true, false); + assertRetentionLeases(indexShard, 0, new long[0], primaryTerm, 3, true, true); } else { - assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, currentTimeMillis::get, primaryTerm, 2, false); + assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, 2, false, false); } } finally { closeShards(indexShard); @@ -191,8 +197,7 @@ public void testCommit() throws IOException { minimumRetainingSequenceNumbers[i] = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE); currentTimeMillis.set(TimeUnit.NANOSECONDS.toMillis(randomNonNegativeLong())); indexShard.addRetentionLease( - Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i, ActionListener.wrap(() -> { - })); + Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i, ActionListener.wrap(() -> {})); } currentTimeMillis.set(TimeUnit.NANOSECONDS.toMillis(Long.MAX_VALUE)); @@ -250,13 +255,10 @@ public void testRetentionLeaseStats() throws IOException { final RetentionLeaseStats stats = indexShard.getRetentionLeaseStats(); assertRetentionLeases( stats.retentionLeases(), - indexShard.indexSettings().getRetentionLeaseMillis(), length, minimumRetainingSequenceNumbers, - () -> 0L, length == 0 ? RetentionLeases.EMPTY.primaryTerm() : indexShard.getOperationPrimaryTerm(), - length, - true); + length); } finally { closeShards(indexShard); } @@ -266,30 +268,39 @@ private void assertRetentionLeases( final IndexShard indexShard, final int size, final long[] minimumRetainingSequenceNumbers, - final LongSupplier currentTimeMillisSupplier, final long primaryTerm, final long version, - final boolean primary) { + final boolean primary, + final boolean expireLeases) { + assertTrue(expireLeases == false || primary); + final RetentionLeases retentionLeases; + if (expireLeases == false) { + if (randomBoolean()) { + retentionLeases = indexShard.getRetentionLeases(); + } else { + final Tuple tuple = indexShard.getRetentionLeases(false); + assertFalse(tuple.v1()); + retentionLeases = tuple.v2(); + } + } else { + final Tuple tuple = indexShard.getRetentionLeases(true); + assertTrue(tuple.v1()); + retentionLeases = tuple.v2(); + } assertRetentionLeases( - indexShard.getEngine().config().retentionLeasesSupplier().get(), - indexShard.indexSettings().getRetentionLeaseMillis(), + retentionLeases, size, minimumRetainingSequenceNumbers, - currentTimeMillisSupplier, primaryTerm, - version, - primary); + version); } private void assertRetentionLeases( final RetentionLeases retentionLeases, - final long retentionLeaseMillis, final int size, final long[] minimumRetainingSequenceNumbers, - final LongSupplier currentTimeMillisSupplier, final long primaryTerm, - final long version, - final boolean primary) { + final long version) { assertThat(retentionLeases.primaryTerm(), equalTo(primaryTerm)); assertThat(retentionLeases.version(), equalTo(version)); final Map idToRetentionLease = new HashMap<>(); @@ -302,10 +313,6 @@ private void assertRetentionLeases( assertThat(idToRetentionLease.keySet(), hasItem(Integer.toString(i))); final RetentionLease retentionLease = idToRetentionLease.get(Integer.toString(i)); assertThat(retentionLease.retainingSequenceNumber(), equalTo(minimumRetainingSequenceNumbers[i])); - if (primary) { - // retention leases can be expired on replicas, so we can only assert on primaries here - assertThat(currentTimeMillisSupplier.getAsLong() - retentionLease.timestamp(), lessThanOrEqualTo(retentionLeaseMillis)); - } assertThat(retentionLease.source(), equalTo("test-" + i)); } }