From d21df2a17a2ed96b0f307505ba0c6c66696e05dd Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 11 Jan 2019 06:23:55 -0800 Subject: [PATCH 01/28] Use Sequence number powered OCC for processing updates (#37308) Updates perform realtime get, perform the requested update and then index the document again using optimistic concurrency control. This PR changes the logic to use sequence numbers instead of versioning. Note that the current versioning logic isn't suffering from the same problem as external OCC requests because the get and indexing is always done on the same primary. Relates #36148 Relates #10708 --- .../action/update/UpdateHelper.java | 24 +++++-------------- .../action/update/UpdateRequestTests.java | 19 --------------- 2 files changed, 6 insertions(+), 37 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java b/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java index 255161c8f3200..a8a5fb8f72f30 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java @@ -163,19 +163,6 @@ Result prepareUpsert(ShardId shardId, UpdateRequest request, final GetResult get return new Result(indexRequest, DocWriteResponse.Result.CREATED, null, null); } - /** - * Calculate the version to use for the update request, using either the existing version if internal versioning is used, or the get - * result document's version if the version type is "FORCE". - */ - static long calculateUpdateVersion(UpdateRequest request, GetResult getResult) { - if (request.versionType() != VersionType.INTERNAL) { - assert request.versionType() == VersionType.FORCE; - return request.version(); // remember, match_any is excluded by the conflict test - } else { - return getResult.getVersion(); - } - } - /** * Calculate a routing value to be used, either the included index request's routing, or retrieved document's routing when defined. */ @@ -195,7 +182,6 @@ static String calculateRouting(GetResult getResult, @Nullable IndexRequest updat * containing a new {@code IndexRequest} to be executed on the primary and replicas. */ Result prepareUpdateIndexRequest(ShardId shardId, UpdateRequest request, GetResult getResult, boolean detectNoop) { - final long updateVersion = calculateUpdateVersion(request, getResult); final IndexRequest currentRequest = request.doc(); final String routing = calculateRouting(getResult, currentRequest); final Tuple> sourceAndContent = XContentHelper.convertToMap(getResult.internalSourceRef(), true); @@ -215,7 +201,8 @@ Result prepareUpdateIndexRequest(ShardId shardId, UpdateRequest request, GetResu } else { final IndexRequest finalIndexRequest = Requests.indexRequest(request.index()) .type(request.type()).id(request.id()).routing(routing) - .source(updatedSourceAsMap, updateSourceContentType).version(updateVersion).versionType(request.versionType()) + .source(updatedSourceAsMap, updateSourceContentType) + .setIfSeqNo(getResult.getSeqNo()).setIfPrimaryTerm(getResult.getPrimaryTerm()) .waitForActiveShards(request.waitForActiveShards()).timeout(request.timeout()) .setRefreshPolicy(request.getRefreshPolicy()); return new Result(finalIndexRequest, DocWriteResponse.Result.UPDATED, updatedSourceAsMap, updateSourceContentType); @@ -228,7 +215,6 @@ Result prepareUpdateIndexRequest(ShardId shardId, UpdateRequest request, GetResu * primary and replicas. */ Result prepareUpdateScriptRequest(ShardId shardId, UpdateRequest request, GetResult getResult, LongSupplier nowInMillis) { - final long updateVersion = calculateUpdateVersion(request, getResult); final IndexRequest currentRequest = request.doc(); final String routing = calculateRouting(getResult, currentRequest); final Tuple> sourceAndContent = XContentHelper.convertToMap(getResult.internalSourceRef(), true); @@ -256,14 +242,16 @@ Result prepareUpdateScriptRequest(ShardId shardId, UpdateRequest request, GetRes case INDEX: final IndexRequest indexRequest = Requests.indexRequest(request.index()) .type(request.type()).id(request.id()).routing(routing) - .source(updatedSourceAsMap, updateSourceContentType).version(updateVersion).versionType(request.versionType()) + .source(updatedSourceAsMap, updateSourceContentType) + .setIfSeqNo(getResult.getSeqNo()).setIfPrimaryTerm(getResult.getPrimaryTerm()) .waitForActiveShards(request.waitForActiveShards()).timeout(request.timeout()) .setRefreshPolicy(request.getRefreshPolicy()); return new Result(indexRequest, DocWriteResponse.Result.UPDATED, updatedSourceAsMap, updateSourceContentType); case DELETE: DeleteRequest deleteRequest = Requests.deleteRequest(request.index()) .type(request.type()).id(request.id()).routing(routing) - .version(updateVersion).versionType(request.versionType()).waitForActiveShards(request.waitForActiveShards()) + .setIfSeqNo(getResult.getSeqNo()).setIfPrimaryTerm(getResult.getPrimaryTerm()) + .waitForActiveShards(request.waitForActiveShards()) .timeout(request.timeout()).setRefreshPolicy(request.getRefreshPolicy()); return new Result(deleteRequest, DocWriteResponse.Result.DELETED, updatedSourceAsMap, updateSourceContentType); default: diff --git a/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java b/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java index 4534fbe23e3c2..5a734352eafb2 100644 --- a/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java @@ -38,7 +38,6 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.env.Environment; -import org.elasticsearch.index.VersionType; import org.elasticsearch.index.get.GetResult; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.script.MockScriptEngine; @@ -570,24 +569,6 @@ public void testRoutingExtraction() throws Exception { assertThat(UpdateHelper.calculateRouting(getResult, indexRequest), equalTo("routing1")); } - @SuppressWarnings("deprecated") // VersionType.FORCE is deprecated - public void testCalculateUpdateVersion() throws Exception { - long randomVersion = randomIntBetween(0, 100); - GetResult getResult = new GetResult("test", "type", "1", 0, 1, randomVersion, true, new BytesArray("{}"), null); - - UpdateRequest request = new UpdateRequest("test", "type1", "1"); - long version = UpdateHelper.calculateUpdateVersion(request, getResult); - - // Use the get result's version - assertThat(version, equalTo(randomVersion)); - - request = new UpdateRequest("test", "type1", "1").versionType(VersionType.FORCE).version(1337); - version = UpdateHelper.calculateUpdateVersion(request, getResult); - - // Use the forced update request version - assertThat(version, equalTo(1337L)); - } - public void testNoopDetection() throws Exception { ShardId shardId = new ShardId("test", "", 0); GetResult getResult = new GetResult("test", "type", "1", 0, 1, 0, true, From fad63298082cccaba6c681aec896e7d636998cfe Mon Sep 17 00:00:00 2001 From: Jake Landis Date: Fri, 11 Jan 2019 08:58:41 -0600 Subject: [PATCH 02/28] add link to freeze from cold phase (#37332) --- docs/reference/ilm/policy-definitions.asciidoc | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/reference/ilm/policy-definitions.asciidoc b/docs/reference/ilm/policy-definitions.asciidoc index ab2c0a039f9d0..1253c87b3943e 100644 --- a/docs/reference/ilm/policy-definitions.asciidoc +++ b/docs/reference/ilm/policy-definitions.asciidoc @@ -93,6 +93,7 @@ The below list shows the actions which are available in each phase. - <> * Cold - <> + - <> * Delete - <> From f4abf9628aa35138f892db93be5a7a3fb3639a5e Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 11 Jan 2019 16:06:48 +0100 Subject: [PATCH 03/28] Mock connections more accurately in DisruptableMockTransport (#37296) This commit moves DisruptableMockTransport to use a more accurate representation of connection management, which allows to use the full connection manager and does not require mocking out any behavior. With this, we can implement restarting nodes in CoordinatorTests. --- .../coordination/CoordinationState.java | 2 +- .../cluster/coordination/Coordinator.java | 2 +- .../cluster/coordination/Join.java | 4 + .../coordination/CoordinatorTests.java | 197 ++++++++++++------ .../snapshots/SnapshotsServiceTests.java | 47 ++--- .../disruption/DisruptableMockTransport.java | 132 ++++++++---- .../DisruptableMockTransportTests.java | 55 ++--- 7 files changed, 267 insertions(+), 172 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java index fc511870b332e..4d542566ccd70 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java @@ -209,7 +209,7 @@ public Join handleStartJoin(StartJoinRequest startJoinRequest) { * @throws CoordinationStateRejectedException if the arguments were incompatible with the current state of this object. */ public boolean handleJoin(Join join) { - assert join.getTargetNode().equals(localNode) : "handling join " + join + " for the wrong node " + localNode; + assert join.targetMatches(localNode) : "handling join " + join + " for the wrong node " + localNode; if (join.getTerm() != getCurrentTerm()) { logger.debug("handleJoin: ignored join due to term mismatch (expected: [{}], actual: [{}])", diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index 233423a391c19..1d7ed0bdc7c2b 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -310,7 +310,7 @@ PublishWithJoinResponse handlePublishRequest(PublishRequest publishRequest) { private static Optional joinWithDestination(Optional lastJoin, DiscoveryNode leader, long term) { if (lastJoin.isPresent() - && lastJoin.get().getTargetNode().getId().equals(leader.getId()) + && lastJoin.get().targetMatches(leader) && lastJoin.get().getTerm() == term) { return lastJoin; } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Join.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Join.java index 1c69c4a62b80a..1b1c3ff7aed11 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Join.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Join.java @@ -78,6 +78,10 @@ public DiscoveryNode getTargetNode() { return targetNode; } + public boolean targetMatches(DiscoveryNode matchingNode) { + return targetNode.getId().equals(matchingNode.getId()); + } + public long getLastAcceptedVersion() { return lastAcceptedVersion; } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java index 36879df5ac06d..b03d05fa36238 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java @@ -26,6 +26,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateUpdateTask; @@ -42,6 +43,10 @@ import org.elasticsearch.cluster.service.ClusterApplier; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; @@ -77,7 +82,6 @@ import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; -import java.util.function.Predicate; import java.util.function.Supplier; import java.util.function.UnaryOperator; import java.util.stream.Collectors; @@ -107,7 +111,6 @@ import static org.elasticsearch.discovery.DiscoverySettings.NO_MASTER_BLOCK_WRITES; import static org.elasticsearch.discovery.PeerFinder.DISCOVERY_FIND_PEERS_INTERVAL_SETTING; import static org.elasticsearch.node.Node.NODE_NAME_SETTING; -import static org.elasticsearch.transport.TransportService.HANDSHAKE_ACTION_NAME; import static org.elasticsearch.transport.TransportService.NOOP_TRANSPORT_INTERCEPTOR; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; @@ -930,7 +933,7 @@ public void testStayCandidateAfterReceivingFollowerCheckFromKnownMaster() { final ClusterNode leader = cluster.getAnyLeader(); final ClusterNode nonLeader = cluster.getAnyNodeExcept(leader); - onNode(nonLeader.getLocalNode(), () -> { + nonLeader.onNode(() -> { logger.debug("forcing {} to become candidate", nonLeader.getId()); synchronized (nonLeader.coordinator.mutex) { nonLeader.coordinator.becomeCandidate("forced"); @@ -1161,6 +1164,11 @@ void runRandomly() { assertThat("may reconnect disconnected nodes, probably unexpected", disconnectedNodes, empty()); assertThat("may reconnect blackholed nodes, probably unexpected", blackholedNodes, empty()); + final List cleanupActions = new ArrayList<>(); + cleanupActions.add(disconnectedNodes::clear); + cleanupActions.add(blackholedNodes::clear); + cleanupActions.add(() -> disruptStorage = false); + final int randomSteps = scaledRandomIntBetween(10, 10000); logger.info("--> start of safety phase of at least [{}] steps", randomSteps); @@ -1183,7 +1191,7 @@ void runRandomly() { if (rarely()) { final ClusterNode clusterNode = getAnyNodePreferringLeaders(); final int newValue = randomInt(); - onNode(clusterNode.getLocalNode(), () -> { + clusterNode.onNode(() -> { logger.debug("----> [runRandomly {}] proposing new value [{}] to [{}]", thisStep, newValue, clusterNode.getId()); clusterNode.submitValue(newValue); @@ -1191,15 +1199,34 @@ void runRandomly() { } else if (rarely()) { final ClusterNode clusterNode = getAnyNodePreferringLeaders(); final boolean autoShrinkVotingConfiguration = randomBoolean(); - onNode(clusterNode.getLocalNode(), + clusterNode.onNode( () -> { logger.debug("----> [runRandomly {}] setting auto-shrink configuration to {} on {}", thisStep, autoShrinkVotingConfiguration, clusterNode.getId()); clusterNode.submitSetAutoShrinkVotingConfiguration(autoShrinkVotingConfiguration); }).run(); + } else if (rarely()) { + // reboot random node + final ClusterNode clusterNode = getAnyNode(); + logger.debug("----> [runRandomly {}] rebooting [{}]", thisStep, clusterNode.getId()); + clusterNode.close(); + clusterNodes.forEach( + cn -> deterministicTaskQueue.scheduleNow(cn.onNode( + new Runnable() { + @Override + public void run() { + cn.transportService.disconnectFromNode(clusterNode.getLocalNode()); + } + + @Override + public String toString() { + return "disconnect from " + clusterNode.getLocalNode() + " after shutdown"; + } + }))); + clusterNodes.replaceAll(cn -> cn == clusterNode ? cn.restartedNode() : cn); } else if (rarely()) { final ClusterNode clusterNode = getAnyNode(); - onNode(clusterNode.getLocalNode(), () -> { + clusterNode.onNode(() -> { logger.debug("----> [runRandomly {}] forcing {} to become candidate", thisStep, clusterNode.getId()); synchronized (clusterNode.coordinator.mutex) { clusterNode.coordinator.becomeCandidate("runRandomly"); @@ -1227,7 +1254,7 @@ void runRandomly() { } } else if (rarely()) { final ClusterNode clusterNode = getAnyNode(); - onNode(clusterNode.getLocalNode(), + clusterNode.onNode( () -> { logger.debug("----> [runRandomly {}] applying initial configuration {} to {}", thisStep, initialConfiguration, clusterNode.getId()); @@ -1252,9 +1279,9 @@ void runRandomly() { assertConsistentStates(); } - disconnectedNodes.clear(); - blackholedNodes.clear(); - disruptStorage = false; + logger.debug("running {} cleanup actions", cleanupActions.size()); + cleanupActions.forEach(Runnable::run); + logger.debug("finished running cleanup actions"); } private void assertConsistentStates() { @@ -1406,18 +1433,28 @@ ClusterNode getAnyLeader() { return randomFrom(allLeaders); } + private final ConnectionStatus preferredUnknownNodeConnectionStatus = + randomFrom(ConnectionStatus.DISCONNECTED, ConnectionStatus.BLACK_HOLE); + private ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) { ConnectionStatus connectionStatus; if (blackholedNodes.contains(sender.getId()) || blackholedNodes.contains(destination.getId())) { connectionStatus = ConnectionStatus.BLACK_HOLE; } else if (disconnectedNodes.contains(sender.getId()) || disconnectedNodes.contains(destination.getId())) { connectionStatus = ConnectionStatus.DISCONNECTED; - } else { + } else if (nodeExists(sender) && nodeExists(destination)) { connectionStatus = ConnectionStatus.CONNECTED; + } else { + connectionStatus = usually() ? preferredUnknownNodeConnectionStatus : + randomFrom(ConnectionStatus.DISCONNECTED, ConnectionStatus.BLACK_HOLE); } return connectionStatus; } + boolean nodeExists(DiscoveryNode node) { + return clusterNodes.stream().anyMatch(cn -> cn.getLocalNode().equals(node)); + } + ClusterNode getAnyMasterEligibleNode() { return randomFrom(clusterNodes.stream().filter(n -> n.getLocalNode().isMasterNode()).collect(Collectors.toList())); } @@ -1486,7 +1523,7 @@ class ClusterNode { private final int nodeIndex; private Coordinator coordinator; - private DiscoveryNode localNode; + private final DiscoveryNode localNode; private final PersistedState persistedState; private FakeClusterApplier clusterApplier; private AckedFakeThreadPoolMasterService masterService; @@ -1496,63 +1533,34 @@ class ClusterNode { private ClusterStateApplyResponse clusterStateApplyResponse = ClusterStateApplyResponse.SUCCEED; ClusterNode(int nodeIndex, boolean masterEligible) { - this.nodeIndex = nodeIndex; - localNode = createDiscoveryNode(masterEligible); - persistedState = new MockPersistedState(0L, - clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, VotingConfiguration.EMPTY_CONFIG, 0L)); - onNode(localNode, this::setUp).run(); + this(nodeIndex, createDiscoveryNode(nodeIndex, masterEligible), + localNode -> new MockPersistedState(0L, + clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, VotingConfiguration.EMPTY_CONFIG, 0L))); } - private DiscoveryNode createDiscoveryNode(boolean masterEligible) { - final TransportAddress address = buildNewFakeTransportAddress(); - return new DiscoveryNode("", "node" + nodeIndex, - UUIDs.randomBase64UUID(random()), // generated deterministically for repeatable tests - address.address().getHostString(), address.getAddress(), address, Collections.emptyMap(), - masterEligible ? EnumSet.allOf(Role.class) : emptySet(), Version.CURRENT); + ClusterNode(int nodeIndex, DiscoveryNode localNode, Function persistedStateSupplier) { + this.nodeIndex = nodeIndex; + this.localNode = localNode; + persistedState = persistedStateSupplier.apply(localNode); + onNodeLog(localNode, this::setUp).run(); } private void setUp() { - mockTransport = new DisruptableMockTransport(logger) { - @Override - protected DiscoveryNode getLocalNode() { - return localNode; - } - + mockTransport = new DisruptableMockTransport(localNode, logger) { @Override - protected ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) { - return Cluster.this.getConnectionStatus(sender, destination); + protected void execute(Runnable runnable) { + deterministicTaskQueue.scheduleNow(onNode(runnable)); } @Override - protected Optional getDisruptedCapturingTransport(DiscoveryNode node, String action) { - final Predicate matchesDestination; - if (action.equals(HANDSHAKE_ACTION_NAME)) { - matchesDestination = n -> n.getLocalNode().getAddress().equals(node.getAddress()); - } else { - matchesDestination = n -> n.getLocalNode().equals(node); - } - return clusterNodes.stream().filter(matchesDestination).findAny().map(cn -> cn.mockTransport); + protected ConnectionStatus getConnectionStatus(DiscoveryNode destination) { + return Cluster.this.getConnectionStatus(getLocalNode(), destination); } @Override - protected void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) { - // handshake needs to run inline as the caller blockingly waits on the result - if (action.equals(HANDSHAKE_ACTION_NAME)) { - onNode(destination, doDelivery).run(); - } else { - deterministicTaskQueue.scheduleNow(onNode(destination, doDelivery)); - } - } - - @Override - protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNode destination) { - if (action.equals(HANDSHAKE_ACTION_NAME)) { - logger.trace("ignoring blackhole and delivering {}", getRequestDescription(requestId, action, destination)); - // handshakes always have a timeout, and are sent in a blocking fashion, so we must respond with an exception. - sendFromTo(destination, getLocalNode(), action, getDisconnectException(requestId, action, destination)); - } else { - super.onBlackholedDuringSend(requestId, action, destination); - } + protected Optional getDisruptableMockTransport(TransportAddress address) { + return clusterNodes.stream().map(cn -> cn.mockTransport) + .filter(transport -> transport.getLocalNode().getAddress().equals(address)).findAny(); } }; @@ -1563,9 +1571,9 @@ protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNo final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); clusterApplier = new FakeClusterApplier(settings, clusterSettings); masterService = new AckedFakeThreadPoolMasterService("test_node", "test", - runnable -> deterministicTaskQueue.scheduleNow(onNode(localNode, runnable))); + runnable -> deterministicTaskQueue.scheduleNow(onNode(runnable))); transportService = mockTransport.createTransportService( - settings, deterministicTaskQueue.getThreadPool(runnable -> onNode(localNode, runnable)), NOOP_TRANSPORT_INTERCEPTOR, + settings, deterministicTaskQueue.getThreadPool(this::onNode), NOOP_TRANSPORT_INTERCEPTOR, a -> localNode, null, emptySet()); final Collection> onJoinValidators = Collections.singletonList((dn, cs) -> extraJoinValidators.forEach(validator -> validator.accept(dn, cs))); @@ -1574,6 +1582,7 @@ protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNo Cluster.this::provideUnicastHosts, clusterApplier, onJoinValidators, Randomness.get()); masterService.setClusterStatePublisher(coordinator); + logger.trace("starting up [{}]", localNode); transportService.start(); transportService.acceptIncomingRequests(); masterService.start(); @@ -1581,6 +1590,37 @@ protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNo coordinator.startInitialJoin(); } + void close() { + logger.trace("taking down [{}]", localNode); + //transportService.stop(); // does blocking stuff :/ + masterService.stop(); + coordinator.stop(); + //transportService.close(); // does blocking stuff :/ + masterService.close(); + coordinator.close(); + } + + ClusterNode restartedNode() { + final TransportAddress address = randomBoolean() ? buildNewFakeTransportAddress() : localNode.getAddress(); + final DiscoveryNode newLocalNode = new DiscoveryNode(localNode.getName(), localNode.getId(), + UUIDs.randomBase64UUID(random()), // generated deterministically for repeatable tests + address.address().getHostString(), address.getAddress(), address, Collections.emptyMap(), + localNode.isMasterNode() ? EnumSet.allOf(Role.class) : emptySet(), Version.CURRENT); + final PersistedState newPersistedState; + try { + BytesStreamOutput outStream = new BytesStreamOutput(); + outStream.setVersion(Version.CURRENT); + persistedState.getLastAcceptedState().writeTo(outStream); + StreamInput inStream = new NamedWriteableAwareStreamInput(outStream.bytes().streamInput(), + new NamedWriteableRegistry(ClusterModule.getNamedWriteables())); + newPersistedState = new MockPersistedState(persistedState.getCurrentTerm(), + ClusterState.readFrom(inStream, newLocalNode)); // adapts it to new localNode instance + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return new ClusterNode(nodeIndex, newLocalNode, node -> newPersistedState); + } + private PersistedState getPersistedState() { return persistedState; } @@ -1615,6 +1655,25 @@ ClusterStateApplyResponse getClusterStateApplyResponse() { return clusterStateApplyResponse; } + Runnable onNode(Runnable runnable) { + final Runnable wrapped = onNodeLog(localNode, runnable); + return new Runnable() { + @Override + public void run() { + if (clusterNodes.contains(ClusterNode.this) == false) { + logger.trace("ignoring runnable {} from node {} as node has been removed from cluster", runnable, localNode); + return; + } + wrapped.run(); + } + + @Override + public String toString() { + return wrapped.toString(); + } + }; + } + void submitSetAutoShrinkVotingConfiguration(final boolean autoShrinkVotingConfiguration) { submitUpdateTask("set master nodes failure tolerance [" + autoShrinkVotingConfiguration + "]", cs -> ClusterState.builder(cs).metaData( @@ -1633,7 +1692,7 @@ AckCollector submitValue(final long value) { AckCollector submitUpdateTask(String source, UnaryOperator clusterStateUpdate) { final AckCollector ackCollector = new AckCollector(); - onNode(localNode, () -> { + onNode(() -> { logger.trace("[{}] submitUpdateTask: enqueueing [{}]", localNode.getId(), source); final long submittedTerm = coordinator.getCurrentTerm(); masterService.submitStateUpdateTask(source, @@ -1698,7 +1757,7 @@ ClusterState getLastAppliedClusterState() { } void applyInitialConfiguration() { - onNode(localNode, () -> { + onNode(() -> { try { coordinator.setInitialConfiguration(initialConfiguration); logger.info("successfully set initial configuration to {}", initialConfiguration); @@ -1734,7 +1793,7 @@ public void setInitialState(ClusterState initialState) { public void onNewClusterState(String source, Supplier clusterStateSupplier, ClusterApplyListener listener) { switch (clusterStateApplyResponse) { case SUCCEED: - deterministicTaskQueue.scheduleNow(onNode(localNode, new Runnable() { + deterministicTaskQueue.scheduleNow(onNode(new Runnable() { @Override public void run() { final ClusterState oldClusterState = clusterApplier.lastAppliedClusterState; @@ -1754,7 +1813,7 @@ public String toString() { })); break; case FAIL: - deterministicTaskQueue.scheduleNow(onNode(localNode, new Runnable() { + deterministicTaskQueue.scheduleNow(onNode(new Runnable() { @Override public void run() { listener.onFailure(source, new ElasticsearchException("cluster state application failed")); @@ -1768,7 +1827,7 @@ public String toString() { break; case HANG: if (randomBoolean()) { - deterministicTaskQueue.scheduleNow(onNode(localNode, new Runnable() { + deterministicTaskQueue.scheduleNow(onNode(new Runnable() { @Override public void run() { final ClusterState oldClusterState = clusterApplier.lastAppliedClusterState; @@ -1796,7 +1855,7 @@ private List provideUnicastHosts(HostsResolver ignored) { } } - public static Runnable onNode(DiscoveryNode node, Runnable runnable) { + public static Runnable onNodeLog(DiscoveryNode node, Runnable runnable) { final String nodeId = "{" + node.getId() + "}{" + node.getEphemeralId() + "}"; return new Runnable() { @Override @@ -1880,6 +1939,14 @@ public void onNodeAck(DiscoveryNode node, Exception e) { } } + private static DiscoveryNode createDiscoveryNode(int nodeIndex, boolean masterEligible) { + final TransportAddress address = buildNewFakeTransportAddress(); + return new DiscoveryNode("", "node" + nodeIndex, + UUIDs.randomBase64UUID(random()), // generated deterministically for repeatable tests + address.address().getHostString(), address.getAddress(), address, Collections.emptyMap(), + masterEligible ? EnumSet.allOf(Role.class) : emptySet(), Version.CURRENT); + } + /** * How to behave with a new cluster state */ diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java index 34160e901006f..b4dbbef65cce2 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java @@ -68,6 +68,7 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; @@ -114,7 +115,6 @@ import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -122,7 +122,6 @@ import static java.util.Collections.emptySet; import static org.elasticsearch.env.Environment.PATH_HOME_SETTING; import static org.elasticsearch.node.Node.NODE_NAME_SETTING; -import static org.elasticsearch.transport.TransportService.HANDSHAKE_ACTION_NAME; import static org.elasticsearch.transport.TransportService.NOOP_TRANSPORT_INTERCEPTOR; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.hasSize; @@ -388,41 +387,26 @@ protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { return new MockSinglePrioritizingExecutor(node.getName(), deterministicTaskQueue); } }); - mockTransport = new DisruptableMockTransport(logger) { + mockTransport = new DisruptableMockTransport(node, logger) { @Override - protected DiscoveryNode getLocalNode() { - return node; - } - - @Override - protected ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) { + protected ConnectionStatus getConnectionStatus(DiscoveryNode destination) { return ConnectionStatus.CONNECTED; } @Override - protected Optional getDisruptedCapturingTransport(DiscoveryNode node, String action) { - final Predicate matchesDestination; - if (action.equals(HANDSHAKE_ACTION_NAME)) { - matchesDestination = n -> n.transportService.getLocalNode().getAddress().equals(node.getAddress()); - } else { - matchesDestination = n -> n.transportService.getLocalNode().equals(node); - } - return testClusterNodes.nodes.values().stream().filter(matchesDestination).findAny().map(cn -> cn.mockTransport); + protected Optional getDisruptableMockTransport(TransportAddress address) { + return testClusterNodes.nodes.values().stream().map(cn -> cn.mockTransport) + .filter(transport -> transport.getLocalNode().getAddress().equals(address)) + .findAny(); } @Override - protected void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) { - // handshake needs to run inline as the caller blockingly waits on the result - final Runnable runnable = CoordinatorTests.onNode(destination, doDelivery); - if (action.equals(HANDSHAKE_ACTION_NAME)) { - runnable.run(); - } else { - deterministicTaskQueue.scheduleNow(runnable); - } + protected void execute(Runnable runnable) { + deterministicTaskQueue.scheduleNow(CoordinatorTests.onNodeLog(getLocalNode(), runnable)); } }; transportService = mockTransport.createTransportService( - settings, deterministicTaskQueue.getThreadPool(runnable -> CoordinatorTests.onNode(node, runnable)), + settings, deterministicTaskQueue.getThreadPool(runnable -> CoordinatorTests.onNodeLog(node, runnable)), NOOP_TRANSPORT_INTERCEPTOR, a -> node, null, emptySet() ); @@ -544,7 +528,16 @@ public void start(ClusterState initialState) { coordinator.start(); masterService.start(); clusterService.getClusterApplierService().setNodeConnectionsService( - new NodeConnectionsService(clusterService.getSettings(), threadPool, transportService)); + new NodeConnectionsService(clusterService.getSettings(), threadPool, transportService) { + @Override + public void connectToNodes(DiscoveryNodes discoveryNodes) { + // override this method as it does blocking calls + for (final DiscoveryNode node : discoveryNodes) { + transportService.connectToNode(node); + } + super.connectToNodes(discoveryNodes); + } + }); clusterService.getClusterApplierService().start(); indicesService.start(); indicesClusterStateService.start(); diff --git a/test/framework/src/main/java/org/elasticsearch/test/disruption/DisruptableMockTransport.java b/test/framework/src/main/java/org/elasticsearch/test/disruption/DisruptableMockTransport.java index 2743c11a2b6e9..2a1101c6d7986 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/disruption/DisruptableMockTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/test/disruption/DisruptableMockTransport.java @@ -20,80 +20,123 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.BoundTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.transport.MockTransport; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.CloseableConnection; import org.elasticsearch.transport.ConnectTransportException; +import org.elasticsearch.transport.ConnectionProfile; import org.elasticsearch.transport.RequestHandlerRegistry; import org.elasticsearch.transport.TransportChannel; +import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportInterceptor; import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportService; import java.io.IOException; import java.util.Optional; +import java.util.Set; +import java.util.function.Function; import static org.elasticsearch.test.ESTestCase.copyWriteable; +import static org.elasticsearch.transport.TransportService.HANDSHAKE_ACTION_NAME; public abstract class DisruptableMockTransport extends MockTransport { + private final DiscoveryNode localNode; private final Logger logger; - public DisruptableMockTransport(Logger logger) { + public DisruptableMockTransport(DiscoveryNode localNode, Logger logger) { + this.localNode = localNode; this.logger = logger; } - protected abstract DiscoveryNode getLocalNode(); + protected abstract ConnectionStatus getConnectionStatus(DiscoveryNode destination); - protected abstract ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination); + protected abstract Optional getDisruptableMockTransport(TransportAddress address); - protected abstract Optional getDisruptedCapturingTransport(DiscoveryNode node, String action); + protected abstract void execute(Runnable runnable); - protected abstract void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery); + protected final void execute(String action, Runnable runnable) { + // handshake needs to run inline as the caller blockingly waits on the result + if (action.equals(HANDSHAKE_ACTION_NAME)) { + runnable.run(); + } else { - protected final void sendFromTo(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) { - handle(sender, destination, action, new Runnable() { - @Override - public void run() { - if (getDisruptedCapturingTransport(destination, action).isPresent()) { - doDelivery.run(); - } else { - logger.trace("unknown destination in {}", this); - } - } + execute(runnable); + } + } - @Override - public String toString() { - return doDelivery.toString(); - } - }); + public DiscoveryNode getLocalNode() { + return localNode; + } + + @Override + public TransportService createTransportService(Settings settings, ThreadPool threadPool, TransportInterceptor interceptor, + Function localNodeFactory, + @Nullable ClusterSettings clusterSettings, Set taskHeaders) { + return new TransportService(settings, this, threadPool, interceptor, localNodeFactory, clusterSettings, taskHeaders); } @Override - protected void onSendRequest(long requestId, String action, TransportRequest request, DiscoveryNode destination) { + public Releasable openConnection(DiscoveryNode node, ConnectionProfile profile, ActionListener listener) { + final Optional matchingTransport = getDisruptableMockTransport(node.getAddress()); + if (matchingTransport.isPresent()) { + listener.onResponse(new CloseableConnection() { + @Override + public DiscoveryNode getNode() { + return node; + } - assert destination.equals(getLocalNode()) == false : "non-local message from " + getLocalNode() + " to itself"; + @Override + public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) + throws TransportException { + onSendRequest(requestId, action, request, matchingTransport.get()); + } + }); + return () -> {}; + } else { + throw new ConnectTransportException(node, "node " + node + " does not exist"); + } + } - sendFromTo(getLocalNode(), destination, action, new Runnable() { + protected void onSendRequest(long requestId, String action, TransportRequest request, + DisruptableMockTransport destinationTransport) { + + assert destinationTransport.getLocalNode().equals(getLocalNode()) == false : + "non-local message from " + getLocalNode() + " to itself"; + + execute(action, new Runnable() { @Override public void run() { - switch (getConnectionStatus(getLocalNode(), destination)) { + switch (getConnectionStatus(destinationTransport.getLocalNode())) { case BLACK_HOLE: - onBlackholedDuringSend(requestId, action, destination); + onBlackholedDuringSend(requestId, action, destinationTransport); break; case DISCONNECTED: - onDisconnectedDuringSend(requestId, action, destination); + onDisconnectedDuringSend(requestId, action, destinationTransport); break; case CONNECTED: - onConnectedDuringSend(requestId, action, request, destination); + onConnectedDuringSend(requestId, action, request, destinationTransport); break; } } @Override public String toString() { - return getRequestDescription(requestId, action, destination); + return getRequestDescription(requestId, action, destinationTransport.getLocalNode()); } }); } @@ -117,20 +160,27 @@ protected String getRequestDescription(long requestId, String action, DiscoveryN requestId, action, getLocalNode(), destination).getFormattedMessage(); } - protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNode destination) { - logger.trace("dropping {}", getRequestDescription(requestId, action, destination)); + protected void onBlackholedDuringSend(long requestId, String action, DisruptableMockTransport destinationTransport) { + if (action.equals(HANDSHAKE_ACTION_NAME)) { + logger.trace("ignoring blackhole and delivering {}", + getRequestDescription(requestId, action, destinationTransport.getLocalNode())); + // handshakes always have a timeout, and are sent in a blocking fashion, so we must respond with an exception. + destinationTransport.execute(action, getDisconnectException(requestId, action, destinationTransport.getLocalNode())); + } else { + logger.trace("dropping {}", getRequestDescription(requestId, action, destinationTransport.getLocalNode())); + } } - protected void onDisconnectedDuringSend(long requestId, String action, DiscoveryNode destination) { - sendFromTo(destination, getLocalNode(), action, getDisconnectException(requestId, action, destination)); + protected void onDisconnectedDuringSend(long requestId, String action, DisruptableMockTransport destinationTransport) { + destinationTransport.execute(action, getDisconnectException(requestId, action, destinationTransport.getLocalNode())); } - protected void onConnectedDuringSend(long requestId, String action, TransportRequest request, DiscoveryNode destination) { - Optional destinationTransport = getDisruptedCapturingTransport(destination, action); - assert destinationTransport.isPresent(); - + protected void onConnectedDuringSend(long requestId, String action, TransportRequest request, + DisruptableMockTransport destinationTransport) { final RequestHandlerRegistry requestHandler = - destinationTransport.get().getRequestHandler(action); + destinationTransport.getRequestHandler(action); + + final DiscoveryNode destination = destinationTransport.getLocalNode(); final String requestDescription = getRequestDescription(requestId, action, destination); @@ -147,10 +197,10 @@ public String getChannelType() { @Override public void sendResponse(final TransportResponse response) { - sendFromTo(destination, getLocalNode(), action, new Runnable() { + execute(action, new Runnable() { @Override public void run() { - if (getConnectionStatus(destination, getLocalNode()) != ConnectionStatus.CONNECTED) { + if (destinationTransport.getConnectionStatus(getLocalNode()) != ConnectionStatus.CONNECTED) { logger.trace("dropping response to {}: channel is not CONNECTED", requestDescription); } else { @@ -167,10 +217,10 @@ public String toString() { @Override public void sendResponse(Exception exception) { - sendFromTo(destination, getLocalNode(), action, new Runnable() { + execute(action, new Runnable() { @Override public void run() { - if (getConnectionStatus(destination, getLocalNode()) != ConnectionStatus.CONNECTED) { + if (destinationTransport.getConnectionStatus(getLocalNode()) != ConnectionStatus.CONNECTED) { logger.trace("dropping response to {}: channel is not CONNECTED", requestDescription); } else { diff --git a/test/framework/src/test/java/org/elasticsearch/test/disruption/DisruptableMockTransportTests.java b/test/framework/src/test/java/org/elasticsearch/test/disruption/DisruptableMockTransportTests.java index 3e98d8245432d..14aa79e87956e 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/disruption/DisruptableMockTransportTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/disruption/DisruptableMockTransportTests.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.node.Node; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.disruption.DisruptableMockTransport.ConnectionStatus; @@ -85,9 +86,6 @@ ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destina public void initTransports() { node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT); node2 = new DiscoveryNode("node2", buildNewFakeTransportAddress(), Version.CURRENT); - List discoNodes = new ArrayList<>(); - discoNodes.add(node1); - discoNodes.add(node2); disconnectedLinks = new HashSet<>(); blackholedLinks = new HashSet<>(); @@ -97,57 +95,37 @@ public void initTransports() { deterministicTaskQueue = new DeterministicTaskQueue( Settings.builder().put(Node.NODE_NAME_SETTING.getKey(), "dummy").build(), random()); - transport1 = new DisruptableMockTransport(logger) { + transport1 = new DisruptableMockTransport(node1, logger) { @Override - protected DiscoveryNode getLocalNode() { - return node1; + protected ConnectionStatus getConnectionStatus(DiscoveryNode destination) { + return DisruptableMockTransportTests.this.getConnectionStatus(getLocalNode(), destination); } @Override - protected ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) { - return DisruptableMockTransportTests.this.getConnectionStatus(sender, destination); + protected Optional getDisruptableMockTransport(TransportAddress address) { + return transports.stream().filter(t -> t.getLocalNode().getAddress().equals(address)).findAny(); } @Override - protected Optional getDisruptedCapturingTransport(DiscoveryNode destination, String action) { - int index = discoNodes.indexOf(destination); - if (index == -1) { - return Optional.empty(); - } else { - return Optional.of(transports.get(index)); - } - } - - @Override - protected void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) { - deterministicTaskQueue.scheduleNow(doDelivery); + protected void execute(Runnable runnable) { + deterministicTaskQueue.scheduleNow(runnable); } }; - transport2 = new DisruptableMockTransport(logger) { + transport2 = new DisruptableMockTransport(node2, logger) { @Override - protected DiscoveryNode getLocalNode() { - return node2; + protected ConnectionStatus getConnectionStatus(DiscoveryNode destination) { + return DisruptableMockTransportTests.this.getConnectionStatus(getLocalNode(), destination); } @Override - protected ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) { - return DisruptableMockTransportTests.this.getConnectionStatus(sender, destination); + protected Optional getDisruptableMockTransport(TransportAddress address) { + return transports.stream().filter(t -> t.getLocalNode().getAddress().equals(address)).findAny(); } @Override - protected Optional getDisruptedCapturingTransport(DiscoveryNode destination, String action) { - int index = discoNodes.indexOf(destination); - if (index == -1) { - return Optional.empty(); - } else { - return Optional.of(transports.get(index)); - } - } - - @Override - protected void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) { - deterministicTaskQueue.scheduleNow(doDelivery); + protected void execute(Runnable runnable) { + deterministicTaskQueue.scheduleNow(runnable); } }; @@ -161,6 +139,9 @@ protected void handle(DiscoveryNode sender, DiscoveryNode destination, String ac service1.start(); service2.start(); + + service1.connectToNode(node2); + service2.connectToNode(node1); } From 1eba1d1df91aa1797a7372bd7ceffb7010f869ad Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 11 Jan 2019 16:09:26 +0100 Subject: [PATCH 04/28] Fix SnapshotDisruptionIT Race Condition (#37358) * Due to a race between retrying the snapshot creation and the failed snapshot create trying to delete the snapshot there is no guarantee that the snapshot is eventually created by retries * Adjusted the assertion accordingly * Closes #36779 --- .../discovery/SnapshotDisruptionIT.java | 39 +++++++++---------- 1 file changed, 18 insertions(+), 21 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java index f6a95a3ed5b35..db3c71c398da7 100644 --- a/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java @@ -133,21 +133,7 @@ public void clusterChanged(ClusterChangedEvent event) { logger.info("--> waiting for disruption to start"); assertTrue(disruptionStarted.await(1, TimeUnit.MINUTES)); - logger.info("--> wait until the snapshot is done"); - assertBusy(() -> { - ClusterState state = dataNodeClient().admin().cluster().prepareState().get().getState(); - SnapshotsInProgress snapshots = state.custom(SnapshotsInProgress.TYPE); - SnapshotDeletionsInProgress snapshotDeletionsInProgress = state.custom(SnapshotDeletionsInProgress.TYPE); - if (snapshots != null && snapshots.entries().size() > 0) { - logger.info("Current snapshot state [{}]", snapshots.entries().get(0).state()); - fail("Snapshot is still running"); - } else if (snapshotDeletionsInProgress != null && snapshotDeletionsInProgress.hasDeletionsInProgress()) { - logger.info("Current snapshot deletion state [{}]", snapshotDeletionsInProgress); - fail("Snapshot deletion is still running"); - } else { - logger.info("Snapshot is no longer in the cluster state"); - } - }, 1, TimeUnit.MINUTES); + assertAllSnapshotsCompleted(); logger.info("--> verify that snapshot was successful or no longer exist"); assertBusy(() -> { @@ -177,14 +163,25 @@ public void clusterChanged(ClusterChangedEvent event) { } } - logger.info("--> verify that snapshot eventually will be created due to retries"); + assertAllSnapshotsCompleted(); + } + + private void assertAllSnapshotsCompleted() throws Exception { + logger.info("--> wait until the snapshot is done"); assertBusy(() -> { - try { - assertSnapshotExists("test-repo", "test-snap-2"); - } catch (SnapshotMissingException ex) { - throw new AssertionError(ex); + ClusterState state = dataNodeClient().admin().cluster().prepareState().get().getState(); + SnapshotsInProgress snapshots = state.custom(SnapshotsInProgress.TYPE); + SnapshotDeletionsInProgress snapshotDeletionsInProgress = state.custom(SnapshotDeletionsInProgress.TYPE); + if (snapshots != null && snapshots.entries().isEmpty() == false) { + logger.info("Current snapshot state [{}]", snapshots.entries().get(0).state()); + fail("Snapshot is still running"); + } else if (snapshotDeletionsInProgress != null && snapshotDeletionsInProgress.hasDeletionsInProgress()) { + logger.info("Current snapshot deletion state [{}]", snapshotDeletionsInProgress); + fail("Snapshot deletion is still running"); + } else { + logger.info("Snapshot is no longer in the cluster state"); } - }, 1, TimeUnit.MINUTES); + }, 1L, TimeUnit.MINUTES); } private void assertSnapshotExists(String repository, String snapshot) { From cae71cddfef98ca9768b4798f081ae89e502e02c Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 11 Jan 2019 07:59:15 -0800 Subject: [PATCH 05/28] Document Seq No powered optimistic concurrency control (#37284) Add documentation to describe the new sequence number powered optimistic concurrency control Relates #36148 Relates #10708 --- docs/reference/docs.asciidoc | 2 + docs/reference/docs/bulk.asciidoc | 11 ++ .../docs/concurrency-control.asciidoc | 114 ++++++++++++ docs/reference/docs/delete.asciidoc | 10 + docs/reference/docs/index_.asciidoc | 173 +++++++++--------- 5 files changed, 227 insertions(+), 83 deletions(-) create mode 100644 docs/reference/docs/concurrency-control.asciidoc diff --git a/docs/reference/docs.asciidoc b/docs/reference/docs.asciidoc index a8ab282853c8f..5c4c471b0a131 100644 --- a/docs/reference/docs.asciidoc +++ b/docs/reference/docs.asciidoc @@ -50,3 +50,5 @@ include::docs/termvectors.asciidoc[] include::docs/multi-termvectors.asciidoc[] include::docs/refresh.asciidoc[] + +include::docs/concurrency-control.asciidoc[] diff --git a/docs/reference/docs/bulk.asciidoc b/docs/reference/docs/bulk.asciidoc index 7ee634ccef649..0aae2365d965e 100644 --- a/docs/reference/docs/bulk.asciidoc +++ b/docs/reference/docs/bulk.asciidoc @@ -197,6 +197,17 @@ size for your particular workload. If using the HTTP API, make sure that the client does not send HTTP chunks, as this will slow things down. +[float] +[[bulk-optimistic-concurrency-control]] +=== Optimistic Concurrency Control + +Each `index` and `delete` action within a bulk API call may include the +`if_seq_no` and `if_primary_term` parameters in their respective action +and meta data lines. The `if_seq_no` and `if_primary_term` parameters control +how operations are executed, based on the last modification to existing +documents. See <> for more details. + + [float] [[bulk-versioning]] === Versioning diff --git a/docs/reference/docs/concurrency-control.asciidoc b/docs/reference/docs/concurrency-control.asciidoc new file mode 100644 index 0000000000000..d457b14068e26 --- /dev/null +++ b/docs/reference/docs/concurrency-control.asciidoc @@ -0,0 +1,114 @@ +[[optimistic-concurrency-control]] +== Optimistic concurrency control + +Elasticsearch is distributed. When documents are created, updated, or deleted, +the new version of the document has to be replicated to other nodes in the cluster. +Elasticsearch is also asynchronous and concurrent, meaning that these replication +requests are sent in parallel, and may arrive at their destination out of sequence. +Elasticsearch needs a way of ensuring that an older version of a document never +overwrites a newer version. + + +To ensure an older version of a document doesn't overwrite a newer version, every +operation performed to a document is assigned a sequence number by the primary +shard that coordinates that change. The sequence number is increased with each +operation and thus newer operations are guaranteed to have a higher sequence +number than older operations. Elasticsearch can then use the sequence number of +operations to make sure they never override a newer document version is never +overridden by a change that has a smaller sequence number assigned to it. + +For example, the following indexing command will create a document and assign it +an initial sequence number and primary term: + +[source,js] +-------------------------------------------------- +PUT products/_doc/1567 +{ + "product" : "r2d2", + "details" : "A resourceful astromech droid" +} +-------------------------------------------------- +// CONSOLE + +You can see the assigned sequence number and primary term in the +the `_seq_no` and `_primary_term` fields of the response: + +[source,js] +-------------------------------------------------- +{ + "_shards" : { + "total" : 2, + "failed" : 0, + "successful" : 1 + }, + "_index" : "products", + "_type" : "_doc", + "_id" : "1567", + "_version" : 1, + "_seq_no" : 362, + "_primary_term" : 2, + "result" : "created" +} +-------------------------------------------------- +// TESTRESPONSE[s/"_seq_no" : \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 2/"_primary_term" : $body._primary_term/] + + +Elasticsearch keeps tracks of the sequence number and primary of the last +operation to have changed each of the document it stores. The sequence number +and primary term are returned in the `_seq_no` and `_primary_term` fields in +the response of the <>: + +[source,js] +-------------------------------------------------- +GET products/_doc/1567 +-------------------------------------------------- +// CONSOLE +// TEST[continued] + +returns: + +[source,js] +-------------------------------------------------- +{ + "_index" : "products", + "_type" : "_doc", + "_id" : "1567", + "_version" : 1, + "_seq_no" : 362, + "_primary_term" : 2, + "found": true, + "_source" : { + "product" : "r2d2", + "details" : "A resourceful astromech droid" + } +} +-------------------------------------------------- +// TESTRESPONSE[s/"_seq_no" : \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 2/"_primary_term" : $body._primary_term/] + + +Note: The <> can return the `_seq_no` and `_primary_term` +for each search hit by requesting the `_seq_no` and `_primary_term` <>. + +The sequence number and the primary term uniquely identify a change. By noting down +the sequence number and primary term returned, you can make sure to only change the +document if no other change was made to it since you retrieved it. This +is done by setting the `if_seq_no` and `if_primary_term` parameters of either the +<> or the <>. + +For example, the following indexing call will make sure to add a tag to the +document without losing any potential change to the description or an addition +of another tag by another API: + +[source,js] +-------------------------------------------------- +PUT products/_doc/1567?if_seq_no=362&if_primary_term=2 +{ + "product" : "r2d2", + "details" : "A resourceful astromech droid", + "tags": ["droid"] +} +-------------------------------------------------- +// CONSOLE +// TEST[continued] +// TEST[catch: conflict] + diff --git a/docs/reference/docs/delete.asciidoc b/docs/reference/docs/delete.asciidoc index 146c822a7bf27..bc6f7b840048d 100644 --- a/docs/reference/docs/delete.asciidoc +++ b/docs/reference/docs/delete.asciidoc @@ -35,6 +35,16 @@ The result of the above delete operation is: // TESTRESPONSE[s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] // TESTRESPONSE[s/"_seq_no" : 5/"_seq_no" : $body._seq_no/] +[float] +[[optimistic-concurrency-control-delete]] +=== Optimistic concurrency control + +Delete operations can be made optional and only be performed if the last +modification to the document was assigned the sequence number and primary +term specified by the `if_seq_no` and `if_primary_term` parameters. If a +mismatch is detected, the operation will result in a `VersionConflictException` +and a status code of 409. See <> for more details. + [float] [[delete-versioning]] === Versioning diff --git a/docs/reference/docs/index_.asciidoc b/docs/reference/docs/index_.asciidoc index 70769b5b67237..572186487f44f 100644 --- a/docs/reference/docs/index_.asciidoc +++ b/docs/reference/docs/index_.asciidoc @@ -79,89 +79,6 @@ Automatic index creation can include a pattern based white/black list, for example, set `action.auto_create_index` to `+aaa*,-bbb*,+ccc*,-*` (+ meaning allowed, and - meaning disallowed). -[float] -[[index-versioning]] -=== Versioning - -Each indexed document is given a version number. The associated -`version` number is returned as part of the response to the index API -request. The index API optionally allows for -http://en.wikipedia.org/wiki/Optimistic_concurrency_control[optimistic -concurrency control] when the `version` parameter is specified. This -will control the version of the document the operation is intended to be -executed against. A good example of a use case for versioning is -performing a transactional read-then-update. Specifying a `version` from -the document initially read ensures no changes have happened in the -meantime. For example: - -[source,js] --------------------------------------------------- -PUT twitter/_doc/1?version=2 -{ - "message" : "elasticsearch now has versioning support, double cool!" -} --------------------------------------------------- -// CONSOLE -// TEST[continued] -// TEST[catch: conflict] - -*NOTE:* versioning is completely real time, and is not affected by the -near real time aspects of search operations. If no version is provided, -then the operation is executed without any version checks. - -By default, internal versioning is used that starts at 1 and increments -with each update, deletes included. Optionally, the version number can be -supplemented with an external value (for example, if maintained in a -database). To enable this functionality, `version_type` should be set to -`external`. The value provided must be a numeric, long value greater or equal to 0, -and less than around 9.2e+18. When using the external version type, instead -of checking for a matching version number, the system checks to see if -the version number passed to the index request is greater than the -version of the currently stored document. If true, the document will be -indexed and the new version number used. If the value provided is less -than or equal to the stored document's version number, a version -conflict will occur and the index operation will fail. - -WARNING: External versioning supports the value 0 as a valid version number. -This allows the version to be in sync with an external versioning system -where version numbers start from zero instead of one. It has the side effect -that documents with version number equal to zero cannot neither be updated -using the <> nor be deleted -using the <> as long as their -version number is equal to zero. - -A nice side effect is that there is no need to maintain strict ordering -of async indexing operations executed as a result of changes to a source -database, as long as version numbers from the source database are used. -Even the simple case of updating the Elasticsearch index using data from -a database is simplified if external versioning is used, as only the -latest version will be used if the index operations are out of order for -whatever reason. - -[float] -==== Version types - -Next to the `internal` & `external` version types explained above, Elasticsearch -also supports other types for specific use cases. Here is an overview of -the different version types and their semantics. - -`internal`:: only index the document if the given version is identical to the version -of the stored document. - -`external` or `external_gt`:: only index the document if the given version is strictly higher -than the version of the stored document *or* if there is no existing document. The given -version will be used as the new version and will be stored with the new document. The supplied -version must be a non-negative long number. - -`external_gte`:: only index the document if the given version is *equal* or higher -than the version of the stored document. If there is no existing document -the operation will succeed as well. The given version will be used as the new version -and will be stored with the new document. The supplied version must be a non-negative long number. - -*NOTE*: The `external_gte` version type is meant for special use cases and -should be used with care. If used incorrectly, it can result in loss of data. -There is another option, `force`, which is deprecated because it can cause -primary and replica shards to diverge. [float] [[operation-type]] @@ -238,6 +155,16 @@ The result of the above index operation is: -------------------------------------------------- // TESTRESPONSE[s/W0tpsmIBdwcYyG50zbta/$body._id/ s/"successful" : 2/"successful" : 1/] +[float] +[[optimistic-concurrency-control-index]] +=== Optimistic concurrency control + +Index operations can be made optional and only be performed if the last +modification to the document was assigned the sequence number and primary +term specified by the `if_seq_no` and `if_primary_term` parameters. If a +mismatch is detected, the operation will result in a `VersionConflictException` +and a status code of 409. See <> for more details. + [float] [[index-routing]] === Routing @@ -380,3 +307,83 @@ PUT twitter/_doc/1?timeout=5m } -------------------------------------------------- // CONSOLE + +[float] +[[index-versioning]] +=== Versioning + +Each indexed document is given a version number. By default, +internal versioning is used that starts at 1 and increments +with each update, deletes included. Optionally, the version number can be +set to an external value (for example, if maintained in a +database). To enable this functionality, `version_type` should be set to +`external`. The value provided must be a numeric, long value greater or equal to 0, +and less than around 9.2e+18. + +When using the external version type, the system checks to see if +the version number passed to the index request is greater than the +version of the currently stored document. If true, the document will be +indexed and the new version number used. If the value provided is less +than or equal to the stored document's version number, a version +conflict will occur and the index operation will fail. For example: + +[source,js] +-------------------------------------------------- +PUT twitter/_doc/1?version=2&version_type=external +{ + "message" : "elasticsearch now has versioning support, double cool!" +} +-------------------------------------------------- +// CONSOLE +// TEST[continued] + +*NOTE:* versioning is completely real time, and is not affected by the +near real time aspects of search operations. If no version is provided, +then the operation is executed without any version checks. + +The above will succeed since the the supplied version of 2 is higher than +the current document version of 1. If the document was already updated +and it's version was set to 2 or higher, the indexing command will fail +and result in a conflict (409 http status code). + +WARNING: External versioning supports the value 0 as a valid version number. +This allows the version to be in sync with an external versioning system +where version numbers start from zero instead of one. It has the side effect +that documents with version number equal to zero cannot neither be updated +using the <> nor be deleted +using the <> as long as their +version number is equal to zero. + +A nice side effect is that there is no need to maintain strict ordering +of async indexing operations executed as a result of changes to a source +database, as long as version numbers from the source database are used. +Even the simple case of updating the Elasticsearch index using data from +a database is simplified if external versioning is used, as only the +latest version will be used if the index operations are out of order for +whatever reason. + +[float] +==== Version types + +Next to the `external` version type explained above, Elasticsearch +also supports other types for specific use cases. Here is an overview of +the different version types and their semantics. + +`internal`:: only index the document if the given version is identical to the version +of the stored document. + +`external` or `external_gt`:: only index the document if the given version is strictly higher +than the version of the stored document *or* if there is no existing document. The given +version will be used as the new version and will be stored with the new document. The supplied +version must be a non-negative long number. + +`external_gte`:: only index the document if the given version is *equal* or higher +than the version of the stored document. If there is no existing document +the operation will succeed as well. The given version will be used as the new version +and will be stored with the new document. The supplied version must be a non-negative long number. + +*NOTE*: The `external_gte` version type is meant for special use cases and +should be used with care. If used incorrectly, it can result in loss of data. +There is another option, `force`, which is deprecated because it can cause +primary and replica shards to diverge. + From 093d4a371cca29d6567f6c2b166eff78468beac9 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 11 Jan 2019 08:07:35 -0800 Subject: [PATCH 06/28] followup to #37284 with additional feedback --- docs/reference/docs/concurrency-control.asciidoc | 4 ++-- docs/reference/docs/index_.asciidoc | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/reference/docs/concurrency-control.asciidoc b/docs/reference/docs/concurrency-control.asciidoc index d457b14068e26..e695e6b5127c9 100644 --- a/docs/reference/docs/concurrency-control.asciidoc +++ b/docs/reference/docs/concurrency-control.asciidoc @@ -14,8 +14,8 @@ operation performed to a document is assigned a sequence number by the primary shard that coordinates that change. The sequence number is increased with each operation and thus newer operations are guaranteed to have a higher sequence number than older operations. Elasticsearch can then use the sequence number of -operations to make sure they never override a newer document version is never -overridden by a change that has a smaller sequence number assigned to it. +operations to make sure a newer document version is never overridden by +a change that has a smaller sequence number assigned to it. For example, the following indexing command will create a document and assign it an initial sequence number and primary term: diff --git a/docs/reference/docs/index_.asciidoc b/docs/reference/docs/index_.asciidoc index 572186487f44f..f706aaca3e034 100644 --- a/docs/reference/docs/index_.asciidoc +++ b/docs/reference/docs/index_.asciidoc @@ -343,13 +343,13 @@ then the operation is executed without any version checks. The above will succeed since the the supplied version of 2 is higher than the current document version of 1. If the document was already updated -and it's version was set to 2 or higher, the indexing command will fail +and its version was set to 2 or higher, the indexing command will fail and result in a conflict (409 http status code). WARNING: External versioning supports the value 0 as a valid version number. This allows the version to be in sync with an external versioning system where version numbers start from zero instead of one. It has the side effect -that documents with version number equal to zero cannot neither be updated +that documents with version number equal to zero can neither be updated using the <> nor be deleted using the <> as long as their version number is equal to zero. @@ -359,7 +359,7 @@ of async indexing operations executed as a result of changes to a source database, as long as version numbers from the source database are used. Even the simple case of updating the Elasticsearch index using data from a database is simplified if external versioning is used, as only the -latest version will be used if the index operations are out of order for +latest version will be used if the index operations arrive out of order for whatever reason. [float] From bb6d8784e71857fd5cea7808e219c606c343b6f7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 11 Jan 2019 17:24:12 +0100 Subject: [PATCH 07/28] Switch indices.get rest after backport of `include_type_name` (#37351) With the `include_type_name` available now for indices.get on 6.x after the backport, the corresponsing yaml test can include anything from 6.7 on. Also changing the RestGetIndicesActionTests base test class. --- .../rest-api-spec/test/indices.get/10_basic.yml | 10 ++++++++-- .../admin/indices/RestGetIndicesActionTests.java | 4 ++-- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get/10_basic.yml index ef028dcdf67e8..e7c0af2ca1422 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get/10_basic.yml @@ -55,8 +55,8 @@ setup: --- "Test include_type_name": - skip: - version: " - 6.99.99" - reason: the include_type_name parameter is not backported to pre 7.0 versions yet + version: " - 6.6.99" + reason: the include_type_name parameter is not supported before 6.7 - do: indices.get: @@ -74,6 +74,12 @@ setup: - is_true: test_index.mappings - is_false: test_index.mappings.type_1 +--- +"Test include_type_name dafaults to false": + - skip: + version: " - 6.99.99" + reason: the include_type_name parameter default is different on 6.x and 7.0, so only test this on 7.0 clusters + - do: indices.get: index: test_index diff --git a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesActionTests.java index 5f157cd298d4d..e4eb0edbb8582 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesActionTests.java @@ -23,8 +23,8 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; -import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.rest.FakeRestRequest; +import org.elasticsearch.test.rest.RestActionTestCase; import java.io.IOException; import java.util.HashMap; @@ -33,7 +33,7 @@ import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; import static org.mockito.Mockito.mock; -public class RestGetIndicesActionTests extends ESTestCase { +public class RestGetIndicesActionTests extends RestActionTestCase { /** * Test that setting the "include_type_name" parameter raises a warning From e4391afd98c99e05ce1ca2163d1a096a0405d72b Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Fri, 11 Jan 2019 17:26:01 +0100 Subject: [PATCH 08/28] Test fix, wait for auto follower to have stopped in the background Relates to #36761 --- .../test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java | 6 ++++++ 1 file changed, 6 insertions(+) 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 1db42dba44e77..286e5badee133 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 @@ -139,7 +139,13 @@ public void testAutoFollowManyIndices() throws Exception { assertThat(autoFollowStats.getNumberOfSuccessfulFollowIndices(), equalTo((long) expectedVal1)); }); + // Delete auto follow pattern and make sure that in the background the auto follower has stopped + // then the leader index created after that should never be auto followed: deleteAutoFollowPatternSetting(); + assertBusy(() -> { + AutoFollowStats autoFollowStats = getAutoFollowStats(); + assertThat(autoFollowStats.getAutoFollowedClusters().size(), equalTo(0)); + }); createLeaderIndex("logs-does-not-count", leaderIndexSettings); putAutoFollowPatterns("my-pattern", new String[] {"logs-*"}); From 953fb9352f439baa162ee5773e82f253f1e70a0e Mon Sep 17 00:00:00 2001 From: David Roberts Date: Fri, 11 Jan 2019 16:31:55 +0000 Subject: [PATCH 09/28] [ML] Update error message for process update (#37363) When this message was first added the model debug config was the only thing that could be updated, but now more aspects of the config can be updated so the message needs to be more general. --- .../ml/job/process/autodetect/AutodetectProcessManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java index 32507df53cef7..ef03b4f9e7160 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java @@ -337,7 +337,7 @@ public void forecastJob(JobTask jobTask, ForecastParams params, Consumer handler) { AutodetectCommunicator communicator = getOpenAutodetectCommunicator(jobTask); if (communicator == null) { - String message = "Cannot process update model debug config because job [" + jobTask.getJobId() + + String message = "Cannot update the job config because job [" + jobTask.getJobId() + "] does not have a corresponding autodetect process"; logger.debug(message); handler.accept(ExceptionsHelper.conflictStatusException(message)); From 955d3aea194b4709a37f88057b8f2dd9a9af760c Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 11 Jan 2019 09:35:38 -0700 Subject: [PATCH 10/28] Mute testRoundRobinWithFailures (#32190) --- .../authc/ldap/support/SessionFactoryLoadBalancingTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/support/SessionFactoryLoadBalancingTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/support/SessionFactoryLoadBalancingTests.java index 87cc73d9a137e..1483b2f474bf0 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/support/SessionFactoryLoadBalancingTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/support/SessionFactoryLoadBalancingTests.java @@ -75,6 +75,7 @@ public void testRoundRobin() throws Exception { } } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/32190") public void testRoundRobinWithFailures() throws Exception { assumeTrue("at least one ldap server should be present for this test", ldapServers.length > 1); logger.debug("using [{}] ldap servers, urls {}", ldapServers.length, ldapUrls()); From 70cee18e56b638525c83054fe795ca1e8ca4f26a Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 11 Jan 2019 13:06:17 -0500 Subject: [PATCH 11/28] Introduce StepListener (#37327) This commit introduces StepListener which provides a simple way to write a flow consisting of multiple asynchronous steps without having nested callbacks. Relates #37291 --- .../elasticsearch/action/StepListener.java | 89 ++++++++++++++ .../util/concurrent/ListenableFuture.java | 8 +- .../action/StepListenerTests.java | 113 ++++++++++++++++++ 3 files changed, 209 insertions(+), 1 deletion(-) create mode 100644 server/src/main/java/org/elasticsearch/action/StepListener.java create mode 100644 server/src/test/java/org/elasticsearch/action/StepListenerTests.java diff --git a/server/src/main/java/org/elasticsearch/action/StepListener.java b/server/src/main/java/org/elasticsearch/action/StepListener.java new file mode 100644 index 0000000000000..efbf8c755d57c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/StepListener.java @@ -0,0 +1,89 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action; + +import org.elasticsearch.common.CheckedConsumer; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.common.util.concurrent.FutureUtils; +import org.elasticsearch.common.util.concurrent.ListenableFuture; + +import java.util.function.Consumer; + +/** + * A {@link StepListener} provides a simple way to write a flow consisting of + * multiple asynchronous steps without having nested callbacks. For example: + * + *
{@code
+ *  void asyncFlowMethod(... ActionListener flowListener) {
+ *    StepListener step1 = new StepListener<>();
+ *    asyncStep1(..., step1);
+
+ *    StepListener step2 = new StepListener<>();
+ *    step1.whenComplete(r1 -> {
+ *      asyncStep2(r1, ..., step2);
+ *    }, flowListener::onFailure);
+ *
+ *    step2.whenComplete(r2 -> {
+ *      R1 r1 = step1.result();
+ *      R r = combine(r1, r2);
+ *     flowListener.onResponse(r);
+ *    }, flowListener::onFailure);
+ *  }
+ * }
+ */ + +public final class StepListener implements ActionListener { + private final ListenableFuture delegate; + + public StepListener() { + this.delegate = new ListenableFuture<>(); + } + + @Override + public void onResponse(Response response) { + delegate.onResponse(response); + } + + @Override + public void onFailure(Exception e) { + delegate.onFailure(e); + } + + /** + * Registers the given actions which are called when this step is completed. If this step is completed successfully, + * the {@code onResponse} is called with the result; otherwise the {@code onFailure} is called with the failure. + * + * @param onResponse is called when this step is completed successfully + * @param onFailure is called when this step is completed with a failure + */ + public void whenComplete(CheckedConsumer onResponse, Consumer onFailure) { + delegate.addListener(ActionListener.wrap(onResponse, onFailure), EsExecutors.newDirectExecutorService(), null); + } + + /** + * Gets the result of this step. This method will throw {@link IllegalStateException} if this step is not completed yet. + */ + public Response result() { + if (delegate.isDone() == false) { + throw new IllegalStateException("step is not completed yet"); + } + return FutureUtils.get(delegate); + } +} diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/ListenableFuture.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/ListenableFuture.java index 725069c5937ed..d4ba603aca458 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/ListenableFuture.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/ListenableFuture.java @@ -60,7 +60,13 @@ public void addListener(ActionListener listener, ExecutorService executor, Th if (done) { run = true; } else { - listeners.add(new Tuple<>(ContextPreservingActionListener.wrapPreservingContext(listener, threadContext), executor)); + final ActionListener wrappedListener; + if (threadContext == null) { + wrappedListener = listener; + } else { + wrappedListener = ContextPreservingActionListener.wrapPreservingContext(listener, threadContext); + } + listeners.add(new Tuple<>(wrappedListener, executor)); run = false; } } diff --git a/server/src/test/java/org/elasticsearch/action/StepListenerTests.java b/server/src/test/java/org/elasticsearch/action/StepListenerTests.java new file mode 100644 index 0000000000000..15e88830e47e9 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/StepListenerTests.java @@ -0,0 +1,113 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; + +import static org.hamcrest.Matchers.equalTo; + +public class StepListenerTests extends ESTestCase { + private ThreadPool threadPool; + + @Before + public void setUpThreadPool() { + threadPool = new TestThreadPool(getTestName()); + } + + @After + public void tearDownThreadPool() { + terminate(threadPool); + } + + public void testSimpleSteps() throws Exception { + CountDownLatch latch = new CountDownLatch(1); + Consumer onFailure = e -> { + latch.countDown(); + fail("test a happy path"); + }; + + StepListener step1 = new StepListener<>(); //[a]sync provide a string + executeAction(() -> step1.onResponse("hello")); + StepListener step2 = new StepListener<>(); //[a]sync calculate the length of the string + step1.whenComplete(str -> executeAction(() -> step2.onResponse(str.length())), onFailure); + step2.whenComplete(length -> executeAction(latch::countDown), onFailure); + latch.await(); + assertThat(step1.result(), equalTo("hello")); + assertThat(step2.result(), equalTo(5)); + } + + public void testAbortOnFailure() throws Exception { + CountDownLatch latch = new CountDownLatch(1); + int failedStep = randomBoolean() ? 1 : 2; + AtomicInteger failureNotified = new AtomicInteger(); + Consumer onFailure = e -> { + failureNotified.getAndIncrement(); + latch.countDown(); + assertThat(e.getMessage(), equalTo("failed at step " + failedStep)); + }; + + StepListener step1 = new StepListener<>(); //[a]sync provide a string + if (failedStep == 1) { + executeAction(() -> step1.onFailure(new RuntimeException("failed at step 1"))); + } else { + executeAction(() -> step1.onResponse("hello")); + } + + StepListener step2 = new StepListener<>(); //[a]sync calculate the length of the string + step1.whenComplete(str -> { + if (failedStep == 2) { + executeAction(() -> step2.onFailure(new RuntimeException("failed at step 2"))); + } else { + executeAction(() -> step2.onResponse(str.length())); + } + }, onFailure); + + step2.whenComplete(length -> latch.countDown(), onFailure); + latch.await(); + assertThat(failureNotified.get(), equalTo(1)); + + if (failedStep == 1) { + assertThat(expectThrows(RuntimeException.class, step1::result).getMessage(), + equalTo("failed at step 1")); + assertThat(expectThrows(RuntimeException.class, step2::result).getMessage(), + equalTo("step is not completed yet")); + } else { + assertThat(step1.result(), equalTo("hello")); + assertThat(expectThrows(RuntimeException.class, step2::result).getMessage(), + equalTo("failed at step 2")); + } + } + + private void executeAction(Runnable runnable) { + if (randomBoolean()) { + threadPool.generic().execute(runnable); + } else { + runnable.run(); + } + } +} From 827ece73c840aab82ef1ab677a83c7c0901ac27e Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 11 Jan 2019 11:11:27 -0700 Subject: [PATCH 12/28] Mute MlConfigMigratorIT.testMigrateConfigs (#37374) --- .../elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java index 33b2484766669..023286af6b9e9 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java @@ -120,6 +120,7 @@ public void testWriteConfigToIndex() throws InterruptedException { assertNull(alreadyMigratedJob.getCustomSettings()); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37374") public void testMigrateConfigs() throws InterruptedException, IOException { // and jobs and datafeeds clusterstate MlMetadata.Builder mlMetadata = new MlMetadata.Builder(); From de52ba1f78edced3060804df7301c305b384009c Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Fri, 11 Jan 2019 13:16:05 -0500 Subject: [PATCH 13/28] Fix RollupDocumentation test to wait for job to stop Also adds some extra state debug information to various log messages --- .../client/documentation/RollupDocumentationIT.java | 11 ++++++++++- .../xpack/core/indexing/AsyncTwoPhaseIndexer.java | 6 +++--- .../rollup/action/TransportStopRollupAction.java | 7 ++++++- 3 files changed, 19 insertions(+), 5 deletions(-) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/RollupDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/RollupDocumentationIT.java index a1fcdbb7bfc83..8125c2f41f4c9 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/RollupDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/RollupDocumentationIT.java @@ -261,6 +261,14 @@ public void testStartRollupJob() throws Exception { } catch (Exception e) { // Swallow any exception, this test does not test actually cancelling. } + // stop job to prevent spamming exceptions on next start request + StopRollupJobRequest stopRequest = new StopRollupJobRequest(id); + stopRequest.waitForCompletion(); + stopRequest.timeout(TimeValue.timeValueSeconds(10)); + + StopRollupJobResponse response = client.rollup().stopRollupJob(stopRequest, RequestOptions.DEFAULT); + assertTrue(response.isAcknowledged()); + // tag::rollup-start-job-execute-listener ActionListener listener = new ActionListener() { @Override @@ -282,7 +290,8 @@ public void onFailure(Exception e) { assertTrue(latch.await(30L, TimeUnit.SECONDS)); // stop job so it can correctly be deleted by the test teardown - rc.stopRollupJob(new StopRollupJobRequest(id), RequestOptions.DEFAULT); + response = rc.stopRollupJob(stopRequest, RequestOptions.DEFAULT); + assertTrue(response.isAcknowledged()); } @SuppressWarnings("unused") diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java index 899226ad48519..df8eeb71e61b1 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java @@ -137,7 +137,8 @@ public synchronized boolean maybeTriggerAsyncJob(long now) { case INDEXING: case STOPPING: case ABORTING: - logger.warn("Schedule was triggered for job [" + getJobId() + "], but prior indexer is still running."); + logger.warn("Schedule was triggered for job [" + getJobId() + "], but prior indexer is still running " + + "(with state [" + currentState + "]"); return false; case STOPPED: @@ -381,8 +382,7 @@ private boolean checkState(IndexerState currentState) { case STOPPING: logger.info("Indexer job encountered [" + IndexerState.STOPPING + "] state, halting indexer."); - doSaveState(finishAndSetState(), getPosition(), () -> { - }); + doSaveState(finishAndSetState(), getPosition(), () -> {}); return false; case STOPPED: diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportStopRollupAction.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportStopRollupAction.java index 2715dd65e844d..09bfa57a39765 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportStopRollupAction.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportStopRollupAction.java @@ -81,10 +81,15 @@ private static ActionListener maybeWrapWithBlockin listener.onResponse(response); } else { listener.onFailure(new ElasticsearchTimeoutException("Timed out after [" + request.timeout().getStringRep() - + "] while waiting for rollup job [" + request.getId() + "] to stop")); + + "] while waiting for rollup job [" + request.getId() + "] to stop. State was [" + + ((RollupJobStatus) jobTask.getStatus()).getIndexerState() + "]")); } } catch (InterruptedException e) { listener.onFailure(e); + } catch (Exception e) { + listener.onFailure(new ElasticsearchTimeoutException("Encountered unexpected error while waiting for " + + "rollup job [" + request.getId() + "] to stop. State was [" + + ((RollupJobStatus) jobTask.getStatus()).getIndexerState() + "].", e)); } }); From 5101e518918ffeb05e679b10504a8bb5c5bd2632 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Fri, 11 Jan 2019 13:29:30 -0600 Subject: [PATCH 14/28] ML: Fix testMigrateConfigs (#37373) * ML: :s/execute/get * Fixing other broken tests * unmuting test --- .../xpack/ml/action/TransportOpenJobActionTests.java | 7 +++++-- .../xpack/ml/integration/BasicDistributedJobsIT.java | 6 ++++-- .../xpack/ml/integration/MlConfigMigratorIT.java | 3 +-- 3 files changed, 10 insertions(+), 6 deletions(-) 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 04dfa5f27502d..7d72ef7f633e1 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 @@ -8,9 +8,11 @@ import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.Version; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -455,8 +457,9 @@ public void testVerifyIndicesPrimaryShardsAreActive() { metaData = new MetaData.Builder(cs.metaData()); routingTable = new RoutingTable.Builder(cs.routingTable()); - - String indexToRemove = randomFrom(TransportOpenJobAction.indicesOfInterest(".ml-anomalies-shared")); + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + String indexToRemove = randomFrom(indexNameExpressionResolver.concreteIndexNames(cs, IndicesOptions.lenientExpandOpen(), + TransportOpenJobAction.indicesOfInterest(".ml-anomalies-shared"))); if (randomBoolean()) { routingTable.remove(indexToRemove); } else { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/BasicDistributedJobsIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/BasicDistributedJobsIT.java index 77041f40e0f4b..7779f4e13d0ea 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/BasicDistributedJobsIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/BasicDistributedJobsIT.java @@ -51,6 +51,7 @@ import java.util.concurrent.TimeUnit; import static org.elasticsearch.persistent.PersistentTasksClusterService.needsReassignment; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.hasEntry; public class BasicDistributedJobsIT extends BaseMlIntegTestCase { @@ -399,8 +400,9 @@ public void testMlStateAndResultsIndicesNotAvailable() throws Exception { String detailedMessage = detail.getMessage(); assertTrue(detailedMessage, detailedMessage.startsWith("Could not open job because no suitable nodes were found, allocation explanation")); - assertTrue(detailedMessage, detailedMessage.endsWith("because not all primary shards are active for the following indices " + - "[.ml-state,.ml-anomalies-shared]]")); + assertThat(detailedMessage, containsString("because not all primary shards are active for the following indices")); + assertThat(detailedMessage, containsString(".ml-state")); + assertThat(detailedMessage, containsString(".ml-anomalies-shared")); logger.info("Start data node"); String nonMlNode = internalCluster().startNode(Settings.builder() diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java index 023286af6b9e9..4993da215afbc 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java @@ -120,7 +120,6 @@ public void testWriteConfigToIndex() throws InterruptedException { assertNull(alreadyMigratedJob.getCustomSettings()); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37374") public void testMigrateConfigs() throws InterruptedException, IOException { // and jobs and datafeeds clusterstate MlMetadata.Builder mlMetadata = new MlMetadata.Builder(); @@ -311,7 +310,7 @@ public void testMigrateConfigsWithoutTasks_GivenMigrationIsDisabled() throws Int } public void assertSnapshot(MlMetadata expectedMlMetadata) throws IOException { - client().admin().indices().prepareRefresh(AnomalyDetectorsIndex.jobStateIndexPattern()).execute(); + client().admin().indices().prepareRefresh(AnomalyDetectorsIndex.jobStateIndexPattern()).get(); SearchResponse searchResponse = client() .prepareSearch(AnomalyDetectorsIndex.jobStateIndexPattern()) .setTypes(ElasticsearchMappings.DOC_TYPE) From 85531f0285839ebc2838129cefe8c2b5dc7ea999 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Fri, 11 Jan 2019 22:43:17 +0200 Subject: [PATCH 15/28] SQL: [Tests] Fix and enable internalClusterTests (#37300) SqlPlugin cannot have more than one public constructor, so for the testing purposes the `getLicenseState()` should be overriden. Fixes: #37191 Co-authored-by: Michael Basnight --- x-pack/plugin/sql/build.gradle | 7 ++- .../xpack/sql/plugin/SqlPlugin.java | 58 +++++++++---------- .../sql/action/AbstractSqlIntegTestCase.java | 30 +++------- .../sql/action/LocalStateSQLXPackPlugin.java | 27 +++++++++ .../xpack/sql/action/SqlActionIT.java | 7 +-- .../sql/action/SqlClearCursorActionIT.java | 12 ++-- .../xpack/sql/action/SqlDisabledIT.java | 13 ++++- .../xpack/sql/action/SqlLicenseIT.java | 6 +- .../sql/action/SqlTranslateActionIT.java | 11 ++-- .../xpack/sql/plugin/SqlPluginTests.java | 12 ++-- 10 files changed, 100 insertions(+), 83 deletions(-) create mode 100644 x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/LocalStateSQLXPackPlugin.java diff --git a/x-pack/plugin/sql/build.gradle b/x-pack/plugin/sql/build.gradle index f5dc3175d4162..cd2f82ee7b308 100644 --- a/x-pack/plugin/sql/build.gradle +++ b/x-pack/plugin/sql/build.gradle @@ -21,10 +21,13 @@ archivesBaseName = 'x-pack-sql' integTest.enabled = false task internalClusterTest(type: RandomizedTestingTask, - group: JavaBasePlugin.VERIFICATION_GROUP -) { + group: JavaBasePlugin.VERIFICATION_GROUP, + dependsOn: unitTest.dependsOn) { include '**/*IT.class' + systemProperty 'es.set.netty.runtime.available.processors', 'false' } +check.dependsOn internalClusterTest +internalClusterTest.mustRunAfter test dependencies { // "org.elasticsearch.plugin:x-pack-core:${version}" doesn't work with idea because the testArtifacts are also here diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/SqlPlugin.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/SqlPlugin.java index 8c759a7d87f8b..9f569206438d2 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/SqlPlugin.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/SqlPlugin.java @@ -50,40 +50,38 @@ public class SqlPlugin extends Plugin implements ActionPlugin { private final boolean enabled; - private final SqlLicenseChecker sqlLicenseChecker; - - SqlPlugin(boolean enabled, SqlLicenseChecker sqlLicenseChecker) { - this.enabled = enabled; - this.sqlLicenseChecker = sqlLicenseChecker; - } + private final SqlLicenseChecker sqlLicenseChecker = new SqlLicenseChecker( + (mode) -> { + XPackLicenseState licenseState = getLicenseState(); + switch (mode) { + case JDBC: + if (licenseState.isJdbcAllowed() == false) { + throw LicenseUtils.newComplianceException("jdbc"); + } + break; + case ODBC: + if (licenseState.isOdbcAllowed() == false) { + throw LicenseUtils.newComplianceException("odbc"); + } + break; + case PLAIN: + if (licenseState.isSqlAllowed() == false) { + throw LicenseUtils.newComplianceException(XPackField.SQL); + } + break; + default: + throw new IllegalArgumentException("Unknown SQL mode " + mode); + } + } + ); public SqlPlugin(Settings settings) { - this(XPackSettings.SQL_ENABLED.get(settings), new SqlLicenseChecker( - (mode) -> { - XPackLicenseState licenseState = XPackPlugin.getSharedLicenseState(); - switch (mode) { - case JDBC: - if (licenseState.isJdbcAllowed() == false) { - throw LicenseUtils.newComplianceException("jdbc"); - } - break; - case ODBC: - if (licenseState.isOdbcAllowed() == false) { - throw LicenseUtils.newComplianceException("odbc"); - } - break; - case PLAIN: - if (licenseState.isSqlAllowed() == false) { - throw LicenseUtils.newComplianceException(XPackField.SQL); - } - break; - default: - throw new IllegalArgumentException("Unknown SQL mode " + mode); - } - } - )); + this.enabled = XPackSettings.SQL_ENABLED.get(settings); } + // overridable by tests + protected XPackLicenseState getLicenseState() { return XPackPlugin.getSharedLicenseState(); } + @Override public Collection createComponents(Client client, ClusterService clusterService, ThreadPool threadPool, ResourceWatcherService resourceWatcherService, ScriptService scriptService, diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/AbstractSqlIntegTestCase.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/AbstractSqlIntegTestCase.java index 0e7d2888f9e16..c741667ba9ebf 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/AbstractSqlIntegTestCase.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/AbstractSqlIntegTestCase.java @@ -5,24 +5,19 @@ */ package org.elasticsearch.xpack.sql.action; -import org.elasticsearch.analysis.common.CommonAnalysisPlugin; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.reindex.ReindexPlugin; +import org.elasticsearch.license.LicenseService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.discovery.TestZenDiscovery; -import org.elasticsearch.xpack.core.XPackPlugin; import org.elasticsearch.xpack.core.XPackSettings; -import java.util.Arrays; import java.util.Collection; +import java.util.Collections; -public abstract class AbstractSqlIntegTestCase extends ESIntegTestCase { +import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE; - @Override - protected boolean ignoreExternalCluster() { - return true; - } +@ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 0, numClientNodes = 0, maxNumDataNodes = 0, transportClientRatio = 0) +public abstract class AbstractSqlIntegTestCase extends ESIntegTestCase { @Override protected Settings nodeSettings(int nodeOrdinal) { @@ -32,29 +27,18 @@ protected Settings nodeSettings(int nodeOrdinal) { settings.put(XPackSettings.WATCHER_ENABLED.getKey(), false); settings.put(XPackSettings.GRAPH_ENABLED.getKey(), false); settings.put(XPackSettings.MACHINE_LEARNING_ENABLED.getKey(), false); - settings.put("xpack.ml.autodetect_process", false); + settings.put(LicenseService.SELF_GENERATED_LICENSE_TYPE.getKey(), "trial"); return settings.build(); } @Override protected Collection> nodePlugins() { - return Arrays.asList(XPackPlugin.class, CommonAnalysisPlugin.class, ReindexPlugin.class); + return Collections.singletonList(LocalStateSQLXPackPlugin.class); } @Override protected Collection> transportClientPlugins() { return nodePlugins(); } - - @Override - protected Settings transportClientSettings() { - // Plugin should be loaded on the transport client as well - return nodeSettings(0); - } - - @Override - protected Collection> getMockPlugins() { - return Arrays.asList(TestZenDiscovery.TestPlugin.class, TestSeedPlugin.class); - } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/LocalStateSQLXPackPlugin.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/LocalStateSQLXPackPlugin.java new file mode 100644 index 0000000000000..1203bfae6776e --- /dev/null +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/LocalStateSQLXPackPlugin.java @@ -0,0 +1,27 @@ +/* + * 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.action; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; +import org.elasticsearch.xpack.sql.plugin.SqlPlugin; + +import java.nio.file.Path; + +public class LocalStateSQLXPackPlugin extends LocalStateCompositeXPackPlugin { + + public LocalStateSQLXPackPlugin(final Settings settings, final Path configPath) throws Exception { + super(settings, configPath); + LocalStateSQLXPackPlugin thisVar = this; + plugins.add(new SqlPlugin(settings) { + @Override + protected XPackLicenseState getLicenseState() { + return thisVar.getLicenseState(); + } + }); + } +} diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlActionIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlActionIT.java index c71d7c274947f..43ea7fe92ebee 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlActionIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlActionIT.java @@ -18,12 +18,11 @@ public class SqlActionIT extends AbstractSqlIntegTestCase { - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") - public void testSqlAction() throws Exception { + public void testSqlAction() { assertAcked(client().admin().indices().prepareCreate("test").get()); client().prepareBulk() - .add(new IndexRequest("test", "doc", "1").source("data", "bar", "count", 42)) - .add(new IndexRequest("test", "doc", "2").source("data", "baz", "count", 43)) + .add(new IndexRequest("test").id("1").source("data", "bar", "count", 42)) + .add(new IndexRequest("test").id("2").source("data", "baz", "count", 43)) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) .get(); ensureYellow("test"); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlClearCursorActionIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlClearCursorActionIT.java index 952104b49eebb..d509c1325fed2 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlClearCursorActionIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlClearCursorActionIT.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.sql.action; -import org.apache.lucene.util.LuceneTestCase.AwaitsFix; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.WriteRequest; @@ -17,16 +16,15 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; -@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") public class SqlClearCursorActionIT extends AbstractSqlIntegTestCase { - public void testSqlClearCursorAction() throws Exception { + public void testSqlClearCursorAction() { assertAcked(client().admin().indices().prepareCreate("test").get()); BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); int indexSize = randomIntBetween(100, 300); logger.info("Indexing {} records", indexSize); for (int i = 0; i < indexSize; i++) { - bulkRequestBuilder.add(new IndexRequest("test", "doc", "id" + i).source("data", "bar", "count", i)); + bulkRequestBuilder.add(new IndexRequest("test").id("id" + i).source("data", "bar", "count", i)); } bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get(); ensureYellow("test"); @@ -50,13 +48,13 @@ public void testSqlClearCursorAction() throws Exception { assertEquals(0, getNumberOfSearchContexts()); } - public void testAutoCursorCleanup() throws Exception { + public void testAutoCursorCleanup() { assertAcked(client().admin().indices().prepareCreate("test").get()); BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); int indexSize = randomIntBetween(100, 300); logger.info("Indexing {} records", indexSize); for (int i = 0; i < indexSize; i++) { - bulkRequestBuilder.add(new IndexRequest("test", "doc", "id" + i).source("data", "bar", "count", i)); + bulkRequestBuilder.add(new IndexRequest("test").id("id" + i).source("data", "bar", "count", i)); } bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get(); ensureYellow("test"); @@ -77,7 +75,7 @@ public void testAutoCursorCleanup() throws Exception { do { sqlQueryResponse = new SqlQueryRequestBuilder(client(), SqlQueryAction.INSTANCE).cursor(sqlQueryResponse.cursor()).get(); fetched += sqlQueryResponse.size(); - } while (sqlQueryResponse.cursor().equals("") == false); + } while (sqlQueryResponse.cursor().isEmpty() == false); assertEquals(indexSize, fetched); SqlClearCursorResponse cleanCursorResponse = new SqlClearCursorRequestBuilder(client(), SqlClearCursorAction.INSTANCE) diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlDisabledIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlDisabledIT.java index 0a56e804a00c6..51be147005173 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlDisabledIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlDisabledIT.java @@ -6,13 +6,23 @@ package org.elasticsearch.xpack.sql.action; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.xpack.core.XPackPlugin; import org.elasticsearch.xpack.core.XPackSettings; +import java.util.Collection; +import java.util.Collections; + import static org.hamcrest.CoreMatchers.either; import static org.hamcrest.CoreMatchers.startsWith; public class SqlDisabledIT extends AbstractSqlIntegTestCase { + @Override + protected Collection> nodePlugins() { + return Collections.singletonList(XPackPlugin.class); + } + @Override protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder() @@ -29,8 +39,7 @@ protected Settings transportClientSettings() { .build(); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") - public void testSqlAction() throws Exception { + public void testSqlAction() { Throwable throwable = expectThrows(Throwable.class, () -> new SqlQueryRequestBuilder(client(), SqlQueryAction.INSTANCE).query("SHOW tables").get()); assertThat(throwable.getMessage(), diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlLicenseIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlLicenseIT.java index 62f7b42c69944..50dda656ab4d1 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlLicenseIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlLicenseIT.java @@ -35,7 +35,7 @@ import static org.elasticsearch.license.XPackLicenseStateTests.randomTrialOrPlatinumMode; import static org.hamcrest.Matchers.equalTo; -@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") +@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37320") public class SqlLicenseIT extends AbstractLicensesIntegrationTestCase { @Override protected boolean ignoreExternalCluster() { @@ -164,8 +164,8 @@ public void testSqlTranslateActionLicense() throws Exception { private void setupTestIndex() { ElasticsearchAssertions.assertAcked(client().admin().indices().prepareCreate("test").get()); client().prepareBulk() - .add(new IndexRequest("test", "doc", "1").source("data", "bar", "count", 42)) - .add(new IndexRequest("test", "doc", "2").source("data", "baz", "count", 43)) + .add(new IndexRequest("test").id("1").source("data", "bar", "count", 42)) + .add(new IndexRequest("test").id("2").source("data", "baz", "count", 43)) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) .get(); } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlTranslateActionIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlTranslateActionIT.java index d86245dcbfae7..3dc41ad9dd362 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlTranslateActionIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlTranslateActionIT.java @@ -17,12 +17,11 @@ public class SqlTranslateActionIT extends AbstractSqlIntegTestCase { - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") - public void testSqlTranslateAction() throws Exception { + public void testSqlTranslateAction() { assertAcked(client().admin().indices().prepareCreate("test").get()); client().prepareBulk() - .add(new IndexRequest("test", "doc", "1").source("data", "bar", "count", 42)) - .add(new IndexRequest("test", "doc", "2").source("data", "baz", "count", 43)) + .add(new IndexRequest("test").id("1").source("data", "bar", "count", 42)) + .add(new IndexRequest("test").id("2").source("data", "baz", "count", 43)) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) .get(); ensureYellow("test"); @@ -33,11 +32,11 @@ public void testSqlTranslateAction() throws Exception { .query("SELECT " + columns + " FROM test ORDER BY count").get(); SearchSourceBuilder source = response.source(); FetchSourceContext fetch = source.fetchSource(); - assertEquals(true, fetch.fetchSource()); + assertTrue(fetch.fetchSource()); assertArrayEquals(new String[] { "data" }, fetch.includes()); assertEquals( singletonList(new DocValueFieldsContext.FieldAndFormat("count", DocValueFieldsContext.USE_DEFAULT_FORMAT)), source.docValueFields()); - assertEquals(singletonList(SortBuilders.fieldSort("count")), source.sorts()); + assertEquals(singletonList(SortBuilders.fieldSort("count").missing("_last").unmappedType("long")), source.sorts()); } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plugin/SqlPluginTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plugin/SqlPluginTests.java index 363254f414c49..be8ac7b9968dc 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plugin/SqlPluginTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plugin/SqlPluginTests.java @@ -25,14 +25,14 @@ public class SqlPluginTests extends ESTestCase { public void testSqlDisabled() { - SqlPlugin plugin = new SqlPlugin(false, new SqlLicenseChecker((mode) -> {})); + Settings settings = Settings.builder().put("xpack.sql.enabled", false).build(); + SqlPlugin plugin = new SqlPlugin(settings); assertThat(plugin.createComponents(mock(Client.class), "cluster", new NamedWriteableRegistry(Cursors.getNamedWriteables())), - empty()); + empty()); assertThat(plugin.getActions(), empty()); assertThat(plugin.getRestHandlers(Settings.EMPTY, mock(RestController.class), - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), - IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, new SettingsFilter(Collections.emptyList()), - mock(IndexNameExpressionResolver.class), () -> mock(DiscoveryNodes.class)), empty()); + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, new SettingsFilter(Collections.emptyList()), + mock(IndexNameExpressionResolver.class), () -> mock(DiscoveryNodes.class)), empty()); } - } From 359222c55c50bf9c15062ca16c71420c41d60100 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Fri, 11 Jan 2019 23:08:00 +0200 Subject: [PATCH 16/28] SQL: Make `FULL` non-reserved keyword in the grammar (#37377) Since `full` can be common as a field name or part of a field name (e.g.: `full.name` or `name.full`), it's nice if it's not a reserved keyword of the grammar so a user can use it without resorting to quotes. Fixes: #37376 --- x-pack/plugin/sql/src/main/antlr/SqlBase.g4 | 2 +- .../xpack/sql/parser/SqlBaseParser.java | 24 +++++++++++++------ .../xpack/sql/parser/SqlParserTests.java | 22 +++++++++++++++++ 3 files changed, 40 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 b/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 index 3bed074b03a2f..6435d80d04073 100644 --- a/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 +++ b/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 @@ -342,7 +342,7 @@ nonReserved | CATALOGS | COLUMNS | CURRENT | DAY | DEBUG | EXECUTABLE | EXPLAIN - | FIRST | FORMAT | FUNCTIONS + | FIRST | FORMAT | FULL | FUNCTIONS | GRAPHVIZ | HOUR | INTERVAL 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 323aeea30eaf2..8ba886404536b 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 @@ -811,6 +811,7 @@ public final StatementContext statement() throws RecognitionException { case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -887,6 +888,7 @@ public final StatementContext statement() throws RecognitionException { case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -957,6 +959,7 @@ public final StatementContext statement() throws RecognitionException { case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -1140,6 +1143,7 @@ public final StatementContext statement() throws RecognitionException { case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -2056,7 +2060,7 @@ public final GroupingExpressionsContext groupingExpressions() throws Recognition match(T__0); setState(332); _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 << 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) | (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(324); expression(); @@ -3870,6 +3874,7 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti case FALSE: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -4257,7 +4262,7 @@ public final PrimaryExpressionContext primaryExpression() throws RecognitionExce { setState(581); _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 << 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 << 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)) { { setState(578); qualifiedName(); @@ -4756,6 +4761,7 @@ public final FunctionExpressionContext functionExpression() throws RecognitionEx case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -4864,7 +4870,7 @@ public final FunctionTemplateContext functionTemplate() throws RecognitionExcept match(T__0); setState(664); _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 << 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) | (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(654); _la = _input.LA(1); @@ -4967,6 +4973,7 @@ public final FunctionNameContext functionName() throws RecognitionException { case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -5809,6 +5816,7 @@ public final IdentifierContext identifier() throws RecognitionException { case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -5899,7 +5907,7 @@ public final TableIdentifierContext tableIdentifier() throws RecognitionExceptio { setState(739); _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 << 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 << 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)) { { setState(736); ((TableIdentifierContext)_localctx).catalog = identifier(); @@ -6101,6 +6109,7 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -6300,6 +6309,7 @@ public static class NonReservedContext extends ParserRuleContext { public TerminalNode EXPLAIN() { return getToken(SqlBaseParser.EXPLAIN, 0); } public TerminalNode FIRST() { return getToken(SqlBaseParser.FIRST, 0); } public TerminalNode FORMAT() { return getToken(SqlBaseParser.FORMAT, 0); } + public TerminalNode FULL() { return getToken(SqlBaseParser.FULL, 0); } public TerminalNode FUNCTIONS() { return getToken(SqlBaseParser.FUNCTIONS, 0); } public TerminalNode GRAPHVIZ() { return getToken(SqlBaseParser.GRAPHVIZ, 0); } public TerminalNode HOUR() { return getToken(SqlBaseParser.HOUR, 0); } @@ -6353,7 +6363,7 @@ public final NonReservedContext nonReserved() throws RecognitionException { { setState(765); _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 << 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 << 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)) ) { _errHandler.recoverInline(this); } else { consume(); @@ -6464,8 +6474,8 @@ private boolean valueExpression_sempred(ValueExpressionContext _localctx, int pr "\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::AAEGIL"+ - "OPRSVWYY\\\\\u035f\2n\3\2\2\2\4q\3\2\2\2\6\u00de\3\2\2\2\b\u00e9\3\2\2"+ + "\2\b\t\22\23\25\25\27\27\31\31\36\36 #$&(++//\62\62\65\6688::AAEGILO"+ + "PRSVWYY\\\\\u035f\2n\3\2\2\2\4q\3\2\2\2\6\u00de\3\2\2\2\b\u00e9\3\2\2"+ "\2\n\u00ed\3\2\2\2\f\u0102\3\2\2\2\16\u0109\3\2\2\2\20\u010b\3\2\2\2\22"+ "\u0113\3\2\2\2\24\u012f\3\2\2\2\26\u0139\3\2\2\2\30\u0143\3\2\2\2\32\u0152"+ "\3\2\2\2\34\u0154\3\2\2\2\36\u015a\3\2\2\2 \u015c\3\2\2\2\"\u0163\3\2"+ diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java index 8e0074798a503..199b4e119d81b 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java @@ -31,6 +31,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.StringJoiner; import static java.util.Collections.nCopies; import static java.util.stream.Collectors.toList; @@ -67,6 +68,27 @@ public void testSelectRightFunction() { assertEquals("RIGHT", f.functionName()); } + public void testsSelectNonReservedKeywords() { + String[] reserved = new String[] { + "ANALYZE", "ANALYZED", "CATALOGS", "COLUMNS", "CURRENT", "DAY", "DEBUG", "EXECUTABLE", "EXPLAIN", + "FIRST", "FORMAT", "FULL", "FUNCTIONS", "GRAPHVIZ", "HOUR", "INTERVAL", "LAST", "LIMIT", + "MAPPED", "MINUTE", "MONTH", "OPTIMIZED", "PARSED", "PHYSICAL", "PLAN", "QUERY", "RLIKE", + "SCHEMAS", "SECOND", "SHOW", "SYS", "TABLES", "TEXT", "TYPE", "TYPES", "VERIFY", "YEAR"}; + StringJoiner sj = new StringJoiner(","); + for (String s : reserved) { + sj.add(s); + } + + Project project = project(parseStatement("SELECT " + sj.toString() + " FROM foo")); + assertEquals(reserved.length, project.projections().size()); + + for (int i = 0; i < project.projections().size(); i++) { + NamedExpression ne = project.projections().get(i); + assertEquals(UnresolvedAttribute.class, ne.getClass()); + assertEquals(reserved[i], ne.name()); + } + } + public void testOrderByField() { Order.OrderDirection dir = randomFrom(Order.OrderDirection.values()); OrderBy ob = orderBy(parseStatement("SELECT * FROM foo ORDER BY bar" + stringForDirection(dir))); From 63fe3c6ed604d2aee3ae5266b35e87f35ff09bee Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 11 Jan 2019 23:26:04 +0100 Subject: [PATCH 17/28] Fix PrimaryAllocationIT Race Condition (#37355) * Fix PrimaryAllocationIT Race Condition * Forcing a stale primary allocation on a green index was tripping the assertion that was removed * Added a test that this case still errors out correctly * Made the ability to wipe stopped datanode's data public on the internal test cluster and used it to ensure correct behaviour on the fixed test * Previously it simply passed because the test finished before the index went green and would NPE when the index was green at the time of the shard store status request, that would then come up empty * Closes #37345 --- .../TransportClusterRerouteAction.java | 6 ++++- .../cluster/routing/PrimaryAllocationIT.java | 25 +++++++++++++++++-- .../test/InternalTestCluster.java | 3 +-- 3 files changed, 29 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java index d5cc35b2205ac..7f29f0bb6db8b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java @@ -113,7 +113,11 @@ private void verifyThenSubmitUpdate(ClusterRerouteRequest request, ActionListene for (Map.Entry> entry : stalePrimaryAllocations.entrySet()) { final String index = entry.getKey(); final ImmutableOpenIntMap> indexStatus = status.get(index); - assert indexStatus != null; + if (indexStatus == null) { + // The index in the stale primary allocation request was green and hence filtered out by the store status + // request. We ignore it here since the relevant exception will be thrown by the reroute action later on. + continue; + } for (AbstractAllocateAllocationCommand command : entry.getValue()) { final List shardStatus = indexStatus.get(command.shardId()); 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 3f826c587e683..b106944e97065 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -265,7 +265,6 @@ public void testForceStaleReplicaToBePromotedToPrimary() throws Exception { assertThat(newHistoryUUIds, hasSize(1)); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37345") public void testForceStaleReplicaToBePromotedToPrimaryOnWrongNode() throws Exception { String master = internalCluster().startMasterOnlyNode(Settings.EMPTY); internalCluster().startDataOnlyNodes(2); @@ -275,7 +274,10 @@ public void testForceStaleReplicaToBePromotedToPrimaryOnWrongNode() throws Excep .put("index.number_of_replicas", 1)).get()); ensureGreen(); createStaleReplicaScenario(master); - internalCluster().startDataOnlyNodes(2); + // Ensure the stopped primary's data is deleted so that it doesn't get picked up by the next datanode we start + internalCluster().wipePendingDataDirectories(); + internalCluster().startDataOnlyNodes(1); + ensureStableCluster(3, master); final int shardId = 0; final List nodeNames = new ArrayList<>(Arrays.asList(internalCluster().getNodeNames())); nodeNames.remove(master); @@ -292,6 +294,25 @@ public void testForceStaleReplicaToBePromotedToPrimaryOnWrongNode() throws Excep equalTo("No data for shard [" + shardId + "] of index [" + idxName + "] found on node [" + nodeWithoutData + ']')); } + public void testForceStaleReplicaToBePromotedForGreenIndex() { + internalCluster().startMasterOnlyNode(Settings.EMPTY); + final List dataNodes = internalCluster().startDataOnlyNodes(2); + final String idxName = "test"; + assertAcked(client().admin().indices().prepareCreate(idxName) + .setSettings(Settings.builder().put("index.number_of_shards", 1) + .put("index.number_of_replicas", 1)).get()); + ensureGreen(); + final String nodeWithoutData = randomFrom(dataNodes); + final int shardId = 0; + IllegalArgumentException iae = expectThrows( + IllegalArgumentException.class, + () -> client().admin().cluster().prepareReroute() + .add(new AllocateStalePrimaryAllocationCommand(idxName, shardId, nodeWithoutData, true)).get()); + assertThat( + iae.getMessage(), + equalTo("[allocate_stale_primary] primary [" + idxName+ "][" + shardId + "] is already assigned")); + } + public void testForceStaleReplicaToBePromotedForMissingIndex() { internalCluster().startMasterOnlyNode(Settings.EMPTY); final String dataNode = internalCluster().startDataOnlyNode(); 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 e6e11dacb749f..e4a11ad414ffe 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -1398,8 +1398,7 @@ private void randomlyResetClients() { } } - private void wipePendingDataDirectories() { - assert Thread.holdsLock(this); + public synchronized void wipePendingDataDirectories() { if (!dataDirToClean.isEmpty()) { try { for (Path path : dataDirToClean) { From 44a1071018b1563a15548fda14f9abbf11985606 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sat, 12 Jan 2019 12:49:48 -0500 Subject: [PATCH 18/28] Make recovery source partially non-blocking (#37291) Today a peer-recovery may run into a deadlock if the value of node_concurrent_recoveries is too high. This happens because the peer-recovery is executed in a blocking fashion. This commit attempts to make the recovery source partially non-blocking. I will make three follow-ups to make it fully non-blocking: (1) send translog operations, (2) primary relocation, (3) send commit files. Relates #36195 --- .../common/util/CancellableThreads.java | 81 ++++++++++++------- .../recovery/PeerRecoverySourceService.java | 13 ++- .../indices/recovery/RecoveryResponse.java | 2 +- .../recovery/RecoverySourceHandler.java | 79 ++++++++++-------- .../common/util/CancellableThreadsTests.java | 37 ++++++++- .../recovery/RecoverySourceHandlerTests.java | 7 +- .../index/shard/IndexShardTestCase.java | 7 +- 7 files changed, 149 insertions(+), 77 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/util/CancellableThreads.java b/server/src/main/java/org/elasticsearch/common/util/CancellableThreads.java index 3037c7a73c0d5..ec3664fd5b469 100644 --- a/server/src/main/java/org/elasticsearch/common/util/CancellableThreads.java +++ b/server/src/main/java/org/elasticsearch/common/util/CancellableThreads.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.common.util; +import org.apache.lucene.util.SetOnce; import org.apache.lucene.util.ThreadInterruptedException; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.Nullable; @@ -38,34 +39,35 @@ public class CancellableThreads { private final Set threads = new HashSet<>(); // needs to be volatile as it is also read outside of synchronized blocks. private volatile boolean cancelled = false; + private final SetOnce onCancel = new SetOnce<>(); private String reason; public synchronized boolean isCancelled() { return cancelled; } - - /** call this will throw an exception if operation was cancelled. - * Override {@link #onCancel(String, Exception)} for custom failure logic */ - public synchronized void checkForCancel() { - if (isCancelled()) { - onCancel(reason, null); - } + public void checkForCancel() { + checkForCancel(null); } - /** - * called if {@link #checkForCancel()} was invoked after the operation was cancelled. - * the default implementation always throws an {@link ExecutionCancelledException}, suppressing - * any other exception that occurred before cancellation - * @param reason reason for failure supplied by the caller of {@link #cancel} - * @param suppressedException any error that was encountered during the execution before the operation was cancelled. - */ - protected void onCancel(String reason, @Nullable Exception suppressedException) { - RuntimeException e = new ExecutionCancelledException("operation was cancelled reason [" + reason + "]"); - if (suppressedException != null) { - e.addSuppressed(suppressedException); + private void checkForCancel(Exception beforeCancelException) { + if (isCancelled()) { + final String reason; + final OnCancel onCancel; + synchronized (this) { + reason = this.reason; + onCancel = this.onCancel.get(); + } + if (onCancel != null) { + onCancel.onCancel(reason, beforeCancelException); + } + // fallback to the default exception + final RuntimeException cancelExp = new ExecutionCancelledException("operation was cancelled reason [" + reason + "]"); + if (beforeCancelException != null) { + cancelExp.addSuppressed(beforeCancelException); + } + throw cancelExp; } - throw e; } private synchronized boolean add() { @@ -125,17 +127,14 @@ public void executeIO(IOInterruptible interruptible) throws IOException { // clear the flag interrupted flag as we are checking for failure.. Thread.interrupted(); } - synchronized (this) { - if (isCancelled()) { - onCancel(reason, ioException != null ? ioException : runtimeException); - } else if (ioException != null) { - // if we're not canceling, we throw the original exception - throw ioException; - } - if (runtimeException != null) { - // if we're not canceling, we throw the original exception - throw runtimeException; - } + checkForCancel(ioException != null ? ioException : runtimeException); + if (ioException != null) { + // if we're not canceling, we throw the original exception + throw ioException; + } + if (runtimeException != null) { + // if we're not canceling, we throw the original exception + throw runtimeException; } if (cancelledByExternalInterrupt) { // restore interrupt flag to at least adhere to expected behavior @@ -185,4 +184,26 @@ public ExecutionCancelledException(StreamInput in) throws IOException { super(in); } } + + /** + * Registers a callback that will be invoked when some running operations are cancelled or {@link #checkForCancel()} is called. + */ + public synchronized void setOnCancel(OnCancel onCancel) { + this.onCancel.set(onCancel); + } + + @FunctionalInterface + public interface OnCancel { + /** + * Called when some running operations are cancelled or {@link #checkForCancel()} is explicitly called. + * If this method throws an exception, cancelling tasks will fail with that exception; otherwise they + * will fail with the default exception {@link ExecutionCancelledException}. + * + * @param reason the reason of the cancellation + * @param beforeCancelException any error that was encountered during the execution before the operations were cancelled. + * @see #checkForCancel() + * @see #setOnCancel(OnCancel) + */ + void onCancel(String reason, @Nullable Exception beforeCancelException); + } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java index 69af8841f4b40..9c30ab156c08a 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java @@ -22,6 +22,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.inject.Inject; @@ -81,7 +83,7 @@ public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexSh } } - private RecoveryResponse recover(final StartRecoveryRequest request) throws IOException { + private void recover(StartRecoveryRequest request, ActionListener listener) throws IOException { final IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); final IndexShard shard = indexService.getShard(request.shardId().id()); @@ -101,18 +103,13 @@ private RecoveryResponse recover(final StartRecoveryRequest request) throws IOEx RecoverySourceHandler handler = ongoingRecoveries.addNewRecovery(request, shard); logger.trace("[{}][{}] starting recovery to {}", request.shardId().getIndex().getName(), request.shardId().id(), request.targetNode()); - try { - return handler.recoverToTarget(); - } finally { - ongoingRecoveries.remove(shard, handler); - } + handler.recoverToTarget(ActionListener.runAfter(listener, () -> ongoingRecoveries.remove(shard, handler))); } class StartRecoveryTransportRequestHandler implements TransportRequestHandler { @Override public void messageReceived(final StartRecoveryRequest request, final TransportChannel channel, Task task) throws Exception { - RecoveryResponse response = recover(request); - channel.sendResponse(response); + recover(request, new HandledTransportAction.ChannelActionListener<>(channel, Actions.START_RECOVERY, request)); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java index 02d4ff5dbc13b..605d202cbaf9b 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java @@ -26,7 +26,7 @@ import java.io.IOException; import java.util.List; -final class RecoveryResponse extends TransportResponse { +public final class RecoveryResponse extends TransportResponse { final List phase1FileNames; final List phase1FileSizes; diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 315af6b4ae084..639ab47799366 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -34,7 +34,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.StopWatch; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.lease.Releasable; @@ -70,6 +69,7 @@ import java.util.List; import java.util.Locale; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import java.util.function.Supplier; @@ -96,22 +96,7 @@ public class RecoverySourceHandler { private final StartRecoveryRequest request; private final int chunkSizeInBytes; private final RecoveryTargetHandler recoveryTarget; - - private final CancellableThreads cancellableThreads = new CancellableThreads() { - @Override - protected void onCancel(String reason, @Nullable Exception suppressedException) { - RuntimeException e; - if (shard.state() == IndexShardState.CLOSED) { // check if the shard got closed on us - e = new IndexShardClosedException(shard.shardId(), "shard is closed and recovery was canceled reason [" + reason + "]"); - } else { - e = new ExecutionCancelledException("recovery was canceled reason [" + reason + "]"); - } - if (suppressedException != null) { - e.addSuppressed(suppressedException); - } - throw e; - } - }; + private final CancellableThreads cancellableThreads = new CancellableThreads(); public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recoveryTarget, final StartRecoveryRequest request, @@ -131,19 +116,37 @@ public StartRecoveryRequest getRequest() { /** * performs the recovery from the local engine to the target */ - public RecoveryResponse recoverToTarget() throws IOException { - runUnderPrimaryPermit(() -> { - final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable(); - ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId()); - if (targetShardRouting == null) { - logger.debug("delaying recovery of {} as it is not listed as assigned to target node {}", request.shardId(), - request.targetNode()); - throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); - } - assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; - }, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ", shard, cancellableThreads, logger); - - try (Closeable ignored = shard.acquireRetentionLockForPeerRecovery()) { + public void recoverToTarget(ActionListener listener) { + final List resources = new CopyOnWriteArrayList<>(); + final Closeable releaseResources = () -> IOUtils.close(resources); + final ActionListener wrappedListener = ActionListener.notifyOnce(listener); + try { + cancellableThreads.setOnCancel((reason, beforeCancelEx) -> { + final RuntimeException e; + if (shard.state() == IndexShardState.CLOSED) { // check if the shard got closed on us + e = new IndexShardClosedException(shard.shardId(), "shard is closed and recovery was canceled reason [" + reason + "]"); + } else { + e = new CancellableThreads.ExecutionCancelledException("recovery was canceled reason [" + reason + "]"); + } + if (beforeCancelEx != null) { + e.addSuppressed(beforeCancelEx); + } + IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e)); + throw e; + }); + runUnderPrimaryPermit(() -> { + final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable(); + ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId()); + if (targetShardRouting == null) { + logger.debug("delaying recovery of {} as it is not listed as assigned to target node {}", request.shardId(), + request.targetNode()); + throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); + } + assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; + }, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ", + shard, cancellableThreads, logger); + final Closeable retentionLock = shard.acquireRetentionLockForPeerRecovery(); + resources.add(retentionLock); final long startingSeqNo; final long requiredSeqNoRangeStart; final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && @@ -217,6 +220,8 @@ public RecoveryResponse recoverToTarget() throws IOException { } final SendSnapshotResult sendSnapshotResult; try (Translog.Snapshot snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo)) { + // we can release the retention lock here because the snapshot itself will retain the required operations. + IOUtils.close(retentionLock, () -> resources.remove(retentionLock)); // we have to capture the max_seen_auto_id_timestamp and the max_seq_no_of_updates to make sure that these values // are at least as high as the corresponding values on the primary when any of these operations were executed on it. final long maxSeenAutoIdTimestamp = shard.getMaxSeenAutoIdTimestamp(); @@ -229,10 +234,16 @@ public RecoveryResponse recoverToTarget() throws IOException { finalizeRecovery(sendSnapshotResult.targetLocalCheckpoint); final long phase1ThrottlingWaitTime = 0L; // TODO: return the actual throttle time - return new RecoveryResponse(sendFileResult.phase1FileNames, sendFileResult.phase1FileSizes, - sendFileResult.phase1ExistingFileNames, sendFileResult.phase1ExistingFileSizes, sendFileResult.totalSize, - sendFileResult.existingTotalSize, sendFileResult.took.millis(), phase1ThrottlingWaitTime, prepareEngineTime.millis(), - sendSnapshotResult.totalOperations, sendSnapshotResult.tookTime.millis()); + assert resources.isEmpty() : "not every resource is released [" + resources + "]"; + IOUtils.close(resources); + wrappedListener.onResponse( + new RecoveryResponse(sendFileResult.phase1FileNames, sendFileResult.phase1FileSizes, + sendFileResult.phase1ExistingFileNames, sendFileResult.phase1ExistingFileSizes, sendFileResult.totalSize, + sendFileResult.existingTotalSize, sendFileResult.took.millis(), phase1ThrottlingWaitTime, prepareEngineTime.millis(), + sendSnapshotResult.totalOperations, sendSnapshotResult.tookTime.millis()) + ); + } catch (Exception e) { + IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e)); } } diff --git a/server/src/test/java/org/elasticsearch/common/util/CancellableThreadsTests.java b/server/src/test/java/org/elasticsearch/common/util/CancellableThreadsTests.java index 2b937730e4750..8b8d17cbf3bf2 100644 --- a/server/src/test/java/org/elasticsearch/common/util/CancellableThreadsTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/CancellableThreadsTests.java @@ -24,7 +24,12 @@ import org.hamcrest.Matchers; import java.io.IOException; +import java.util.Arrays; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.elasticsearch.common.util.CancellableThreads.ExecutionCancelledException; +import static org.hamcrest.Matchers.equalTo; public class CancellableThreadsTests extends ESTestCase { public static class CustomException extends RuntimeException { @@ -39,6 +44,8 @@ public IOCustomException(String msg) { } } + static class ThrowOnCancelException extends RuntimeException { + } private class TestPlan { public final int id; @@ -167,6 +174,19 @@ public void testCancellableThreads() throws InterruptedException { } readyForCancel.await(); + final boolean throwInOnCancel = randomBoolean(); + final AtomicInteger invokeTimes = new AtomicInteger(); + cancellableThreads.setOnCancel((reason, beforeCancelException) -> { + invokeTimes.getAndIncrement(); + if (throwInOnCancel) { + ThrowOnCancelException e = new ThrowOnCancelException(); + if (beforeCancelException != null) { + e.addSuppressed(beforeCancelException); + } + throw e; + } + }); + cancellableThreads.cancel("test"); for (Thread thread : threads) { thread.join(20000); @@ -181,7 +201,11 @@ public void testCancellableThreads() throws InterruptedException { assertNull(exceptions[i]); } else { // in all other cases, we expect a cancellation exception. - assertThat(exceptions[i], Matchers.instanceOf(CancellableThreads.ExecutionCancelledException.class)); + if (throwInOnCancel) { + assertThat(exceptions[i], Matchers.instanceOf(ThrowOnCancelException.class)); + } else { + assertThat(exceptions[i], Matchers.instanceOf(ExecutionCancelledException.class)); + } if (plan.exceptAfterCancel) { assertThat(exceptions[i].getSuppressed(), Matchers.arrayContaining( @@ -191,8 +215,17 @@ public void testCancellableThreads() throws InterruptedException { assertThat(exceptions[i].getSuppressed(), Matchers.emptyArray()); } } - assertThat(interrupted[plan.id], Matchers.equalTo(plan.presetInterrupt)); + assertThat(interrupted[plan.id], equalTo(plan.presetInterrupt)); + } + assertThat(invokeTimes.longValue(), + equalTo(Arrays.stream(plans).filter(p -> p.exceptBeforeCancel == false && p.exitBeforeCancel == false).count())); + if (throwInOnCancel) { + expectThrows(ThrowOnCancelException.class, cancellableThreads::checkForCancel); + } else { + expectThrows(ExecutionCancelledException.class, cancellableThreads::checkForCancel); } + assertThat(invokeTimes.longValue(), + equalTo(Arrays.stream(plans).filter(p -> p.exceptBeforeCancel == false && p.exitBeforeCancel == false).count() + 1)); } } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 3f6a8072d86d5..17e88d2864d05 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -35,6 +35,7 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.UUIDs; @@ -433,7 +434,11 @@ SendSnapshotResult phase2(long startingSeqNo, long requiredSeqNoRangeStart, long } }; - expectThrows(IndexShardRelocatedException.class, handler::recoverToTarget); + PlainActionFuture future = new PlainActionFuture<>(); + expectThrows(IndexShardRelocatedException.class, () -> { + handler.recoverToTarget(future); + future.actionGet(); + }); assertFalse(phase1Called.get()); assertFalse(prepareTargetForTranslogCalled.get()); assertFalse(phase2Called.get()); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 2e13cd6e66543..367e0ded60775 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -23,6 +23,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -67,6 +68,7 @@ import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; import org.elasticsearch.indices.recovery.RecoveryFailedException; +import org.elasticsearch.indices.recovery.RecoveryResponse; import org.elasticsearch.indices.recovery.RecoverySourceHandler; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryTarget; @@ -604,7 +606,10 @@ protected final void recoverUnstartedReplica(final IndexShard replica, (int) ByteSizeUnit.MB.toBytes(1)); primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable, Collections.emptySet()); - recovery.recoverToTarget(); + + PlainActionFuture future = new PlainActionFuture<>(); + recovery.recoverToTarget(future); + future.actionGet(); recoveryTarget.markAsDone(); } From 03be4dbaca182a360e0a6cf3aa547739f44f1308 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sat, 12 Jan 2019 14:43:19 -0800 Subject: [PATCH 19/28] Introduce retention lease persistence (#37375) This commit introduces the persistence of retention leases by persisting them in index commits and recovering them when recovering a shard from store. --- docs/reference/indices/flush.asciidoc | 4 +- .../elasticsearch/index/engine/Engine.java | 1 + .../index/engine/InternalEngine.java | 10 +- .../index/engine/SoftDeletesPolicy.java | 15 ++- .../index/seqno/ReplicationTracker.java | 11 ++ .../index/seqno/RetentionLease.java | 107 +++++++++++++++++ .../elasticsearch/index/shard/IndexShard.java | 11 ++ .../index/engine/InternalEngineTests.java | 27 ++++- .../index/seqno/RetentionLeaseTests.java | 70 +++++++++++- .../shard/IndexShardRetentionLeaseTests.java | 53 +++++++++ .../index/engine/EngineTestCase.java | 108 ++++++++++++++---- 11 files changed, 386 insertions(+), 31 deletions(-) diff --git a/docs/reference/indices/flush.asciidoc b/docs/reference/indices/flush.asciidoc index c6bf60182fd76..a0027756ab57a 100644 --- a/docs/reference/indices/flush.asciidoc +++ b/docs/reference/indices/flush.asciidoc @@ -102,7 +102,8 @@ which returns something similar to: "max_seq_no" : "-1", "sync_id" : "AVvFY-071siAOuFGEO9P", <1> "max_unsafe_auto_id_timestamp" : "-1", - "min_retained_seq_no": "0" + "min_retained_seq_no" : "0", + "retention_leases" : "id:replica-0;retaining_seq_no:0;timestamp:1547235588;source:replica" }, "num_docs" : 0 } @@ -117,6 +118,7 @@ which returns something similar to: // TESTRESPONSE[s/"translog_uuid" : "hnOG3xFcTDeoI_kvvvOdNA"/"translog_uuid": $body.indices.twitter.shards.0.0.commit.user_data.translog_uuid/] // TESTRESPONSE[s/"history_uuid" : "XP7KDJGiS1a2fHYiFL5TXQ"/"history_uuid": $body.indices.twitter.shards.0.0.commit.user_data.history_uuid/] // TESTRESPONSE[s/"sync_id" : "AVvFY-071siAOuFGEO9P"/"sync_id": $body.indices.twitter.shards.0.0.commit.user_data.sync_id/] +// TESTRESPONSE[s/"retention_leases" : "id:replica-0;retaining_seq_no:0;timestamp:1547235588;source:replica"/"retention_leases": $body.indices.twitter.shards.0.0.commit.user_data.retention_leases/] <1> the `sync id` marker [float] diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index 008b85331030d..e1df104d338df 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -113,6 +113,7 @@ public abstract class Engine implements Closeable { public static final String SYNC_COMMIT_ID = "sync_id"; public static final String HISTORY_UUID_KEY = "history_uuid"; public static final String MIN_RETAINED_SEQNO = "min_retained_seq_no"; + public static final String RETENTION_LEASES = "retention_leases"; public static final String MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID = "max_unsafe_auto_id_timestamp"; protected final ShardId shardId; diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index d0e55fc13eeda..df52c6bc0213f 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -51,6 +51,7 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lucene.LoggerInfoStream; import org.elasticsearch.common.lucene.Lucene; @@ -74,6 +75,7 @@ import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; import org.elasticsearch.index.seqno.LocalCheckpointTracker; +import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ElasticsearchMergePolicy; @@ -2336,7 +2338,13 @@ protected void commitIndexWriter(final IndexWriter writer, final Translog transl commitData.put(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, Long.toString(maxUnsafeAutoIdTimestamp.get())); commitData.put(HISTORY_UUID_KEY, historyUUID); if (softDeleteEnabled) { - commitData.put(Engine.MIN_RETAINED_SEQNO, Long.toString(softDeletesPolicy.getMinRetainedSeqNo())); + /* + * We sample these from the policy (which occurs under a lock) to ensure that we have a consistent view of the minimum + * retained sequence number, and the retention leases. + */ + final Tuple> retentionPolicy = softDeletesPolicy.getRetentionPolicy(); + commitData.put(Engine.MIN_RETAINED_SEQNO, Long.toString(retentionPolicy.v1())); + commitData.put(Engine.RETENTION_LEASES, RetentionLease.encodeRetentionLeases(retentionPolicy.v2())); } logger.trace("committing writer with commit data [{}]", commitData); return commitData.entrySet().iterator(); diff --git a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java index c957902d8df77..a2452d4b53eb9 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java @@ -21,6 +21,7 @@ import org.apache.lucene.document.LongPoint; import org.apache.lucene.search.Query; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.seqno.RetentionLease; @@ -45,6 +46,7 @@ final class SoftDeletesPolicy { private long retentionOperations; // The min seq_no value that is retained - ops after this seq# should exist in the Lucene index. private long minRetainedSeqNo; + private Collection retentionLeases; // provides the retention leases used to calculate the minimum sequence number to retain private final Supplier> retentionLeasesSupplier; @@ -57,6 +59,7 @@ final class SoftDeletesPolicy { this.retentionOperations = retentionOperations; this.minRetainedSeqNo = minRetainedSeqNo; this.retentionLeasesSupplier = Objects.requireNonNull(retentionLeasesSupplier); + retentionLeases = retentionLeasesSupplier.get(); this.localCheckpointOfSafeCommit = SequenceNumbers.NO_OPS_PERFORMED; this.retentionLockCount = 0; } @@ -106,7 +109,11 @@ private synchronized void releaseRetentionLock() { * Operations whose seq# is least this value should exist in the Lucene index. */ synchronized long getMinRetainedSeqNo() { - // Do not advance if the retention lock is held + return getRetentionPolicy().v1(); + } + + public synchronized Tuple> getRetentionPolicy() { + // do not advance if the retention lock is held if (retentionLockCount == 0) { /* * This policy retains operations for two purposes: peer-recovery and querying changes history. @@ -119,8 +126,8 @@ synchronized long getMinRetainedSeqNo() { */ // calculate the minimum sequence number to retain based on retention leases - final long minimumRetainingSequenceNumber = retentionLeasesSupplier - .get() + retentionLeases = retentionLeasesSupplier.get(); + final long minimumRetainingSequenceNumber = retentionLeases .stream() .mapToLong(RetentionLease::retainingSequenceNumber) .min() @@ -139,7 +146,7 @@ synchronized long getMinRetainedSeqNo() { */ minRetainedSeqNo = Math.max(minRetainedSeqNo, minSeqNoToRetain); } - return minRetainedSeqNo; + return Tuple.tuple(minRetainedSeqNo, retentionLeases); } /** 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 4298e5712bfc6..f309512ec98b6 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -185,6 +185,17 @@ public synchronized void addOrUpdateRetentionLease(final String id, final long r retentionLeases.put(id, new RetentionLease(id, retainingSequenceNumber, currentTimeMillisSupplier.getAsLong(), source)); } + /** + * Updates retention leases on a replica. + * + * @param retentionLeases the retention leases + */ + public synchronized void updateRetentionLeasesOnReplica(final Collection retentionLeases) { + assert primaryMode == false; + this.retentionLeases.clear(); + this.retentionLeases.putAll(retentionLeases.stream().collect(Collectors.toMap(RetentionLease::id, Function.identity()))); + } + public static class CheckpointState implements Writeable { /** diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java index 076b707a5df42..f763759261385 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java @@ -19,6 +19,13 @@ package org.elasticsearch.index.seqno; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Locale; +import java.util.Objects; +import java.util.stream.Collectors; + /** * A "shard history retention lease" (or "retention lease" for short) is conceptually a marker containing a retaining sequence number such * that all operations with sequence number at least that retaining sequence number will be retained during merge operations (which could @@ -81,18 +88,118 @@ public String source() { * @param source the source of the retention lease */ public RetentionLease(final String id, final long retainingSequenceNumber, final long timestamp, final String source) { + Objects.requireNonNull(id); + if (id.isEmpty()) { + throw new IllegalArgumentException("retention lease ID can not be empty"); + } + if (id.contains(":") || id.contains(";") || id.contains(",")) { + // retention lease IDs can not contain these characters because they are used in encoding retention leases + throw new IllegalArgumentException("retention lease ID can not contain any of [:;,] but was [" + id + "]"); + } if (retainingSequenceNumber < SequenceNumbers.UNASSIGNED_SEQ_NO) { throw new IllegalArgumentException("retention lease retaining sequence number [" + retainingSequenceNumber + "] out of range"); } if (timestamp < 0) { throw new IllegalArgumentException("retention lease timestamp [" + timestamp + "] out of range"); } + Objects.requireNonNull(source); + if (source.isEmpty()) { + throw new IllegalArgumentException("retention lease source can not be empty"); + } + if (source.contains(":") || source.contains(";") || source.contains(",")) { + // retention lease sources can not contain these characters because they are used in encoding retention leases + throw new IllegalArgumentException("retention lease source can not contain any of [:;,] but was [" + source + "]"); + } this.id = id; this.retainingSequenceNumber = retainingSequenceNumber; this.timestamp = timestamp; this.source = source; } + /** + * Encodes a retention lease as a string. This encoding can be decoded by {@link #decodeRetentionLease(String)}. The retention lease is + * encoded in the format id:{id};retaining_seq_no:{retainingSequenecNumber};timestamp:{timestamp};source:{source}. + * + * @param retentionLease the retention lease + * @return the encoding of the retention lease + */ + static String encodeRetentionLease(final RetentionLease retentionLease) { + Objects.requireNonNull(retentionLease); + return String.format( + Locale.ROOT, + "id:%s;retaining_seq_no:%d;timestamp:%d;source:%s", + retentionLease.id(), + retentionLease.retainingSequenceNumber(), + retentionLease.timestamp(), + retentionLease.source()); + } + + /** + * Encodes a collection of retention leases as a string. This encoding can be decoed by {@link #decodeRetentionLeases(String)}. The + * encoding is a comma-separated encoding of each retention lease as encoded by {@link #encodeRetentionLease(RetentionLease)}. + * + * @param retentionLeases the retention leases + * @return the encoding of the retention leases + */ + public static String encodeRetentionLeases(final Collection retentionLeases) { + Objects.requireNonNull(retentionLeases); + return retentionLeases.stream().map(RetentionLease::encodeRetentionLease).collect(Collectors.joining(",")); + } + + /** + * Decodes a retention lease encoded by {@link #encodeRetentionLease(RetentionLease)}. + * + * @param encodedRetentionLease an encoded retention lease + * @return the decoded retention lease + */ + static RetentionLease decodeRetentionLease(final String encodedRetentionLease) { + Objects.requireNonNull(encodedRetentionLease); + final String[] fields = encodedRetentionLease.split(";"); + assert fields.length == 4 : Arrays.toString(fields); + assert fields[0].matches("id:[^:;,]+") : fields[0]; + final String id = fields[0].substring("id:".length()); + assert fields[1].matches("retaining_seq_no:\\d+") : fields[1]; + final long retainingSequenceNumber = Long.parseLong(fields[1].substring("retaining_seq_no:".length())); + assert fields[2].matches("timestamp:\\d+") : fields[2]; + final long timestamp = Long.parseLong(fields[2].substring("timestamp:".length())); + assert fields[3].matches("source:[^:;,]+") : fields[3]; + final String source = fields[3].substring("source:".length()); + return new RetentionLease(id, retainingSequenceNumber, timestamp, source); + } + + /** + * Decodes retention leases encoded by {@link #encodeRetentionLeases(Collection)}. + * + * @param encodedRetentionLeases an encoded collection of retention leases + * @return the decoded retention leases + */ + public static Collection decodeRetentionLeases(final String encodedRetentionLeases) { + Objects.requireNonNull(encodedRetentionLeases); + if (encodedRetentionLeases.isEmpty()) { + return Collections.emptyList(); + } + assert Arrays.stream(encodedRetentionLeases.split(",")) + .allMatch(s -> s.matches("id:[^:;,]+;retaining_seq_no:\\d+;timestamp:\\d+;source:[^:;,]+")) + : encodedRetentionLeases; + return Arrays.stream(encodedRetentionLeases.split(",")).map(RetentionLease::decodeRetentionLease).collect(Collectors.toList()); + } + + @Override + public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + final RetentionLease that = (RetentionLease) o; + return Objects.equals(id, that.id) && + retainingSequenceNumber == that.retainingSequenceNumber && + timestamp == that.timestamp && + Objects.equals(source, that.source); + } + + @Override + public int hashCode() { + return Objects.hash(id, retainingSequenceNumber, timestamp, source); + } + @Override public String toString() { return "RetentionLease{" + 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 66b8e607b5c76..270ce20819102 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -106,6 +106,7 @@ import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.search.stats.ShardSearchStats; import org.elasticsearch.index.seqno.ReplicationTracker; +import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.PrimaryReplicaSyncer.ResyncTask; @@ -140,6 +141,7 @@ import java.nio.channels.ClosedByInterruptException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.List; @@ -1416,6 +1418,7 @@ private void innerOpenEngineAndTranslog() throws IOException { final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); final long globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID); replicationTracker.updateGlobalCheckpointOnReplica(globalCheckpoint, "read from translog checkpoint"); + replicationTracker.updateRetentionLeasesOnReplica(getRetentionLeases(store.readLastCommittedSegmentsInfo())); trimUnsafeCommits(); synchronized (mutex) { verifyNotClosed(); @@ -1435,6 +1438,14 @@ private void innerOpenEngineAndTranslog() throws IOException { assert recoveryState.getStage() == RecoveryState.Stage.TRANSLOG : "TRANSLOG stage expected but was: " + recoveryState.getStage(); } + static Collection getRetentionLeases(final SegmentInfos segmentInfos) { + final String committedRetentionLeases = segmentInfos.getUserData().get(Engine.RETENTION_LEASES); + if (committedRetentionLeases == null) { + return Collections.emptyList(); + } + return RetentionLease.decodeRetentionLeases(committedRetentionLeases); + } + private void trimUnsafeCommits() throws IOException { assert currentEngineReference.get() == null || currentEngineReference.get() instanceof ReadOnlyEngine : "a write engine is running"; final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index f88aaedd6991f..796d7eb0c60ec 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -116,6 +116,7 @@ import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.seqno.ReplicationTracker; +import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexSearcherWrapper; @@ -140,6 +141,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Base64; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -5241,13 +5243,14 @@ public void testKeepMinRetainedSeqNoByMergePolicy() throws IOException { final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + final AtomicReference> leasesHolder = new AtomicReference<>(Collections.emptyList()); final List operations = generateSingleDocHistory(true, randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "2"); Randomness.shuffle(operations); Set existingSeqNos = new HashSet<>(); store = createStore(); - engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, - globalCheckpoint::get)); + engine = createEngine( + config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get, leasesHolder::get)); assertThat(engine.getMinRetainedSeqNo(), equalTo(0L)); long lastMinRetainedSeqNo = engine.getMinRetainedSeqNo(); for (Engine.Operation op : operations) { @@ -5261,6 +5264,18 @@ public void testKeepMinRetainedSeqNoByMergePolicy() throws IOException { if (randomBoolean()) { globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpointTracker().getCheckpoint())); } + if (randomBoolean()) { + final int length = randomIntBetween(0, 8); + final List leases = new ArrayList<>(length); + for (int i = 0; i < length; i++) { + final String id = randomAlphaOfLength(8); + final long retainingSequenceNumber = randomLongBetween(0L, Math.max(0L, globalCheckpoint.get())); + final long timestamp = randomLongBetween(0L, Long.MAX_VALUE); + final String source = randomAlphaOfLength(8); + leases.add(new RetentionLease(id, retainingSequenceNumber, timestamp, source)); + } + leasesHolder.set(leases); + } if (rarely()) { settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); @@ -5273,6 +5288,14 @@ public void testKeepMinRetainedSeqNoByMergePolicy() throws IOException { engine.flush(true, true); assertThat(Long.parseLong(engine.getLastCommittedSegmentInfos().userData.get(Engine.MIN_RETAINED_SEQNO)), equalTo(engine.getMinRetainedSeqNo())); + final Collection leases = leasesHolder.get(); + if (leases.isEmpty()) { + assertThat(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.RETENTION_LEASES), equalTo("")); + } else { + assertThat( + engine.getLastCommittedSegmentInfos().getUserData().get(Engine.RETENTION_LEASES), + equalTo(RetentionLease.encodeRetentionLeases(leases))); + } } if (rarely()) { engine.forceMerge(randomBoolean()); diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java index a5e4af5d0e6a3..c4340a381ce25 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java @@ -21,12 +21,34 @@ import org.elasticsearch.test.ESTestCase; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasToString; public class RetentionLeaseTests extends ESTestCase { + public void testInvalidId() { + final String id = "id" + randomFrom(":", ";", ","); + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> new RetentionLease(id, randomNonNegativeLong(), randomNonNegativeLong(), "source")); + assertThat(e, hasToString(containsString("retention lease ID can not contain any of [:;,] but was [" + id + "]"))); + } + + public void testEmptyId() { + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> new RetentionLease("", randomNonNegativeLong(), randomNonNegativeLong(), "source")); + assertThat(e, hasToString(containsString("retention lease ID can not be empty"))); + } + public void testRetainingSequenceNumberOutOfRange() { final long retainingSequenceNumber = randomLongBetween(Long.MIN_VALUE, UNASSIGNED_SEQ_NO - 1); final IllegalArgumentException e = expectThrows( @@ -42,9 +64,51 @@ public void testTimestampOutOfRange() { final IllegalArgumentException e = expectThrows( IllegalArgumentException.class, () -> new RetentionLease("id", randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE), timestamp, "source")); - assertThat( - e, - hasToString(containsString("retention lease timestamp [" + timestamp + "] out of range"))); + assertThat(e, hasToString(containsString("retention lease timestamp [" + timestamp + "] out of range"))); + } + + public void testInvalidSource() { + final String source = "source" + randomFrom(":", ";", ","); + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> new RetentionLease("id", randomNonNegativeLong(), randomNonNegativeLong(), source)); + assertThat(e, hasToString(containsString("retention lease source can not contain any of [:;,] but was [" + source + "]"))); + } + + public void testEmptySource() { + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> new RetentionLease("id", randomNonNegativeLong(), randomNonNegativeLong(), "")); + assertThat(e, hasToString(containsString("retention lease source can not be empty"))); + } + + public void testRetentionLeaseEncoding() { + final String id = randomAlphaOfLength(8); + final long retainingSequenceNumber = randomNonNegativeLong(); + final long timestamp = randomNonNegativeLong(); + final String source = randomAlphaOfLength(8); + final RetentionLease retentionLease = new RetentionLease(id, retainingSequenceNumber, timestamp, source); + assertThat(RetentionLease.decodeRetentionLease(RetentionLease.encodeRetentionLease(retentionLease)), equalTo(retentionLease)); + } + + public void testRetentionLeasesEncoding() { + final int length = randomIntBetween(0, 8); + final List retentionLeases = new ArrayList<>(length); + for (int i = 0; i < length; i++) { + final String id = randomAlphaOfLength(8); + final long retainingSequenceNumber = randomNonNegativeLong(); + final long timestamp = randomNonNegativeLong(); + final String source = randomAlphaOfLength(8); + final RetentionLease retentionLease = new RetentionLease(id, retainingSequenceNumber, timestamp, source); + retentionLeases.add(retentionLease); + } + final Collection decodedRetentionLeases = + RetentionLease.decodeRetentionLeases(RetentionLease.encodeRetentionLeases(retentionLeases)); + if (length == 0) { + assertThat(decodedRetentionLeases, empty()); + } else { + assertThat(decodedRetentionLeases, contains(retentionLeases.toArray(new RetentionLease[0]))); + } } } 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 bd2a33617eecf..e95b52280ca2e 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java @@ -19,9 +19,14 @@ package org.elasticsearch.index.shard; +import org.apache.lucene.index.SegmentInfos; +import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.cluster.routing.RecoverySource; +import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -33,9 +38,11 @@ import java.util.Map; import java.util.concurrent.ExecutorService; 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.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; @@ -127,6 +134,52 @@ public void testExpiration() throws IOException { } } + public void testCommit() throws IOException { + final Settings settings = Settings.builder() + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING.getKey(), Long.MAX_VALUE, TimeUnit.NANOSECONDS) + .build(); + final IndexShard indexShard = newStartedShard( + true, + settings, + new InternalEngineFactory()); + try { + final int length = randomIntBetween(0, 8); + final long[] minimumRetainingSequenceNumbers = new long[length]; + for (int i = 0; i < length; i++) { + minimumRetainingSequenceNumbers[i] = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE); + currentTimeMillis.set(TimeUnit.NANOSECONDS.toMillis(randomNonNegativeLong())); + indexShard.addOrUpdateRetentionLease(Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i); + } + + currentTimeMillis.set(TimeUnit.NANOSECONDS.toMillis(Long.MAX_VALUE)); + + // force a commit + indexShard.flush(new FlushRequest().force(true)); + + // the committed retention leases should equal our current retention leases + final SegmentInfos segmentCommitInfos = indexShard.store().readLastCommittedSegmentsInfo(); + assertTrue(segmentCommitInfos.getUserData().containsKey(Engine.RETENTION_LEASES)); + final Collection retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get(); + assertThat(IndexShard.getRetentionLeases(segmentCommitInfos), contains(retentionLeases.toArray(new RetentionLease[0]))); + + // when we recover, we should recover the retention leases + final IndexShard recoveredShard = reinitShard( + indexShard, + ShardRoutingHelper.initWithSameId(indexShard.routingEntry(), RecoverySource.ExistingStoreRecoverySource.INSTANCE)); + try { + recoverShardFromStore(recoveredShard); + assertThat( + recoveredShard.getEngine().config().retentionLeasesSupplier().get(), + contains(retentionLeases.toArray(new RetentionLease[0]))); + } finally { + closeShards(recoveredShard); + } + } finally { + closeShards(indexShard); + } + } + private void assertRetentionLeases( final IndexShard indexShard, final int size, diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 58059cd30e382..8b463f33b9081 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -578,44 +578,112 @@ public EngineConfig config(IndexSettings indexSettings, Store store, Path transl public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy, ReferenceManager.RefreshListener refreshListener, Sort indexSort, LongSupplier globalCheckpointSupplier) { - return config(indexSettings, store, translogPath, mergePolicy, refreshListener, null, indexSort, globalCheckpointSupplier, - new NoneCircuitBreakerService()); + return config( + indexSettings, + store, + translogPath, + mergePolicy, + refreshListener, + indexSort, + globalCheckpointSupplier, + globalCheckpointSupplier == null ? null : Collections::emptyList); + } + + public EngineConfig config( + final IndexSettings indexSettings, + final Store store, + final Path translogPath, + final MergePolicy mergePolicy, + final ReferenceManager.RefreshListener refreshListener, + final Sort indexSort, + final LongSupplier globalCheckpointSupplier, + final Supplier> retentionLeasesSupplier) { + return config( + indexSettings, + store, + translogPath, + mergePolicy, + refreshListener, + null, + indexSort, + globalCheckpointSupplier, + retentionLeasesSupplier, + new NoneCircuitBreakerService()); } public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy, ReferenceManager.RefreshListener externalRefreshListener, ReferenceManager.RefreshListener internalRefreshListener, - Sort indexSort, @Nullable final LongSupplier maybeGlobalCheckpointSupplier, + Sort indexSort, @Nullable LongSupplier maybeGlobalCheckpointSupplier, CircuitBreakerService breakerService) { - IndexWriterConfig iwc = newIndexWriterConfig(); - TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, BigArrays.NON_RECYCLING_INSTANCE); - Engine.EventListener listener = new Engine.EventListener() { - @Override - public void onFailedEngine(String reason, @Nullable Exception e) { - // we don't need to notify anybody in this test - } - }; + return config( + indexSettings, + store, + translogPath, + mergePolicy, + externalRefreshListener, + internalRefreshListener, + indexSort, + maybeGlobalCheckpointSupplier, + maybeGlobalCheckpointSupplier == null ? null : Collections::emptyList, + breakerService); + } + + public EngineConfig config( + final IndexSettings indexSettings, + final Store store, + final Path translogPath, + final MergePolicy mergePolicy, + final ReferenceManager.RefreshListener externalRefreshListener, + final ReferenceManager.RefreshListener internalRefreshListener, + final Sort indexSort, + final @Nullable LongSupplier maybeGlobalCheckpointSupplier, + final @Nullable Supplier> maybeRetentionLeasesSupplier, + final CircuitBreakerService breakerService) { + final IndexWriterConfig iwc = newIndexWriterConfig(); + final TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, BigArrays.NON_RECYCLING_INSTANCE); + final Engine.EventListener listener = new Engine.EventListener() {}; // we don't need to notify anybody in this test final List extRefreshListenerList = - externalRefreshListener == null ? emptyList() : Collections.singletonList(externalRefreshListener); + externalRefreshListener == null ? emptyList() : Collections.singletonList(externalRefreshListener); final List intRefreshListenerList = - internalRefreshListener == null ? emptyList() : Collections.singletonList(internalRefreshListener); + internalRefreshListener == null ? emptyList() : Collections.singletonList(internalRefreshListener); final LongSupplier globalCheckpointSupplier; final Supplier> retentionLeasesSupplier; if (maybeGlobalCheckpointSupplier == null) { + assert maybeRetentionLeasesSupplier == null; final ReplicationTracker replicationTracker = new ReplicationTracker( shardId, allocationId.getId(), indexSettings, SequenceNumbers.NO_OPS_PERFORMED, update -> {}, () -> 0L); globalCheckpointSupplier = replicationTracker; retentionLeasesSupplier = replicationTracker::getRetentionLeases; } else { + assert maybeRetentionLeasesSupplier != null; globalCheckpointSupplier = maybeGlobalCheckpointSupplier; - retentionLeasesSupplier = Collections::emptyList; + retentionLeasesSupplier = maybeRetentionLeasesSupplier; } - EngineConfig config = new EngineConfig(shardId, allocationId.getId(), threadPool, indexSettings, null, store, - mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), listener, - IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, - TimeValue.timeValueMinutes(5), extRefreshListenerList, intRefreshListenerList, indexSort, - breakerService, globalCheckpointSupplier, retentionLeasesSupplier, primaryTerm::get, tombstoneDocSupplier()); - return config; + return new EngineConfig( + shardId, + allocationId.getId(), + threadPool, + indexSettings, + null, + store, + mergePolicy, + iwc.getAnalyzer(), + iwc.getSimilarity(), + new CodecService(null, logger), + listener, + IndexSearcher.getDefaultQueryCache(), + IndexSearcher.getDefaultQueryCachingPolicy(), + translogConfig, + TimeValue.timeValueMinutes(5), + extRefreshListenerList, + intRefreshListenerList, + indexSort, + breakerService, + globalCheckpointSupplier, + retentionLeasesSupplier, + primaryTerm::get, + tombstoneDocSupplier()); } protected static final BytesReference B_1 = new BytesArray(new byte[]{1}); From a4339ec7e9b6e6cc2ca4511048cae5915ad17b58 Mon Sep 17 00:00:00 2001 From: Costin Leau Date: Sun, 13 Jan 2019 01:40:22 +0200 Subject: [PATCH 20/28] SQL: Use declared source for error messages (#37161) Improve error messages by returning the original SQL statement declaration instead of trying to reproduce it as the casing and whitespaces are not preserved accurately leading to small differences. Close #37161 --- .../sql/qa/multi_node/RestSqlMultinodeIT.java | 2 +- .../xpack/sql/qa/security/UserFunctionIT.java | 12 +- .../sql/qa/single_node/CliExplainIT.java | 12 +- .../xpack/sql/qa/jdbc/DebugCsvSpec.java | 2 +- .../sql/qa/src/main/resources/agg.csv-spec | 6 +- .../sql/qa/src/main/resources/docs.csv-spec | 104 ++++++------- .../qa/src/main/resources/functions.csv-spec | 38 ++--- .../sql/qa/src/main/resources/math.csv-spec | 86 +++++------ .../sql/qa/src/main/resources/select.csv-spec | 142 +++++++++--------- .../xpack/sql/analysis/analyzer/Analyzer.java | 9 +- .../xpack/sql/expression/Literal.java | 2 +- .../sql/expression/function/Function.java | 16 +- .../function/UnresolvedFunction.java | 2 +- .../expression/function/aggregate/Max.java | 6 +- .../expression/function/aggregate/Min.java | 4 +- .../function/aggregate/NumericAggregate.java | 2 +- .../function/aggregate/Percentile.java | 4 +- .../function/aggregate/PercentileRank.java | 4 +- .../sql/expression/function/scalar/Cast.java | 12 -- .../scalar/ConfigurationFunction.java | 6 - .../scalar/datetime/BaseDateTimeFunction.java | 16 +- .../scalar/math/BinaryNumericFunction.java | 4 +- .../expression/function/scalar/math/E.java | 8 +- .../expression/function/scalar/math/Pi.java | 8 +- .../scalar/string/BinaryStringFunction.java | 4 +- .../string/BinaryStringNumericFunction.java | 2 +- .../string/BinaryStringStringFunction.java | 2 +- .../function/scalar/string/Concat.java | 6 +- .../function/scalar/string/Insert.java | 8 +- .../function/scalar/string/Locate.java | 6 +- .../function/scalar/string/Replace.java | 8 +- .../function/scalar/string/Substring.java | 8 +- .../expression/predicate/BinaryPredicate.java | 28 ---- .../xpack/sql/expression/predicate/Range.java | 14 +- .../predicate/logical/BinaryLogic.java | 2 +- .../sql/expression/predicate/logical/Not.java | 4 +- .../predicate/operator/arithmetic/Neg.java | 10 +- .../operator/comparison/BinaryComparison.java | 13 +- .../predicate/operator/comparison/In.java | 8 - .../xpack/sql/parser/AbstractBuilder.java | 10 ++ .../xpack/sql/parser/ExpressionBuilder.java | 65 +++++--- .../analyzer/VerifierErrorMessagesTests.java | 44 +++--- .../function/NamedExpressionTests.java | 41 +++-- .../scalar/datetime/DayOfYearTests.java | 3 +- .../xpack/sql/optimizer/OptimizerTests.java | 2 +- .../sql/parser/EscapedFunctionsTests.java | 36 ++--- .../xpack/sql/parser/ExpressionTests.java | 52 ++++--- .../xpack/sql/parser/SqlParserTests.java | 17 ++- .../xpack/sql/planner/QueryFolderTests.java | 32 ++-- .../sql/planner/QueryTranslatorTests.java | 18 ++- .../sql/type/DataTypeConversionTests.java | 8 +- 51 files changed, 458 insertions(+), 500 deletions(-) diff --git a/x-pack/plugin/sql/qa/multi-node/src/test/java/org/elasticsearch/xpack/sql/qa/multi_node/RestSqlMultinodeIT.java b/x-pack/plugin/sql/qa/multi-node/src/test/java/org/elasticsearch/xpack/sql/qa/multi_node/RestSqlMultinodeIT.java index 95066306cc2c2..16ed3053ab46f 100644 --- a/x-pack/plugin/sql/qa/multi-node/src/test/java/org/elasticsearch/xpack/sql/qa/multi_node/RestSqlMultinodeIT.java +++ b/x-pack/plugin/sql/qa/multi-node/src/test/java/org/elasticsearch/xpack/sql/qa/multi_node/RestSqlMultinodeIT.java @@ -108,7 +108,7 @@ private Map responseToMap(Response response) throws IOException private void assertCount(RestClient client, int count) throws IOException { Map expected = new HashMap<>(); String mode = randomMode(); - expected.put("columns", singletonList(columnInfo(mode, "COUNT(1)", "long", JDBCType.BIGINT, 20))); + expected.put("columns", singletonList(columnInfo(mode, "COUNT(*)", "long", JDBCType.BIGINT, 20))); expected.put("rows", singletonList(singletonList(count))); Request request = new Request("POST", "/_sql"); diff --git a/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/UserFunctionIT.java b/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/UserFunctionIT.java index 639ffd17e34f3..66492bab8f56f 100644 --- a/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/UserFunctionIT.java +++ b/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/UserFunctionIT.java @@ -60,7 +60,7 @@ protected String getProtocol() { @Before private void setUpUsers() throws IOException { int usersCount = name.getMethodName().startsWith("testSingle") ? 1 : randomIntBetween(5, 15); - users = new ArrayList(usersCount); + users = new ArrayList<>(usersCount); users.addAll(randomUnique(() -> randomAlphaOfLengthBetween(1, 15), usersCount)); for (String user : users) { createUser(user, MINIMAL_ACCESS_ROLE); @@ -80,7 +80,7 @@ public void testSingleRandomUser() throws IOException { Map expected = new HashMap<>(); expected.put("columns", Arrays.asList( - columnInfo(mode, "USER", "keyword", JDBCType.VARCHAR, 0))); + columnInfo(mode, "USER()", "keyword", JDBCType.VARCHAR, 0))); expected.put("rows", Arrays.asList(Arrays.asList(randomUserName))); Map actual = runSql(randomUserName, mode, SQL); @@ -96,7 +96,7 @@ public void testSingleRandomUserWithWhereEvaluatingTrue() throws IOException { Map expected = new HashMap<>(); expected.put("columns", Arrays.asList( - columnInfo(mode, "USER", "keyword", JDBCType.VARCHAR, 0))); + columnInfo(mode, "USER()", "keyword", JDBCType.VARCHAR, 0))); expected.put("rows", Arrays.asList(Arrays.asList(randomUserName), Arrays.asList(randomUserName), Arrays.asList(randomUserName))); @@ -114,7 +114,7 @@ public void testSingleRandomUserWithWhereEvaluatingFalse() throws IOException { Map expected = new HashMap<>(); expected.put("columns", Arrays.asList( - columnInfo(mode, "USER", "keyword", JDBCType.VARCHAR, 0))); + columnInfo(mode, "USER()", "keyword", JDBCType.VARCHAR, 0))); expected.put("rows", Collections.>emptyList()); String anotherRandomUserName = randomValueOtherThan(randomUserName, () -> randomAlphaOfLengthBetween(1, 15)); Map actual = runSql(randomUserName, mode, SQL + " FROM test WHERE USER()='" + anotherRandomUserName + "' LIMIT 3"); @@ -129,7 +129,7 @@ public void testMultipleRandomUsersAccess() throws IOException { Map expected = new HashMap<>(); expected.put("columns", Arrays.asList( - columnInfo(mode, "USER", "keyword", JDBCType.VARCHAR, 0))); + columnInfo(mode, "USER()", "keyword", JDBCType.VARCHAR, 0))); expected.put("rows", Arrays.asList(Arrays.asList(randomlyPickedUsername))); Map actual = runSql(randomlyPickedUsername, mode, SQL); @@ -147,7 +147,7 @@ public void testSingleUserSelectFromIndex() throws IOException { Map expected = new HashMap<>(); expected.put("columns", Arrays.asList( - columnInfo(mode, "USER", "keyword", JDBCType.VARCHAR, 0))); + columnInfo(mode, "USER()", "keyword", JDBCType.VARCHAR, 0))); expected.put("rows", Arrays.asList(Arrays.asList(randomUserName), Arrays.asList(randomUserName), Arrays.asList(randomUserName))); diff --git a/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/CliExplainIT.java b/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/CliExplainIT.java index c2027ccbfcc6b..58c5ae5c78e91 100644 --- a/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/CliExplainIT.java +++ b/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/CliExplainIT.java @@ -64,7 +64,7 @@ public void testExplainWithWhere() throws IOException { assertThat(readLine(), startsWith("----------")); assertThat(readLine(), startsWith("With[{}]")); assertThat(readLine(), startsWith("\\_Project[[?*]]")); - assertThat(readLine(), startsWith(" \\_Filter[?i == 2]")); + assertThat(readLine(), startsWith(" \\_Filter[i = 2#")); assertThat(readLine(), startsWith(" \\_UnresolvedRelation[[][index=test],null,Unknown index [test]]")); assertEquals("", readLine()); @@ -72,14 +72,14 @@ public void testExplainWithWhere() throws IOException { containsString("plan")); assertThat(readLine(), startsWith("----------")); assertThat(readLine(), startsWith("Project[[i{f}#")); - assertThat(readLine(), startsWith("\\_Filter[i{f}#")); + assertThat(readLine(), startsWith("\\_Filter[i = 2#")); assertThat(readLine(), startsWith(" \\_EsRelation[test][i{f}#")); assertEquals("", readLine()); assertThat(command("EXPLAIN (PLAN OPTIMIZED) SELECT * FROM test WHERE i = 2"), containsString("plan")); assertThat(readLine(), startsWith("----------")); assertThat(readLine(), startsWith("Project[[i{f}#")); - assertThat(readLine(), startsWith("\\_Filter[i{f}#")); + assertThat(readLine(), startsWith("\\_Filter[i = 2#")); assertThat(readLine(), startsWith(" \\_EsRelation[test][i{f}#")); assertEquals("", readLine()); @@ -124,20 +124,20 @@ public void testExplainWithCount() throws IOException { assertThat(command("EXPLAIN (PLAN PARSED) SELECT COUNT(*) FROM test"), containsString("plan")); assertThat(readLine(), startsWith("----------")); assertThat(readLine(), startsWith("With[{}]")); - assertThat(readLine(), startsWith("\\_Project[[?COUNT(?*)]]")); + assertThat(readLine(), startsWith("\\_Project[[?COUNT(*)]]")); assertThat(readLine(), startsWith(" \\_UnresolvedRelation[[][index=test],null,Unknown index [test]]")); assertEquals("", readLine()); assertThat(command("EXPLAIN " + (randomBoolean() ? "" : "(PLAN ANALYZED) ") + "SELECT COUNT(*) FROM test"), containsString("plan")); assertThat(readLine(), startsWith("----------")); - assertThat(readLine(), startsWith("Aggregate[[],[COUNT(1)#")); + assertThat(readLine(), startsWith("Aggregate[[],[COUNT(*)#")); assertThat(readLine(), startsWith("\\_EsRelation[test][i{f}#")); assertEquals("", readLine()); assertThat(command("EXPLAIN (PLAN OPTIMIZED) SELECT COUNT(*) FROM test"), containsString("plan")); assertThat(readLine(), startsWith("----------")); - assertThat(readLine(), startsWith("Aggregate[[],[COUNT(1)#")); + assertThat(readLine(), startsWith("Aggregate[[],[COUNT(*)#")); assertThat(readLine(), startsWith("\\_EsRelation[test][i{f}#")); assertEquals("", readLine()); diff --git a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DebugCsvSpec.java b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DebugCsvSpec.java index 8f07fd879b73d..d5a633e5ea388 100644 --- a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DebugCsvSpec.java +++ b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DebugCsvSpec.java @@ -20,7 +20,7 @@ import static org.elasticsearch.xpack.sql.qa.jdbc.CsvTestUtils.executeCsvQuery; import static org.elasticsearch.xpack.sql.qa.jdbc.CsvTestUtils.specParser; -@TestLogging(JdbcTestUtils.SQL_TRACE) +@TestLogging("org.elasticsearch.xpack.sql:TRACE") public abstract class DebugCsvSpec extends SpecBaseIntegrationTestCase { private final CsvTestCase testCase; diff --git a/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec index bdb94321b76d5..3717afefab3bd 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec @@ -169,7 +169,7 @@ FROM "test_emp" GROUP BY "TEMP" ORDER BY "TEMP" LIMIT 20; aggAndOrderByCastedValue SELECT CHAR_LENGTH(SPACE(CAST(languages AS SMALLINT))), COUNT(*) FROM test_emp GROUP BY 1 ORDER BY 1 DESC; -CHAR_LENGTH(SPACE(CAST(languages AS SMALLINT))):i| COUNT(1):l +CHAR_LENGTH(SPACE(CAST(languages AS SMALLINT))):i| COUNT(*):l -------------------------------------------------+--------------- 5 |21 4 |18 @@ -180,9 +180,9 @@ null |10 ; aggAndOrderByCastedFunctionValue -SELECT ROUND(SQRT(CAST(EXP(languages) AS SMALLINT)), 2), COUNT(*) FROM test_emp GROUP BY 1 ORDER BY 1 DESC; +SELECT ROUND(SQRT(CAST(EXP(languages) AS SMALLINT)),2), COUNT(*) FROM test_emp GROUP BY 1 ORDER BY 1 DESC; -ROUND(SQRT(CAST(EXP(languages) AS SMALLINT)),2):d| COUNT(1):l +ROUND(SQRT(CAST(EXP(languages) AS SMALLINT)),2):d| COUNT(*):l -------------------------------------------------+--------------- 12.17 |21 7.42 |18 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 2903292b1adff..e6bde4795a85f 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 @@ -1124,8 +1124,8 @@ stringAscii // tag::stringAscii SELECT ASCII('Elastic'); -ASCII(Elastic) ---------------- +ASCII('Elastic') +---------------- 69 // end::stringAscii ; @@ -1134,8 +1134,8 @@ stringBitLength // tag::stringBitLength SELECT BIT_LENGTH('Elastic'); -BIT_LENGTH(Elastic) -------------------- +BIT_LENGTH('Elastic') +--------------------- 56 // end::stringBitLength ; @@ -1154,8 +1154,8 @@ stringCharLength // tag::stringCharLength SELECT CHAR_LENGTH('Elastic'); -CHAR_LENGTH(Elastic) --------------------- +CHAR_LENGTH('Elastic') +---------------------- 7 // end::stringCharLength ; @@ -1164,8 +1164,8 @@ stringConcat // tag::stringConcat SELECT CONCAT('Elasticsearch', ' SQL'); -CONCAT(Elasticsearch, SQL) --------------------------- +CONCAT('Elasticsearch', ' SQL') +------------------------------- Elasticsearch SQL // end::stringConcat ; @@ -1174,8 +1174,8 @@ stringInsert // tag::stringInsert SELECT INSERT('Elastic ', 8, 1, 'search'); -INSERT(Elastic ,8,1,search) ---------------------------- +INSERT('Elastic ', 8, 1, 'search') +---------------------------------- Elasticsearch // end::stringInsert ; @@ -1184,8 +1184,8 @@ stringLCase // tag::stringLCase SELECT LCASE('Elastic'); -LCASE(Elastic) ---------------- +LCASE('Elastic') +---------------- elastic // end::stringLCase ; @@ -1194,8 +1194,8 @@ stringLeft // tag::stringLeft SELECT LEFT('Elastic',3); -LEFT(Elastic,3) ---------------- +LEFT('Elastic',3) +----------------- Ela // end::stringLeft ; @@ -1204,8 +1204,8 @@ stringLength // tag::stringLength SELECT LENGTH('Elastic '); -LENGTH(Elastic ) ------------------- +LENGTH('Elastic ') +-------------------- 7 // end::stringLength ; @@ -1214,8 +1214,8 @@ stringLocateWoStart // tag::stringLocateWoStart SELECT LOCATE('a', 'Elasticsearch'); -LOCATE(a,Elasticsearch) ------------------------ +LOCATE('a', 'Elasticsearch') +---------------------------- 3 // end::stringLocateWoStart ; @@ -1224,8 +1224,8 @@ stringLocateWithStart // tag::stringLocateWithStart SELECT LOCATE('a', 'Elasticsearch', 5); -LOCATE(a,Elasticsearch,5) -------------------------- +LOCATE('a', 'Elasticsearch', 5) +------------------------------- 10 // end::stringLocateWithStart ; @@ -1234,8 +1234,8 @@ stringLTrim // tag::stringLTrim SELECT LTRIM(' Elastic'); -LTRIM( Elastic) ------------------ +LTRIM(' Elastic') +------------------- Elastic // end::stringLTrim ; @@ -1244,8 +1244,8 @@ stringOctetLength // tag::stringOctetLength SELECT OCTET_LENGTH('Elastic'); -OCTET_LENGTH(Elastic) -------------------- +OCTET_LENGTH('Elastic') +----------------------- 7 // end::stringOctetLength ; @@ -1254,8 +1254,8 @@ stringPosition // tag::stringPosition SELECT POSITION('Elastic', 'Elasticsearch'); -POSITION(Elastic,Elasticsearch) -------------------------------- +POSITION('Elastic', 'Elasticsearch') +------------------------------------ 1 // end::stringPosition ; @@ -1264,8 +1264,8 @@ stringRepeat // tag::stringRepeat SELECT REPEAT('La', 3); - REPEAT(La,3) ---------------- + REPEAT('La', 3) +---------------- LaLaLa // end::stringRepeat ; @@ -1274,8 +1274,8 @@ stringReplace // tag::stringReplace SELECT REPLACE('Elastic','El','Fant'); - REPLACE(Elastic,El,Fant) ------------------------------ +REPLACE('Elastic','El','Fant') +------------------------------ Fantastic // end::stringReplace ; @@ -1284,8 +1284,8 @@ stringRight // tag::stringRight SELECT RIGHT('Elastic',3); -RIGHT(Elastic,3) ----------------- +RIGHT('Elastic',3) +------------------ tic // end::stringRight ; @@ -1294,8 +1294,8 @@ stringRTrim // tag::stringRTrim SELECT RTRIM('Elastic '); -RTRIM(Elastic ) ------------------ +RTRIM('Elastic ') +------------------- Elastic // end::stringRTrim ; @@ -1316,8 +1316,8 @@ stringSubString // tag::stringSubString SELECT SUBSTRING('Elasticsearch', 0, 7); -SUBSTRING(Elasticsearch,0,7) ----------------------------- +SUBSTRING('Elasticsearch', 0, 7) +-------------------------------- Elastic // end::stringSubString ; @@ -1326,8 +1326,8 @@ stringUCase // tag::stringUCase SELECT UCASE('Elastic'); -UCASE(Elastic) ---------------- +UCASE('Elastic') +---------------- ELASTIC // end::stringUCase ; @@ -1417,7 +1417,7 @@ mathInlineAcos // tag::mathInlineAcos SELECT ACOS(COS(PI())), PI(); - ACOS(COS(PI)) | PI + ACOS(COS(PI())) | PI() -----------------+----------------- 3.141592653589793|3.141592653589793 // end::mathInlineAcos @@ -1467,8 +1467,8 @@ mathInlineCeiling // tag::mathInlineCeiling SELECT CEIL(125.01), CEILING(-125.99); - CEIL(125.01) | CEIL(-125.99) ----------------+--------------- + CEIL(125.01) |CEILING(-125.99) +---------------+---------------- 126 |-125 // end::mathInlineCeiling ; @@ -1507,7 +1507,7 @@ mathInlineDegrees // tag::mathInlineDegrees SELECT DEGREES(PI() * 2), DEGREES(PI()); -DEGREES((PI) * 2)| DEGREES(PI) +DEGREES(PI() * 2)| DEGREES(PI()) -----------------+--------------- 360.0 |180.0 // end::mathInlineDegrees @@ -1517,7 +1517,7 @@ mathEulersNumber // tag::mathEulersNumber SELECT E(), CEIL(E()); - E | CEIL(E) + E() | CEIL(E()) -----------------+--------------- 2.718281828459045|3 // end::mathEulersNumber @@ -1527,7 +1527,7 @@ mathExpInline // tag::mathExpInline SELECT EXP(1), E(), EXP(2), E() * E(); - EXP(1) | E | EXP(2) | (E) * (E) + EXP(1) | E() | EXP(2) | E() * E() -----------------+-----------------+----------------+------------------ 2.718281828459045|2.718281828459045|7.38905609893065|7.3890560989306495 // end::mathExpInline @@ -1537,7 +1537,7 @@ mathExpm1Inline // tag::mathExpm1Inline SELECT E(), EXP(2), EXPM1(2); - E | EXP(2) | EXPM1(2) + E() | EXP(2) | EXPM1(2) -----------------+----------------+---------------- 2.718281828459045|7.38905609893065|6.38905609893065 // end::mathExpm1Inline @@ -1567,8 +1567,8 @@ mathInlineLog10 // tag::mathInlineLog10 SELECT LOG10(5), LOG(5)/LOG(10); - LOG10(5) |(LOG(5)) / (LOG(10)) -------------------+-------------------- + LOG10(5) | LOG(5)/LOG(10) +------------------+----------------------- 0.6989700043360189|0.6989700043360187 // end::mathInlineLog10 ; @@ -1577,7 +1577,7 @@ mathPINumber // tag::mathPINumber SELECT PI(); - PI + PI() ----------------- 3.141592653589793 // end::mathPINumber @@ -1587,7 +1587,7 @@ mathInlinePowerPositive // tag::mathInlinePowerPositive SELECT POWER(3, 2), POWER(3, 3); - POWER(3,2) | POWER(3,3) + POWER(3, 2) | POWER(3, 3) ---------------+--------------- 9.0 |27.0 // end::mathInlinePowerPositive @@ -1597,7 +1597,7 @@ mathInlinePowerNegative // tag::mathInlinePowerNegative SELECT POWER(5, -1), POWER(5, -2); - POWER(5,-1) | POWER(5,-2) + POWER(5, -1) | POWER(5, -2) ---------------+--------------- 0.2 |0.04 // end::mathInlinePowerNegative @@ -1607,7 +1607,7 @@ mathInlineRadians // tag::mathInlineRadians SELECT RADIANS(90), PI()/2; - RADIANS(90) | (PI) / 2 + RADIANS(90) | PI()/2 ------------------+------------------ 1.5707963267948966|1.5707963267948966 // end::mathInlineRadians @@ -1677,7 +1677,7 @@ mathInlineSqrt // tag::mathInlineSqrt SELECT SQRT(EXP(2)), E(), SQRT(25); - SQRT(EXP(2)) | E | SQRT(25) + SQRT(EXP(2)) | E() | SQRT(25) -----------------+-----------------+--------------- 2.718281828459045|2.718281828459045|5.0 // end::mathInlineSqrt diff --git a/x-pack/plugin/sql/qa/src/main/resources/functions.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/functions.csv-spec index 6fec225df0c50..a89013cb32bca 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/functions.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/functions.csv-spec @@ -263,18 +263,18 @@ SELECT POSITION('x',LCASE("first_name")) pos, "first_name" FROM "test_emp" WHERE ; selectPositionWithLcaseAndConditionWithGroupByAndOrderBy -SELECT POSITION('m',LCASE("first_name")), COUNT(*) pos FROM "test_emp" +SELECT POSITION('m',LCASE("first_name")) AS pos, COUNT(*) c FROM "test_emp" WHERE POSITION('m',LCASE("first_name")) != 0 GROUP BY POSITION('m',LCASE("first_name")) ORDER BY POSITION('m',LCASE("first_name")) DESC; -POSITION(m,LCASE(first_name)):i| pos:l --------------------------------+--------------- -9 |1 -7 |1 -4 |2 -3 |6 -2 |1 -1 |9 +pos:i| c:l +-----+----- +9 |1 +7 |1 +4 |2 +3 |6 +2 |1 +1 |9 ; selectInsertWithPositionAndCondition @@ -299,8 +299,8 @@ ReMzi |3 selectLocateAndInsertWithLocateWithConditionAndThreeParameters SELECT LOCATE('a',"first_name",7) pos, INSERT("first_name",LOCATE('a',"first_name",7),1,'AAA') FROM "test_emp" WHERE LOCATE('a',"first_name",7) > 0; - pos:i |INSERT(first_name,LOCATE(a,first_name,7),1,AAA):s ----------------+----------------------------------------------- + pos:i |INSERT("first_name",LOCATE('a',"first_name",7),1,'AAA'):s +---------------+--------------------------------------------------------- 8 |ChirstiAAAn 7 |DuangkAAAew 8 |PrasadrAAAm @@ -317,8 +317,8 @@ SELECT LOCATE('a',"first_name",7) pos, INSERT("first_name",LOCATE('a',"first_nam selectLocateAndInsertWithLocateWithConditionAndTwoParameters SELECT LOCATE('a',"first_name") pos, INSERT("first_name",LOCATE('a',"first_name"),1,'AAA') FROM "test_emp" WHERE LOCATE('a',"first_name") > 0 ORDER BY "first_name" LIMIT 10; - pos:i |INSERT(first_name,LOCATE(a,first_name),1,AAA):s ----------------+--------------------------------------------- + pos:i |INSERT("first_name",LOCATE('a',"first_name"),1,'AAA'):s +---------------+------------------------------------------------------- 5 |AlejAAAndro 3 |AmAAAbile 7 |ArumugAAAm @@ -455,23 +455,23 @@ AlejandRo |1 checkColumnNameWithNestedArithmeticFunctionCallsOnTableColumn -SELECT CHAR(emp_no % 10000) FROM "test_emp" WHERE emp_no > 10064 ORDER BY emp_no LIMIT 1; +SELECT CHAR(emp_no % 10000) AS c FROM "test_emp" WHERE emp_no > 10064 ORDER BY emp_no LIMIT 1; -CHAR((emp_no) % 10000):s +c:s A ; checkColumnNameWithComplexNestedArithmeticFunctionCallsOnTableColumn1 -SELECT CHAR(emp_no % (7000 + 3000)) FROM "test_emp" WHERE emp_no > 10065 ORDER BY emp_no LIMIT 1; +SELECT CHAR(emp_no % (7000 + 3000)) AS c FROM "test_emp" WHERE emp_no > 10065 ORDER BY emp_no LIMIT 1; -CHAR((emp_no) % (7000 + 3000)):s +c:s B ; checkColumnNameWithComplexNestedArithmeticFunctionCallsOnTableColumn2 -SELECT CHAR((emp_no % (emp_no - 1 + 1)) + 67) FROM "test_emp" WHERE emp_no > 10066 ORDER BY emp_no LIMIT 1; +SELECT CHAR((emp_no % (emp_no - 1 + 1)) + 67) AS c FROM "test_emp" WHERE emp_no > 10066 ORDER BY emp_no LIMIT 1; -CHAR(((emp_no) % (((emp_no) - 1) + 1)) + 67):s +c:s C ; diff --git a/x-pack/plugin/sql/qa/src/main/resources/math.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/math.csv-spec index 2df93b3795443..19eda7b92f9c0 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/math.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/math.csv-spec @@ -2,27 +2,27 @@ truncateWithAsciiHavingAndOrderBy SELECT TRUNCATE(ASCII(LEFT(first_name, 1)), 1), COUNT(*) count FROM test_emp GROUP BY ASCII(LEFT(first_name, 1)) HAVING COUNT(*) > 5 ORDER BY TRUNCATE(ASCII(LEFT(first_name, 1)), 1) DESC; -TRUNCATE(ASCII(LEFT(first_name,1)),1):i| count:l ----------------------------------------+--------------- -null |10 -66 |7 -72 |6 -75 |7 -77 |9 -83 |11 +TRUNCATE(ASCII(LEFT(first_name, 1)), 1):i| count:l +-----------------------------------------+--------------- +null |10 +66 |7 +72 |6 +75 |7 +77 |9 +83 |11 ; truncateWithNoSecondParameterWithAsciiHavingAndOrderBy SELECT TRUNCATE(ASCII(LEFT(first_name, 1))), COUNT(*) count FROM test_emp GROUP BY ASCII(LEFT(first_name, 1)) HAVING COUNT(*) > 5 ORDER BY TRUNCATE(ASCII(LEFT(first_name, 1))) DESC; -TRUNCATE(ASCII(LEFT(first_name,1)),0):i| count:l ----------------------------------------+--------------- -null |10 -66 |7 -72 |6 -75 |7 -77 |9 -83 |11 +TRUNCATE(ASCII(LEFT(first_name, 1))):i| count:l +--------------------------------------+--------------- +null |10 +66 |7 +72 |6 +75 |7 +77 |9 +83 |11 ; roundWithGroupByAndOrderBy @@ -97,21 +97,21 @@ SELECT MIN(salary) mi, MAX(salary) ma, COUNT(*) c, TRUNCATE(AVG(salary)) tr FROM minMaxTruncateAndRoundOfAverageWithHavingRoundAndTruncate SELECT MIN(salary) mi, MAX(salary) ma, YEAR(hire_date) year, ROUND(AVG(languages), 1), TRUNCATE(AVG(languages), 1), COUNT(*) FROM test_emp GROUP BY YEAR(hire_date) HAVING ROUND(AVG(languages), 1) > 2.5 AND TRUNCATE(AVG(languages), 1) <= 3.0 ORDER BY YEAR(hire_date); - mi:i | ma:i | year:i |ROUND(AVG(languages),1):d|TRUNCATE(AVG(languages),1):d| COUNT(1):l ----------------+---------------+---------------+-------------------------+----------------------------+--------------- -25324 |70011 |1986 |3.0 |3.0 |15 -25945 |73578 |1987 |2.9 |2.8 |9 -25976 |74970 |1988 |3.0 |3.0 |13 -31120 |71165 |1989 |3.1 |3.0 |12 -30404 |58715 |1992 |3.0 |3.0 |3 -35742 |67492 |1993 |2.8 |2.7 |4 -45656 |45656 |1995 |3.0 |3.0 |1 + mi:i | ma:i | year:i |ROUND(AVG(languages), 1):d|TRUNCATE(AVG(languages), 1):d| COUNT(*):l +---------------+---------------+---------------+--------------------------+-----------------------------+--------------- +25324 |70011 |1986 |3.0 |3.0 |15 +25945 |73578 |1987 |2.9 |2.8 |9 +25976 |74970 |1988 |3.0 |3.0 |13 +31120 |71165 |1989 |3.1 |3.0 |12 +30404 |58715 |1992 |3.0 |3.0 |3 +35742 |67492 |1993 |2.8 |2.7 |4 +45656 |45656 |1995 |3.0 |3.0 |1 ; minMaxRoundWithHavingRound -SELECT MIN(salary) mi, MAX(salary) ma, YEAR(hire_date) year, ROUND(AVG(languages), 1), COUNT(*) FROM test_emp GROUP BY YEAR(hire_date) HAVING ROUND(AVG(languages), 1) > 2.5 ORDER BY YEAR(hire_date); +SELECT MIN(salary) mi, MAX(salary) ma, YEAR(hire_date) year, ROUND(AVG(languages),1), COUNT(*) FROM test_emp GROUP BY YEAR(hire_date) HAVING ROUND(AVG(languages), 1) > 2.5 ORDER BY YEAR(hire_date); - mi:i | ma:i | year:i |ROUND(AVG(languages),1):d| COUNT(1):l + mi:i | ma:i | year:i |ROUND(AVG(languages),1):d| COUNT(*):l ---------------+---------------+---------------+-------------------------+--------------- 26436 |74999 |1984 |3.1 |11 31897 |61805 |1985 |3.5 |11 @@ -127,9 +127,9 @@ SELECT MIN(salary) mi, MAX(salary) ma, YEAR(hire_date) year, ROUND(AVG(languages ; groupByAndOrderByTruncateWithPositiveParameter -SELECT TRUNCATE(AVG(salary), 2), AVG(salary), COUNT(*) FROM test_emp GROUP BY TRUNCATE(salary, 2) ORDER BY TRUNCATE(salary, 2) DESC LIMIT 10; +SELECT TRUNCATE(AVG(salary),2), AVG(salary), COUNT(*) FROM test_emp GROUP BY TRUNCATE(salary, 2) ORDER BY TRUNCATE(salary, 2) DESC LIMIT 10; -TRUNCATE(AVG(salary),2):d| AVG(salary):d | COUNT(1):l +TRUNCATE(AVG(salary),2):d| AVG(salary):d | COUNT(*):l -------------------------+---------------+--------------- 74999.0 |74999.0 |1 74970.0 |74970.0 |1 @@ -144,9 +144,9 @@ TRUNCATE(AVG(salary),2):d| AVG(salary):d | COUNT(1):l ; groupByAndOrderByRoundWithPositiveParameter -SELECT ROUND(AVG(salary), 2), AVG(salary), COUNT(*) FROM test_emp GROUP BY ROUND(salary, 2) ORDER BY ROUND(salary, 2) DESC LIMIT 10; +SELECT ROUND(AVG(salary),2), AVG(salary), COUNT(*) FROM test_emp GROUP BY ROUND(salary, 2) ORDER BY ROUND(salary, 2) DESC LIMIT 10; -ROUND(AVG(salary),2):d| AVG(salary):d | COUNT(1):l +ROUND(AVG(salary),2):d| AVG(salary):d | COUNT(*):l ----------------------+---------------+--------------- 74999.0 |74999.0 |1 74970.0 |74970.0 |1 @@ -163,16 +163,16 @@ ROUND(AVG(salary),2):d| AVG(salary):d | COUNT(1):l groupByAndOrderByRoundWithNoSecondParameter SELECT ROUND(AVG(salary)), ROUND(salary) rounded, AVG(salary), COUNT(*) FROM test_emp GROUP BY rounded ORDER BY rounded DESC LIMIT 10; -ROUND(AVG(salary),0):d| rounded:i | AVG(salary):d | COUNT(1):l -----------------------+---------------+---------------+--------------- -74999.0 |74999 |74999.0 |1 -74970.0 |74970 |74970.0 |1 -74572.0 |74572 |74572.0 |1 -73851.0 |73851 |73851.0 |1 -73717.0 |73717 |73717.0 |1 -73578.0 |73578 |73578.0 |1 -71165.0 |71165 |71165.0 |1 -70011.0 |70011 |70011.0 |1 -69904.0 |69904 |69904.0 |1 -68547.0 |68547 |68547.0 |1 +ROUND(AVG(salary)):d| rounded:i | AVG(salary):d | COUNT(*):l +--------------------+---------------+---------------+--------------- +74999.0 |74999 |74999.0 |1 +74970.0 |74970 |74970.0 |1 +74572.0 |74572 |74572.0 |1 +73851.0 |73851 |73851.0 |1 +73717.0 |73717 |73717.0 |1 +73578.0 |73578 |73578.0 |1 +71165.0 |71165 |71165.0 |1 +70011.0 |70011 |70011.0 |1 +69904.0 |69904 |69904.0 |1 +68547.0 |68547 |68547.0 |1 ; diff --git a/x-pack/plugin/sql/qa/src/main/resources/select.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/select.csv-spec index 2aa7d9bdc7b51..684a451e33173 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/select.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/select.csv-spec @@ -5,39 +5,39 @@ equalsSelectClause SELECT CAST(4 = 4 AS STRING), CAST(NOT 4 = 4 AS STRING), CAST(3 = 4 AS STRING), CAST(NOT 3 = 4 AS STRING), CAST(1 = null AS STRING), CAST(NOT null = 1 AS STRING); - CAST(4 == 4 AS VARCHAR):s | CAST(NOT(4 == 4) AS VARCHAR):s | CAST(3 == 4 AS VARCHAR):s | CAST(NOT(3 == 4) AS VARCHAR):s | CAST(1 == null AS VARCHAR):s | CAST(NOT(null == 1) AS VARCHAR):s -----------------------------+---------------------------------+----------------------------+---------------------------------+-------------------------------+----------------------------------- -true |false |false |true |null |null +CAST(4 = 4 AS STRING):s|CAST(NOT 4 = 4 AS STRING):s|CAST(3 = 4 AS STRING):s|CAST(NOT 3 = 4 AS STRING):s|CAST(1 = null AS STRING):s|CAST(NOT null = 1 AS STRING):s +-----------------------+---------------------------+-----------------------+---------------------------+--------------------------+---------------------------- +true |false |false |true |null |null ; notEqualsSelectClause SELECT CAST(4 != 4 AS STRING), CAST(NOT 4 != 4 AS STRING), CAST(3 != 4 AS STRING), CAST(NOT 3 != 4 AS STRING), CAST(1 != null AS STRING), CAST(NOT 1 != null AS STRING); - CAST(4 != 4 AS VARCHAR):s | CAST(NOT(4 != 4) AS VARCHAR):s | CAST(3 != 4 AS VARCHAR):s | CAST(NOT(3 != 4) AS VARCHAR):s | CAST(1 != null AS VARCHAR):s | CAST(NOT(1 != null) AS VARCHAR):s -----------------------------+---------------------------------+----------------------------+---------------------------------+-------------------------------+----------------------------------- -false |true |true |false |null |null +CAST(4 != 4 AS STRING):s|CAST(NOT 4 != 4 AS STRING):s|CAST(3 != 4 AS STRING):s|CAST(NOT 3 != 4 AS STRING):s|CAST(1 != null AS STRING):s|CAST(NOT 1 != null AS STRING):s +------------------------+----------------------------+------------------------+----------------------------+---------------------------+------------------------------- +false |true |true |false |null |null ; equalSelectClauseWithTableColumns SELECT CAST(languages = 2 AS STRING), CAST(NOT languages = 2 AS STRING), CAST(languages = null AS STRING), CAST(NOT languages = null AS STRING) FROM "test_emp" WHERE emp_no IN(10018, 10019, 10020) ORDER BY emp_no; - CAST((languages) == 2 AS VARCHAR):s | CAST(NOT((languages) == 2) AS VARCHAR):s | CAST((languages) == null AS VARCHAR):s | CAST(NOT((languages) == null) AS VARCHAR):s ---------------------------------------+-------------------------------------------+-----------------------------------------+--------------------------------------------- -true |false |null |null -false |true |null |null -null |null |null |null +CAST(languages = 2 AS STRING):s|CAST(NOT languages = 2 AS STRING):s|CAST(languages = null AS STRING):s|CAST(NOT languages = null AS STRING):s +-------------------------------+-----------------------------------+----------------------------------+-------------------------------------- +true |false |null |null +false |true |null |null +null |null |null |null ; notEqualsAndNotEqualsSelectClauseWithTableColumns SELECT CAST(languages != 2 AS STRING), CAST(NOT languages != 2 AS STRING), CAST(languages != null AS STRING), CAST(NOT languages != null AS STRING) FROM "test_emp" WHERE emp_no IN(10018, 10019, 10020) ORDER BY emp_no; - CAST((languages) != 2 AS VARCHAR):s | CAST(NOT((languages) != 2) AS VARCHAR):s | CAST((languages) != null AS VARCHAR):s | CAST(NOT((languages) != null) AS VARCHAR):s ---------------------------------------+-------------------------------------------+-----------------------------------------+--------------------------------------------- -false |true |null |null -true |false |null |null -null |null |null |null +CAST(languages != 2 AS STRING):s|CAST(NOT languages != 2 AS STRING):s|CAST(languages != null AS STRING):s|CAST(NOT languages != null AS STRING):s +--------------------------------+------------------------------------+-----------------------------------+--------------------------------------- +false |true |null |null +true |false |null |null +null |null |null |null ; @@ -48,27 +48,27 @@ null |null selectWithOrAndNullHandling SELECT CAST(true OR null AS STRING), CAST(null OR true AS STRING), CAST(false OR null AS STRING), CAST(null OR false AS STRING), CAST(null OR null AS STRING); - CAST(true OR null AS VARCHAR):s | CAST(null OR true AS VARCHAR):s | CAST(false OR null AS VARCHAR):s | CAST(null OR false AS VARCHAR):s | CAST(null OR null AS VARCHAR):s -----------------------------------+----------------------------------+-----------------------------------+-----------------------------------+--------------------------------- -true |true |null |null |null +CAST(true OR null AS STRING):s|CAST(null OR true AS STRING):s|CAST(false OR null AS STRING):s|CAST(null OR false AS STRING):s|CAST(null OR null AS STRING):s +------------------------------+------------------------------+-------------------------------+-------------------------------+---------------------------- +true |true |null |null |null ; selectWithAndAndNullHandling SELECT CAST(true AND null AS STRING), CAST(null AND true AS STRING), CAST(false AND null AS STRING), CAST(null AND false AS STRING), CAST(null AND null AS STRING); - CAST(true AND null AS VARCHAR):s | CAST(null AND true AS VARCHAR):s | CAST(false AND null AS VARCHAR):s | CAST(null AND false AS VARCHAR):s | CAST(null AND null AS VARCHAR):s ------------------------------------+-----------------------------------+------------------------------------+------------------------------------+---------------------------------- -null |null |false |false |null +CAST(true AND null AS STRING):s|CAST(null AND true AS STRING):s|CAST(false AND null AS STRING):s|CAST(null AND false AS STRING):s|CAST(null AND null AS STRING):s +-------------------------------+-------------------------------+--------------------------------+--------------------------------+----------------------------- +null |null |false |false |null ; selectWithOrAndAndAndNullHandling_WithTableColumns SELECT CAST(languages = 2 OR null AS STRING), CAST(languages = 2 AND null AS STRING) FROM test_emp WHERE emp_no BETWEEN 10018 AND 10020 ORDER BY emp_no; - CAST(((languages) == 2) OR null AS VARCHAR):s | CAST(((languages) == 2) AND null AS VARCHAR):s ------------------------------------------------+------------------------------------------------ -true |null -null |false -null |null +CAST(languages = 2 OR null AS STRING):s|CAST(languages = 2 AND null AS STRING):s +---------------------------------------+---------------------------------------- +true |null +null |false +null |null ; @@ -76,45 +76,45 @@ null |null // SELECT with IN // inWithLiterals -SELECT 1 IN (1, 2, 3), 1 IN (2, 3); +SELECT 1 IN (1,2, 3), 1 IN (2, 3); - 1 IN (1, 2, 3):b | 1 IN (2, 3):b --------------------+------------- -true |false +1 IN (1,2, 3) | 1 IN (2, 3) +---------------+--------------- +true |false ; inWithLiteralsAndFunctions SELECT 1 IN (2 - 1, 2, 3), abs(-1) IN (2, 3, abs(4 - 5)); - 1 IN (2 - 1, 2, 3) | ABS(-1) IN (2, 3, ABS(4 - 5)) ----------------------+------------------------------ -true |true +1 IN (2 - 1, 2, 3)|abs(-1) IN (2, 3, abs(4 - 5)) +------------------+----------------------------- +true |true ; inWithLiteralsAndNegation SELECT 1 NOT IN (1, 1 + 1, 3), 1 NOT IN (2, 3); - NOT(1 IN (1, 1 + 1, 3)) | NOT(1 IN (2, 3)) ---------------------------+----------------- -false |true +1 NOT IN (1, 1 + 1, 3)|1 NOT IN (2, 3) +----------------------+--------------- +false |true ; // Need to CAST as STRING since for boolean types the jdbc CSV translates null -> false inWithNullHandling SELECT CAST(2 IN (1, null, 3) AS STRING), CAST(3 IN (1, null, 3) AS STRING), CAST(null IN (1, null, 3) AS STRING), CAST(null IN (1, 2, 3) AS STRING); - CAST(2 IN (1, null, 3) AS VARCHAR):s | CAST(3 IN (1, null, 3) AS VARCHAR):s | CAST(null IN (1, null, 3) AS VARCHAR):s | CAST(null IN (1, 2, 3) AS VARCHAR):s ----------------------------------------+--------------------------------------+------------------------------------------+-------------------------------------- -null |true |null |null +CAST(2 IN (1, null, 3) AS STRING):s|CAST(3 IN (1, null, 3) AS STRING):s|CAST(null IN (1, null, 3) AS STRING):s|CAST(null IN (1, 2, 3) AS STRING):s +-----------------------------------+-----------------------------------+--------------------------------------+----------------------------------- +null |true |null |null ; inWithNullHandlingAndNegation SELECT CAST(NOT 2 IN (1, null, 3) AS STRING), CAST(3 NOT IN (1, null, 3) AS STRING), CAST(NOT null IN (1, null, 3) AS STRING), CAST(null NOT IN (1, 2, 3) AS STRING); - CAST(NOT(2 IN (1, null, 3)) AS VARCHAR):s | CAST(NOT(3 IN (1, null, 3)) AS VARCHAR):s | CAST(NOT(null IN (1, null, 3)) AS VARCHAR):s | CAST(NOT(null IN (1, 2, 3)) AS VARCHAR):s ---------------------------------------------+--------------------------------------------+-----------------------------------------------+------------------------------------------- -null |false |null |null +CAST(NOT 2 IN (1, null, 3) AS STRING):s|CAST(3 NOT IN (1, null, 3) AS STRING):s|CAST(NOT null IN (1, null, 3) AS STRING):s|CAST(null NOT IN (1, 2, 3) AS STRING):s +---------------------------------------+---------------------------------------+------------------------------------------+--------------------------------------- +null |false |null |null ; // @@ -134,34 +134,34 @@ false inWithTableColumnAndFunction SELECT emp_no IN (10000, 10000 + 1, abs(-10000 - 2)) FROM test_emp WHERE emp_no BETWEEN 10001 AND 10004 ORDER BY emp_no; - emp_no IN (10000, 10000 + 1, ABS(-10000 - 2)):b ------------------------------------------------- -true -true -false -false +emp_no IN (10000, 10000 + 1, abs(-10000 - 2)) +--------------------------------------------- +true +true +false +false ; inWithTableColumnAndNegation SELECT emp_no NOT IN (10000, 10000 + 1, 10002) FROM test_emp WHERE emp_no BETWEEN 10001 AND 10004 ORDER BY emp_no; - NOT(emp_no IN (10000, 10000 + 1, 10002)):b -------------------------------------------- -false -false -true -true +emp_no NOT IN (10000, 10000 + 1, 10002) +--------------------------------------- +false +false +true +true ; inWithTableColumnAndComplexFunctions SELECT emp_no IN (1, abs(1 - 10002), 3) OR emp_no NOT IN (10000, 10000 + 2, 10003) FROM test_emp WHERE emp_no BETWEEN 10001 AND 10004 ORDER BY emp_no; -(emp_no IN (1, ABS(1 - 10002), 3)) OR (NOT(emp_no IN (10000, 10000 + 2, 10003))):b ----------------------------------------------------------------------------------- -true -false -false -true +emp_no IN (1, abs(1 - 10002), 3) OR emp_no NOT IN (10000, 10000 + 2, 10003) +--------------------------------------------------------------------------- +true +false +false +true ; @@ -169,19 +169,19 @@ true inWithTableColumnAndNullHandling SELECT emp_no, CAST(languages IN (2, 3) AS STRING), CAST(languages IN (2, null, 3) AS STRING) FROM test_emp WHERE emp_no BETWEEN 10018 AND 10020 ORDER BY emp_no; - emp_no:i | CAST(languages IN (2, 3) AS VARCHAR):s | CAST(languages IN (2, null, 3) AS VARCHAR):s -----------+-----------------------------------------+---------------------------------------------- -10018 |true |true -10019 |false |null -10020 |null |null + emp_no:i |CAST(languages IN (2, 3) AS STRING):s|CAST(languages IN (2, null, 3) AS STRING):s +---------------+-------------------------------------+------------------------------------------- +10018 |true |true +10019 |false |null +10020 |null |null ; inWithTableColumnAndNullHandlingAndNegation SELECT emp_no, CAST(languages NOT IN (2, 3) AS STRING), CAST(NOT languages IN (2, null, 3) AS STRING) FROM test_emp WHERE emp_no BETWEEN 10018 AND 10020 ORDER BY emp_no; - emp_no:i | CAST(NOT(languages IN (2, 3)) AS VARCHAR):s | CAST(NOT(languages IN (2, null, 3)) AS VARCHAR):s -----------+----------------------------------------------+--------------------------------------------------- -10018 |false |false -10019 |true |null -10020 |null |null + emp_no:i |CAST(languages NOT IN (2, 3) AS STRING):s|CAST(NOT languages IN (2, null, 3) AS STRING):s +---------------+-----------------------------------------+----------------------------------------------- +10018 |false |false +10019 |true |null +10020 |null |null ; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java index 090fea80b14c4..097758976da84 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java @@ -772,7 +772,7 @@ private Expression collectResolvedAndReplace(Expression e, Map> seen for (Function seenFunction : list) { if (uf.arguments().equals(seenFunction.arguments())) { // Special check for COUNT: an already seen COUNT function will be returned only if its DISTINCT property - // matches the one from the unresolved function to be checked. + // matches the one from the unresolved function to be checked. if (seenFunction instanceof Count) { if (uf.sameAs((Count) seenFunction)) { return seenFunction; @@ -899,8 +899,7 @@ private List assignAliases(List expr return new Alias(c.source(), ((NamedExpression) c.field()).name(), c); } } - //TODO: maybe add something closer to SQL - return new Alias(child.source(), child.toString(), child); + return new Alias(child.source(), child.sourceText(), child); }, UnresolvedAlias.class); newExpr.add(expr.equals(transformed) ? expr : transformed); } @@ -1031,7 +1030,7 @@ private Expression rule(Expression e, List seen) { } private boolean functionsEquals(Function f, Function seenFunction) { - return f.name().equals(seenFunction.name()) && f.arguments().equals(seenFunction.arguments()); + return f.sourceText().equals(seenFunction.sourceText()) && f.arguments().equals(seenFunction.arguments()); } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/Literal.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/Literal.java index 1d029f3133039..579fd934b48ea 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/Literal.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/Literal.java @@ -36,7 +36,7 @@ public Literal(Source source, Object value, DataType dataType) { } public Literal(Source source, String name, Object value, DataType dataType) { - super(source, name == null ? String.valueOf(value) : name, emptyList(), null); + super(source, name == null ? source.text() : name, emptyList(), null); this.dataType = dataType; this.value = DataTypeConversion.convert(value, dataType); } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/Function.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/Function.java index f63145f6a25b5..06171d43cc036 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/Function.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/Function.java @@ -14,7 +14,6 @@ import org.elasticsearch.xpack.sql.util.StringUtils; import java.util.List; -import java.util.StringJoiner; /** * Any SQL expression with parentheses, like {@code MAX()}, or {@code ABS()}. A @@ -33,7 +32,7 @@ protected Function(Source source, List children, ExpressionId id, bo // cannot detect name yet so override the name super(source, null, children, id, synthetic); functionName = StringUtils.camelCaseToUnderscore(getClass().getSimpleName()); - name = functionName() + functionArgs(); + name = source.text(); } public final List arguments() { @@ -52,7 +51,7 @@ public Nullability nullable() { @Override public String toString() { - return name() + "#" + id(); + return sourceText() + "#" + id(); } public String functionName() { @@ -64,16 +63,7 @@ public String functionId() { return id().toString(); } - protected String functionArgs() { - StringJoiner sj = new StringJoiner(",", "(", ")"); - for (Expression child : children()) { - String val = child instanceof NamedExpression && child.resolved() ? Expressions.name(child) : child.toString(); - sj.add(val); - } - return sj.toString(); - } - public boolean functionEquals(Function f) { return f != null && getClass() == f.getClass() && arguments().equals(f.arguments()); } -} +} \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java index 82b4d95a9b47b..52bbed17346b3 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java @@ -166,7 +166,7 @@ public String unresolvedMessage() { @Override public String toString() { - return UNRESOLVED_PREFIX + functionName() + functionArgs(); + return UNRESOLVED_PREFIX + sourceText(); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Max.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Max.java index 5df73793677ed..e66dfdebc6b36 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Max.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Max.java @@ -8,8 +8,8 @@ import org.elasticsearch.xpack.sql.expression.Expression; import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; -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.util.List; @@ -45,6 +45,6 @@ public String innerName() { @Override protected TypeResolution resolveType() { - return Expressions.typeMustBeNumericOrDate(field(), functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeNumericOrDate(field(), sourceText(), ParamOrdinal.DEFAULT); } -} +} \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Min.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Min.java index 09ee16619cd80..bc3c5952f3896 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Min.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Min.java @@ -8,8 +8,8 @@ import org.elasticsearch.xpack.sql.expression.Expression; import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; -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.util.List; @@ -48,6 +48,6 @@ public String innerName() { @Override protected TypeResolution resolveType() { - return Expressions.typeMustBeNumericOrDate(field(), functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeNumericOrDate(field(), sourceText(), ParamOrdinal.DEFAULT); } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/NumericAggregate.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/NumericAggregate.java index 7a6fe313544e7..bfe0d2ded7e34 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/NumericAggregate.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/NumericAggregate.java @@ -25,7 +25,7 @@ abstract class NumericAggregate extends AggregateFunction { @Override protected TypeResolution resolveType() { - return Expressions.typeMustBeNumeric(field(), functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeNumeric(field(), sourceText(), ParamOrdinal.DEFAULT); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Percentile.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Percentile.java index ee5304b915c48..593466f4c4773 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Percentile.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Percentile.java @@ -10,8 +10,8 @@ import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; import org.elasticsearch.xpack.sql.expression.Foldables; -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.util.List; @@ -52,7 +52,7 @@ protected TypeResolution resolveType() { return resolution; } - return Expressions.typeMustBeNumeric(percent, functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeNumeric(percent, sourceText(), ParamOrdinal.DEFAULT); } public Expression percent() { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/PercentileRank.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/PercentileRank.java index c8f51476af3d3..72614f8265f48 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/PercentileRank.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/PercentileRank.java @@ -10,8 +10,8 @@ import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; import org.elasticsearch.xpack.sql.expression.Foldables; -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.util.List; @@ -52,7 +52,7 @@ protected TypeResolution resolveType() { return resolution; } - return Expressions.typeMustBeNumeric(value, functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeNumeric(value, sourceText(), ParamOrdinal.DEFAULT); } public Expression value() { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/Cast.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/Cast.java index 04f9b13ff3329..fd82d2bb4db23 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/Cast.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/Cast.java @@ -111,16 +111,4 @@ public boolean equals(Object obj) { return Objects.equals(dataType, other.dataType()) && Objects.equals(field(), other.field()); } - - @Override - public String toString() { - return functionName() + "(" + field().toString() + " AS " + to().sqlName() + ")#" + id(); - } - - @Override - public String name() { - StringBuilder sb = new StringBuilder(super.name()); - sb.insert(sb.length() - 1, " AS " + to().sqlName()); - return sb.toString(); - } } 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 39ee00d9b7253..a24fba1b13569 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 @@ -12,7 +12,6 @@ import org.elasticsearch.xpack.sql.session.Configuration; import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataType; -import org.elasticsearch.xpack.sql.util.StringUtils; import java.util.List; import java.util.Objects; @@ -55,11 +54,6 @@ public boolean foldable() { @Override public abstract Object fold(); - @Override - protected String functionArgs() { - return StringUtils.EMPTY; - } - @Override public ScriptTemplate asScript() { return asScript(this); diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/BaseDateTimeFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/BaseDateTimeFunction.java index cf4fab276c12b..345498afd00f1 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/BaseDateTimeFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/BaseDateTimeFunction.java @@ -10,8 +10,8 @@ import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; import org.elasticsearch.xpack.sql.expression.function.scalar.UnaryScalarFunction; -import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.tree.NodeInfo; +import org.elasticsearch.xpack.sql.tree.Source; import java.time.ZoneId; import java.time.ZonedDateTime; @@ -20,17 +20,10 @@ abstract class BaseDateTimeFunction extends UnaryScalarFunction { private final ZoneId zoneId; - private final String name; BaseDateTimeFunction(Source source, Expression field, ZoneId zoneId) { super(source, field); this.zoneId = zoneId; - - StringBuilder sb = new StringBuilder(super.name()); - // add timezone as last argument - sb.insert(sb.length() - 1, " [" + zoneId.getId() + "]"); - - this.name = sb.toString(); } @Override @@ -42,17 +35,12 @@ protected final NodeInfo info() { @Override protected TypeResolution resolveType() { - return Expressions.typeMustBeDate(field(), functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeDate(field(), sourceText(), ParamOrdinal.DEFAULT); } public ZoneId zoneId() { return zoneId; } - - @Override - public String name() { - return name; - } @Override public boolean foldable() { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/BinaryNumericFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/BinaryNumericFunction.java index 5f04edec65051..f3369bf14a457 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/BinaryNumericFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/BinaryNumericFunction.java @@ -36,12 +36,12 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution resolution = Expressions.typeMustBeNumeric(left(), functionName(), ParamOrdinal.FIRST); + TypeResolution resolution = Expressions.typeMustBeNumeric(left(), sourceText(), ParamOrdinal.FIRST); if (resolution.unresolved()) { return resolution; } - return Expressions.typeMustBeNumeric(right(), functionName(), ParamOrdinal.SECOND); + return Expressions.typeMustBeNumeric(right(), sourceText(), ParamOrdinal.SECOND); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/E.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/E.java index 843ecccb265af..265b96984b581 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/E.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/E.java @@ -11,10 +11,9 @@ import org.elasticsearch.xpack.sql.expression.function.scalar.math.MathProcessor.MathOperation; import org.elasticsearch.xpack.sql.expression.gen.script.Params; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; -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 org.elasticsearch.xpack.sql.util.StringUtils; public class E extends MathFunction { @@ -39,11 +38,6 @@ public Object fold() { return Math.E; } - @Override - protected String functionArgs() { - return StringUtils.EMPTY; - } - @Override public ScriptTemplate asScript() { return TEMPLATE; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/Pi.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/Pi.java index f6dabb1f211dd..7fb966c3201a3 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/Pi.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/Pi.java @@ -11,10 +11,9 @@ import org.elasticsearch.xpack.sql.expression.function.scalar.math.MathProcessor.MathOperation; import org.elasticsearch.xpack.sql.expression.gen.script.Params; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; -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 org.elasticsearch.xpack.sql.util.StringUtils; public class Pi extends MathFunction { @@ -39,11 +38,6 @@ public Object fold() { return Math.PI; } - @Override - protected String functionArgs() { - return StringUtils.EMPTY; - } - @Override public ScriptTemplate asScript() { return TEMPLATE; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringFunction.java index eb0d9423e9d3d..fd294564b642c 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringFunction.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.string; import org.elasticsearch.xpack.sql.expression.Expression; +import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; import org.elasticsearch.xpack.sql.expression.FieldAttribute; import org.elasticsearch.xpack.sql.expression.function.scalar.BinaryScalarFunction; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; @@ -15,7 +16,6 @@ import java.util.Objects; import java.util.function.BiFunction; -import static org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; import static org.elasticsearch.xpack.sql.expression.Expressions.typeMustBeString; import static org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder.paramsBuilder; @@ -42,7 +42,7 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution resolution = typeMustBeString(left(), functionName(), ParamOrdinal.FIRST); + TypeResolution resolution = typeMustBeString(left(), sourceText(), ParamOrdinal.FIRST); if (resolution.unresolved()) { return resolution; } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringNumericFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringNumericFunction.java index 1c15e3ec5d73f..d9f767d1ce81a 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringNumericFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringNumericFunction.java @@ -26,7 +26,7 @@ public BinaryStringNumericFunction(Source source, Expression left, Expression ri @Override protected TypeResolution resolveSecondParameterInputType(Expression e) { - return Expressions.typeMustBeNumeric(e,functionName(), Expressions.ParamOrdinal.SECOND); + return Expressions.typeMustBeNumeric(e, sourceText(), Expressions.ParamOrdinal.SECOND); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringStringFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringStringFunction.java index 9a937d4647f50..51189f6a4efff 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringStringFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringStringFunction.java @@ -21,7 +21,7 @@ public BinaryStringStringFunction(Source source, Expression left, Expression rig @Override protected TypeResolution resolveSecondParameterInputType(Expression e) { - return Expressions.typeMustBeString(e, functionName(), Expressions.ParamOrdinal.SECOND); + return Expressions.typeMustBeString(e, sourceText(), Expressions.ParamOrdinal.SECOND); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Concat.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Concat.java index 9a132c012a817..1f9833133a98a 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Concat.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Concat.java @@ -13,8 +13,8 @@ import org.elasticsearch.xpack.sql.expression.function.scalar.BinaryScalarFunction; import org.elasticsearch.xpack.sql.expression.gen.pipeline.Pipe; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; -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 static org.elasticsearch.xpack.sql.expression.function.scalar.string.ConcatFunctionProcessor.process; @@ -37,12 +37,12 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution sourceResolution = Expressions.typeMustBeString(left(), functionName(), ParamOrdinal.FIRST); + TypeResolution sourceResolution = Expressions.typeMustBeString(left(), sourceText(), ParamOrdinal.FIRST); if (sourceResolution.unresolved()) { return sourceResolution; } - return Expressions.typeMustBeString(right(), functionName(), ParamOrdinal.SECOND); + return Expressions.typeMustBeString(right(), sourceText(), ParamOrdinal.SECOND); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Insert.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Insert.java index 7f15e9ce6117f..8e6fc2052928e 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Insert.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Insert.java @@ -46,22 +46,22 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution sourceResolution = Expressions.typeMustBeString(source, functionName(), ParamOrdinal.FIRST); + TypeResolution sourceResolution = Expressions.typeMustBeString(source, sourceText(), ParamOrdinal.FIRST); if (sourceResolution.unresolved()) { return sourceResolution; } - TypeResolution startResolution = Expressions.typeMustBeNumeric(start, functionName(), ParamOrdinal.SECOND); + TypeResolution startResolution = Expressions.typeMustBeNumeric(start, sourceText(), ParamOrdinal.SECOND); if (startResolution.unresolved()) { return startResolution; } - TypeResolution lengthResolution = Expressions.typeMustBeNumeric(length, functionName(), ParamOrdinal.THIRD); + TypeResolution lengthResolution = Expressions.typeMustBeNumeric(length, sourceText(), ParamOrdinal.THIRD); if (lengthResolution.unresolved()) { return lengthResolution; } - return Expressions.typeMustBeString(replacement, functionName(), ParamOrdinal.FOURTH); + return Expressions.typeMustBeString(replacement, sourceText(), ParamOrdinal.FOURTH); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Locate.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Locate.java index cf3d7ed6e6a12..042ec1a736373 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Locate.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Locate.java @@ -48,19 +48,19 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution patternResolution = Expressions.typeMustBeString(pattern, functionName(), ParamOrdinal.FIRST); + TypeResolution patternResolution = Expressions.typeMustBeString(pattern, sourceText(), ParamOrdinal.FIRST); if (patternResolution.unresolved()) { return patternResolution; } - TypeResolution sourceResolution = Expressions.typeMustBeString(source, functionName(), ParamOrdinal.SECOND); + TypeResolution sourceResolution = Expressions.typeMustBeString(source, sourceText(), ParamOrdinal.SECOND); if (sourceResolution.unresolved()) { return sourceResolution; } return start == null ? TypeResolution.TYPE_RESOLVED : - Expressions.typeMustBeNumeric(start, functionName(), ParamOrdinal.THIRD); + Expressions.typeMustBeNumeric(start, sourceText(), ParamOrdinal.THIRD); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Replace.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Replace.java index 82994fae9f000..cd960f1f3b3ea 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Replace.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Replace.java @@ -44,17 +44,17 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution sourceResolution = Expressions.typeMustBeString(source, functionName(), ParamOrdinal.FIRST); + TypeResolution sourceResolution = Expressions.typeMustBeString(source, sourceText(), ParamOrdinal.FIRST); if (sourceResolution.unresolved()) { return sourceResolution; } - TypeResolution patternResolution = Expressions.typeMustBeString(pattern, functionName(), ParamOrdinal.SECOND); + TypeResolution patternResolution = Expressions.typeMustBeString(pattern, sourceText(), ParamOrdinal.SECOND); if (patternResolution.unresolved()) { return patternResolution; } - return Expressions.typeMustBeString(replacement, functionName(), ParamOrdinal.THIRD); + return Expressions.typeMustBeString(replacement, sourceText(), ParamOrdinal.THIRD); } @Override @@ -124,4 +124,4 @@ public Expression replaceChildren(List newChildren) { return new Replace(source(), newChildren.get(0), newChildren.get(1), newChildren.get(2)); } -} +} \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Substring.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Substring.java index c049551f90ef9..75178e73fce46 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Substring.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Substring.java @@ -45,17 +45,17 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution sourceResolution = Expressions.typeMustBeString(source, functionName(), ParamOrdinal.FIRST); + TypeResolution sourceResolution = Expressions.typeMustBeString(source, sourceText(), ParamOrdinal.FIRST); if (sourceResolution.unresolved()) { return sourceResolution; } - TypeResolution startResolution = Expressions.typeMustBeNumeric(start, functionName(), ParamOrdinal.SECOND); + TypeResolution startResolution = Expressions.typeMustBeNumeric(start, sourceText(), ParamOrdinal.SECOND); if (startResolution.unresolved()) { return startResolution; } - return Expressions.typeMustBeNumeric(length, functionName(), ParamOrdinal.THIRD); + return Expressions.typeMustBeNumeric(length, sourceText(), ParamOrdinal.THIRD); } @Override @@ -124,4 +124,4 @@ public Expression replaceChildren(List newChildren) { return new Substring(source(), newChildren.get(0), newChildren.get(1), newChildren.get(2)); } -} +} \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/BinaryPredicate.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/BinaryPredicate.java index 277fa7e4998e5..eb7265dc29bc8 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/BinaryPredicate.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/BinaryPredicate.java @@ -6,8 +6,6 @@ package org.elasticsearch.xpack.sql.expression.predicate; import org.elasticsearch.xpack.sql.expression.Expression; -import org.elasticsearch.xpack.sql.expression.Expressions; -import org.elasticsearch.xpack.sql.expression.Literal; import org.elasticsearch.xpack.sql.expression.function.scalar.BinaryScalarFunction; import org.elasticsearch.xpack.sql.tree.Source; @@ -21,12 +19,10 @@ */ public abstract class BinaryPredicate> extends BinaryScalarFunction { - private final String name; private final F function; protected BinaryPredicate(Source source, Expression left, Expression right, F function) { super(source, left, right); - this.name = name(left, right, function.symbol()); this.function = function; } @@ -65,11 +61,6 @@ public boolean equals(Object obj) { && Objects.equals(right(), other.right()); } - @Override - public String name() { - return name; - } - public String symbol() { return function.symbol(); } @@ -77,23 +68,4 @@ public String symbol() { public F function() { return function; } - - private static String name(Expression left, Expression right, String symbol) { - StringBuilder sb = new StringBuilder(); - sb.append(Expressions.name(left)); - if (!(left instanceof Literal)) { - sb.insert(0, "("); - sb.append(")"); - } - sb.append(" "); - sb.append(symbol); - sb.append(" "); - int pos = sb.length(); - sb.append(Expressions.name(right)); - if (!(right instanceof Literal)) { - sb.insert(pos, "("); - sb.append(")"); - } - return sb.toString(); - } } \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/Range.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/Range.java index a189b7fda0eeb..aa1a3a32e5541 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/Range.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/Range.java @@ -18,8 +18,8 @@ import org.elasticsearch.xpack.sql.expression.predicate.operator.comparison.BinaryComparison; import org.elasticsearch.xpack.sql.expression.predicate.operator.comparison.BinaryComparisonPipe; import org.elasticsearch.xpack.sql.expression.predicate.operator.comparison.BinaryComparisonProcessor.BinaryComparisonOperation; -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.util.List; @@ -33,7 +33,6 @@ // BETWEEN or range - is a mix of gt(e) AND lt(e) public class Range extends ScalarFunction { - private final String name; private final Expression value, lower, upper; private final boolean includeLower, includeUpper; @@ -45,12 +44,6 @@ public Range(Source source, Expression value, Expression lower, boolean includeL this.upper = upper; this.includeLower = includeLower; this.includeUpper = includeUpper; - this.name = name(value, lower, upper, includeLower, includeUpper); - } - - @Override - public String name() { - return name; } @Override @@ -212,9 +205,4 @@ private static String name(Expression value, Expression lower, Expression upper, return sb.toString(); } - - @Override - public String toString() { - return name(); - } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/BinaryLogic.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/BinaryLogic.java index d4bce9feab3ec..930636657fc48 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/BinaryLogic.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/BinaryLogic.java @@ -27,7 +27,7 @@ public DataType dataType() { @Override protected TypeResolution resolveInputType(Expression e, Expressions.ParamOrdinal paramOrdinal) { - return Expressions.typeMustBeBoolean(e, functionName(), paramOrdinal); + return Expressions.typeMustBeBoolean(e, sourceText(), paramOrdinal); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/Not.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/Not.java index 8d326c570292b..bbadaa6ef6430 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/Not.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/Not.java @@ -12,8 +12,8 @@ import org.elasticsearch.xpack.sql.expression.gen.processor.Processor; import org.elasticsearch.xpack.sql.expression.gen.script.Scripts; import org.elasticsearch.xpack.sql.expression.predicate.Negatable; -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; public class Not extends UnaryScalarFunction { @@ -37,7 +37,7 @@ protected TypeResolution resolveType() { if (DataType.BOOLEAN == field().dataType()) { return TypeResolution.TYPE_RESOLVED; } - return Expressions.typeMustBeBoolean(field(), functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeBoolean(field(), sourceText(), ParamOrdinal.DEFAULT); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/arithmetic/Neg.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/arithmetic/Neg.java index 265a1eb993c94..3ada9a523a13c 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/arithmetic/Neg.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/arithmetic/Neg.java @@ -8,13 +8,12 @@ import org.elasticsearch.xpack.sql.expression.Expression; import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; -import org.elasticsearch.xpack.sql.expression.NamedExpression; import org.elasticsearch.xpack.sql.expression.function.scalar.UnaryScalarFunction; import org.elasticsearch.xpack.sql.expression.gen.processor.Processor; import org.elasticsearch.xpack.sql.expression.gen.script.Scripts; import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.UnaryArithmeticProcessor.UnaryArithmeticOperation; -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; /** @@ -38,7 +37,7 @@ protected Neg replaceChild(Expression newChild) { @Override protected TypeResolution resolveType() { - return Expressions.typeMustBeNumeric(field(), functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeNumeric(field(), sourceText(), ParamOrdinal.DEFAULT); } @Override @@ -51,11 +50,6 @@ public DataType dataType() { return field().dataType(); } - @Override - public String name() { - return "-" + (field() instanceof NamedExpression && field().resolved() ? Expressions.name(field()) : field().toString()); - } - @Override public String processScript(String script) { return Scripts.formatTemplate(Scripts.SQL_SCRIPTS + ".neg(" + script + ")"); diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/BinaryComparison.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/BinaryComparison.java index 815fa10b12fd1..4e7473907c890 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/BinaryComparison.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/BinaryComparison.java @@ -40,18 +40,7 @@ protected Pipe makePipe() { return new BinaryComparisonPipe(source(), this, Expressions.pipe(left()), Expressions.pipe(right()), function()); } - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(left()); - sb.append(" "); - sb.append(symbol()); - sb.append(" "); - sb.append(right()); - return sb.toString(); - } - public static Integer compare(Object left, Object right) { return Comparisons.compare(left, right); } -} +} \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/In.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/In.java index f9e6b72fb5958..f8f0bb35b504e 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/In.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/In.java @@ -21,7 +21,6 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Objects; -import java.util.StringJoiner; import java.util.stream.Collectors; import static org.elasticsearch.common.logging.LoggerMessageFormat.format; @@ -85,13 +84,6 @@ public Boolean fold() { return InProcessor.apply(value.fold(), Foldables.valuesOf(list, value.dataType())); } - @Override - public String name() { - StringJoiner sj = new StringJoiner(", ", " IN (", ")"); - list.forEach(e -> sj.add(Expressions.name(e))); - return Expressions.name(value) + sj.toString(); - } - @Override public ScriptTemplate asScript() { ScriptTemplate leftScript = asScript(value); diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/AbstractBuilder.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/AbstractBuilder.java index 1d6c264321787..81c2e7578ccb9 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/AbstractBuilder.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/AbstractBuilder.java @@ -92,6 +92,16 @@ static Source source(Token token) { return new Source(new Location(token.getLine(), token.getCharPositionInLine()), text); } + Source source(ParserRuleContext begin, ParserRuleContext end) { + Check.notNull(begin, "begin is null"); + Check.notNull(end, "end is null"); + Token start = begin.start; + Token stop = end.stop != null ? end.stop : begin.stop; + Interval interval = new Interval(start.getStartIndex(), stop.getStopIndex()); + String text = start.getInputStream().getText(interval); + return new Source(new Location(start.getLine(), start.getCharPositionInLine()), text); + } + /** * Retrieves the raw text of the node (without interpreting it as a string literal). */ 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 e103976760fb3..132c390337628 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 @@ -11,6 +11,7 @@ import org.antlr.v4.runtime.tree.TerminalNode; import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.xpack.sql.SqlIllegalArgumentException; import org.elasticsearch.xpack.sql.expression.Alias; import org.elasticsearch.xpack.sql.expression.Exists; @@ -215,7 +216,7 @@ public Expression visitPredicated(PredicatedContext ctx) { } PredicateContext pCtx = ctx.predicate(); - Source source = source(pCtx); + Source source = source(ctx.valueExpression(), ctx); Expression e = null; switch (pCtx.kind.getType()) { @@ -320,7 +321,7 @@ public Object visitArithmeticUnary(ArithmeticUnaryContext ctx) { if (value instanceof Literal) { // Minus already processed together with literal number return value; } - return new Neg(source(ctx.operator), value); + return new Neg(source(ctx), value); default: throw new ParsingException(source, "Unknown arithmetic {}", source.text()); } @@ -331,7 +332,7 @@ public Object visitArithmeticBinary(ArithmeticBinaryContext ctx) { Expression left = expression(ctx.left); Expression right = expression(ctx.right); - Source source = source(ctx.operator); + Source source = source(ctx); switch (ctx.operator.getType()) { case SqlBaseParser.ASTERISK: @@ -611,7 +612,7 @@ public Literal visitInterval(IntervalContext interval) { Interval timeInterval = value instanceof Period ? new IntervalYearMonth((Period) value, intervalType) : new IntervalDayTime((Duration) value, intervalType); - return new Literal(source(interval), text(interval), timeInterval, timeInterval.dataType()); + return new Literal(source(interval), timeInterval, timeInterval.dataType()); } private TemporalAmount of(NumberContext valueNumeric, TimeUnit unit) { @@ -689,23 +690,24 @@ public Expression visitStringLiteral(StringLiteralContext ctx) { @Override public Literal visitDecimalLiteral(DecimalLiteralContext ctx) { - String string = (hasMinusFromParent(ctx) ? "-" : "") + ctx.getText(); + Tuple tuple = withMinus(ctx); + try { - return new Literal(source(ctx), Double.valueOf(StringUtils.parseDouble(string)), DataType.DOUBLE); + return new Literal(tuple.v1(), Double.valueOf(StringUtils.parseDouble(tuple.v2())), DataType.DOUBLE); } catch (SqlIllegalArgumentException siae) { - throw new ParsingException(source(ctx), siae.getMessage()); + throw new ParsingException(tuple.v1(), siae.getMessage()); } } @Override public Literal visitIntegerLiteral(IntegerLiteralContext ctx) { - String string = (hasMinusFromParent(ctx) ? "-" : "") + ctx.getText(); + Tuple tuple = withMinus(ctx); long value; try { - value = Long.valueOf(StringUtils.parseLong(string)); + value = Long.valueOf(StringUtils.parseLong(tuple.v2())); } catch (SqlIllegalArgumentException siae) { - throw new ParsingException(source(ctx), siae.getMessage()); + throw new ParsingException(tuple.v1(), siae.getMessage()); } Object val = Long.valueOf(value); @@ -715,7 +717,7 @@ public Literal visitIntegerLiteral(IntegerLiteralContext ctx) { type = DataType.INTEGER; val = Integer.valueOf((int) value); } - return new Literal(source(ctx), val, type); + return new Literal(tuple.v1(), val, type); } @Override @@ -876,7 +878,30 @@ public Literal visitGuidEscapedLiteral(GuidEscapedLiteralContext ctx) { return new Literal(source(ctx), string, DataType.KEYWORD); } - private boolean hasMinusFromParent(SqlBaseParser.NumberContext ctx) { + /** + * Return the source and the value of the given number, + * taking into account MINUS (-) if needed. + */ + private static Tuple withMinus(NumberContext ctx) { + String string = ctx.getText(); + Source source = minusAwareSource(ctx); + + if (source != null) { + string = "-" + string; + } else { + source = source(ctx); + } + + return new Tuple<>(source, string); + } + + /** + * Checks the presence of MINUS (-) in the parent and if found, + * returns the parent source or null otherwise. + * Parsing of the value should not depend on the returned source + * as it might contain extra spaces. + */ + private static Source minusAwareSource(SqlBaseParser.NumberContext ctx) { ParserRuleContext parentCtx = ctx.getParent(); if (parentCtx != null) { if (parentCtx instanceof SqlBaseParser.NumericLiteralContext) { @@ -886,17 +911,23 @@ private boolean hasMinusFromParent(SqlBaseParser.NumberContext ctx) { if (parentCtx != null && parentCtx instanceof SqlBaseParser.ValueExpressionDefaultContext) { parentCtx = parentCtx.getParent(); if (parentCtx != null && parentCtx instanceof SqlBaseParser.ArithmeticUnaryContext) { - return ((ArithmeticUnaryContext) parentCtx).MINUS() != null; + if (((ArithmeticUnaryContext) parentCtx).MINUS() != null) { + return source(parentCtx); + } } } } } else if (parentCtx instanceof SqlBaseParser.IntervalContext) { IntervalContext ic = (IntervalContext) parentCtx; - return ic.sign != null && ic.sign.getType() == SqlBaseParser.MINUS; + if (ic.sign != null && ic.sign.getType() == SqlBaseParser.MINUS) { + return source(ic); + } } else if (parentCtx instanceof SqlBaseParser.SysTypesContext) { - return ((SysTypesContext) parentCtx).MINUS() != null; + if (((SysTypesContext) parentCtx).MINUS() != null) { + return source(parentCtx); + } } } - return false; + return null; } -} +} \ No newline at end of file diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java index 6366ba85503b0..2d20d92e17826 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java @@ -239,7 +239,7 @@ public void testGroupByOrderByAliasedInSelectAllowed() { } public void testGroupByOrderByScalarOverNonGrouped() { - assertEquals("1:50: Cannot order by non-grouped column [YEAR(date [Z])], expected [text]", + assertEquals("1:50: Cannot order by non-grouped column [YEAR(date)], expected [text]", error("SELECT MAX(int) FROM test GROUP BY text ORDER BY YEAR(date)")); } @@ -249,7 +249,7 @@ public void testGroupByOrderByFieldFromGroupByFunction() { } public void testGroupByOrderByScalarOverNonGrouped_WithHaving() { - assertEquals("1:71: Cannot order by non-grouped column [YEAR(date [Z])], expected [text]", + assertEquals("1:71: Cannot order by non-grouped column [YEAR(date)], expected [text]", error("SELECT MAX(int) FROM test GROUP BY text HAVING MAX(int) > 10 ORDER BY YEAR(date)")); } @@ -373,12 +373,12 @@ public void testInNestedWithDifferentDataTypesFromLeftValue_WhereClause() { } public void testNotSupportedAggregateOnDate() { - assertEquals("1:8: [AVG] argument must be [numeric], found value [date] type [date]", + assertEquals("1:8: [AVG(date)] argument must be [numeric], found value [date] type [date]", error("SELECT AVG(date) FROM test")); } public void testNotSupportedAggregateOnString() { - assertEquals("1:8: [MAX] argument must be [numeric or date], found value [keyword] type [keyword]", + assertEquals("1:8: [MAX(keyword)] argument must be [numeric or date], found value [keyword] type [keyword]", error("SELECT MAX(keyword) FROM test")); } @@ -388,53 +388,53 @@ public void testInvalidTypeForStringFunction_WithOneArg() { } public void testInvalidTypeForNumericFunction_WithOneArg() { - assertEquals("1:8: [COS] argument must be [numeric], found value [foo] type [keyword]", + assertEquals("1:8: [COS] argument must be [numeric], found value ['foo'] type [keyword]", error("SELECT COS('foo')")); } public void testInvalidTypeForBooleanFunction_WithOneArg() { - assertEquals("1:8: [NOT] argument must be [boolean], found value [foo] type [keyword]", + assertEquals("1:8: [NOT 'foo'] argument must be [boolean], found value ['foo'] type [keyword]", error("SELECT NOT 'foo'")); } public void testInvalidTypeForStringFunction_WithTwoArgs() { - assertEquals("1:8: [CONCAT] first argument must be [string], found value [1] type [integer]", + assertEquals("1:8: [CONCAT(1, 'bar')] first argument must be [string], found value [1] type [integer]", error("SELECT CONCAT(1, 'bar')")); - assertEquals("1:8: [CONCAT] second argument must be [string], found value [2] type [integer]", + assertEquals("1:8: [CONCAT('foo', 2)] second argument must be [string], found value [2] type [integer]", error("SELECT CONCAT('foo', 2)")); } public void testInvalidTypeForNumericFunction_WithTwoArgs() { - assertEquals("1:8: [TRUNCATE] first argument must be [numeric], found value [foo] type [keyword]", + assertEquals("1:8: [TRUNCATE('foo', 2)] first argument must be [numeric], found value ['foo'] type [keyword]", error("SELECT TRUNCATE('foo', 2)")); - assertEquals("1:8: [TRUNCATE] second argument must be [numeric], found value [bar] type [keyword]", + assertEquals("1:8: [TRUNCATE(1.2, 'bar')] second argument must be [numeric], found value ['bar'] type [keyword]", error("SELECT TRUNCATE(1.2, 'bar')")); } public void testInvalidTypeForBooleanFuntion_WithTwoArgs() { - assertEquals("1:8: [OR] first argument must be [boolean], found value [1] type [integer]", + assertEquals("1:8: [1 OR true] first argument must be [boolean], found value [1] type [integer]", error("SELECT 1 OR true")); - assertEquals("1:8: [OR] second argument must be [boolean], found value [2] type [integer]", + assertEquals("1:8: [true OR 2] second argument must be [boolean], found value [2] type [integer]", error("SELECT true OR 2")); } public void testInvalidTypeForFunction_WithThreeArgs() { - assertEquals("1:8: [REPLACE] first argument must be [string], found value [1] type [integer]", + assertEquals("1:8: [REPLACE(1, 'foo', 'bar')] first argument must be [string], found value [1] type [integer]", error("SELECT REPLACE(1, 'foo', 'bar')")); - assertEquals("1:8: [REPLACE] second argument must be [string], found value [2] type [integer]", + assertEquals("1:8: [REPLACE('text', 2, 'bar')] second argument must be [string], found value [2] type [integer]", error("SELECT REPLACE('text', 2, 'bar')")); - assertEquals("1:8: [REPLACE] third argument must be [string], found value [3] type [integer]", + assertEquals("1:8: [REPLACE('text', 'foo', 3)] third argument must be [string], found value [3] type [integer]", error("SELECT REPLACE('text', 'foo', 3)")); } public void testInvalidTypeForFunction_WithFourArgs() { - assertEquals("1:8: [INSERT] first argument must be [string], found value [1] type [integer]", + assertEquals("1:8: [INSERT(1, 1, 2, 'new')] first argument must be [string], found value [1] type [integer]", error("SELECT INSERT(1, 1, 2, 'new')")); - assertEquals("1:8: [INSERT] second argument must be [numeric], found value [foo] type [keyword]", + assertEquals("1:8: [INSERT('text', 'foo', 2, 'new')] second argument must be [numeric], found value ['foo'] type [keyword]", error("SELECT INSERT('text', 'foo', 2, 'new')")); - assertEquals("1:8: [INSERT] third argument must be [numeric], found value [bar] type [keyword]", + assertEquals("1:8: [INSERT('text', 1, 'bar', 'new')] third argument must be [numeric], found value ['bar'] type [keyword]", error("SELECT INSERT('text', 1, 'bar', 'new')")); - assertEquals("1:8: [INSERT] fourth argument must be [string], found value [3] type [integer]", + assertEquals("1:8: [INSERT('text', 1, 2, 3)] fourth argument must be [string], found value [3] type [integer]", error("SELECT INSERT('text', 1, 2, 3)")); } @@ -508,7 +508,7 @@ public void testAggsInWhere() { } public void testHistogramInFilter() { - assertEquals("1:63: Cannot filter on grouping function [HISTOGRAM(date)], use its argument instead", + assertEquals("1:63: Cannot filter on grouping function [HISTOGRAM(date, INTERVAL 1 MONTH)], use its argument instead", error("SELECT HISTOGRAM(date, INTERVAL 1 MONTH) AS h FROM test WHERE " + "HISTOGRAM(date, INTERVAL 1 MONTH) > CAST('2000-01-01' AS DATE) GROUP BY h")); } @@ -522,8 +522,8 @@ public void testHistogramInHaving() { public void testGroupByScalarOnTopOfGrouping() { assertEquals( - "1:14: Cannot combine [HISTOGRAM(date)] grouping function inside GROUP BY, " - + "found [MONTH_OF_YEAR(HISTOGRAM(date) [Z])]; consider moving the expression inside the histogram", + "1:14: Cannot combine [HISTOGRAM(date, INTERVAL 1 MONTH)] grouping function inside " + + "GROUP BY, found [MONTH(HISTOGRAM(date, INTERVAL 1 MONTH))]; consider moving the expression inside the histogram", error("SELECT MONTH(HISTOGRAM(date, INTERVAL 1 MONTH)) AS h FROM test GROUP BY h")); } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/NamedExpressionTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/NamedExpressionTests.java index d723e79274ed9..4ac4450620626 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/NamedExpressionTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/NamedExpressionTests.java @@ -14,6 +14,8 @@ import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.Mul; import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.Neg; import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.Sub; +import org.elasticsearch.xpack.sql.tree.Location; +import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataType; import org.elasticsearch.xpack.sql.type.EsField; @@ -23,29 +25,40 @@ public class NamedExpressionTests extends ESTestCase { public void testArithmeticFunctionName() { - Add add = new Add(EMPTY, l(5), l(2)); - assertEquals("5 + 2", add.name()); + String e = "5 + 2"; + Add add = new Add(s(e), l(5), l(2)); + assertEquals(e, add.sourceText()); - Div div = new Div(EMPTY, l(5), l(2)); - assertEquals("5 / 2", div.name()); + e = "5 / 2"; + Div div = new Div(s(e), l(5), l(2)); + assertEquals(e, div.sourceText()); - Mod mod = new Mod(EMPTY, l(5), l(2)); - assertEquals("5 % 2", mod.name()); + e = "5%2"; + Mod mod = new Mod(s(e), l(5), l(2)); + assertEquals(e, mod.sourceText()); - Mul mul = new Mul(EMPTY, l(5), l(2)); - assertEquals("5 * 2", mul.name()); + e = "5 * 2"; + Mul mul = new Mul(s(e), l(5), l(2)); + assertEquals(e, mul.sourceText()); - Sub sub = new Sub(EMPTY, l(5), l(2)); - assertEquals("5 - 2", sub.name()); + e = "5 -2"; + Sub sub = new Sub(s(e), l(5), l(2)); + assertEquals(e, sub.sourceText()); - Neg neg = new Neg(EMPTY, l(5)); - assertEquals("-5", neg.name()); + e = " - 5"; + Neg neg = new Neg(s(e), l(5)); + assertEquals(e, neg.sourceText()); } public void testNameForArithmeticFunctionAppliedOnTableColumn() { FieldAttribute fa = new FieldAttribute(EMPTY, "myField", new EsField("myESField", DataType.INTEGER, emptyMap(), true)); - Add add = new Add(EMPTY, fa, l(10)); - assertEquals("(myField) + 10", add.name()); + String e = "myField + 10"; + Add add = new Add(s(e), fa, l(10)); + assertEquals(e, add.sourceText()); + } + + private static Source s(String text) { + return new Source(Location.EMPTY, text); } private static Literal l(Object value) { diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/DayOfYearTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/DayOfYearTests.java index 6bd4a8fe1baca..2a7af2916373e 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/DayOfYearTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/DayOfYearTests.java @@ -7,6 +7,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.sql.expression.Literal; +import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataType; import java.time.ZoneId; @@ -27,6 +28,6 @@ private Object extract(Object value, ZoneId zoneId) { } private DayOfYear build(Object value, ZoneId zoneId) { - return new DayOfYear(null, new Literal(null, value, DataType.DATE), zoneId); + return new DayOfYear(Source.EMPTY, new Literal(Source.EMPTY, value, DataType.DATE), zoneId); } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerTests.java index bc5bc0985cb29..700097c46163a 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerTests.java @@ -585,7 +585,7 @@ public void testNullEqualsWithNullLiteralBecomesIsNull() { BooleanLiteralsOnTheRight swapLiteralsToRight = new BooleanLiteralsOnTheRight(); BinaryComparisonSimplification bcSimpl = new BinaryComparisonSimplification(); FieldAttribute fa = getFieldAttribute(); - Source source = new Source(1, 10, StringUtils.EMPTY); + Source source = new Source(1, 10, "IS_NULL(a)"); Expression e = bcSimpl.rule(swapLiteralsToRight.rule(new NullEquals(source, fa, NULL))); assertEquals(IsNull.class, e.getClass()); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/EscapedFunctionsTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/EscapedFunctionsTests.java index 9492d63b7d323..5969f8e5ed2cd 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/EscapedFunctionsTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/EscapedFunctionsTests.java @@ -80,12 +80,12 @@ private Function function(String name) { public void testFunctionNoArg() { Function f = function("SCORE()"); - assertEquals("SCORE", f.functionName()); + assertEquals("{fn SCORE()}", f.sourceText()); } public void testFunctionOneArg() { Function f = function("ABS(foo)"); - assertEquals("ABS", f.functionName()); + assertEquals("{fn ABS(foo)}", f.sourceText()); assertEquals(1, f.arguments().size()); Expression arg = f.arguments().get(0); assertThat(arg, instanceOf(UnresolvedAttribute.class)); @@ -95,75 +95,77 @@ public void testFunctionOneArg() { public void testFunctionOneArgFunction() { Function f = function("ABS({fn SCORE()})"); - assertEquals("ABS", f.functionName()); + assertEquals("{fn ABS({fn SCORE()})}", f.sourceText()); assertEquals(1, f.arguments().size()); Expression arg = f.arguments().get(0); assertThat(arg, instanceOf(UnresolvedFunction.class)); UnresolvedFunction uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("SCORE")); + assertThat(uf.sourceText(), is("{fn SCORE()}")); } public void testFunctionFloorWithExtract() { - Function f = function("CAST({fn FLOOR({fn EXTRACT(YEAR FROM \"foo\")})} AS int)"); - assertEquals("CAST", f.functionName()); + String e = "CAST({fn FLOOR({fn EXTRACT(YEAR FROM \"foo\")})} AS int)"; + Function f = function(e); + assertEquals(e, f.sourceText()); assertEquals(1, f.arguments().size()); Expression arg = f.arguments().get(0); assertThat(arg, instanceOf(UnresolvedFunction.class)); f = (Function) arg; - assertEquals("FLOOR", f.functionName()); + assertEquals("{fn FLOOR({fn EXTRACT(YEAR FROM \"foo\")})}", f.sourceText()); assertEquals(1, f.arguments().size()); arg = f.arguments().get(0); assertThat(arg, instanceOf(UnresolvedFunction.class)); UnresolvedFunction uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("YEAR")); + assertThat(uf.sourceText(), is("EXTRACT(YEAR FROM \"foo\")")); } public void testFunctionWithFunctionWithArg() { Function f = function("POWER(foo, {fn POWER({fn SCORE()}, {fN SCORE()})})"); - assertEquals("POWER", f.functionName()); + assertEquals("{fn POWER(foo, {fn POWER({fn SCORE()}, {fN SCORE()})})}", f.sourceText()); assertEquals(2, f.arguments().size()); Expression arg = f.arguments().get(1); assertThat(arg, instanceOf(UnresolvedFunction.class)); UnresolvedFunction uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("POWER")); + assertThat(uf.sourceText(), is("{fn POWER({fn SCORE()}, {fN SCORE()})}")); assertEquals(2, uf.arguments().size()); List args = uf.arguments(); arg = args.get(0); assertThat(arg, instanceOf(UnresolvedFunction.class)); uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("SCORE")); + assertThat(uf.sourceText(), is("{fn SCORE()}")); arg = args.get(1); assertThat(arg, instanceOf(UnresolvedFunction.class)); uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("SCORE")); + assertThat(uf.sourceText(), is("{fN SCORE()}")); } public void testFunctionWithFunctionWithArgAndParams() { - Function f = (Function) parser.createExpression("POWER(?, {fn POWER({fn ABS(?)}, {fN ABS(?)})})", + String e = "POWER(?, {fn POWER({fn ABS(?)}, {fN ABS(?)})})"; + Function f = (Function) parser.createExpression(e, asList(new SqlTypedParamValue(DataType.LONG.esType, 1), new SqlTypedParamValue(DataType.LONG.esType, 1), new SqlTypedParamValue(DataType.LONG.esType, 1))); - assertEquals("POWER", f.functionName()); + assertEquals(e, f.sourceText()); assertEquals(2, f.arguments().size()); Expression arg = f.arguments().get(1); assertThat(arg, instanceOf(UnresolvedFunction.class)); UnresolvedFunction uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("POWER")); + assertEquals("{fn POWER({fn ABS(?)}, {fN ABS(?)})}", uf.sourceText()); assertEquals(2, uf.arguments().size()); List args = uf.arguments(); arg = args.get(0); assertThat(arg, instanceOf(UnresolvedFunction.class)); uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("ABS")); + assertThat(uf.sourceText(), is("{fn ABS(?)}")); arg = args.get(1); assertThat(arg, instanceOf(UnresolvedFunction.class)); uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("ABS")); + assertThat(uf.sourceText(), is("{fN ABS(?)}")); } public void testDateLiteral() { 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 e56bd4ecd11ac..2e34e947944d4 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 @@ -26,7 +26,6 @@ import java.util.Locale; import static java.lang.String.format; -import static org.hamcrest.core.StringStartsWith.startsWith; public class ExpressionTests extends ESTestCase { @@ -36,7 +35,7 @@ public void testTokenFunctionName() { Expression lt = parser.createExpression("LEFT()"); assertEquals(UnresolvedFunction.class, lt.getClass()); UnresolvedFunction uf = (UnresolvedFunction) lt; - assertEquals("LEFT", uf.functionName()); + assertEquals("LEFT()", uf.sourceText()); } public void testLiteralBoolean() { @@ -182,10 +181,10 @@ private TemporalAmount intervalOf(String query) { } public void testLiteralTimesLiteral() { - Expression expr = parser.createExpression("10*2"); + Expression expr = parser.createExpression("10 *2"); assertEquals(Mul.class, expr.getClass()); Mul mul = (Mul) expr; - assertEquals("10 * 2", mul.name()); + assertEquals("10 *2", mul.sourceText()); assertEquals(DataType.INTEGER, mul.dataType()); } @@ -193,39 +192,46 @@ public void testFunctionTimesLiteral() { Expression expr = parser.createExpression("PI()*2"); assertEquals(Mul.class, expr.getClass()); Mul mul = (Mul) expr; - assertEquals("(PI) * 2", mul.name()); + assertEquals("PI()*2", mul.sourceText()); + } + + public void testNegativeLiteral() { + Expression expr = parser.createExpression("- 6"); + assertEquals(Literal.class, expr.getClass()); + assertEquals("- 6", expr.sourceText()); } public void testComplexArithmetic() { - Expression expr = parser.createExpression("-(((a-2)-(-3))+b)"); + String sql = "-(((a-2)-(-3))+b)"; + Expression expr = parser.createExpression(sql); assertEquals(Neg.class, expr.getClass()); Neg neg = (Neg) expr; - assertThat(neg.name(), startsWith("-(((a) - 2) - -3) + (b)#")); + assertEquals(sql, neg.sourceText()); assertEquals(1, neg.children().size()); assertEquals(Add.class, neg.children().get(0).getClass()); Add add = (Add) neg.children().get(0); - assertEquals("(((a) - 2) - -3) + (b)", add.name()); + assertEquals("((a-2)-(-3))+b", add.sourceText()); assertEquals(2, add.children().size()); assertEquals("?b", add.children().get(1).toString()); assertEquals(Sub.class, add.children().get(0).getClass()); Sub sub1 = (Sub) add.children().get(0); - assertEquals("((a) - 2) - -3", sub1.name()); + assertEquals("(a-2)-(-3)", sub1.sourceText()); assertEquals(2, sub1.children().size()); assertEquals(Literal.class, sub1.children().get(1).getClass()); - assertEquals("-3", ((Literal) sub1.children().get(1)).name()); + assertEquals("-3", ((Literal) sub1.children().get(1)).sourceText()); assertEquals(Sub.class, sub1.children().get(0).getClass()); Sub sub2 = (Sub) sub1.children().get(0); assertEquals(2, sub2.children().size()); assertEquals("?a", sub2.children().get(0).toString()); assertEquals(Literal.class, sub2.children().get(1).getClass()); - assertEquals("2", ((Literal) sub2.children().get(1)).name()); + assertEquals("2", ((Literal) sub2.children().get(1)).sourceText()); } public void testEquals() { Expression expr = parser.createExpression("a = 10"); assertEquals(Equals.class, expr.getClass()); Equals eq = (Equals) expr; - assertEquals("(a) == 10", eq.name()); + assertEquals("a = 10", eq.sourceText()); assertEquals(2, eq.children().size()); } @@ -233,7 +239,7 @@ public void testNullEquals() { Expression expr = parser.createExpression("a <=> 10"); assertEquals(NullEquals.class, expr.getClass()); NullEquals nullEquals = (NullEquals) expr; - assertEquals("(a) <=> 10", nullEquals.name()); + assertEquals("a <=> 10", nullEquals.sourceText()); assertEquals(2, nullEquals.children().size()); } @@ -241,12 +247,12 @@ public void testNotEquals() { Expression expr = parser.createExpression("a != 10"); assertEquals(NotEquals.class, expr.getClass()); NotEquals neq = (NotEquals) expr; - assertEquals("(a) != 10", neq.name()); + assertEquals("a != 10", neq.sourceText()); assertEquals(2, neq.children().size()); } public void testCastWithUnquotedDataType() { - Expression expr = parser.createExpression("CAST(10*2 AS long)"); + Expression expr = parser.createExpression("CAST(10* 2 AS long)"); assertEquals(Cast.class, expr.getClass()); Cast cast = (Cast) expr; assertEquals(DataType.INTEGER, cast.from()); @@ -254,7 +260,7 @@ public void testCastWithUnquotedDataType() { assertEquals(DataType.LONG, cast.dataType()); assertEquals(Mul.class, cast.field().getClass()); Mul mul = (Mul) cast.field(); - assertEquals("10 * 2", mul.name()); + assertEquals("10* 2", mul.sourceText()); assertEquals(DataType.INTEGER, mul.dataType()); } @@ -267,7 +273,7 @@ public void testCastWithQuotedDataType() { assertEquals(DataType.LONG, cast.dataType()); assertEquals(Mul.class, cast.field().getClass()); Mul mul = (Mul) cast.field(); - assertEquals("10 * 2", mul.name()); + assertEquals("10*2", mul.sourceText()); assertEquals(DataType.INTEGER, mul.dataType()); } @@ -285,20 +291,22 @@ public void testConvertWithUnquotedDataType() { assertEquals(DataType.LONG, cast.dataType()); assertEquals(Mul.class, cast.field().getClass()); Mul mul = (Mul) cast.field(); - assertEquals("10 * 2", mul.name()); + assertEquals("10*2", mul.sourceText()); assertEquals(DataType.INTEGER, mul.dataType()); } public void testConvertWithQuotedDataType() { - Expression expr = parser.createExpression("CONVERT(10*2, \"LonG\")"); + String e = "CONVERT(10*2, \"LonG\")"; + Expression expr = parser.createExpression(e); assertEquals(Cast.class, expr.getClass()); Cast cast = (Cast) expr; + assertEquals(e, cast.sourceText()); assertEquals(DataType.INTEGER, cast.from()); assertEquals(DataType.LONG, cast.to()); assertEquals(DataType.LONG, cast.dataType()); assertEquals(Mul.class, cast.field().getClass()); Mul mul = (Mul) cast.field(); - assertEquals("10 * 2", mul.name()); + assertEquals("10*2", mul.sourceText()); assertEquals(DataType.INTEGER, mul.dataType()); } @@ -334,7 +342,7 @@ public void testCurrentTimestamp() { Expression expr = parser.createExpression("CURRENT_TIMESTAMP"); assertEquals(UnresolvedFunction.class, expr.getClass()); UnresolvedFunction ur = (UnresolvedFunction) expr; - assertEquals("CURRENT_TIMESTAMP", ur.name()); + assertEquals("CURRENT_TIMESTAMP", ur.sourceText()); assertEquals(0, ur.children().size()); } @@ -342,7 +350,7 @@ public void testCurrentTimestampPrecision() { Expression expr = parser.createExpression("CURRENT_TIMESTAMP(4)"); assertEquals(UnresolvedFunction.class, expr.getClass()); UnresolvedFunction ur = (UnresolvedFunction) expr; - assertEquals("CURRENT_TIMESTAMP", ur.name()); + assertEquals("CURRENT_TIMESTAMP(4)", ur.sourceText()); assertEquals(1, ur.children().size()); Expression child = ur.children().get(0); assertEquals(Literal.class, child.getClass()); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java index 199b4e119d81b..a29feff881637 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.sql.parser; import com.google.common.base.Joiner; + import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.sql.expression.NamedExpression; import org.elasticsearch.xpack.sql.expression.Order; @@ -15,6 +16,7 @@ import org.elasticsearch.xpack.sql.expression.predicate.fulltext.MatchQueryPredicate; import org.elasticsearch.xpack.sql.expression.predicate.fulltext.MultiMatchQueryPredicate; import org.elasticsearch.xpack.sql.expression.predicate.fulltext.StringQueryPredicate; +import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.Add; import org.elasticsearch.xpack.sql.expression.predicate.operator.comparison.In; import org.elasticsearch.xpack.sql.parser.SqlBaseParser.BooleanExpressionContext; import org.elasticsearch.xpack.sql.parser.SqlBaseParser.QueryPrimaryDefaultContext; @@ -60,12 +62,17 @@ public void testSelectField() { public void testSelectScore() { UnresolvedFunction f = singleProjection(project(parseStatement("SELECT SCORE() FROM foo")), UnresolvedFunction.class); - assertEquals("SCORE", f.functionName()); + assertEquals("SCORE()", f.sourceText()); + } + + public void testSelectAddWithParanthesis() { + Add f = singleProjection(project(parseStatement("SELECT (1 + 2)")), Add.class); + assertEquals("1 + 2", f.sourceText()); } public void testSelectRightFunction() { UnresolvedFunction f = singleProjection(project(parseStatement("SELECT RIGHT()")), UnresolvedFunction.class); - assertEquals("RIGHT", f.functionName()); + assertEquals("RIGHT()", f.sourceText()); } public void testsSelectNonReservedKeywords() { @@ -102,13 +109,13 @@ public void testOrderByField() { public void testOrderByScore() { Order.OrderDirection dir = randomFrom(Order.OrderDirection.values()); - OrderBy ob = orderBy(parseStatement("SELECT * FROM foo ORDER BY SCORE()" + stringForDirection(dir))); + OrderBy ob = orderBy(parseStatement("SELECT * FROM foo ORDER BY SCORE( )" + stringForDirection(dir))); assertThat(ob.order(), hasSize(1)); Order o = ob.order().get(0); assertEquals(dir, o.direction()); assertThat(o.child(), instanceOf(UnresolvedFunction.class)); UnresolvedFunction f = (UnresolvedFunction) o.child(); - assertEquals("SCORE", f.functionName()); + assertEquals("SCORE( )", f.sourceText()); } public void testOrderByTwo() { @@ -299,7 +306,7 @@ public void testLimitStackOverflowForInAndLiteralsIsNotApplied() { In in = (In) filter.condition(); assertEquals("?a", in.value().toString()); assertEquals(noChildren, in.list().size()); - assertThat(in.list().get(0).toString(), startsWith("(a) + (b)#")); + assertThat(in.list().get(0).toString(), startsWith("a + b#")); } public void testDecrementOfDepthCounter() { diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryFolderTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryFolderTests.java index c20f4e9d632af..214d935251994 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryFolderTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryFolderTests.java @@ -76,7 +76,7 @@ public void testLocalExecWithPrunedFilterWithFunction() { assertEquals(EmptyExecutable.class, le.executable().getClass()); EmptyExecutable ee = (EmptyExecutable) le.executable(); assertEquals(1, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("E{c}#")); + assertThat(ee.output().get(0).toString(), startsWith("E(){c}#")); } public void testLocalExecWithPrunedFilterWithFunctionAndAggregation() { @@ -86,7 +86,7 @@ public void testLocalExecWithPrunedFilterWithFunctionAndAggregation() { assertEquals(EmptyExecutable.class, le.executable().getClass()); EmptyExecutable ee = (EmptyExecutable) le.executable(); assertEquals(1, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("E{c}#")); + assertThat(ee.output().get(0).toString(), startsWith("E(){c}#")); } public void testLocalExecWithoutFromClause() { @@ -96,9 +96,9 @@ public void testLocalExecWithoutFromClause() { assertEquals(SingletonExecutable.class, le.executable().getClass()); SingletonExecutable ee = (SingletonExecutable) le.executable(); assertEquals(3, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("E{c}#")); - assertThat(ee.output().get(1).toString(), startsWith("foo{c}#")); - assertThat(ee.output().get(2).toString(), startsWith("ABS(10){c}#")); + assertThat(ee.output().get(0).toString(), startsWith("E(){c}#")); + assertThat(ee.output().get(1).toString(), startsWith("'foo'{c}#")); + assertThat(ee.output().get(2).toString(), startsWith("abs(10){c}#")); } public void testLocalExecWithoutFromClauseWithPrunedFilter() { @@ -108,7 +108,7 @@ public void testLocalExecWithoutFromClauseWithPrunedFilter() { assertEquals(EmptyExecutable.class, le.executable().getClass()); EmptyExecutable ee = (EmptyExecutable) le.executable(); assertEquals(1, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("E{c}#")); + assertThat(ee.output().get(0).toString(), startsWith("E(){c}#")); } public void testFoldingOfIsNull() { @@ -137,7 +137,7 @@ public void testFoldingToLocalExecBooleanAndNull_HavingClause() { EmptyExecutable ee = (EmptyExecutable) le.executable(); assertEquals(2, ee.output().size()); assertThat(ee.output().get(0).toString(), startsWith("keyword{f}#")); - assertThat(ee.output().get(1).toString(), startsWith("MAX(int){a->")); + assertThat(ee.output().get(1).toString(), startsWith("max(int){a->")); } public void testFoldingBooleanOrNull_WhereClause() { @@ -159,7 +159,7 @@ public void testFoldingBooleanOrNull_HavingClause() { "\"lang\":\"painless\",\"params\":{\"v0\":10}},")); assertEquals(2, ee.output().size()); assertThat(ee.output().get(0).toString(), startsWith("keyword{f}#")); - assertThat(ee.output().get(1).toString(), startsWith("MAX(int){a->")); + assertThat(ee.output().get(1).toString(), startsWith("max(int){a->")); } public void testFoldingOfIsNotNull() { @@ -208,7 +208,7 @@ public void testFoldingToLocalExecWithProjectWithGroupBy_WithOrderAndLimit() { EmptyExecutable ee = (EmptyExecutable) le.executable(); assertEquals(2, ee.output().size()); assertThat(ee.output().get(0).toString(), startsWith("keyword{f}#")); - assertThat(ee.output().get(1).toString(), startsWith("MAX(int){a->")); + assertThat(ee.output().get(1).toString(), startsWith("max(int){a->")); } public void testFoldingToLocalExecWithProjectWithGroupBy_WithHaving_WithOrderAndLimit() { @@ -219,7 +219,7 @@ public void testFoldingToLocalExecWithProjectWithGroupBy_WithHaving_WithOrderAnd EmptyExecutable ee = (EmptyExecutable) le.executable(); assertEquals(2, ee.output().size()); assertThat(ee.output().get(0).toString(), startsWith("keyword{f}#")); - assertThat(ee.output().get(1).toString(), startsWith("MAX(int){a->")); + assertThat(ee.output().get(1).toString(), startsWith("max(int){a->")); } public void testGroupKeyTypes_Boolean() { @@ -232,7 +232,7 @@ public void testGroupKeyTypes_Boolean() { "\"lang\":\"painless\",\"params\":{\"v0\":\"int\",\"v1\":10}},\"missing_bucket\":true," + "\"value_type\":\"boolean\",\"order\":\"asc\"}}}]}}}")); assertEquals(2, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("COUNT(1){a->")); + assertThat(ee.output().get(0).toString(), startsWith("count(*){a->")); assertThat(ee.output().get(1).toString(), startsWith("a{s->")); } @@ -246,7 +246,7 @@ public void testGroupKeyTypes_Integer() { "\"lang\":\"painless\",\"params\":{\"v0\":\"int\",\"v1\":10}},\"missing_bucket\":true," + "\"value_type\":\"long\",\"order\":\"asc\"}}}]}}}")); assertEquals(2, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("COUNT(1){a->")); + assertThat(ee.output().get(0).toString(), startsWith("count(*){a->")); assertThat(ee.output().get(1).toString(), startsWith("a{s->")); } @@ -260,7 +260,7 @@ public void testGroupKeyTypes_Rational() { "\"lang\":\"painless\",\"params\":{\"v0\":\"int\"}},\"missing_bucket\":true," + "\"value_type\":\"double\",\"order\":\"asc\"}}}]}}}")); assertEquals(2, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("COUNT(1){a->")); + assertThat(ee.output().get(0).toString(), startsWith("count(*){a->")); assertThat(ee.output().get(1).toString(), startsWith("a{s->")); } @@ -274,7 +274,7 @@ public void testGroupKeyTypes_String() { "\"lang\":\"painless\",\"params\":{\"v0\":\"keyword\"}},\"missing_bucket\":true," + "\"value_type\":\"string\",\"order\":\"asc\"}}}]}}}")); assertEquals(2, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("COUNT(1){a->")); + assertThat(ee.output().get(0).toString(), startsWith("count(*){a->")); assertThat(ee.output().get(1).toString(), startsWith("a{s->")); } @@ -288,7 +288,7 @@ public void testGroupKeyTypes_IP() { "\"lang\":\"painless\",\"params\":{\"v0\":\"keyword\",\"v1\":\"IP\"}}," + "\"missing_bucket\":true,\"value_type\":\"ip\",\"order\":\"asc\"}}}]}}}")); assertEquals(2, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("COUNT(1){a->")); + assertThat(ee.output().get(0).toString(), startsWith("count(*){a->")); assertThat(ee.output().get(1).toString(), startsWith("a{s->")); } @@ -303,7 +303,7 @@ public void testGroupKeyTypes_Date() { "\"v0\":\"date\",\"v1\":\"P1Y2M\",\"v2\":\"INTERVAL_YEAR_TO_MONTH\"}},\"missing_bucket\":true," + "\"value_type\":\"date\",\"order\":\"asc\"}}}]}}}")); assertEquals(2, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("COUNT(1){a->")); + assertThat(ee.output().get(0).toString(), startsWith("count(*){a->")); assertThat(ee.output().get(1).toString(), startsWith("a{s->")); } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java index 8ca1d64bd12fd..23beae2fd586e 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java @@ -277,7 +277,7 @@ public void testTranslateIsNullExpression_HavingClause_Painless() { AggFilter aggFilter = translation.aggFilter; assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.isNull(params.a0))", aggFilter.scriptTemplate().toString()); - assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=MAX(int){a->")); + assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=max(int){a->")); } public void testTranslateIsNotNullExpression_HavingClause_Painless() { @@ -290,7 +290,7 @@ public void testTranslateIsNotNullExpression_HavingClause_Painless() { AggFilter aggFilter = translation.aggFilter; assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.isNotNull(params.a0))", aggFilter.scriptTemplate().toString()); - assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=MAX(int){a->")); + assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=max(int){a->")); } public void testTranslateInExpression_WhereClause() { @@ -328,8 +328,10 @@ public void testTranslateInExpressionInvalidValues_WhereClause() { Expression condition = ((Filter) p.children().get(0)).condition(); assertFalse(condition.foldable()); SqlIllegalArgumentException ex = expectThrows(SqlIllegalArgumentException.class, () -> QueryTranslator.toQuery(condition, false)); - assertEquals("Line 1:52: Comparisons against variables are not (currently) supported; " + - "offender [keyword] in [keyword IN (foo, bar, keyword)]", ex.getMessage()); + assertEquals( + "Line 1:52: Comparisons against variables are not (currently) supported; " + + "offender [keyword] in [keyword IN ('foo', 'bar', keyword)]", + ex.getMessage()); } public void testTranslateInExpression_WhereClause_Painless() { @@ -358,7 +360,7 @@ public void testTranslateInExpression_HavingClause_Painless() { AggFilter aggFilter = translation.aggFilter; assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.in(params.a0, params.v0))", aggFilter.scriptTemplate().toString()); - assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=MAX(int){a->")); + assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=max(int){a->")); assertThat(aggFilter.scriptTemplate().params().toString(), endsWith(", {v=[10, 20]}]")); } @@ -372,7 +374,7 @@ public void testTranslateInExpression_HavingClause_PainlessOneArg() { AggFilter aggFilter = translation.aggFilter; assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.in(params.a0, params.v0))", aggFilter.scriptTemplate().toString()); - assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=MAX(int){a->")); + assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=max(int){a->")); assertThat(aggFilter.scriptTemplate().params().toString(), endsWith(", {v=[10]}]")); } @@ -387,7 +389,7 @@ public void testTranslateInExpression_HavingClause_PainlessAndNullHandling() { AggFilter aggFilter = translation.aggFilter; assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.in(params.a0, params.v0))", aggFilter.scriptTemplate().toString()); - assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=MAX(int){a->")); + assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=max(int){a->")); assertThat(aggFilter.scriptTemplate().params().toString(), endsWith(", {v=[10, null, 20, 30]}]")); } @@ -406,7 +408,7 @@ public void testTranslateMathFunction_HavingClause_Painless() { assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(InternalSqlScriptUtils." + operation.name().toLowerCase(Locale.ROOT) + "(params.a0),params.v0))", aggFilter.scriptTemplate().toString()); - assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=MAX(int){a->")); + assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=max(int){a->")); assertThat(aggFilter.scriptTemplate().params().toString(), endsWith(", {v=10}]")); } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/type/DataTypeConversionTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/type/DataTypeConversionTests.java index 4aa3cba3d7f18..a44ce44d0f904 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/type/DataTypeConversionTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/type/DataTypeConversionTests.java @@ -9,12 +9,13 @@ import org.elasticsearch.xpack.sql.SqlIllegalArgumentException; import org.elasticsearch.xpack.sql.TestUtils; import org.elasticsearch.xpack.sql.expression.Literal; +import org.elasticsearch.xpack.sql.tree.Location; +import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataTypeConversion.Conversion; import java.time.ZonedDateTime; import static org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTimeTestUtils.dateTime; -import static org.elasticsearch.xpack.sql.tree.Source.EMPTY; import static org.elasticsearch.xpack.sql.type.DataType.BOOLEAN; import static org.elasticsearch.xpack.sql.type.DataType.BYTE; import static org.elasticsearch.xpack.sql.type.DataType.DATE; @@ -291,9 +292,10 @@ public void testStringToIp() { } public void testIpToString() { + Source s = new Source(Location.EMPTY, "10.0.0.1"); Conversion ipToString = conversionFor(IP, KEYWORD); - assertEquals("10.0.0.1", ipToString.convert(new Literal(EMPTY, "10.0.0.1", IP))); + assertEquals("10.0.0.1", ipToString.convert(new Literal(s, "10.0.0.1", IP))); Conversion stringToIp = conversionFor(KEYWORD, IP); - assertEquals("10.0.0.1", ipToString.convert(stringToIp.convert(Literal.of(EMPTY, "10.0.0.1")))); + assertEquals("10.0.0.1", ipToString.convert(stringToIp.convert(Literal.of(s, "10.0.0.1")))); } } From 96cfa000a50a6bcee33298e2d2afd9e560684a8a Mon Sep 17 00:00:00 2001 From: Peter Dyson Date: Sun, 13 Jan 2019 16:24:34 +1000 Subject: [PATCH 21/28] [DOCS] copy_to only works one level deep, not recursively (#37249) --- docs/reference/mapping/params/copy-to.asciidoc | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/reference/mapping/params/copy-to.asciidoc b/docs/reference/mapping/params/copy-to.asciidoc index 447d7baf274f4..6a1aef62bc391 100644 --- a/docs/reference/mapping/params/copy-to.asciidoc +++ b/docs/reference/mapping/params/copy-to.asciidoc @@ -61,3 +61,7 @@ Some important points: * It is the field _value_ which is copied, not the terms (which result from the analysis process). * The original <> field will not be modified to show the copied values. * The same value can be copied to multiple fields, with `"copy_to": [ "field_1", "field_2" ]` +* You cannot copy recursively via intermediary fields such as a `copy_to` on +`field_1` to `field_2` and `copy_to` on `field_2` to `field_3` expecting +indexing into `field_1` will eventuate in `field_3`, instead use copy_to +directly to multiple fields from the originating field. \ No newline at end of file From 6fd57d90dad0df06aece223e5e8e672b8d0bc908 Mon Sep 17 00:00:00 2001 From: Albert Zaharovits Date: Sun, 13 Jan 2019 15:26:23 +0200 Subject: [PATCH 22/28] Security Audit includes HTTP method for requests (#37322) Adds another field, named "request.method", to the structured logfile audit. This field is present for all events associated with a REST request (not a transport request) and the value is one of GET, POST, PUT, DELETE, OPTIONS, HEAD, PATCH, TRACE and CONNECT. --- .../en/security/auditing/event-types.asciidoc | 3 +++ .../security/auditing/output-logfile.asciidoc | 3 +-- .../core/src/main/config/log4j2.properties | 2 ++ .../audit/logfile/LoggingAuditTrail.java | 18 ++++++++++-------- .../audit/logfile/LoggingAuditTrailTests.java | 7 +++++++ 5 files changed, 23 insertions(+), 10 deletions(-) diff --git a/x-pack/docs/en/security/auditing/event-types.asciidoc b/x-pack/docs/en/security/auditing/event-types.asciidoc index 442877f31b5d3..417b26cbd09b3 100644 --- a/x-pack/docs/en/security/auditing/event-types.asciidoc +++ b/x-pack/docs/en/security/auditing/event-types.asciidoc @@ -118,6 +118,9 @@ common ones): This is URL encoded. `url.query` :: The query part of the URL (after "?", if present) of the REST request associated with this event. This is URL encoded. +`request.method` :: The HTTP method of the REST request associated with this + event. It is one of GET, POST, PUT, DELETE, OPTIONS, + HEAD, PATCH, TRACE and CONNECT. `request.body` :: The full content of the REST request associated with this event, if enabled. This contains the query body. The body is escaped according to the JSON RFC 4627. diff --git a/x-pack/docs/en/security/auditing/output-logfile.asciidoc b/x-pack/docs/en/security/auditing/output-logfile.asciidoc index d9e7eb81c3f3a..382307e893f6d 100644 --- a/x-pack/docs/en/security/auditing/output-logfile.asciidoc +++ b/x-pack/docs/en/security/auditing/output-logfile.asciidoc @@ -18,8 +18,7 @@ For more information, see {ref}/logging.html#configuring-logging-levels[configur [[audit-log-entry-format]] === Log entry format -The log entries in the `_audit.log` file -have the following format: +The log entries in the `_audit.log` file have the following format: - Each log entry is a one line JSON document and each one is printed on a separate line. - The fields of a log entry are ordered. However, if a field does not have a value it diff --git a/x-pack/plugin/core/src/main/config/log4j2.properties b/x-pack/plugin/core/src/main/config/log4j2.properties index 21b0732fed418..52b6ce3950690 100644 --- a/x-pack/plugin/core/src/main/config/log4j2.properties +++ b/x-pack/plugin/core/src/main/config/log4j2.properties @@ -22,6 +22,7 @@ appender.audit_rolling.layout.pattern = {\ %varsNotEmpty{, "realm":"%enc{%map{realm}}{JSON}"}\ %varsNotEmpty{, "url.path":"%enc{%map{url.path}}{JSON}"}\ %varsNotEmpty{, "url.query":"%enc{%map{url.query}}{JSON}"}\ + %varsNotEmpty{, "request.method":"%enc{%map{request.method}}{JSON}"}\ %varsNotEmpty{, "request.body":"%enc{%map{request.body}}{JSON}"}\ %varsNotEmpty{, "request.id":"%enc{%map{request.id}}{JSON}"}\ %varsNotEmpty{, "action":"%enc{%map{action}}{JSON}"}\ @@ -51,6 +52,7 @@ appender.audit_rolling.layout.pattern = {\ # "realm" name of a realm that has generated an "authentication_failed" or an "authentication_successful"; the subject is not yet authenticated # "url.path" the URI component between the port and the query string; it is percent (URL) encoded # "url.query" the URI component after the path and before the fragment; it is percent (URL) encoded +# "request.method" the method of the HTTP request, i.e. one of GET, POST, PUT, DELETE, OPTIONS, HEAD, PATCH, TRACE, CONNECT # "request.body" the content of the request body entity, JSON escaped # "request.id" a synthentic identifier for the incoming request, this is unique per incoming request, and consistent across all audit events generated by that request # "action" an action is the most granular operation that is authorized and this identifies it in a namespaced way (internal) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrail.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrail.java index cdeee882c1bda..045140e331f28 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrail.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrail.java @@ -100,6 +100,7 @@ public class LoggingAuditTrail implements AuditTrail, ClusterStateListener { public static final String REALM_FIELD_NAME = "realm"; public static final String URL_PATH_FIELD_NAME = "url.path"; public static final String URL_QUERY_FIELD_NAME = "url.query"; + public static final String REQUEST_METHOD_FIELD_NAME = "request.method"; public static final String REQUEST_BODY_FIELD_NAME = "request.body"; public static final String REQUEST_ID_FIELD_NAME = "request.id"; public static final String ACTION_FIELD_NAME = "action"; @@ -211,7 +212,7 @@ public void authenticationSuccess(String requestId, String realm, User user, Res .with(EVENT_TYPE_FIELD_NAME, REST_ORIGIN_FIELD_VALUE) .with(EVENT_ACTION_FIELD_NAME, "authentication_success") .with(REALM_FIELD_NAME, realm) - .withRestUri(request) + .withRestUriAndMethod(request) .withRequestId(requestId) .withPrincipal(user) .withRestOrigin(request) @@ -276,7 +277,7 @@ public void anonymousAccessDenied(String requestId, RestRequest request) { final StringMapMessage logEntry = new LogEntryBuilder() .with(EVENT_TYPE_FIELD_NAME, REST_ORIGIN_FIELD_VALUE) .with(EVENT_ACTION_FIELD_NAME, "anonymous_access_denied") - .withRestUri(request) + .withRestUriAndMethod(request) .withRestOrigin(request) .withRequestBody(request) .withRequestId(requestId) @@ -316,7 +317,7 @@ public void authenticationFailed(String requestId, RestRequest request) { final StringMapMessage logEntry = new LogEntryBuilder() .with(EVENT_TYPE_FIELD_NAME, REST_ORIGIN_FIELD_VALUE) .with(EVENT_ACTION_FIELD_NAME, "authentication_failed") - .withRestUri(request) + .withRestUriAndMethod(request) .withRestOrigin(request) .withRequestBody(request) .withRequestId(requestId) @@ -357,7 +358,7 @@ public void authenticationFailed(String requestId, AuthenticationToken token, Re .with(EVENT_TYPE_FIELD_NAME, REST_ORIGIN_FIELD_VALUE) .with(EVENT_ACTION_FIELD_NAME, "authentication_failed") .with(PRINCIPAL_FIELD_NAME, token.principal()) - .withRestUri(request) + .withRestUriAndMethod(request) .withRestOrigin(request) .withRequestBody(request) .withRequestId(requestId) @@ -401,7 +402,7 @@ public void authenticationFailed(String requestId, String realm, AuthenticationT .with(EVENT_ACTION_FIELD_NAME, "realm_authentication_failed") .with(REALM_FIELD_NAME, realm) .with(PRINCIPAL_FIELD_NAME, token.principal()) - .withRestUri(request) + .withRestUriAndMethod(request) .withRestOrigin(request) .withRequestBody(request) .withRequestId(requestId) @@ -468,7 +469,7 @@ public void tamperedRequest(String requestId, RestRequest request) { final StringMapMessage logEntry = new LogEntryBuilder() .with(EVENT_TYPE_FIELD_NAME, REST_ORIGIN_FIELD_VALUE) .with(EVENT_ACTION_FIELD_NAME, "tampered_request") - .withRestUri(request) + .withRestUriAndMethod(request) .withRestOrigin(request) .withRequestBody(request) .withRequestId(requestId) @@ -617,7 +618,7 @@ public void runAsDenied(String requestId, Authentication authentication, RestReq .with(EVENT_TYPE_FIELD_NAME, REST_ORIGIN_FIELD_VALUE) .with(EVENT_ACTION_FIELD_NAME, "run_as_denied") .with(PRINCIPAL_ROLES_FIELD_NAME, roleNames) - .withRestUri(request) + .withRestUriAndMethod(request) .withRunAsSubject(authentication) .withRestOrigin(request) .withRequestBody(request) @@ -637,7 +638,7 @@ private class LogEntryBuilder { logEntry = new StringMapMessage(LoggingAuditTrail.this.entryCommonFields.commonFields); } - LogEntryBuilder withRestUri(RestRequest request) { + LogEntryBuilder withRestUriAndMethod(RestRequest request) { final int queryStringIndex = request.uri().indexOf('?'); int queryStringLength = request.uri().indexOf('#'); if (queryStringLength < 0) { @@ -651,6 +652,7 @@ LogEntryBuilder withRestUri(RestRequest request) { if (queryStringIndex > -1) { logEntry.with(URL_QUERY_FIELD_NAME, request.uri().substring(queryStringIndex + 1, queryStringLength)); } + logEntry.with(REQUEST_METHOD_FIELD_NAME, request.method().toString()); return this; } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java index da4823a3f3a69..817ed2a2358d0 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java @@ -245,6 +245,7 @@ public void testAnonymousAccessDeniedRest() throws Exception { .put(LoggingAuditTrail.EVENT_ACTION_FIELD_NAME, "anonymous_access_denied") .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) + .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) @@ -346,6 +347,7 @@ public void testAuthenticationFailedRest() throws Exception { .put(LoggingAuditTrail.PRINCIPAL_FIELD_NAME, mockToken.principal()) .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) + .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) @@ -386,6 +388,7 @@ public void testAuthenticationFailedRestNoToken() throws Exception { .put(LoggingAuditTrail.PRINCIPAL_FIELD_NAME, null) .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) + .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) @@ -468,6 +471,7 @@ public void testAuthenticationFailedRealmRest() throws Exception { .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) .put(LoggingAuditTrail.PRINCIPAL_FIELD_NAME, mockToken.principal()) .put(LoggingAuditTrail.ACTION_FIELD_NAME, null) + .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) @@ -627,6 +631,7 @@ public void testTamperedRequestRest() throws Exception { .put(LoggingAuditTrail.EVENT_ACTION_FIELD_NAME, "tampered_request") .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) + .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) @@ -891,6 +896,7 @@ public void testAuthenticationSuccessRest() throws Exception { .put(LoggingAuditTrail.REALM_FIELD_NAME, realm) .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) + .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) @@ -1080,6 +1086,7 @@ private Tuple prepareRestContent(String uri, InetSocke } builder.withRemoteAddress(remoteAddress); builder.withParams(params); + builder.withMethod(randomFrom(RestRequest.Method.values())); return new Tuple<>(content, builder.build()); } From a566bacbc8e263a95e0975ff2639e095d4a74af8 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Sun, 13 Jan 2019 19:33:39 +0200 Subject: [PATCH 23/28] Upgrade ASM for java 12 compatability (#37385) Closes #37371 --- x-pack/test/feature-aware/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/test/feature-aware/build.gradle b/x-pack/test/feature-aware/build.gradle index 11b0e67183c8f..7e7ed3ee3a862 100644 --- a/x-pack/test/feature-aware/build.gradle +++ b/x-pack/test/feature-aware/build.gradle @@ -1,7 +1,7 @@ apply plugin: 'elasticsearch.build' dependencies { - compile 'org.ow2.asm:asm:6.2' + compile 'org.ow2.asm:asm:7.0' compile "org.elasticsearch:elasticsearch:${version}" compile "org.elasticsearch.plugin:x-pack-core:${version}" testCompile "org.elasticsearch.test:framework:${version}" From d54f88f62c1ab9f0d25f90c49b8184a1e0eb4a65 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Mon, 14 Jan 2019 08:32:45 +0100 Subject: [PATCH 24/28] Remove unused empty constructors from suggestions classes (#37295) We recently migrated suggestions to `Writeable`. That allows us to also clean up empty constructors and methods that called them as they are no longer needed. They are replaced by constructors that accept a `StreamInput` instance. --- .../customsuggester/CustomSuggestion.java | 12 ------- .../elasticsearch/search/suggest/Suggest.java | 33 +++---------------- .../completion/CompletionSuggestion.java | 26 +++------------ .../suggest/phrase/PhraseSuggestion.java | 20 +---------- .../search/suggest/term/TermSuggestion.java | 22 +------------ .../search/SearchModuleTests.java | 7 +--- 6 files changed, 12 insertions(+), 108 deletions(-) diff --git a/plugins/examples/custom-suggester/src/main/java/org/elasticsearch/example/customsuggester/CustomSuggestion.java b/plugins/examples/custom-suggester/src/main/java/org/elasticsearch/example/customsuggester/CustomSuggestion.java index f7ec27b7af002..05c7880e9ec24 100644 --- a/plugins/examples/custom-suggester/src/main/java/org/elasticsearch/example/customsuggester/CustomSuggestion.java +++ b/plugins/examples/custom-suggester/src/main/java/org/elasticsearch/example/customsuggester/CustomSuggestion.java @@ -77,11 +77,6 @@ public String getDummy() { return dummy; } - @Override - protected Entry newEntry() { - return new Entry(); - } - @Override protected Entry newEntry(StreamInput in) throws IOException { return new Entry(in); @@ -123,11 +118,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(dummy); } - @Override - protected Option newOption() { - return new Option(); - } - @Override protected Option newOption(StreamInput in) throws IOException { return new Option(in); @@ -178,8 +168,6 @@ public static class Option extends Suggest.Suggestion.Entry.Option { private String dummy; - public Option() {} - public Option(Text text, float score, String dummy) { super(text, score); this.dummy = dummy; diff --git a/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java b/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java index b2f2a336d3684..8d78116e15a15 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java @@ -33,7 +33,6 @@ import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParserUtils; import org.elasticsearch.rest.action.search.RestSearchAction; @@ -46,7 +45,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; @@ -78,10 +76,6 @@ public class Suggest implements Iterable>> suggestMap; - private Suggest() { - this(Collections.emptyList()); - } - public Suggest(List>> suggestions) { // we sort suggestions by their names to ensure iteration over suggestions are consistent // this is needed as we need to fill in suggestion docs in SearchPhaseController#sortDocs @@ -259,16 +253,11 @@ public int hashCode() { */ public abstract static class Suggestion implements Iterable, NamedWriteable, ToXContentFragment { - private static final String NAME = "suggestion"; - public static final int TYPE = 0; - protected String name; - protected int size; + protected final String name; + protected final int size; protected final List entries = new ArrayList<>(5); - protected Suggestion() { - } - public Suggestion(String name, int size) { this.name = name; this.size = size; // The suggested term size specified in request, only used for merging shard responses @@ -380,7 +369,6 @@ public void trim() { } } - protected abstract T newEntry(); protected abstract T newEntry(StreamInput in) throws IOException; @Override @@ -608,7 +596,6 @@ public int hashCode() { return Objects.hash(text, offset, length, options); } - protected abstract O newOption(); protected abstract O newOption(StreamInput in) throws IOException; @Override @@ -653,8 +640,8 @@ public abstract static class Option implements Writeable, ToXContentFragment { public static final ParseField SCORE = new ParseField("score"); public static final ParseField COLLATE_MATCH = new ParseField("collate_match"); - private Text text; - private Text highlighted; + private final Text text; + private final Text highlighted; private float score; private Boolean collateMatch; @@ -673,8 +660,6 @@ public Option(Text text, float score) { this(text, null, score); } - public Option() {} - public Option(StreamInput in) throws IOException { text = in.readText(); score = in.readFloat(); @@ -776,14 +761,6 @@ public int hashCode() { @Override public String toString() { - try { - XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint(); - builder.startObject(); - toXContent(builder, EMPTY_PARAMS); - builder.endObject(); - return Strings.toString(builder); - } catch (IOException e) { - return "{ \"error\" : \"" + e.getMessage() + "\"}"; - } + return Strings.toString(this, true, true); } } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java index 5dea2d8c1a0ef..0fb7e4b3f43f2 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java @@ -71,11 +71,8 @@ public final class CompletionSuggestion extends Suggest.Suggestion> contexts = Collections.emptyMap(); - private ScoreDoc doc; + private final Map> contexts; + private final ScoreDoc doc; private SearchHit hit; public static final ParseField CONTEXTS = new ParseField("contexts"); @@ -292,10 +279,6 @@ public Option(int docID, Text text, float score, Map> contex this.contexts = Objects.requireNonNull(contexts, "context map cannot be null"); } - protected Option() { - super(); - } - public Option(StreamInput in) throws IOException { super(in); this.doc = Lucene.readScoreDoc(in); @@ -455,5 +438,4 @@ public String toString() { } } } - } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestion.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestion.java index 39932049dfd8f..a29c6f08a7d8b 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestion.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestion.java @@ -24,7 +24,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.text.Text; import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ContextParser; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.search.suggest.Suggest; @@ -44,8 +43,6 @@ public class PhraseSuggestion extends Suggest.Suggestion @Deprecated public static final int TYPE = 3; - public PhraseSuggestion() {} - public PhraseSuggestion(String name, int size) { super(name, size); } @@ -64,11 +61,6 @@ public int getWriteableType() { return TYPE; } - @Override - protected Entry newEntry() { - return new Entry(); - } - @Override protected Entry newEntry(StreamInput in) throws IOException { return new Entry(in); @@ -132,19 +124,13 @@ public void addOption(Option option) { static { declareCommonFields(PARSER); - PARSER.declareObjectArray(Entry::addOptions, (ContextParser) (p, c) -> Option.fromXContent(p), - new ParseField(OPTIONS)); + PARSER.declareObjectArray(Entry::addOptions, (p, c) -> Option.fromXContent(p), new ParseField(OPTIONS)); } public static Entry fromXContent(XContentParser parser) { return PARSER.apply(parser, null); } - @Override - protected Option newOption() { - return new Option(); - } - @Override protected Option newOption(StreamInput in) throws IOException { return new Option(in); @@ -169,10 +155,6 @@ public int hashCode() { public static class Option extends Suggestion.Entry.Option { - public Option() { - super(); - } - public Option(Text text, Text highlighted, float score, Boolean collateMatch) { super(text, highlighted, score, collateMatch); } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggestion.java b/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggestion.java index b8e334f92835c..a1ffdd5670940 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggestion.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggestion.java @@ -52,8 +52,6 @@ public class TermSuggestion extends Suggestion { private SortBy sort; - public TermSuggestion() {} - public TermSuggestion(String name, int size, SortBy sort) { super(name, size); this.sort = sort; @@ -61,7 +59,6 @@ public TermSuggestion(String name, int size, SortBy sort) { public TermSuggestion(StreamInput in) throws IOException { super(in); - if (in.getVersion().onOrAfter(Version.V_7_0_0)) { sort = SortBy.readFromStream(in); } @@ -80,7 +77,6 @@ public int compare(Suggestion.Entry.Option first, Suggestion.Entry.Option second } return FREQUENCY.compare(first, second); } - } // Same behaviour as comparators in suggest module, but for SuggestedWord @@ -105,7 +101,6 @@ public int compare(Suggestion.Entry.Option first, Suggestion.Entry.Option second // third criteria: term text return first.getText().compareTo(second.getText()); } - } @Override @@ -154,11 +149,6 @@ public static TermSuggestion fromXContent(XContentParser parser, String name) th return suggestion; } - @Override - protected Entry newEntry() { - return new Entry(); - } - @Override protected Entry newEntry(StreamInput in) throws IOException { return new Entry(in); @@ -184,17 +174,12 @@ public Entry(Text text, int offset, int length) { super(text, offset, length); } - public Entry() {} + private Entry() {} public Entry(StreamInput in) throws IOException { super(in); } - @Override - protected Option newOption() { - return new Option(); - } - @Override protected Option newOption(StreamInput in) throws IOException { return new Option(in); @@ -236,10 +221,6 @@ protected void mergeInto(Suggestion.Entry.Option otherOption) { freq += ((Option) otherOption).freq; } - protected Option() { - super(); - } - public void setFreq(int freq) { this.freq = freq; } @@ -283,6 +264,5 @@ public static Option fromXContent(XContentParser parser) { return PARSER.apply(parser, null); } } - } } diff --git a/server/src/test/java/org/elasticsearch/search/SearchModuleTests.java b/server/src/test/java/org/elasticsearch/search/SearchModuleTests.java index 321d50278b8be..f2d250fa1f80f 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchModuleTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchModuleTests.java @@ -44,11 +44,11 @@ import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicParser; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder; -import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.DerivativePipelineAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.DerivativePipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.InternalDerivative; import org.elasticsearch.search.aggregations.pipeline.MovAvgModel; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.SimpleModel; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; @@ -579,11 +579,6 @@ private static class TestSuggestion extends Suggestion { super(in); } - @Override - protected Entry newEntry() { - return null; - } - @Override protected Entry newEntry(StreamInput in) throws IOException { return null; From 374e24c7fd09dd80437138e6ca25803abd8469f3 Mon Sep 17 00:00:00 2001 From: Ioannis Kakavas Date: Mon, 14 Jan 2019 10:50:24 +0200 Subject: [PATCH 25/28] Mute SslNullCipherTests on JDK12 JDK12 doesn't support NULL cipher for TLS by default. This commit mutes these tests on JDK12 until we decide whether we need to keep or remove them --- .../xpack/security/transport/ssl/SslNullCipherTests.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslNullCipherTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslNullCipherTests.java index 7427c5a67e92d..f57be31632868 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslNullCipherTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslNullCipherTests.java @@ -7,6 +7,7 @@ import org.elasticsearch.action.DocWriteResponse.Result; import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.bootstrap.JavaVersion; import org.elasticsearch.client.Client; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.SecurityIntegTestCase; @@ -22,6 +23,11 @@ public static void muteInFips() { assumeFalse("Can't run in a FIPS JVM", inFipsJvm()); } + @BeforeClass + public static void muteInJDK12() { + assumeFalse("https://github.com/elastic/elasticsearch/issues/37403", JavaVersion.current().equals(JavaVersion.parse("12"))); + } + @Override public boolean transportSSLEnabled() { return true; From d44a6f9fbcc4f71116a5b6eb0573a786d67a58ae Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 14 Jan 2019 03:54:34 -0500 Subject: [PATCH 26/28] Simplify SyncedFlushService flow with StepListener (#37383) Today the SyncedFlushService flow is written with multiple nested callbacks which are hard to read. This commit replaces them with sequential step listeners. --- .../elasticsearch/action/StepListener.java | 3 +- .../indices/flush/SyncedFlushService.java | 81 ++++++++----------- 2 files changed, 35 insertions(+), 49 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/StepListener.java b/server/src/main/java/org/elasticsearch/action/StepListener.java index efbf8c755d57c..160ba23da246f 100644 --- a/server/src/main/java/org/elasticsearch/action/StepListener.java +++ b/server/src/main/java/org/elasticsearch/action/StepListener.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.common.util.concurrent.ListenableFuture; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; /** @@ -84,6 +85,6 @@ public Response result() { if (delegate.isDone() == false) { throw new IllegalStateException("step is not completed yet"); } - return FutureUtils.get(delegate); + return FutureUtils.get(delegate, 0L, TimeUnit.NANOSECONDS); // this future is done already - use a non-blocking method. } } diff --git a/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java b/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java index e1cd85faaef5d..9bc4e4ead1269 100644 --- a/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java +++ b/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java @@ -24,6 +24,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.StepListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.SyncedFlushResponse; import org.elasticsearch.action.support.IndicesOptions; @@ -219,56 +220,40 @@ private void innerAttemptSyncedFlush(final ShardId shardId, return; } - final ActionListener> presyncListener = - new ActionListener>() { - @Override - public void onResponse(final Map presyncResponses) { - if (presyncResponses.isEmpty()) { - actionListener.onResponse(new ShardsSyncedFlushResult(shardId, totalShards, - "all shards failed to commit on pre-sync")); - return; - } - final ActionListener inflightOpsListener = new ActionListener() { - @Override - public void onResponse(InFlightOpsResponse response) { - final int inflight = response.opCount(); - assert inflight >= 0; - if (inflight != 0) { - actionListener.onResponse(new ShardsSyncedFlushResult(shardId, totalShards, "[" + inflight + - "] ongoing operations on primary")); - } else { - // 3. now send the sync request to all the shards; - final String sharedSyncId = sharedExistingSyncId(presyncResponses); - if (sharedSyncId != null) { - assert presyncResponses.values().stream().allMatch(r -> r.existingSyncId.equals(sharedSyncId)) : - "Not all shards have the same existing sync id [" + sharedSyncId + "], responses [" + - presyncResponses + "]"; - reportSuccessWithExistingSyncId(shardId, sharedSyncId, activeShards, totalShards, - presyncResponses, actionListener); - }else { - String syncId = UUIDs.randomBase64UUID(); - sendSyncRequests(syncId, activeShards, state, presyncResponses, shardId, totalShards, actionListener); - } - } - } - - @Override - public void onFailure(Exception e) { - actionListener.onFailure(e); - } - }; - // 2. fetch in flight operations - getInflightOpsCount(shardId, state, shardRoutingTable, inflightOpsListener); + // 1. send pre-sync flushes to all replicas + final StepListener> presyncStep = new StepListener<>(); + sendPreSyncRequests(activeShards, state, shardId, presyncStep); + + // 2. fetch in flight operations + final StepListener inflightOpsStep = new StepListener<>(); + presyncStep.whenComplete(presyncResponses -> { + if (presyncResponses.isEmpty()) { + actionListener.onResponse(new ShardsSyncedFlushResult(shardId, totalShards, "all shards failed to commit on pre-sync")); + } else { + getInflightOpsCount(shardId, state, shardRoutingTable, inflightOpsStep); } - - @Override - public void onFailure(Exception e) { - actionListener.onFailure(e); + }, actionListener::onFailure); + + // 3. now send the sync request to all the shards + inflightOpsStep.whenComplete(inFlightOpsResponse -> { + final Map presyncResponses = presyncStep.result(); + final int inflight = inFlightOpsResponse.opCount(); + assert inflight >= 0; + if (inflight != 0) { + actionListener.onResponse( + new ShardsSyncedFlushResult(shardId, totalShards, "[" + inflight + "] ongoing operations on primary")); + } else { + final String sharedSyncId = sharedExistingSyncId(presyncResponses); + if (sharedSyncId != null) { + assert presyncResponses.values().stream().allMatch(r -> r.existingSyncId.equals(sharedSyncId)) : + "Not all shards have the same existing sync id [" + sharedSyncId + "], responses [" + presyncResponses + "]"; + reportSuccessWithExistingSyncId(shardId, sharedSyncId, activeShards, totalShards, presyncResponses, actionListener); + }else { + String syncId = UUIDs.randomBase64UUID(); + sendSyncRequests(syncId, activeShards, state, presyncResponses, shardId, totalShards, actionListener); + } } - }; - - // 1. send pre-sync flushes to all replicas - sendPreSyncRequests(activeShards, state, shardId, presyncListener); + }, actionListener::onFailure); } catch (Exception e) { actionListener.onFailure(e); } From 89b45f1fc6ed7bb46df5071f7ad34d80e6e87761 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Mon, 14 Jan 2019 11:14:38 +0100 Subject: [PATCH 27/28] Remove deprecated pipeline request contructors (#37366) The constructors in PutPipelineRequest and SimulatePipelineRequest that guess the xContent type from the provided source are deprecated since 6.0 and each have a counterpart that takes the xContent type as an explicit argument. Removing these ctors together with the builders and methods in ClusterAdminClient that don't have the xContent type as argument. --- .../action/ingest/PutPipelineRequest.java | 10 ---------- .../action/ingest/PutPipelineRequestBuilder.java | 5 ----- .../action/ingest/SimulatePipelineRequest.java | 13 +------------ .../ingest/SimulatePipelineRequestBuilder.java | 10 ---------- .../elasticsearch/client/ClusterAdminClient.java | 13 ------------- .../client/support/AbstractClient.java | 10 ---------- .../elasticsearch/common/io/UTF8StreamWriter.java | 8 -------- 7 files changed, 1 insertion(+), 68 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java index abff28bcf553c..46a34717c89c0 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import java.io.IOException; @@ -38,15 +37,6 @@ public class PutPipelineRequest extends AcknowledgedRequest private BytesReference source; private XContentType xContentType; - /** - * Create a new pipeline request - * @deprecated use {@link #PutPipelineRequest(String, BytesReference, XContentType)} to avoid content type auto-detection - */ - @Deprecated - public PutPipelineRequest(String id, BytesReference source) { - this(id, source, XContentHelper.xContentType(source)); - } - /** * Create a new pipeline request with the id and source along with the content type of the source */ diff --git a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequestBuilder.java index 1919d98c7e1f9..aa2aff8bb9915 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequestBuilder.java @@ -31,11 +31,6 @@ public PutPipelineRequestBuilder(ElasticsearchClient client, PutPipelineAction a super(client, action, new PutPipelineRequest()); } - @Deprecated - public PutPipelineRequestBuilder(ElasticsearchClient client, PutPipelineAction action, String id, BytesReference source) { - super(client, action, new PutPipelineRequest(id, source)); - } - public PutPipelineRequestBuilder(ElasticsearchClient client, PutPipelineAction action, String id, BytesReference source, XContentType xContentType) { super(client, action, new PutPipelineRequest(id, source, xContentType)); diff --git a/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java b/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java index 7514a41f5756b..4c2736e3d86de 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java @@ -26,11 +26,11 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.VersionType; import org.elasticsearch.ingest.ConfigurationUtils; import org.elasticsearch.ingest.IngestDocument; +import org.elasticsearch.ingest.IngestDocument.MetaData; import org.elasticsearch.ingest.IngestService; import org.elasticsearch.ingest.Pipeline; @@ -41,8 +41,6 @@ import java.util.Map; import java.util.Objects; -import static org.elasticsearch.ingest.IngestDocument.MetaData; - public class SimulatePipelineRequest extends ActionRequest implements ToXContentObject { private String id; @@ -50,15 +48,6 @@ public class SimulatePipelineRequest extends ActionRequest implements ToXContent private BytesReference source; private XContentType xContentType; - /** - * Create a new request - * @deprecated use {@link #SimulatePipelineRequest(BytesReference, XContentType)} that does not attempt content autodetection - */ - @Deprecated - public SimulatePipelineRequest(BytesReference source) { - this(source, XContentHelper.xContentType(source)); - } - /** * Creates a new request with the given source and its content type */ diff --git a/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequestBuilder.java index 7b7d31603aa6b..1a5e863bd55ee 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequestBuilder.java @@ -33,16 +33,6 @@ public SimulatePipelineRequestBuilder(ElasticsearchClient client, SimulatePipeli super(client, action, new SimulatePipelineRequest()); } - /** - * Create a new builder for {@link SimulatePipelineRequest}s - * @deprecated use {@link #SimulatePipelineRequestBuilder(ElasticsearchClient, SimulatePipelineAction, BytesReference, XContentType)} to - * avoid content type auto-detection on the source bytes - */ - @Deprecated - public SimulatePipelineRequestBuilder(ElasticsearchClient client, SimulatePipelineAction action, BytesReference source) { - super(client, action, new SimulatePipelineRequest(source)); - } - /** * Create a new builder for {@link SimulatePipelineRequest}s */ diff --git a/server/src/main/java/org/elasticsearch/client/ClusterAdminClient.java b/server/src/main/java/org/elasticsearch/client/ClusterAdminClient.java index 624c79996371a..42aaed10d6172 100644 --- a/server/src/main/java/org/elasticsearch/client/ClusterAdminClient.java +++ b/server/src/main/java/org/elasticsearch/client/ClusterAdminClient.java @@ -576,13 +576,6 @@ public interface ClusterAdminClient extends ElasticsearchClient { */ ActionFuture putPipeline(PutPipelineRequest request); - /** - * Stores an ingest pipeline - * @deprecated use {@link #preparePutPipeline(String, BytesReference, XContentType)} - */ - @Deprecated - PutPipelineRequestBuilder preparePutPipeline(String id, BytesReference source); - /** * Stores an ingest pipeline */ @@ -633,12 +626,6 @@ public interface ClusterAdminClient extends ElasticsearchClient { */ ActionFuture simulatePipeline(SimulatePipelineRequest request); - /** - * Simulates an ingest pipeline - */ - @Deprecated - SimulatePipelineRequestBuilder prepareSimulatePipeline(BytesReference source); - /** * Simulates an ingest pipeline */ diff --git a/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java b/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java index d642101e1c3e9..3fc931a85c0f7 100644 --- a/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java +++ b/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java @@ -1066,11 +1066,6 @@ public ActionFuture putPipeline(PutPipelineRequest request return execute(PutPipelineAction.INSTANCE, request); } - @Override - public PutPipelineRequestBuilder preparePutPipeline(String id, BytesReference source) { - return new PutPipelineRequestBuilder(this, PutPipelineAction.INSTANCE, id, source); - } - @Override public PutPipelineRequestBuilder preparePutPipeline(String id, BytesReference source, XContentType xContentType) { return new PutPipelineRequestBuilder(this, PutPipelineAction.INSTANCE, id, source, xContentType); @@ -1121,11 +1116,6 @@ public ActionFuture simulatePipeline(SimulatePipelineR return execute(SimulatePipelineAction.INSTANCE, request); } - @Override - public SimulatePipelineRequestBuilder prepareSimulatePipeline(BytesReference source) { - return new SimulatePipelineRequestBuilder(this, SimulatePipelineAction.INSTANCE, source); - } - @Override public SimulatePipelineRequestBuilder prepareSimulatePipeline(BytesReference source, XContentType xContentType) { return new SimulatePipelineRequestBuilder(this, SimulatePipelineAction.INSTANCE, source, xContentType); diff --git a/server/src/main/java/org/elasticsearch/common/io/UTF8StreamWriter.java b/server/src/main/java/org/elasticsearch/common/io/UTF8StreamWriter.java index 0ed5fd498de91..cb038d12fc2ec 100644 --- a/server/src/main/java/org/elasticsearch/common/io/UTF8StreamWriter.java +++ b/server/src/main/java/org/elasticsearch/common/io/UTF8StreamWriter.java @@ -326,12 +326,4 @@ public void reset() { _index = 0; _outputStream = null; } - - /** - * @deprecated Replaced by {@link #setOutput(OutputStream)} - */ - @Deprecated - public UTF8StreamWriter setOutputStream(OutputStream out) { - return this.setOutput(out); - } } \ No newline at end of file From 4b1368183d741bb1f5818d24b638eb38e0954d24 Mon Sep 17 00:00:00 2001 From: Jiyu-Zhang-Zendesk <36034626+Jiyu-Zhang-Zendesk@users.noreply.github.com> Date: Mon, 14 Jan 2019 11:20:25 +0100 Subject: [PATCH 28/28] Update analysis.asciidoc (#37404) STConvert plugin is made by Medcl to convert between Simplified Chinese and Traditional Chinese. It's widely used by the Search Community for Chinese --- docs/plugins/analysis.asciidoc | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/plugins/analysis.asciidoc b/docs/plugins/analysis.asciidoc index 875c87124ef45..0a0bbf090495f 100644 --- a/docs/plugins/analysis.asciidoc +++ b/docs/plugins/analysis.asciidoc @@ -58,6 +58,7 @@ A number of analysis plugins have been contributed by our community: * https://github.com/ofir123/elasticsearch-network-analysis[Network Addresses Analysis Plugin] (by Ofir123) * https://github.com/medcl/elasticsearch-analysis-string2int[String2Integer Analysis Plugin] (by Medcl) * https://github.com/ZarHenry96/elasticsearch-dandelion-plugin[Dandelion Analysis Plugin] (by ZarHenry96) +* https://github.com/medcl/elasticsearch-analysis-stconvert[STConvert Analysis Plugin] (by Medcl) include::analysis-icu.asciidoc[]