From 35a59229274fdc59c909cf058229f0988bd683d5 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 16 Nov 2017 09:22:40 -0700 Subject: [PATCH 01/31] Delete unneeded nio client (#27408) This is a follow up to #27132. As that PR greatly simplified the connection logic inside a low level transport implementation, much of the functionality provided by the NioClient class is no longer necessary. This commit removes that class. --- .../transport/nio/NioClient.java | 62 -------------- .../transport/nio/NioShutdown.java | 8 +- .../transport/nio/NioTransport.java | 20 ++--- .../transport/nio/NioClientTests.java | 81 ------------------- 4 files changed, 8 insertions(+), 163 deletions(-) delete mode 100644 test/framework/src/main/java/org/elasticsearch/transport/nio/NioClient.java delete mode 100644 test/framework/src/test/java/org/elasticsearch/transport/nio/NioClientTests.java diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioClient.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioClient.java deleted file mode 100644 index 74a9eb46a23c8..0000000000000 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioClient.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.transport.nio; - -import org.elasticsearch.transport.nio.channel.ChannelFactory; -import org.elasticsearch.transport.nio.channel.NioSocketChannel; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.concurrent.Semaphore; -import java.util.function.Supplier; - -public class NioClient { - - private final OpenChannels openChannels; - private final Supplier selectorSupplier; - private final ChannelFactory channelFactory; - private final Semaphore semaphore = new Semaphore(Integer.MAX_VALUE); - - NioClient(OpenChannels openChannels, Supplier selectorSupplier, ChannelFactory channelFactory) { - this.openChannels = openChannels; - this.selectorSupplier = selectorSupplier; - this.channelFactory = channelFactory; - } - - public void close() { - semaphore.acquireUninterruptibly(Integer.MAX_VALUE); - } - - NioSocketChannel initiateConnection(InetSocketAddress address) throws IOException { - boolean allowedToConnect = semaphore.tryAcquire(); - if (allowedToConnect == false) { - return null; - } - - try { - SocketSelector selector = selectorSupplier.get(); - NioSocketChannel nioSocketChannel = channelFactory.openNioChannel(address, selector); - openChannels.clientChannelOpened(nioSocketChannel); - return nioSocketChannel; - } finally { - semaphore.release(); - } - } -} diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioShutdown.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioShutdown.java index 8dc87f80f8aa3..320da6a88d186 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioShutdown.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioShutdown.java @@ -34,13 +34,7 @@ public NioShutdown(Logger logger) { this.logger = logger; } - void orderlyShutdown(OpenChannels openChannels, NioClient client, ArrayList acceptors, - ArrayList socketSelectors) { - // Close the client. This ensures that no new send connections will be opened. Client could be null if exception was - // throw on start up - if (client != null) { - client.close(); - } + void orderlyShutdown(OpenChannels openChannels, ArrayList acceptors, ArrayList socketSelectors) { // Start by closing the server channels. Once these are closed, we are guaranteed to no accept new connections openChannels.closeServerChannels(); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java index 381f2841136e9..fc8d361b82ef5 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java @@ -72,7 +72,8 @@ public class NioTransport extends TcpTransport { private final ConcurrentMap profileToChannelFactory = newConcurrentMap(); private final ArrayList acceptors = new ArrayList<>(); private final ArrayList socketSelectors = new ArrayList<>(); - private NioClient client; + private RoundRobinSelectorSupplier clientSelectorSupplier; + private ChannelFactory clientChannelFactory; private int acceptorNumber; public NioTransport(Settings settings, ThreadPool threadPool, NetworkService networkService, BigArrays bigArrays, @@ -111,10 +112,8 @@ protected void sendMessage(NioChannel channel, BytesReference reference, ActionL @Override protected NioChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) throws IOException { - NioSocketChannel channel = client.initiateConnection(node.getAddress().address()); - if (channel == null) { - throw new ElasticsearchException("client is shutdown"); - } + NioSocketChannel channel = clientChannelFactory.openNioChannel(node.getAddress().address(), clientSelectorSupplier.get()); + openChannels.clientChannelOpened(channel); channel.addConnectListener(connectListener); return channel; } @@ -137,7 +136,8 @@ protected void doStart() { } } - client = createClient(); + clientSelectorSupplier = new RoundRobinSelectorSupplier(socketSelectors); + clientChannelFactory = new ChannelFactory(new ProfileSettings(settings, "default"), contextSetter); if (NetworkService.NETWORK_SERVER.get(settings)) { int acceptorCount = NioTransport.NIO_ACCEPTOR_COUNT.get(settings); @@ -178,7 +178,7 @@ protected void doStart() { @Override protected void stopInternal() { NioShutdown nioShutdown = new NioShutdown(logger); - nioShutdown.orderlyShutdown(openChannels, client, acceptors, socketSelectors); + nioShutdown.orderlyShutdown(openChannels, acceptors, socketSelectors); profileToChannelFactory.clear(); socketSelectors.clear(); @@ -193,10 +193,4 @@ final void exceptionCaught(NioSocketChannel channel, Throwable cause) { final Throwable t = unwrapped != null ? unwrapped : cause; onException(channel, t instanceof Exception ? (Exception) t : new ElasticsearchException(t)); } - - private NioClient createClient() { - Supplier selectorSupplier = new RoundRobinSelectorSupplier(socketSelectors); - ChannelFactory channelFactory = new ChannelFactory(new ProfileSettings(settings, "default"), contextSetter); - return new NioClient(openChannels, selectorSupplier, channelFactory); - } } diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/NioClientTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/NioClientTests.java deleted file mode 100644 index 6b376af066474..0000000000000 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/NioClientTests.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.transport.nio; - -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.transport.nio.channel.ChannelFactory; -import org.elasticsearch.transport.nio.channel.NioSocketChannel; -import org.junit.Before; - -import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.function.Supplier; - -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public class NioClientTests extends ESTestCase { - - private NioClient client; - private SocketSelector selector; - private ChannelFactory channelFactory; - private OpenChannels openChannels = new OpenChannels(logger); - private InetSocketAddress address; - - @Before - @SuppressWarnings("unchecked") - public void setUpClient() { - channelFactory = mock(ChannelFactory.class); - selector = mock(SocketSelector.class); - ArrayList selectors = new ArrayList<>(); - selectors.add(selector); - Supplier selectorSupplier = new RoundRobinSelectorSupplier(selectors); - client = new NioClient(openChannels, selectorSupplier, channelFactory); - address = new InetSocketAddress(InetAddress.getLoopbackAddress(), 0); - } - - public void testCreateConnection() throws IOException, InterruptedException { - NioSocketChannel channel1 = mock(NioSocketChannel.class); - - when(channelFactory.openNioChannel(eq(address), eq(selector))).thenReturn(channel1); - - NioSocketChannel nioSocketChannel = client.initiateConnection(address); - - assertEquals(channel1, nioSocketChannel); - } - - public void testConnectionException() throws IOException, InterruptedException { - IOException ioException = new IOException(); - - when(channelFactory.openNioChannel(eq(address), eq(selector))).thenThrow(ioException); - - expectThrows(IOException.class, () -> client.initiateConnection(address)); - } - - public void testCloseDoesNotAllowConnections() throws IOException { - client.close(); - - assertNull(client.initiateConnection(address)); - } -} From 80ef9bbdb1cb42583eff3180e5565b8cf1305064 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 16 Nov 2017 11:19:36 -0700 Subject: [PATCH 02/31] Remove parameterization from TcpTransport (#27407) This commit is a follow up to the work completed in #27132. Essentially it transitions two more methods (sendMessage and getLocalAddress) from Transport to TcpChannel. With this change, there is no longer a need for TcpTransport to be aware of the specific type of channel a transport returns. So that class is no longer parameterized by channel type. --- .../elasticsearch/transport/TcpChannel.java | 22 ++- .../elasticsearch/transport/TcpTransport.java | 126 ++++++++---------- .../transport/TcpTransportChannel.java | 10 +- .../transport/TcpTransportTests.java | 90 +++++++------ .../transport/netty4/Netty4Transport.java | 28 +--- .../transport/netty4/NettyTcpChannel.java | 33 ++++- .../transport/netty4/Netty4TransportIT.java | 4 +- .../netty4/NettyTransportMultiPortTests.java | 11 +- .../netty4/SimpleNetty4TransportTests.java | 4 +- .../AbstractSimpleTransportTestCase.java | 2 +- .../transport/MockTcpTransport.java | 44 +++--- .../transport/nio/NioTransport.java | 34 +++-- .../nio/channel/NioServerSocketChannel.java | 8 ++ .../nio/channel/NioSocketChannel.java | 18 +++ .../transport/MockTcpTransportTests.java | 6 +- .../nio/SimpleNioTransportTests.java | 5 +- 16 files changed, 239 insertions(+), 206 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/transport/TcpChannel.java b/core/src/main/java/org/elasticsearch/transport/TcpChannel.java index f429e71f4a874..ee2be4ed736c9 100644 --- a/core/src/main/java/org/elasticsearch/transport/TcpChannel.java +++ b/core/src/main/java/org/elasticsearch/transport/TcpChannel.java @@ -19,19 +19,19 @@ package org.elasticsearch.transport; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.unit.TimeValue; import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.SocketAddress; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.concurrent.ExecutionException; @@ -80,6 +80,22 @@ public interface TcpChannel extends Releasable { */ boolean isOpen(); + /** + * Returns the local address for this channel. + * + * @return the local address of this channel. + */ + InetSocketAddress getLocalAddress(); + + /** + * Sends a tcp message to the channel. The listener will be executed once the send process has been + * completed. + * + * @param reference to send to channel + * @param listener to execute upon send completion + */ + void sendMessage(BytesReference reference, ActionListener listener); + /** * Closes the channel. * diff --git a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java index 4092eb6256988..d5aed81ae805b 100644 --- a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -118,7 +118,7 @@ import static org.elasticsearch.common.transport.NetworkExceptionHelper.isConnectException; import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; -public abstract class TcpTransport extends AbstractLifecycleComponent implements Transport { +public abstract class TcpTransport extends AbstractLifecycleComponent implements Transport { public static final String TRANSPORT_SERVER_WORKER_THREAD_NAME_PREFIX = "transport_server_worker"; public static final String TRANSPORT_CLIENT_BOSS_THREAD_NAME_PREFIX = "transport_client_boss"; @@ -199,8 +199,8 @@ public abstract class TcpTransport extends AbstractL protected final ConcurrentMap connectedNodes = newConcurrentMap(); protected final ConcurrentMap profileBoundAddresses = newConcurrentMap(); - private final Map> serverChannels = newConcurrentMap(); - private final Set acceptedChannels = Collections.newSetFromMap(new ConcurrentHashMap<>()); + private final Map> serverChannels = newConcurrentMap(); + private final Set acceptedChannels = Collections.newSetFromMap(new ConcurrentHashMap<>()); protected final KeyedLock connectionLock = new KeyedLock<>(); private final NamedWriteableRegistry namedWriteableRegistry; @@ -340,10 +340,10 @@ protected void doRunInLifecycle() throws Exception { for (Map.Entry entry : connectedNodes.entrySet()) { DiscoveryNode node = entry.getKey(); NodeChannels channels = entry.getValue(); - for (Channel channel : channels.getChannels()) { - internalSendMessage(channel, pingHeader, new SendMetricListener(pingHeader.length()) { + for (TcpChannel channel : channels.getChannels()) { + internalSendMessage(channel, pingHeader, new SendMetricListener(pingHeader.length()) { @Override - protected void innerInnerOnResponse(Channel channel) { + protected void innerInnerOnResponse(TcpChannel channel) { successfulPings.inc(); } @@ -397,12 +397,12 @@ public void onFailure(Exception e) { public final class NodeChannels implements Connection { private final Map typeMapping; - private final List channels; + private final List channels; private final DiscoveryNode node; private final AtomicBoolean closed = new AtomicBoolean(false); private final Version version; - NodeChannels(DiscoveryNode node, List channels, ConnectionProfile connectionProfile, Version handshakeVersion) { + NodeChannels(DiscoveryNode node, List channels, ConnectionProfile connectionProfile, Version handshakeVersion) { this.node = node; this.channels = Collections.unmodifiableList(channels); assert channels.size() == connectionProfile.getNumConnections() : "expected channels size to be == " @@ -420,11 +420,11 @@ public Version getVersion() { return version; } - public List getChannels() { + public List getChannels() { return channels; } - public Channel channel(TransportRequestOptions.Type type) { + public TcpChannel channel(TransportRequestOptions.Type type) { ConnectionProfile.ConnectionTypeHandle connectionTypeHandle = typeMapping.get(type); if (connectionTypeHandle == null) { throw new IllegalArgumentException("no type channel for [" + type + "]"); @@ -477,7 +477,7 @@ public void sendRequest(long requestId, String action, TransportRequest request, if (closed.get()) { throw new NodeNotConnectedException(node, "connection already closed"); } - Channel channel = channel(options.type()); + TcpChannel channel = channel(options.type()); sendRequestToChannel(this.node, channel, requestId, action, request, options, getVersion(), (byte) 0); } @@ -594,13 +594,13 @@ public final NodeChannels openConnection(DiscoveryNode node, ConnectionProfile c try { int numConnections = connectionProfile.getNumConnections(); assert numConnections > 0 : "A connection profile must be configured with at least one connection"; - List channels = new ArrayList<>(numConnections); - List> connectionFutures = new ArrayList<>(numConnections); + List channels = new ArrayList<>(numConnections); + List> connectionFutures = new ArrayList<>(numConnections); for (int i = 0; i < numConnections; ++i) { try { - PlainActionFuture connectFuture = PlainActionFuture.newFuture(); + PlainActionFuture connectFuture = PlainActionFuture.newFuture(); connectionFutures.add(connectFuture); - Channel channel = initiateChannel(node, connectionProfile.getConnectTimeout(), connectFuture); + TcpChannel channel = initiateChannel(node, connectionProfile.getConnectTimeout(), connectFuture); channels.add(channel); } catch (Exception e) { // If there was an exception when attempting to instantiate the raw channels, we close all of the channels @@ -618,7 +618,7 @@ public final NodeChannels openConnection(DiscoveryNode node, ConnectionProfile c } // If we make it past the block above, we have successfully established connections for all of the channels - final Channel handshakeChannel = channels.get(0); // one channel is guaranteed by the connection profile + final TcpChannel handshakeChannel = channels.get(0); // one channel is guaranteed by the connection profile handshakeChannel.addCloseListener(ActionListener.wrap(() -> cancelHandshakeForChannel(handshakeChannel))); Version version; try { @@ -635,7 +635,7 @@ public final NodeChannels openConnection(DiscoveryNode node, ConnectionProfile c transportService.onConnectionOpened(nodeChannels); final NodeChannels finalNodeChannels = nodeChannels; final AtomicBoolean runOnce = new AtomicBoolean(false); - Consumer onClose = c -> { + Consumer onClose = c -> { assert c.isOpen() == false : "channel is still open when onClose is called"; // we only need to disconnect from the nodes once since all other channels // will also try to run this we protect it from running multiple times. @@ -772,15 +772,15 @@ protected InetSocketAddress bindToPort(final String name, final InetAddress host final AtomicReference boundSocket = new AtomicReference<>(); boolean success = portsRange.iterate(portNumber -> { try { - Channel channel = bind(name, new InetSocketAddress(hostAddress, portNumber)); + TcpChannel channel = bind(name, new InetSocketAddress(hostAddress, portNumber)); synchronized (serverChannels) { - List list = serverChannels.get(name); + List list = serverChannels.get(name); if (list == null) { list = new ArrayList<>(); serverChannels.put(name, list); } list.add(channel); - boundSocket.set(getLocalAddress(channel)); + boundSocket.set(channel.getLocalAddress()); } } catch (Exception e) { lastException.set(e); @@ -937,9 +937,9 @@ protected final void doStop() { closeLock.writeLock().lock(); try { // first stop to accept any incoming connections so nobody can connect to this transport - for (Map.Entry> entry : serverChannels.entrySet()) { + for (Map.Entry> entry : serverChannels.entrySet()) { String profile = entry.getKey(); - List channels = entry.getValue(); + List channels = entry.getValue(); ActionListener closeFailLogger = ActionListener.wrap(c -> {}, e -> logger.warn(() -> new ParameterizedMessage("Error closing serverChannel for profile [{}]", profile), e)); channels.forEach(c -> c.addCloseListener(closeFailLogger)); @@ -979,7 +979,7 @@ protected final void doStop() { } } - protected void onException(Channel channel, Exception e) { + protected void onException(TcpChannel channel, Exception e) { if (!lifecycle.started()) { // just close and ignore - we are already stopped and just need to make sure we release all resources TcpChannel.closeChannel(channel, false); @@ -1014,9 +1014,9 @@ protected void onException(Channel channel, Exception e) { // in case we are able to return data, serialize the exception content and sent it back to the client if (channel.isOpen()) { BytesArray message = new BytesArray(e.getMessage().getBytes(StandardCharsets.UTF_8)); - final SendMetricListener closeChannel = new SendMetricListener(message.length()) { + final SendMetricListener closeChannel = new SendMetricListener(message.length()) { @Override - protected void innerInnerOnResponse(Channel channel) { + protected void innerInnerOnResponse(TcpChannel channel) { TcpChannel.closeChannel(channel, false); } @@ -1036,34 +1036,19 @@ protected void innerOnFailure(Exception e) { } } - protected void serverAcceptedChannel(Channel channel) { + protected void serverAcceptedChannel(TcpChannel channel) { boolean addedOnThisCall = acceptedChannels.add(channel); assert addedOnThisCall : "Channel should only be added to accept channel set once"; channel.addCloseListener(ActionListener.wrap(() -> acceptedChannels.remove(channel))); } - /** - * Returns the channels local address - */ - protected abstract InetSocketAddress getLocalAddress(Channel channel); - /** * Binds to the given {@link InetSocketAddress} * * @param name the profile name * @param address the address to bind to */ - protected abstract Channel bind(String name, InetSocketAddress address) throws IOException; - - /** - * Sends message to channel. The listener's onResponse method will be called when the send is complete unless an exception - * is thrown during the send. If an exception is thrown, the listener's onException method will be called. - * - * @param channel the destination channel - * @param reference the byte reference for the message - * @param listener the listener to call when the operation has completed - */ - protected abstract void sendMessage(Channel channel, BytesReference reference, ActionListener listener); + protected abstract TcpChannel bind(String name, InetSocketAddress address) throws IOException; /** * Initiate a single tcp socket channel to a node. Implementations do not have to observe the connectTimeout. @@ -1075,7 +1060,7 @@ protected void serverAcceptedChannel(Channel channel) { * @return the pending connection * @throws IOException if an I/O exception occurs while opening the channel */ - protected abstract Channel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) + protected abstract TcpChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) throws IOException; /** @@ -1088,7 +1073,7 @@ public boolean canCompress(TransportRequest request) { return compress && (!(request instanceof BytesTransportRequest)); } - private void sendRequestToChannel(final DiscoveryNode node, final Channel targetChannel, final long requestId, final String action, + private void sendRequestToChannel(final DiscoveryNode node, final TcpChannel channel, final long requestId, final String action, final TransportRequest request, TransportRequestOptions options, Version channelVersion, byte status) throws IOException, TransportException { @@ -1120,9 +1105,9 @@ private void sendRequestToChannel(final DiscoveryNode node, final Channel target BytesReference message = buildMessage(requestId, status, node.getVersion(), request, stream); final TransportRequestOptions finalOptions = options; // this might be called in a different thread - SendListener onRequestSent = new SendListener(stream, + SendListener onRequestSent = new SendListener(channel, stream, () -> transportService.onRequestSent(node, requestId, action, request, finalOptions), message.length()); - internalSendMessage(targetChannel, message, onRequestSent); + internalSendMessage(channel, message, onRequestSent); addedReleaseListener = true; } finally { if (!addedReleaseListener) { @@ -1134,13 +1119,13 @@ private void sendRequestToChannel(final DiscoveryNode node, final Channel target /** * sends a message to the given channel, using the given callbacks. */ - private void internalSendMessage(Channel targetChannel, BytesReference message, SendMetricListener listener) { + private void internalSendMessage(TcpChannel channel, BytesReference message, SendMetricListener listener) { try { - sendMessage(targetChannel, message, listener); + channel.sendMessage(message, listener); } catch (Exception ex) { // call listener to ensure that any resources are released listener.onFailure(ex); - onException(targetChannel, ex); + onException(channel, ex); } } @@ -1153,12 +1138,12 @@ private void internalSendMessage(Channel targetChannel, BytesReference message, * @param requestId the request ID this response replies to * @param action the action this response replies to */ - public void sendErrorResponse(Version nodeVersion, Channel channel, final Exception error, final long requestId, + public void sendErrorResponse(Version nodeVersion, TcpChannel channel, final Exception error, final long requestId, final String action) throws IOException { try (BytesStreamOutput stream = new BytesStreamOutput()) { stream.setVersion(nodeVersion); RemoteTransportException tx = new RemoteTransportException( - nodeName(), new TransportAddress(getLocalAddress(channel)), action, error); + nodeName(), new TransportAddress(channel.getLocalAddress()), action, error); threadPool.getThreadContext().writeTo(stream); stream.writeException(tx); byte status = 0; @@ -1167,7 +1152,7 @@ public void sendErrorResponse(Version nodeVersion, Channel channel, final Except final BytesReference bytes = stream.bytes(); final BytesReference header = buildHeader(requestId, status, nodeVersion, bytes.length()); CompositeBytesReference message = new CompositeBytesReference(header, bytes); - SendListener onResponseSent = new SendListener(null, + SendListener onResponseSent = new SendListener(channel, null, () -> transportService.onResponseSent(requestId, action, error), message.length()); internalSendMessage(channel, message, onResponseSent); } @@ -1178,12 +1163,12 @@ public void sendErrorResponse(Version nodeVersion, Channel channel, final Except * * @see #sendErrorResponse(Version, TcpChannel, Exception, long, String) for sending back errors to the caller */ - public void sendResponse(Version nodeVersion, Channel channel, final TransportResponse response, final long requestId, + public void sendResponse(Version nodeVersion, TcpChannel channel, final TransportResponse response, final long requestId, final String action, TransportResponseOptions options) throws IOException { sendResponse(nodeVersion, channel, response, requestId, action, options, (byte) 0); } - private void sendResponse(Version nodeVersion, Channel channel, final TransportResponse response, final long requestId, + private void sendResponse(Version nodeVersion, TcpChannel channel, final TransportResponse response, final long requestId, final String action, TransportResponseOptions options, byte status) throws IOException { if (compress) { options = TransportResponseOptions.builder(options).withCompress(true).build(); @@ -1202,7 +1187,7 @@ private void sendResponse(Version nodeVersion, Channel channel, final TransportR final TransportResponseOptions finalOptions = options; // this might be called in a different thread - SendListener listener = new SendListener(stream, + SendListener listener = new SendListener(channel, stream, () -> transportService.onResponseSent(requestId, action, response, finalOptions), message.length()); internalSendMessage(channel, message, listener); addedReleaseListener = true; @@ -1355,7 +1340,7 @@ public HttpOnTransportException(StreamInput in) throws IOException { /** * This method handles the message receive part for both request and responses */ - public final void messageReceived(BytesReference reference, Channel channel, String profileName, + public final void messageReceived(BytesReference reference, TcpChannel channel, String profileName, InetSocketAddress remoteAddress, int messageLengthBytes) throws IOException { final int totalMessageSize = messageLengthBytes + TcpHeader.MARKER_BYTES_SIZE + TcpHeader.MESSAGE_LENGTH_SIZE; readBytesMetric.inc(totalMessageSize); @@ -1494,8 +1479,9 @@ private void handleException(final TransportResponseHandler handler, Throwable e }); } - protected String handleRequest(Channel channel, String profileName, final StreamInput stream, long requestId, int messageLengthBytes, - Version version, InetSocketAddress remoteAddress, byte status) throws IOException { + protected String handleRequest(TcpChannel channel, String profileName, final StreamInput stream, long requestId, + int messageLengthBytes, Version version, InetSocketAddress remoteAddress, byte status) + throws IOException { final String action = stream.readString(); transportService.onRequestReceived(requestId, action); TransportChannel transportChannel = null; @@ -1514,7 +1500,7 @@ protected String handleRequest(Channel channel, String profileName, final Stream } else { getInFlightRequestBreaker().addWithoutBreaking(messageLengthBytes); } - transportChannel = new TcpTransportChannel<>(this, channel, transportName, action, requestId, version, profileName, + transportChannel = new TcpTransportChannel(this, channel, transportName, action, requestId, version, profileName, messageLengthBytes); final TransportRequest request = reg.newRequest(stream); request.remoteAddress(new TransportAddress(remoteAddress)); @@ -1525,7 +1511,7 @@ protected String handleRequest(Channel channel, String profileName, final Stream } catch (Exception e) { // the circuit breaker tripped if (transportChannel == null) { - transportChannel = new TcpTransportChannel<>(this, channel, transportName, action, requestId, version, profileName, 0); + transportChannel = new TcpTransportChannel(this, channel, transportName, action, requestId, version, profileName, 0); } try { transportChannel.sendResponse(e); @@ -1611,7 +1597,8 @@ public void writeTo(StreamOutput out) throws IOException { } } - protected Version executeHandshake(DiscoveryNode node, Channel channel, TimeValue timeout) throws IOException, InterruptedException { + protected Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout) + throws IOException, InterruptedException { numHandshakes.inc(); final long requestId = newRequestId(); final HandshakeResponseHandler handler = new HandshakeResponseHandler(channel); @@ -1671,7 +1658,7 @@ public long newRequestId() { /** * Called once the channel is closed for instance due to a disconnect or a closed socket etc. */ - private void cancelHandshakeForChannel(Channel channel) { + private void cancelHandshakeForChannel(TcpChannel channel) { final Optional first = pendingHandshakes.entrySet().stream() .filter((entry) -> entry.getValue().channel == channel).map(Map.Entry::getKey).findFirst(); if (first.isPresent()) { @@ -1699,7 +1686,7 @@ protected final void ensureOpen() { /** * This listener increments the transmitted bytes metric on success. */ - private abstract class SendMetricListener extends NotifyOnceListener { + private abstract class SendMetricListener extends NotifyOnceListener { private final long messageSize; private SendMetricListener(long messageSize) { @@ -1707,31 +1694,34 @@ private SendMetricListener(long messageSize) { } @Override - protected final void innerOnResponse(T object) { + protected final void innerOnResponse(org.elasticsearch.transport.TcpChannel object) { transmittedBytesMetric.inc(messageSize); innerInnerOnResponse(object); } - protected abstract void innerInnerOnResponse(T object); + protected abstract void innerInnerOnResponse(org.elasticsearch.transport.TcpChannel object); } - private final class SendListener extends SendMetricListener { + private final class SendListener extends SendMetricListener { + private final TcpChannel channel; private final Releasable optionalReleasable; private final Runnable transportAdaptorCallback; - private SendListener(Releasable optionalReleasable, Runnable transportAdaptorCallback, long messageLength) { + private SendListener(TcpChannel channel, Releasable optionalReleasable, Runnable transportAdaptorCallback, long messageLength) { super(messageLength); + this.channel = channel; this.optionalReleasable = optionalReleasable; this.transportAdaptorCallback = transportAdaptorCallback; } @Override - protected void innerInnerOnResponse(Channel channel) { + protected void innerInnerOnResponse(TcpChannel channel) { release(); } @Override protected void innerOnFailure(Exception e) { + logger.warn(() -> new ParameterizedMessage("send message failed [channel: {}]", channel), e); release(); } diff --git a/core/src/main/java/org/elasticsearch/transport/TcpTransportChannel.java b/core/src/main/java/org/elasticsearch/transport/TcpTransportChannel.java index 3267548e91434..eb4c244c7a920 100644 --- a/core/src/main/java/org/elasticsearch/transport/TcpTransportChannel.java +++ b/core/src/main/java/org/elasticsearch/transport/TcpTransportChannel.java @@ -23,8 +23,8 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; -public final class TcpTransportChannel implements TransportChannel { - private final TcpTransport transport; +public final class TcpTransportChannel implements TransportChannel { + private final TcpTransport transport; private final Version version; private final String action; private final long requestId; @@ -32,9 +32,9 @@ public final class TcpTransportChannel implements Tr private final long reservedBytes; private final AtomicBoolean released = new AtomicBoolean(); private final String channelType; - private final Channel channel; + private final TcpChannel channel; - TcpTransportChannel(TcpTransport transport, Channel channel, String channelType, String action, + TcpTransportChannel(TcpTransport transport, TcpChannel channel, String channelType, String action, long requestId, Version version, String profileName, long reservedBytes) { this.version = version; this.channel = channel; @@ -97,7 +97,7 @@ public Version getVersion() { return version; } - public Channel getChannel() { + public TcpChannel getChannel() { return channel; } } diff --git a/core/src/test/java/org/elasticsearch/transport/TcpTransportTests.java b/core/src/test/java/org/elasticsearch/transport/TcpTransportTests.java index 19ada600cc105..275c6dbaeb291 100644 --- a/core/src/test/java/org/elasticsearch/transport/TcpTransportTests.java +++ b/core/src/test/java/org/elasticsearch/transport/TcpTransportTests.java @@ -39,7 +39,6 @@ import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import static org.hamcrest.Matchers.equalTo; @@ -172,57 +171,23 @@ public void testEnsureVersionCompatibility() { public void testCompressRequest() throws IOException { final boolean compressed = randomBoolean(); - final AtomicBoolean called = new AtomicBoolean(false); Req request = new Req(randomRealisticUnicodeOfLengthBetween(10, 100)); ThreadPool threadPool = new TestThreadPool(TcpTransportTests.class.getName()); - AtomicReference exceptionReference = new AtomicReference<>(); + AtomicReference messageCaptor = new AtomicReference<>(); try { - TcpTransport transport = new TcpTransport( + TcpTransport transport = new TcpTransport( "test", Settings.builder().put("transport.tcp.compress", compressed).build(), threadPool, new BigArrays(Settings.EMPTY, null), null, null, null) { - @Override - protected InetSocketAddress getLocalAddress(FakeChannel o) { - return null; - } @Override protected FakeChannel bind(String name, InetSocketAddress address) throws IOException { return null; } - @Override - protected void sendMessage(FakeChannel o, BytesReference reference, ActionListener listener) { - try { - StreamInput streamIn = reference.streamInput(); - streamIn.skip(TcpHeader.MARKER_BYTES_SIZE); - int len = streamIn.readInt(); - long requestId = streamIn.readLong(); - assertEquals(42, requestId); - byte status = streamIn.readByte(); - Version version = Version.fromId(streamIn.readInt()); - assertEquals(Version.CURRENT, version); - assertEquals(compressed, TransportStatus.isCompress(status)); - called.compareAndSet(false, true); - if (compressed) { - final int bytesConsumed = TcpHeader.HEADER_SIZE; - streamIn = CompressorFactory.compressor(reference.slice(bytesConsumed, reference.length() - bytesConsumed)) - .streamInput(streamIn); - } - threadPool.getThreadContext().readHeaders(streamIn); - assertEquals("foobar", streamIn.readString()); - Req readReq = new Req(""); - readReq.readFrom(streamIn); - assertEquals(request.value, readReq.value); - } catch (IOException e) { - exceptionReference.set(e); - } - } - @Override protected FakeChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, - ActionListener connectListener) throws IOException { - FakeChannel fakeChannel = new FakeChannel(); - return fakeChannel; + ActionListener connectListener) throws IOException { + return new FakeChannel(messageCaptor); } @Override @@ -233,18 +198,41 @@ public long getNumOpenServerConnections() { @Override public NodeChannels getConnection(DiscoveryNode node) { int numConnections = MockTcpTransport.LIGHT_PROFILE.getNumConnections(); - ArrayList fakeChannels = new ArrayList<>(numConnections); + ArrayList fakeChannels = new ArrayList<>(numConnections); for (int i = 0; i < numConnections; ++i) { - fakeChannels.add(new FakeChannel()); + fakeChannels.add(new FakeChannel(messageCaptor)); } return new NodeChannels(node, fakeChannels, MockTcpTransport.LIGHT_PROFILE, Version.CURRENT); } }; + DiscoveryNode node = new DiscoveryNode("foo", buildNewFakeTransportAddress(), Version.CURRENT); Transport.Connection connection = transport.getConnection(node); connection.sendRequest(42, "foobar", request, TransportRequestOptions.EMPTY); - assertTrue(called.get()); - assertNull("IOException while sending message.", exceptionReference.get()); + + BytesReference reference = messageCaptor.get(); + assertNotNull(reference); + + StreamInput streamIn = reference.streamInput(); + streamIn.skip(TcpHeader.MARKER_BYTES_SIZE); + int len = streamIn.readInt(); + long requestId = streamIn.readLong(); + assertEquals(42, requestId); + byte status = streamIn.readByte(); + Version version = Version.fromId(streamIn.readInt()); + assertEquals(Version.CURRENT, version); + assertEquals(compressed, TransportStatus.isCompress(status)); + if (compressed) { + final int bytesConsumed = TcpHeader.HEADER_SIZE; + streamIn = CompressorFactory.compressor(reference.slice(bytesConsumed, reference.length() - bytesConsumed)) + .streamInput(streamIn); + } + threadPool.getThreadContext().readHeaders(streamIn); + assertEquals("foobar", streamIn.readString()); + Req readReq = new Req(""); + readReq.readFrom(streamIn); + assertEquals(request.value, readReq.value); + } finally { ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS); } @@ -252,6 +240,12 @@ public NodeChannels getConnection(DiscoveryNode node) { private static final class FakeChannel implements TcpChannel { + private final AtomicReference messageCaptor; + + FakeChannel(AtomicReference messageCaptor) { + this.messageCaptor = messageCaptor; + } + @Override public void close() { } @@ -268,6 +262,16 @@ public void setSoLinger(int value) throws IOException { public boolean isOpen() { return false; } + + @Override + public InetSocketAddress getLocalAddress() { + return null; + } + + @Override + public void sendMessage(BytesReference reference, ActionListener listener) { + messageCaptor.set(reference); + } } private static final class Req extends TransportRequest { diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java index 9cdefc292f22f..29ff3967d6dea 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java @@ -42,7 +42,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.SuppressForbidden; -import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.lease.Releasables; @@ -57,6 +56,7 @@ import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TcpChannel; import org.elasticsearch.transport.TcpTransport; import org.elasticsearch.transport.TransportRequestOptions; @@ -79,7 +79,7 @@ * longer. Med is for the typical search / single doc index. And High for things like cluster state. Ping is reserved for * sending out ping requests to other nodes. */ -public class Netty4Transport extends TcpTransport { +public class Netty4Transport extends TcpTransport { static { Netty4Utils.setup(); @@ -249,7 +249,7 @@ public long getNumOpenServerConnections() { } @Override - protected NettyTcpChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener listener) + protected NettyTcpChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener listener) throws IOException { ChannelFuture channelFuture = bootstrap.connect(node.getAddress().address()); Channel channel = channelFuture.channel(); @@ -279,28 +279,6 @@ protected NettyTcpChannel initiateChannel(DiscoveryNode node, TimeValue connectT return nettyChannel; } - @Override - protected void sendMessage(NettyTcpChannel channel, BytesReference reference, ActionListener listener) { - final ChannelFuture future = channel.getLowLevelChannel().writeAndFlush(Netty4Utils.toByteBuf(reference)); - future.addListener(f -> { - if (f.isSuccess()) { - listener.onResponse(channel); - } else { - final Throwable cause = f.cause(); - Netty4Utils.maybeDie(cause); - logger.warn((Supplier) () -> - new ParameterizedMessage("write and flush on the network layer failed (channel: {})", channel), cause); - assert cause instanceof Exception; - listener.onFailure((Exception) cause); - } - }); - } - - @Override - protected InetSocketAddress getLocalAddress(NettyTcpChannel channel) { - return (InetSocketAddress) channel.getLowLevelChannel().localAddress(); - } - @Override protected NettyTcpChannel bind(String name, InetSocketAddress address) { Channel channel = serverBootstraps.get(name).bind(address).syncUninterruptibly().channel(); diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyTcpChannel.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyTcpChannel.java index c18c3c4fe1f11..12ab34a32af75 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyTcpChannel.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyTcpChannel.java @@ -20,10 +20,15 @@ package org.elasticsearch.transport.netty4; import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelOption; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.logging.log4j.util.Supplier; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.transport.TcpChannel; +import java.net.InetSocketAddress; import java.util.concurrent.CompletableFuture; public class NettyTcpChannel implements TcpChannel { @@ -48,10 +53,6 @@ public class NettyTcpChannel implements TcpChannel { }); } - public Channel getLowLevelChannel() { - return channel; - } - @Override public void close() { channel.close(); @@ -71,4 +72,28 @@ public void setSoLinger(int value) { public boolean isOpen() { return channel.isOpen(); } + + @Override + public InetSocketAddress getLocalAddress() { + return (InetSocketAddress) channel.localAddress(); + } + + @Override + public void sendMessage(BytesReference reference, ActionListener listener) { + final ChannelFuture future = channel.writeAndFlush(Netty4Utils.toByteBuf(reference)); + future.addListener(f -> { + if (f.isSuccess()) { + listener.onResponse(this); + } else { + final Throwable cause = f.cause(); + Netty4Utils.maybeDie(cause); + assert cause instanceof Exception; + listener.onFailure((Exception) cause); + } + }); + } + + public Channel getLowLevelChannel() { + return channel; + } } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportIT.java index 3eb5adc8d067d..04a2b8131f9ee 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportIT.java @@ -36,6 +36,7 @@ import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TcpChannel; import org.elasticsearch.transport.TcpTransport; import org.elasticsearch.transport.Transport; @@ -108,7 +109,8 @@ public ExceptionThrowingNetty4Transport( super(settings, threadPool, networkService, bigArrays, namedWriteableRegistry, circuitBreakerService); } - protected String handleRequest(NettyTcpChannel channel, String profileName, + @Override + protected String handleRequest(TcpChannel channel, String profileName, StreamInput stream, long requestId, int messageLengthBytes, Version version, InetSocketAddress remoteAddress, byte status) throws IOException { String action = super.handleRequest(channel, profileName, stream, requestId, messageLengthBytes, version, diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/NettyTransportMultiPortTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/NettyTransportMultiPortTests.java index 295e7ab389ceb..cde939bab8dd0 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/NettyTransportMultiPortTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/NettyTransportMultiPortTests.java @@ -30,7 +30,6 @@ import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TcpTransport; -import org.elasticsearch.transport.TransportService; import org.junit.Before; import java.util.Collections; @@ -59,7 +58,7 @@ public void testThatNettyCanBindToMultiplePorts() throws Exception { .build(); ThreadPool threadPool = new TestThreadPool("tst"); - try (TcpTransport transport = startTransport(settings, threadPool)) { + try (TcpTransport transport = startTransport(settings, threadPool)) { assertEquals(1, transport.profileBoundAddresses().size()); assertEquals(1, transport.boundAddress().boundAddresses().length); } finally { @@ -75,7 +74,7 @@ public void testThatDefaultProfileInheritsFromStandardSettings() throws Exceptio .build(); ThreadPool threadPool = new TestThreadPool("tst"); - try (TcpTransport transport = startTransport(settings, threadPool)) { + try (TcpTransport transport = startTransport(settings, threadPool)) { assertEquals(1, transport.profileBoundAddresses().size()); assertEquals(1, transport.boundAddress().boundAddresses().length); } finally { @@ -108,7 +107,7 @@ public void testThatDefaultProfilePortOverridesGeneralConfiguration() throws Exc .build(); ThreadPool threadPool = new TestThreadPool("tst"); - try (TcpTransport transport = startTransport(settings, threadPool)) { + try (TcpTransport transport = startTransport(settings, threadPool)) { assertEquals(0, transport.profileBoundAddresses().size()); assertEquals(1, transport.boundAddress().boundAddresses().length); } finally { @@ -116,9 +115,9 @@ public void testThatDefaultProfilePortOverridesGeneralConfiguration() throws Exc } } - private TcpTransport startTransport(Settings settings, ThreadPool threadPool) { + private TcpTransport startTransport(Settings settings, ThreadPool threadPool) { BigArrays bigArrays = new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService()); - TcpTransport transport = new Netty4Transport(settings, threadPool, new NetworkService(Collections.emptyList()), + TcpTransport transport = new Netty4Transport(settings, threadPool, new NetworkService(Collections.emptyList()), bigArrays, new NamedWriteableRegistry(Collections.emptyList()), new NoneCircuitBreakerService()); transport.start(); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java index 47259a7c613eb..b2126b1b61185 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java @@ -58,7 +58,7 @@ public static MockTransportService nettyFromThreadPool(Settings settings, Thread BigArrays.NON_RECYCLING_INSTANCE, namedWriteableRegistry, new NoneCircuitBreakerService()) { @Override - protected Version executeHandshake(DiscoveryNode node, NettyTcpChannel channel, TimeValue timeout) throws IOException, + protected Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout) throws IOException, InterruptedException { if (doHandshake) { return super.executeHandshake(node, channel, timeout); @@ -90,7 +90,7 @@ protected MockTransportService build(Settings settings, Version version, Cluster protected void closeConnectionChannel(Transport transport, Transport.Connection connection) throws IOException { final Netty4Transport t = (Netty4Transport) transport; @SuppressWarnings("unchecked") - final TcpTransport.NodeChannels channels = (TcpTransport.NodeChannels) connection; + final TcpTransport.NodeChannels channels = (TcpTransport.NodeChannels) connection; TcpChannel.closeChannels(channels.getChannels().subList(0, randomIntBetween(1, channels.getChannels().size())), true); } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java index 2cd4ef94ae0e7..a45411324b0fc 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java @@ -1976,7 +1976,7 @@ public void testTcpHandshake() throws IOException, InterruptedException { MockTcpTransport transport = new MockTcpTransport(Settings.EMPTY, threadPool, BigArrays.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(Collections.emptyList())) { @Override - protected String handleRequest(MockChannel mockChannel, String profileName, StreamInput stream, long requestId, + protected String handleRequest(TcpChannel mockChannel, String profileName, StreamInput stream, long requestId, int messageLengthBytes, Version version, InetSocketAddress remoteAddress, byte status) throws IOException { return super.handleRequest(mockChannel, profileName, stream, requestId, messageLengthBytes, version, remoteAddress, diff --git a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java index 4b1da5c212621..5d5e14b406177 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java @@ -68,7 +68,7 @@ * that need real networking. This implementation is a test only implementation that implements * the networking layer in the worst possible way since it blocks and uses a thread per request model. */ -public class MockTcpTransport extends TcpTransport { +public class MockTcpTransport extends TcpTransport { /** * A pre-built light connection profile that shares a single connection across all @@ -109,11 +109,6 @@ public MockTcpTransport(Settings settings, ThreadPool threadPool, BigArrays bigA this.mockVersion = mockVersion; } - @Override - protected InetSocketAddress getLocalAddress(MockChannel mockChannel) { - return mockChannel.localAddress; - } - @Override protected MockChannel bind(final String name, InetSocketAddress address) throws IOException { MockServerSocket socket = new MockServerSocket(); @@ -176,7 +171,7 @@ private void readMessage(MockChannel mockChannel, StreamInput input) throws IOEx } @Override - protected MockChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) + protected MockChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) throws IOException { InetSocketAddress address = node.getAddress().address(); final MockSocket socket = new MockSocket(); @@ -222,22 +217,6 @@ private void configureSocket(Socket socket) throws SocketException { socket.setReuseAddress(TCP_REUSE_ADDRESS.get(settings)); } - @Override - protected void sendMessage(MockChannel mockChannel, BytesReference reference, ActionListener listener) { - try { - synchronized (mockChannel) { - final Socket socket = mockChannel.activeChannel; - OutputStream outputStream = new BufferedOutputStream(socket.getOutputStream()); - reference.writeTo(outputStream); - outputStream.flush(); - } - listener.onResponse(mockChannel); - } catch (IOException e) { - listener.onFailure(e); - onException(mockChannel, e); - } - } - @Override public long getNumOpenServerConnections() { return 1; @@ -401,6 +380,25 @@ public boolean isOpen() { return isOpen.get(); } + @Override + public InetSocketAddress getLocalAddress() { + return localAddress; + } + + @Override + public void sendMessage(BytesReference reference, ActionListener listener) { + try { + synchronized (this) { + OutputStream outputStream = new BufferedOutputStream(activeChannel.getOutputStream()); + reference.writeTo(outputStream); + outputStream.flush(); + } + listener.onResponse(this); + } catch (IOException e) { + listener.onFailure(e); + onException(this, e); + } + } } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java index fc8d361b82ef5..27cb73f98f8c2 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java @@ -23,7 +23,6 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Setting; @@ -33,6 +32,7 @@ import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TcpChannel; import org.elasticsearch.transport.TcpTransport; import org.elasticsearch.transport.Transports; import org.elasticsearch.transport.nio.channel.ChannelFactory; @@ -54,7 +54,7 @@ import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadFactory; -public class NioTransport extends TcpTransport { +public class NioTransport extends TcpTransport { public static final String TRANSPORT_WORKER_THREAD_NAME_PREFIX = Transports.NIO_TRANSPORT_WORKER_THREAD_NAME_PREFIX; public static final String TRANSPORT_ACCEPTOR_THREAD_NAME_PREFIX = Transports.NIO_TRANSPORT_ACCEPTOR_THREAD_NAME_PREFIX; @@ -87,11 +87,6 @@ public long getNumOpenServerConnections() { return openChannels.serverChannelsCount(); } - @Override - protected InetSocketAddress getLocalAddress(NioChannel channel) { - return channel.getLocalAddress(); - } - @Override protected NioServerSocketChannel bind(String name, InetSocketAddress address) throws IOException { ChannelFactory channelFactory = this.profileToChannelFactory.get(name); @@ -100,21 +95,22 @@ protected NioServerSocketChannel bind(String name, InetSocketAddress address) th } @Override - protected void sendMessage(NioChannel channel, BytesReference reference, ActionListener listener) { - if (channel instanceof NioSocketChannel) { - NioSocketChannel nioSocketChannel = (NioSocketChannel) channel; - nioSocketChannel.getWriteContext().sendMessage(reference, listener); - } else { - logger.error("cannot send message to channel of this type [{}]", channel.getClass()); - } - } - - @Override - protected NioChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) + protected NioChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) throws IOException { NioSocketChannel channel = clientChannelFactory.openNioChannel(node.getAddress().address(), clientSelectorSupplier.get()); openChannels.clientChannelOpened(channel); - channel.addConnectListener(connectListener); + // TODO: Temporary conversion due to types + channel.addConnectListener(new ActionListener() { + @Override + public void onResponse(NioChannel nioChannel) { + connectListener.onResponse(nioChannel); + } + + @Override + public void onFailure(Exception e) { + connectListener.onFailure(e); + } + }); return channel; } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannel.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannel.java index fab6fa22c6b16..5b365fbd36c46 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannel.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannel.java @@ -19,6 +19,9 @@ package org.elasticsearch.transport.nio.channel; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.transport.TcpChannel; import org.elasticsearch.transport.nio.AcceptingSelector; import java.io.IOException; @@ -39,6 +42,11 @@ public ChannelFactory getChannelFactory() { return channelFactory; } + @Override + public void sendMessage(BytesReference reference, ActionListener listener) { + throw new UnsupportedOperationException("Cannot send a message to a server channel."); + } + @Override public String toString() { return "NioServerSocketChannel{" + diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioSocketChannel.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioSocketChannel.java index 5e4e323094199..520cefd27f9d7 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioSocketChannel.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioSocketChannel.java @@ -20,6 +20,8 @@ package org.elasticsearch.transport.nio.channel; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.transport.TcpChannel; import org.elasticsearch.transport.nio.NetworkBytesReference; import org.elasticsearch.transport.nio.SocketSelector; @@ -46,6 +48,22 @@ public NioSocketChannel(String profile, SocketChannel socketChannel, SocketSelec this.socketSelector = selector; } + @Override + public void sendMessage(BytesReference reference, ActionListener listener) { + // TODO: Temporary conversion due to types + writeContext.sendMessage(reference, new ActionListener() { + @Override + public void onResponse(NioChannel nioChannel) { + listener.onResponse(nioChannel); + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }); + } + @Override public void closeFromSelector() throws IOException { assert socketSelector.isOnCurrentThread() : "Should only call from selector thread"; diff --git a/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java b/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java index bbe3c13442cec..6844b55cadc3a 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java @@ -40,7 +40,7 @@ protected MockTransportService build(Settings settings, Version version, Cluster Transport transport = new MockTcpTransport(settings, threadPool, BigArrays.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(Collections.emptyList()), version) { @Override - protected Version executeHandshake(DiscoveryNode node, MockChannel mockChannel, TimeValue timeout) throws IOException, + protected Version executeHandshake(DiscoveryNode node, TcpChannel mockChannel, TimeValue timeout) throws IOException, InterruptedException { if (doHandshake) { return super.executeHandshake(node, mockChannel, timeout); @@ -58,8 +58,8 @@ protected Version executeHandshake(DiscoveryNode node, MockChannel mockChannel, @Override protected void closeConnectionChannel(Transport transport, Transport.Connection connection) throws IOException { final MockTcpTransport t = (MockTcpTransport) transport; - @SuppressWarnings("unchecked") final TcpTransport.NodeChannels channels = - (TcpTransport.NodeChannels) connection; + @SuppressWarnings("unchecked") final TcpTransport.NodeChannels channels = + (TcpTransport.NodeChannels) connection; TcpChannel.closeChannels(channels.getChannels().subList(0, randomIntBetween(1, channels.getChannels().size())), true); } diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java index 04f1b424142c5..bc02a89a5c18d 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java @@ -39,7 +39,6 @@ import org.elasticsearch.transport.TcpTransport; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.transport.nio.channel.NioChannel; import java.io.IOException; import java.net.InetAddress; @@ -62,7 +61,7 @@ public static MockTransportService nioFromThreadPool(Settings settings, ThreadPo BigArrays.NON_RECYCLING_INSTANCE, namedWriteableRegistry, new NoneCircuitBreakerService()) { @Override - protected Version executeHandshake(DiscoveryNode node, NioChannel channel, TimeValue timeout) throws IOException, + protected Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout) throws IOException, InterruptedException { if (doHandshake) { return super.executeHandshake(node, channel, timeout); @@ -100,7 +99,7 @@ protected MockTransportService build(Settings settings, Version version, Cluster @Override protected void closeConnectionChannel(Transport transport, Transport.Connection connection) throws IOException { @SuppressWarnings("unchecked") - TcpTransport.NodeChannels channels = (TcpTransport.NodeChannels) connection; + TcpTransport.NodeChannels channels = (TcpTransport.NodeChannels) connection; TcpChannel.closeChannels(channels.getChannels().subList(0, randomIntBetween(1, channels.getChannels().size())), true); } From 9766b858d01c55b37a0254e41243bcee7a088e8c Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 16 Nov 2017 18:38:54 +0000 Subject: [PATCH 03/31] Prepare for bump to 6.0.1 on the master branch (#27391) An assortment of fixes, particularly to version number calculations, in preparation for the bump to 6.0.1. --- build.gradle | 13 ++ .../main/java/org/elasticsearch/Version.java | 63 ++++++++-- .../java/org/elasticsearch/VersionTests.java | 2 +- .../org/elasticsearch/test/VersionUtils.java | 111 ++++++++---------- .../elasticsearch/test/VersionUtilsTests.java | 99 ++++++++++++++-- 5 files changed, 207 insertions(+), 81 deletions(-) diff --git a/build.gradle b/build.gradle index d63973f53897a..e498101a16a78 100644 --- a/build.gradle +++ b/build.gradle @@ -81,6 +81,7 @@ List versions = [] // keep track of the previous major version's last minor, so we know where wire compat begins int prevMinorIndex = -1 // index in the versions list of the last minor from the prev major int lastPrevMinor = -1 // the minor version number from the prev major we most recently seen +int prevBugfixIndex = -1 // index in the versions list of the last bugfix release from the prev major for (String line : versionLines) { /* Note that this skips alphas and betas which is fine because they aren't * compatible with anything. */ @@ -108,12 +109,19 @@ for (String line : versionLines) { lastPrevMinor = minor } } + if (major == prevMajor) { + prevBugfixIndex = versions.size() - 1 + } } } if (versions.toSorted { it.id } != versions) { println "Versions: ${versions}" throw new GradleException("Versions.java contains out of order version constants") } +if (prevBugfixIndex != -1) { + versions[prevBugfixIndex] = new Version(versions[prevBugfixIndex].major, versions[prevBugfixIndex].minor, + versions[prevBugfixIndex].bugfix, versions[prevBugfixIndex].suffix, true) +} if (currentVersion.bugfix == 0) { // If on a release branch, after the initial release of that branch, the bugfix version will // be bumped, and will be != 0. On master and N.x branches, we want to test against the @@ -262,6 +270,11 @@ subprojects { ext.projectSubstitutions["org.elasticsearch.distribution.rpm:elasticsearch:${indexCompatVersions[-1]}"] = ':distribution:bwc-release-snapshot' ext.projectSubstitutions["org.elasticsearch.distribution.zip:elasticsearch:${indexCompatVersions[-1]}"] = ':distribution:bwc-release-snapshot' } + } else if (indexCompatVersions[-2].snapshot) { + /* This is a terrible hack for the bump to 6.0.1 which will be fixed by #27397 */ + ext.projectSubstitutions["org.elasticsearch.distribution.deb:elasticsearch:${indexCompatVersions[-2]}"] = ':distribution:bwc-release-snapshot' + ext.projectSubstitutions["org.elasticsearch.distribution.rpm:elasticsearch:${indexCompatVersions[-2]}"] = ':distribution:bwc-release-snapshot' + ext.projectSubstitutions["org.elasticsearch.distribution.zip:elasticsearch:${indexCompatVersions[-2]}"] = ':distribution:bwc-release-snapshot' } project.afterEvaluate { configurations.all { diff --git a/core/src/main/java/org/elasticsearch/Version.java b/core/src/main/java/org/elasticsearch/Version.java index 687061c3e7fa1..e1f5a5da14c1a 100644 --- a/core/src/main/java/org/elasticsearch/Version.java +++ b/core/src/main/java/org/elasticsearch/Version.java @@ -28,6 +28,11 @@ import org.elasticsearch.monitor.jvm.JvmInfo; import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; public class Version implements Comparable { /* @@ -363,19 +368,23 @@ public int compareTo(Version other) { * is a beta or RC release then the version itself is returned. */ public Version minimumCompatibilityVersion() { - final int bwcMajor; - final int bwcMinor; - // TODO: remove this entirely, making it static for each version - if (major == 6) { // we only specialize for current major here - bwcMajor = Version.V_5_6_0.major; - bwcMinor = Version.V_5_6_0.minor; - } else if (major == 7) { // we only specialize for current major here - return V_6_1_0; - } else { - bwcMajor = major; - bwcMinor = 0; + if (major >= 6) { + // all major versions from 6 onwards are compatible with last minor series of the previous major + final List declaredVersions = getDeclaredVersions(getClass()); + Version bwcVersion = null; + for (int i = declaredVersions.size() - 1; i >= 0; i--) { + final Version candidateVersion = declaredVersions.get(i); + if (candidateVersion.major == major - 1 && candidateVersion.isRelease() && after(candidateVersion)) { + if (bwcVersion != null && candidateVersion.minor < bwcVersion.minor) { + break; + } + bwcVersion = candidateVersion; + } + } + return bwcVersion == null ? this : bwcVersion; } - return Version.min(this, fromId(bwcMajor * 1000000 + bwcMinor * 10000 + 99)); + + return Version.min(this, fromId((int) major * 1000000 + 0 * 10000 + 99)); } /** @@ -485,4 +494,34 @@ public boolean isRC() { public boolean isRelease() { return build == 99; } + + /** + * Extracts a sorted list of declared version constants from a class. + * The argument would normally be Version.class but is exposed for + * testing with other classes-containing-version-constants. + */ + public static List getDeclaredVersions(final Class versionClass) { + final Field[] fields = versionClass.getFields(); + final List versions = new ArrayList<>(fields.length); + for (final Field field : fields) { + final int mod = field.getModifiers(); + if (false == Modifier.isStatic(mod) && Modifier.isFinal(mod) && Modifier.isPublic(mod)) { + continue; + } + if (field.getType() != Version.class) { + continue; + } + if ("CURRENT".equals(field.getName())) { + continue; + } + assert field.getName().matches("V(_\\d+)+(_(alpha|beta|rc)\\d+)?") : field.getName(); + try { + versions.add(((Version) field.get(null))); + } catch (final IllegalAccessException e) { + throw new RuntimeException(e); + } + } + Collections.sort(versions); + return versions; + } } diff --git a/core/src/test/java/org/elasticsearch/VersionTests.java b/core/src/test/java/org/elasticsearch/VersionTests.java index 89c6eaa9e9a42..693a2cf9a91a9 100644 --- a/core/src/test/java/org/elasticsearch/VersionTests.java +++ b/core/src/test/java/org/elasticsearch/VersionTests.java @@ -337,7 +337,7 @@ public void testIsCompatible() { assertTrue(isCompatible(Version.V_5_6_0, Version.V_6_0_0_alpha2)); assertFalse(isCompatible(Version.fromId(2000099), Version.V_6_0_0_alpha2)); assertFalse(isCompatible(Version.fromId(2000099), Version.V_5_0_0)); - assertTrue(isCompatible(Version.fromString("6.1.0"), Version.fromString("7.0.0"))); + assertFalse(isCompatible(Version.fromString("6.0.0"), Version.fromString("7.0.0"))); assertFalse(isCompatible(Version.fromString("6.0.0-alpha1"), Version.fromString("7.0.0"))); assertFalse("only compatible with the latest minor", isCompatible(VersionUtils.getPreviousMinorVersion(), Version.fromString("7.0.0"))); diff --git a/test/framework/src/main/java/org/elasticsearch/test/VersionUtils.java b/test/framework/src/main/java/org/elasticsearch/test/VersionUtils.java index 74a9b58a78e37..8fb5f7b81fa83 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/VersionUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/test/VersionUtils.java @@ -23,10 +23,7 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.Tuple; -import java.lang.reflect.Field; -import java.lang.reflect.Modifier; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Optional; @@ -49,72 +46,64 @@ public class VersionUtils { * guarantees in v1 and versions without the guranteees in v2 */ static Tuple, List> resolveReleasedVersions(Version current, Class versionClass) { - Field[] fields = versionClass.getFields(); - List versions = new ArrayList<>(fields.length); - for (final Field field : fields) { - final int mod = field.getModifiers(); - if (false == Modifier.isStatic(mod) && Modifier.isFinal(mod) && Modifier.isPublic(mod)) { - continue; - } - if (field.getType() != Version.class) { - continue; - } - if ("CURRENT".equals(field.getName())) { - continue; - } - assert field.getName().matches("V(_\\d+)+(_(alpha|beta|rc)\\d+)?") : field.getName(); - try { - versions.add(((Version) field.get(null))); - } catch (final IllegalAccessException e) { - throw new RuntimeException(e); - } - } - Collections.sort(versions); + List versions = Version.getDeclaredVersions(versionClass); Version last = versions.remove(versions.size() - 1); assert last.equals(current) : "The highest version must be the current one " - + "but was [" + versions.get(versions.size() - 1) + "] and current was [" + current + "]"; - - if (current.revision != 0) { - /* If we are in a stable branch there should be no unreleased version constants - * because we don't expect to release any new versions in older branches. If there - * are extra constants then gradle will yell about it. */ + + "but was [" + last + "] and current was [" + current + "]"; + + /* In the 5.x series prior to 5.6, unreleased version constants had an + * `_UNRELEASED` suffix, and when making the first release on a minor release + * branch the last, unreleased, version constant from the previous minor branch + * was dropped. After 5.6, there is no `_UNRELEASED` suffix on version constants' + * names and, additionally, they are not dropped when a new minor release branch + * starts. + * + * This means that in 6.x and later series the last release _in each + * minor branch_ is unreleased, whereas in 5.x it's more complicated: There were + * (sometimes, and sometimes multiple) minor branches containing no releases, each + * of which contains a single version constant of the form 5.n.0, and these + * branches always followed a branch that _did_ contain a version of the + * form 5.m.p (p>0). All versions strictly before the last 5.m version are released, + * and all other 5.* versions are unreleased. + */ + + if (current.major == 5 && current.revision != 0) { + /* The current (i.e. latest) version is 5.a.b, b nonzero, which + * means that all other versions are released. */ return new Tuple<>(unmodifiableList(versions), singletonList(current)); } - /* If we are on a patch release then we know that at least the version before the - * current one is unreleased. If it is released then gradle would be complaining. */ - int unreleasedIndex = versions.size() - 1; - while (true) { - if (unreleasedIndex < 0) { - throw new IllegalArgumentException("Couldn't find first non-alpha release"); - } - /* We don't support backwards compatibility for alphas, betas, and rcs. But - * they were released so we add them to the released list. Usually this doesn't - * matter to consumers, but consumers that do care should filter non-release - * versions. */ - if (versions.get(unreleasedIndex).isRelease()) { - break; + final List unreleased = new ArrayList<>(); + unreleased.add(current); + Version prevConsideredVersion = current; + + for (int i = versions.size() - 1; i >= 0; i--) { + Version currConsideredVersion = versions.get(i); + if (currConsideredVersion.major == 5) { + unreleased.add(currConsideredVersion); + versions.remove(i); + if (currConsideredVersion.revision != 0) { + /* Currently considering the latest version in the 5.x series, + * which is (a) unreleased and (b) the only such. So we're done. */ + break; + } + /* ... else we're on a version of the form 5.n.0, and have not yet + * considered a version of the form 5.n.m (m>0), so this entire branch + * is unreleased, so carry on looking for a branch containing releases. + */ + } else if (currConsideredVersion.major != prevConsideredVersion.major + || currConsideredVersion.minor != prevConsideredVersion.minor) { + /* Have moved to the end of a new minor branch, so this is + * an unreleased version. */ + unreleased.add(currConsideredVersion); + versions.remove(i); } - unreleasedIndex--; - } + prevConsideredVersion = currConsideredVersion; - Version unreleased = versions.remove(unreleasedIndex); - if (unreleased.revision == 0) { - /* - * If the last unreleased version is itself a patch release then Gradle enforces that there is yet another unreleased version - * before that. However, we have to skip alpha/betas/RCs too (e.g., consider when the version constants are ..., 5.6.3, 5.6.4, - * 6.0.0-alpha1, ..., 6.0.0-rc1, 6.0.0-rc2, 6.0.0, 6.1.0 on the 6.x branch. In this case, we will have pruned 6.0.0 and 6.1.0 as - * unreleased versions, but we also need to prune 5.6.4. At this point though, unreleasedIndex will be pointing to 6.0.0-rc2, so - * we have to skip backwards until we find a non-alpha/beta/RC again. Then we can prune that version as an unreleased version - * too. - */ - do { - unreleasedIndex--; - } while (versions.get(unreleasedIndex).isRelease() == false); - Version earlierUnreleased = versions.remove(unreleasedIndex); - return new Tuple<>(unmodifiableList(versions), unmodifiableList(Arrays.asList(earlierUnreleased, unreleased, current))); } - return new Tuple<>(unmodifiableList(versions), unmodifiableList(Arrays.asList(unreleased, current))); + + Collections.reverse(unreleased); + return new Tuple<>(unmodifiableList(versions), unmodifiableList(unreleased)); } private static final List RELEASED_VERSIONS; diff --git a/test/framework/src/test/java/org/elasticsearch/test/VersionUtilsTests.java b/test/framework/src/test/java/org/elasticsearch/test/VersionUtilsTests.java index 73a7001f5d214..3be5ec5913dd2 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/VersionUtilsTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/VersionUtilsTests.java @@ -101,7 +101,7 @@ public void testRandomVersionBetween() { assertEquals(unreleased, VersionUtils.randomVersionBetween(random(), unreleased, unreleased)); } - static class TestReleaseBranch { + public static class TestReleaseBranch { public static final Version V_5_3_0 = Version.fromString("5.3.0"); public static final Version V_5_3_1 = Version.fromString("5.3.1"); public static final Version V_5_3_2 = Version.fromString("5.3.2"); @@ -118,7 +118,7 @@ public void testResolveReleasedVersionsForReleaseBranch() { assertEquals(singletonList(TestReleaseBranch.V_5_4_1), unreleased); } - static class TestStableBranch { + public static class TestStableBranch { public static final Version V_5_3_0 = Version.fromString("5.3.0"); public static final Version V_5_3_1 = Version.fromString("5.3.1"); public static final Version V_5_3_2 = Version.fromString("5.3.2"); @@ -134,7 +134,7 @@ public void testResolveReleasedVersionsForUnreleasedStableBranch() { assertEquals(Arrays.asList(TestStableBranch.V_5_3_2, TestStableBranch.V_5_4_0), unreleased); } - static class TestStableBranchBehindStableBranch { + public static class TestStableBranchBehindStableBranch { public static final Version V_5_3_0 = Version.fromString("5.3.0"); public static final Version V_5_3_1 = Version.fromString("5.3.1"); public static final Version V_5_3_2 = Version.fromString("5.3.2"); @@ -142,7 +142,7 @@ static class TestStableBranchBehindStableBranch { public static final Version V_5_5_0 = Version.fromString("5.5.0"); public static final Version CURRENT = V_5_5_0; } - public void testResolveReleasedVersionsForStableBtranchBehindStableBranch() { + public void testResolveReleasedVersionsForStableBranchBehindStableBranch() { Tuple, List> t = VersionUtils.resolveReleasedVersions(TestStableBranchBehindStableBranch.CURRENT, TestStableBranchBehindStableBranch.class); List released = t.v1(); @@ -152,7 +152,7 @@ public void testResolveReleasedVersionsForStableBtranchBehindStableBranch() { TestStableBranchBehindStableBranch.V_5_5_0), unreleased); } - static class TestUnstableBranch { + public static class TestUnstableBranch { public static final Version V_5_3_0 = Version.fromString("5.3.0"); public static final Version V_5_3_1 = Version.fromString("5.3.1"); public static final Version V_5_3_2 = Version.fromString("5.3.2"); @@ -173,6 +173,87 @@ public void testResolveReleasedVersionsForUnstableBranch() { assertEquals(Arrays.asList(TestUnstableBranch.V_5_3_2, TestUnstableBranch.V_5_4_0, TestUnstableBranch.V_6_0_0_beta1), unreleased); } + public static class TestNewMajorRelease { + public static final Version V_5_6_0 = Version.fromString("5.6.0"); + public static final Version V_5_6_1 = Version.fromString("5.6.1"); + public static final Version V_5_6_2 = Version.fromString("5.6.2"); + public static final Version V_6_0_0_alpha1 = Version.fromString("6.0.0-alpha1"); + public static final Version V_6_0_0_alpha2 = Version.fromString("6.0.0-alpha2"); + public static final Version V_6_0_0_beta1 = Version.fromString("6.0.0-beta1"); + public static final Version V_6_0_0_beta2 = Version.fromString("6.0.0-beta2"); + public static final Version V_6_0_0 = Version.fromString("6.0.0"); + public static final Version V_6_0_1 = Version.fromString("6.0.1"); + public static final Version CURRENT = V_6_0_1; + } + + public void testResolveReleasedVersionsAtNewMajorRelease() { + Tuple, List> t = VersionUtils.resolveReleasedVersions(TestNewMajorRelease.CURRENT, + TestNewMajorRelease.class); + List released = t.v1(); + List unreleased = t.v2(); + assertEquals(Arrays.asList(TestNewMajorRelease.V_5_6_0, TestNewMajorRelease.V_5_6_1, + TestNewMajorRelease.V_6_0_0_alpha1, TestNewMajorRelease.V_6_0_0_alpha2, + TestNewMajorRelease.V_6_0_0_beta1, TestNewMajorRelease.V_6_0_0_beta2, + TestNewMajorRelease.V_6_0_0), released); + assertEquals(Arrays.asList(TestNewMajorRelease.V_5_6_2, TestNewMajorRelease.V_6_0_1), unreleased); + } + + public static class TestVersionBumpIn6x { + public static final Version V_5_6_0 = Version.fromString("5.6.0"); + public static final Version V_5_6_1 = Version.fromString("5.6.1"); + public static final Version V_5_6_2 = Version.fromString("5.6.2"); + public static final Version V_6_0_0_alpha1 = Version.fromString("6.0.0-alpha1"); + public static final Version V_6_0_0_alpha2 = Version.fromString("6.0.0-alpha2"); + public static final Version V_6_0_0_beta1 = Version.fromString("6.0.0-beta1"); + public static final Version V_6_0_0_beta2 = Version.fromString("6.0.0-beta2"); + public static final Version V_6_0_0 = Version.fromString("6.0.0"); + public static final Version V_6_0_1 = Version.fromString("6.0.1"); + public static final Version V_6_1_0 = Version.fromString("6.1.0"); + public static final Version CURRENT = V_6_1_0; + } + + public void testResolveReleasedVersionsAtVersionBumpIn6x() { + Tuple, List> t = VersionUtils.resolveReleasedVersions(TestVersionBumpIn6x.CURRENT, + TestVersionBumpIn6x.class); + List released = t.v1(); + List unreleased = t.v2(); + assertEquals(Arrays.asList(TestVersionBumpIn6x.V_5_6_0, TestVersionBumpIn6x.V_5_6_1, + TestVersionBumpIn6x.V_6_0_0_alpha1, TestVersionBumpIn6x.V_6_0_0_alpha2, + TestVersionBumpIn6x.V_6_0_0_beta1, TestVersionBumpIn6x.V_6_0_0_beta2, + TestVersionBumpIn6x.V_6_0_0), released); + assertEquals(Arrays.asList(TestVersionBumpIn6x.V_5_6_2, TestVersionBumpIn6x.V_6_0_1, TestVersionBumpIn6x.V_6_1_0), unreleased); + } + + public static class TestNewMinorBranchIn6x { + public static final Version V_5_6_0 = Version.fromString("5.6.0"); + public static final Version V_5_6_1 = Version.fromString("5.6.1"); + public static final Version V_5_6_2 = Version.fromString("5.6.2"); + public static final Version V_6_0_0_alpha1 = Version.fromString("6.0.0-alpha1"); + public static final Version V_6_0_0_alpha2 = Version.fromString("6.0.0-alpha2"); + public static final Version V_6_0_0_beta1 = Version.fromString("6.0.0-beta1"); + public static final Version V_6_0_0_beta2 = Version.fromString("6.0.0-beta2"); + public static final Version V_6_0_0 = Version.fromString("6.0.0"); + public static final Version V_6_0_1 = Version.fromString("6.0.1"); + public static final Version V_6_1_0 = Version.fromString("6.1.0"); + public static final Version V_6_1_1 = Version.fromString("6.1.1"); + public static final Version V_6_1_2 = Version.fromString("6.1.2"); + public static final Version V_6_2_0 = Version.fromString("6.2.0"); + public static final Version CURRENT = V_6_2_0; + } + + public void testResolveReleasedVersionsAtNewMinorBranchIn6x() { + Tuple, List> t = VersionUtils.resolveReleasedVersions(TestNewMinorBranchIn6x.CURRENT, + TestNewMinorBranchIn6x.class); + List released = t.v1(); + List unreleased = t.v2(); + assertEquals(Arrays.asList(TestNewMinorBranchIn6x.V_5_6_0, TestNewMinorBranchIn6x.V_5_6_1, + TestNewMinorBranchIn6x.V_6_0_0_alpha1, TestNewMinorBranchIn6x.V_6_0_0_alpha2, + TestNewMinorBranchIn6x.V_6_0_0_beta1, TestNewMinorBranchIn6x.V_6_0_0_beta2, + TestNewMinorBranchIn6x.V_6_0_0, TestNewMinorBranchIn6x.V_6_1_0, TestNewMinorBranchIn6x.V_6_1_1), released); + assertEquals(Arrays.asList(TestNewMinorBranchIn6x.V_5_6_2, TestNewMinorBranchIn6x.V_6_0_1, + TestNewMinorBranchIn6x.V_6_1_2, TestNewMinorBranchIn6x.V_6_2_0), unreleased); + } + /** * Tests that {@link Version#minimumCompatibilityVersion()} and {@link VersionUtils#allReleasedVersions()} * agree with the list of wire and index compatible versions we build in gradle. @@ -181,8 +262,9 @@ public void testGradleVersionsMatchVersionUtils() { // First check the index compatible versions VersionsFromProperty indexCompatible = new VersionsFromProperty("tests.gradle_index_compat_versions"); List released = VersionUtils.allReleasedVersions().stream() - // Java lists some non-index compatible versions but gradle does not include them. - .filter(v -> v.major == Version.CURRENT.major || v.major == Version.CURRENT.major - 1) + /* Java lists all versions from the 5.x series onwards, but we only want to consider + * ones that we're supposed to be compatible with. */ + .filter(v -> v.onOrAfter(Version.CURRENT.minimumIndexCompatibilityVersion())) /* Gradle will never include *released* alphas or betas because it will prefer * the unreleased branch head. Gradle is willing to use branch heads that are * beta or rc so that we have *something* to test against even though we @@ -199,6 +281,9 @@ public void testGradleVersionsMatchVersionUtils() { /* Gradle skips the current version because being backwards compatible * with yourself is implied. Java lists the version because it is useful. */ .filter(v -> v != Version.CURRENT) + /* Java lists all versions from the 5.x series onwards, but we only want to consider + * ones that we're supposed to be compatible with. */ + .filter(v -> v.onOrAfter(Version.CURRENT.minimumIndexCompatibilityVersion())) /* Note that gradle skips alphas because they don't have any backwards * compatibility guarantees but keeps the last beta and rc in a branch * on when there are only betas an RCs in that branch so that we have From 492edb91b9b3c353e953889f539e10785718a575 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 15 Nov 2017 09:54:32 +0000 Subject: [PATCH 04/31] Bump version to 6.0.1 --- core/src/main/java/org/elasticsearch/Version.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/Version.java b/core/src/main/java/org/elasticsearch/Version.java index e1f5a5da14c1a..e5b2fa25e832f 100644 --- a/core/src/main/java/org/elasticsearch/Version.java +++ b/core/src/main/java/org/elasticsearch/Version.java @@ -128,6 +128,9 @@ public class Version implements Comparable { public static final int V_6_0_0_ID = 6000099; public static final Version V_6_0_0 = new Version(V_6_0_0_ID, org.apache.lucene.util.Version.LUCENE_7_0_1); + public static final int V_6_0_1_ID = 6000199; + public static final Version V_6_0_1 = + new Version(V_6_0_1_ID, org.apache.lucene.util.Version.LUCENE_7_0_1); public static final int V_6_1_0_ID = 6010099; public static final Version V_6_1_0 = new Version(V_6_1_0_ID, org.apache.lucene.util.Version.LUCENE_7_1_0); @@ -136,8 +139,6 @@ public class Version implements Comparable { new Version(V_7_0_0_alpha1_ID, org.apache.lucene.util.Version.LUCENE_7_1_0); public static final Version CURRENT = V_7_0_0_alpha1; - // unreleased versions must be added to the above list with the suffix _UNRELEASED (with the exception of CURRENT) - static { assert CURRENT.luceneVersion.equals(org.apache.lucene.util.Version.LATEST) : "Version must be upgraded to [" + org.apache.lucene.util.Version.LATEST + "] is still set to [" + CURRENT.luceneVersion + "]"; @@ -153,6 +154,8 @@ public static Version fromId(int id) { return V_7_0_0_alpha1; case V_6_1_0_ID: return V_6_1_0; + case V_6_0_1_ID: + return V_6_0_1; case V_6_0_0_ID: return V_6_0_0; case V_6_0_0_rc2_ID: From f761a0e0e47b1d8787b2c537253394dd8a807673 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 16 Nov 2017 18:24:06 -0700 Subject: [PATCH 05/31] Remove unneeded Throwable handling in nio (#27412) This is related to #27260. In the nio transport work we do not catch or handle `Throwable`. There are a few places where we have exception handlers that accept `Throwable`. This commit removes those cases. --- .../java/org/elasticsearch/transport/nio/NioTransport.java | 6 ++---- .../org/elasticsearch/transport/nio/SocketEventHandler.java | 4 ++-- .../org/elasticsearch/transport/nio/TcpReadHandler.java | 5 ++--- .../elasticsearch/transport/nio/channel/TcpReadContext.java | 2 +- .../transport/nio/SocketEventHandlerTests.java | 2 +- .../transport/nio/TestingSocketEventHandler.java | 5 +---- .../transport/nio/channel/TcpReadContextTests.java | 4 ++-- 7 files changed, 11 insertions(+), 17 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java index 27cb73f98f8c2..2902b0bccfd9d 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java @@ -184,9 +184,7 @@ protected SocketEventHandler getSocketEventHandler() { return new SocketEventHandler(logger, this::exceptionCaught, openChannels); } - final void exceptionCaught(NioSocketChannel channel, Throwable cause) { - final Throwable unwrapped = ExceptionsHelper.unwrap(cause, ElasticsearchException.class); - final Throwable t = unwrapped != null ? unwrapped : cause; - onException(channel, t instanceof Exception ? (Exception) t : new ElasticsearchException(t)); + final void exceptionCaught(NioSocketChannel channel, Exception exception) { + onException(channel, exception); } } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketEventHandler.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketEventHandler.java index b04ecc4ea9a6f..46292f63d1bda 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketEventHandler.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketEventHandler.java @@ -34,10 +34,10 @@ */ public class SocketEventHandler extends EventHandler { - private final BiConsumer exceptionHandler; + private final BiConsumer exceptionHandler; private final Logger logger; - public SocketEventHandler(Logger logger, BiConsumer exceptionHandler, OpenChannels openChannels) { + public SocketEventHandler(Logger logger, BiConsumer exceptionHandler, OpenChannels openChannels) { super(logger, openChannels); this.exceptionHandler = exceptionHandler; this.logger = logger; diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/TcpReadHandler.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/TcpReadHandler.java index b41d87a0c0998..2cb59ed95d1c5 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/TcpReadHandler.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/TcpReadHandler.java @@ -32,10 +32,9 @@ public TcpReadHandler(NioTransport transport) { this.transport = transport; } - public void handleMessage(BytesReference reference, NioSocketChannel channel, String profileName, - int messageBytesLength) { + public void handleMessage(BytesReference reference, NioSocketChannel channel, int messageBytesLength) { try { - transport.messageReceived(reference, channel, profileName, channel.getRemoteAddress(), messageBytesLength); + transport.messageReceived(reference, channel, channel.getProfile(), channel.getRemoteAddress(), messageBytesLength); } catch (IOException e) { handleException(channel, e); } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpReadContext.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpReadContext.java index ee56f6f6f9763..57aa16ce15e3b 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpReadContext.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpReadContext.java @@ -80,7 +80,7 @@ public int read() throws IOException { // A message length of 6 bytes it is just a ping. Ignore for now. if (messageLengthWithHeader != 6) { - handler.handleMessage(messageWithoutHeader, channel, channel.getProfile(), messageWithoutHeader.length()); + handler.handleMessage(messageWithoutHeader, channel, messageWithoutHeader.length()); } } catch (Exception e) { handler.handleException(channel, e); diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketEventHandlerTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketEventHandlerTests.java index b1c6fab2065a9..abbe023b97c0b 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketEventHandlerTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketEventHandlerTests.java @@ -43,7 +43,7 @@ public class SocketEventHandlerTests extends ESTestCase { - private BiConsumer exceptionHandler; + private BiConsumer exceptionHandler; private SocketEventHandler handler; private NioSocketChannel channel; diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/TestingSocketEventHandler.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/TestingSocketEventHandler.java index 7d3cf97ee08ee..65759cf770552 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/TestingSocketEventHandler.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/TestingSocketEventHandler.java @@ -30,11 +30,8 @@ public class TestingSocketEventHandler extends SocketEventHandler { - private final Logger logger; - - public TestingSocketEventHandler(Logger logger, BiConsumer exceptionHandler, OpenChannels openChannels) { + public TestingSocketEventHandler(Logger logger, BiConsumer exceptionHandler, OpenChannels openChannels) { super(logger, exceptionHandler, openChannels); - this.logger = logger; } private Set hasConnectedMap = Collections.newSetFromMap(new WeakHashMap<>()); diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpReadContextTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpReadContextTests.java index fc8d7e48ab0bf..2b4db98010238 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpReadContextTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpReadContextTests.java @@ -72,7 +72,7 @@ public void testSuccessfulRead() throws IOException { readContext.read(); - verify(handler).handleMessage(new BytesArray(bytes), channel, PROFILE, messageLength); + verify(handler).handleMessage(new BytesArray(bytes), channel, messageLength); assertEquals(1024 * 16, bufferCapacity.get()); BytesArray bytesArray = new BytesArray(new byte[10]); @@ -110,7 +110,7 @@ public void testPartialRead() throws IOException { assertEquals(1024 * 16 - fullPart1.length, bufferCapacity.get()); CompositeBytesReference reference = new CompositeBytesReference(new BytesArray(part1), new BytesArray(part2)); - verify(handler).handleMessage(reference, channel, PROFILE, messageLength + messageLength); + verify(handler).handleMessage(reference, channel, messageLength + messageLength); } public void testReadThrowsIOException() throws IOException { From 53462f6499617c6152718a5b7b2e618a8d093dba Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Fri, 17 Nov 2017 10:25:21 +0100 Subject: [PATCH 06/31] Make fields optional in multi_match query and rely on index.query.default_field by default (#27380) * Make fields optional in multi_match query and rely on index.query.default_field by default This commit adds the ability to send `multi_match` query without providing any `fields`. When no fields are provided the `multi_match` query will use the fields defined in the index setting `index.query.default_field` (which in turns defaults to `*`). The same behavior is already implemented in `query_string` and `simple_query_string` so this change just applies the heuristic to `multi_match` queries. Relying on `index.query.default_field` rather than `*` is safer for big mappings that break the 1024 field expansion limit added in 7.0 for all text queries. For these kind of mappings the admin can change the `index.query.default_field` in order to make sure that exploratory queries using `multi_match`, `query_string` or `simple_query_string` do not throw an exception. --- .../index/query/MultiMatchQueryBuilder.java | 57 +++++++++---- .../query/MultiMatchQueryBuilderTests.java | 80 ++++++++++++++++--- .../query-dsl/multi-match-query.asciidoc | 5 ++ 3 files changed, 117 insertions(+), 25 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java index 6063b8a120491..c9a5e7e52e968 100644 --- a/core/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -36,9 +37,11 @@ import org.elasticsearch.index.search.MatchQuery; import org.elasticsearch.index.search.MultiMatchQuery; import org.elasticsearch.index.search.QueryParserHelper; +import org.elasticsearch.index.search.QueryStringQueryParser; import java.io.IOException; import java.util.HashMap; +import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Objects; @@ -55,7 +58,6 @@ public class MultiMatchQueryBuilder extends AbstractQueryBuilder fieldsBoosts) throws IOException { @@ -778,7 +789,9 @@ protected Query doToQuery(QueryShardContext context) throws IOException { if (cutoffFrequency != null) { multiMatchQuery.setCommonTermsCutoff(cutoffFrequency); } - multiMatchQuery.setLenient(lenient); + if (lenient != null) { + multiMatchQuery.setLenient(lenient); + } multiMatchQuery.setZeroTermsQuery(zeroTermsQuery); multiMatchQuery.setAutoGenerateSynonymsPhraseQuery(autoGenerateSynonymsPhraseQuery); multiMatchQuery.setTranspositions(fuzzyTranspositions); @@ -793,8 +806,20 @@ protected Query doToQuery(QueryShardContext context) throws IOException { } } } - - Map newFieldsBoosts = QueryParserHelper.resolveMappingFields(context, fieldsBoosts); + Map newFieldsBoosts; + if (fieldsBoosts.isEmpty()) { + // no fields provided, defaults to index.query.default_field + List defaultFields = context.defaultFields(); + boolean isAllField = defaultFields.size() == 1 && Regex.isMatchAllPattern(defaultFields.get(0)); + if (isAllField && lenient == null) { + // Sets leniency to true if not explicitly + // set in the request + multiMatchQuery.setLenient(true); + } + newFieldsBoosts = QueryParserHelper.resolveMappingFields(context, QueryParserHelper.parseFieldsAndWeights(defaultFields)); + } else { + newFieldsBoosts = QueryParserHelper.resolveMappingFields(context, fieldsBoosts); + } return multiMatchQuery.parse(type, newFieldsBoosts, value, minimumShouldMatch); } diff --git a/core/src/test/java/org/elasticsearch/index/query/MultiMatchQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/MultiMatchQueryBuilderTests.java index a0afe28a17bce..e81edb7dcf95d 100644 --- a/core/src/test/java/org/elasticsearch/index/query/MultiMatchQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/MultiMatchQueryBuilderTests.java @@ -32,8 +32,10 @@ import org.apache.lucene.search.PointRangeQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.lucene.search.MultiPhrasePrefixQuery; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.index.query.MultiMatchQueryBuilder.Type; import org.elasticsearch.index.search.MatchQuery; @@ -41,6 +43,7 @@ import org.elasticsearch.test.AbstractQueryTestCase; import java.io.IOException; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -66,18 +69,28 @@ protected MultiMatchQueryBuilder doCreateTestQueryBuilder() { assumeTrue("test with date fields runs only when at least a type is registered", getCurrentTypes().length > 0); } - // creates the query with random value and field name - Object value; + final Object value; if (fieldName.equals(STRING_FIELD_NAME)) { value = getRandomQueryText(); } else { value = getRandomValueForFieldName(fieldName); } - MultiMatchQueryBuilder query = new MultiMatchQueryBuilder(value, fieldName); - // field with random boost - if (randomBoolean()) { - query.field(fieldName, randomFloat() * 10); + + final MultiMatchQueryBuilder query; + if (rarely()) { + query = new MultiMatchQueryBuilder(value, fieldName); + if (randomBoolean()) { + query.lenient(randomBoolean()); + } + // field with random boost + if (randomBoolean()) { + query.field(fieldName, randomFloat() * 10); + } + } else { + query = new MultiMatchQueryBuilder(value); + query.lenient(true); } + // sets other parameters of the multi match query if (randomBoolean()) { query.type(randomFrom(MultiMatchQueryBuilder.Type.values())); @@ -112,9 +125,6 @@ protected MultiMatchQueryBuilder doCreateTestQueryBuilder() { if (randomBoolean()) { query.tieBreaker(randomFloat()); } - if (randomBoolean()) { - query.lenient(randomBoolean()); - } if (randomBoolean()) { query.cutoffFrequency((float) 10 / randomIntBetween(1, 100)); } @@ -338,4 +348,56 @@ public void testToFuzzyQuery() throws Exception { assertEquals(expected, query); } + + public void testDefaultField() throws Exception { + assumeTrue("test runs only when at least a type is registered", getCurrentTypes().length > 0); + QueryShardContext context = createShardContext(); + MultiMatchQueryBuilder builder = new MultiMatchQueryBuilder("hello"); + // should pass because we set lenient to true when default field is `*` + Query query = builder.toQuery(context); + assertThat(query, instanceOf(DisjunctionMaxQuery.class)); + + context.getIndexSettings().updateIndexMetaData( + newIndexMeta("index", context.getIndexSettings().getSettings(), Settings.builder().putList("index.query.default_field", + STRING_FIELD_NAME, STRING_FIELD_NAME_2 + "^5").build()) + ); + + MultiMatchQueryBuilder qb = new MultiMatchQueryBuilder("hello"); + query = qb.toQuery(context); + DisjunctionMaxQuery expected = new DisjunctionMaxQuery( + Arrays.asList( + new TermQuery(new Term(STRING_FIELD_NAME, "hello")), + new BoostQuery(new TermQuery(new Term(STRING_FIELD_NAME_2, "hello")), 5.0f) + ), 0.0f + ); + assertEquals(expected, query); + + context.getIndexSettings().updateIndexMetaData( + newIndexMeta("index", context.getIndexSettings().getSettings(), Settings.builder().putList("index.query.default_field", + STRING_FIELD_NAME, STRING_FIELD_NAME_2 + "^5", INT_FIELD_NAME).build()) + ); + // should fail because lenient defaults to false + IllegalArgumentException exc = expectThrows(IllegalArgumentException.class, () -> qb.toQuery(context)); + assertThat(exc, instanceOf(NumberFormatException.class)); + assertThat(exc.getMessage(), equalTo("For input string: \"hello\"")); + + // explicitly sets lenient + qb.lenient(true); + query = qb.toQuery(context); + expected = new DisjunctionMaxQuery( + Arrays.asList( + new TermQuery(new Term(STRING_FIELD_NAME, "hello")), + new BoostQuery(new TermQuery(new Term(STRING_FIELD_NAME_2, "hello")), 5.0f), + new MatchNoDocsQuery("failed [mapped_int] query, caused by number_format_exception:[For input string: \"hello\"]") + ), 0.0f + ); + assertEquals(expected, query); + } + + private static IndexMetaData newIndexMeta(String name, Settings oldIndexSettings, Settings indexSettings) { + Settings build = Settings.builder().put(oldIndexSettings) + .put(indexSettings) + .build(); + return IndexMetaData.builder(name).settings(build).build(); + } } diff --git a/docs/reference/query-dsl/multi-match-query.asciidoc b/docs/reference/query-dsl/multi-match-query.asciidoc index 217d3a7d21165..edb6ff11da7de 100644 --- a/docs/reference/query-dsl/multi-match-query.asciidoc +++ b/docs/reference/query-dsl/multi-match-query.asciidoc @@ -58,6 +58,11 @@ GET /_search <1> The `subject` field is three times as important as the `message` field. +If no `fields` are provided, the `multi_match` query defaults to the `index.query.default_field` +index settings, which in turn defaults to `*`. `*` extracts all fields in the mapping that +are eligible to term queries and filters the metadata fields. All extracted fields are then +combined to build a query. + WARNING: There is a limit of no more than 1024 fields being queried at once. [[multi-match-types]] From c8598fd573e8545cf620278185e99c1b631ee432 Mon Sep 17 00:00:00 2001 From: Scott Newson Date: Fri, 17 Nov 2017 04:10:50 -0700 Subject: [PATCH 07/31] Fix line-break in README.textile (#27416) --- README.textile | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/README.textile b/README.textile index ac40517f9781c..63c45abfe0e38 100644 --- a/README.textile +++ b/README.textile @@ -208,8 +208,7 @@ In order to create a distribution, simply run the @gradle assemble@ command in t The distribution for each project will be created under the @build/distributions@ directory in that project. -See the "TESTING":TESTING.asciidoc file for more information about -running the Elasticsearch test suite. +See the "TESTING":TESTING.asciidoc file for more information about running the Elasticsearch test suite. h3. Upgrading from Elasticsearch 1.x? From 08a257327f24c40e48d4af806681c32892de2a9a Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 17 Nov 2017 14:12:06 +0000 Subject: [PATCH 08/31] Remove newline from log message (#27425) It leads to harder-to-parse logs that look like this: ``` 1> [2017-11-16T20:46:21,804][INFO ][o.e.t.r.y.ClientYamlTestClient] Adding header Content-Type 1> with value application/json 1> [2017-11-16T20:46:21,812][INFO ][o.e.t.r.y.ClientYamlTestClient] Adding header Content-Type 1> with value application/json 1> [2017-11-16T20:46:21,820][INFO ][o.e.t.r.y.ClientYamlTestClient] Adding header Content-Type 1> with value application/json 1> [2017-11-16T20:46:21,966][INFO ][o.e.t.r.y.ClientYamlTestClient] Adding header Content-Type 1> with value application/json ``` --- .../org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java index b4704bd9ed896..6e2f43ae75281 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java @@ -165,7 +165,7 @@ public ClientYamlTestResponse callApi(String apiName, Map params Header[] requestHeaders = new Header[headers.size()]; int index = 0; for (Map.Entry header : headers.entrySet()) { - logger.info("Adding header {}\n with value {}", header.getKey(), header.getValue()); + logger.info("Adding header {} with value {}", header.getKey(), header.getValue()); requestHeaders[index++] = new BasicHeader(header.getKey(), header.getValue()); } From c91b7cad831552188fd5956fa94e6efa670160bb Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Fri, 17 Nov 2017 15:45:26 +0100 Subject: [PATCH 09/31] [#27380] Adjust bwc for multi_match lenient option --- .../org/elasticsearch/index/query/MultiMatchQueryBuilder.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java index c9a5e7e52e968..0411b955b6547 100644 --- a/core/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java @@ -225,7 +225,7 @@ public MultiMatchQueryBuilder(StreamInput in) throws IOException { fuzzyRewrite = in.readOptionalString(); useDisMax = in.readOptionalBoolean(); tieBreaker = in.readOptionalFloat(); - if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + if (in.getVersion().onOrAfter(Version.V_6_1_0)) { lenient = in.readOptionalBoolean(); } else { lenient = in.readBoolean(); @@ -257,7 +257,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeOptionalString(fuzzyRewrite); out.writeOptionalBoolean(useDisMax); out.writeOptionalFloat(tieBreaker); - if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + if (out.getVersion().onOrAfter(Version.V_6_1_0)) { out.writeOptionalBoolean(lenient); } else { out.writeBoolean(lenient == null ? MatchQuery.DEFAULT_LENIENCY : lenient); From 76203e72bd445ab36ed19ad8a9a79188f2574958 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 17 Nov 2017 17:27:19 +0100 Subject: [PATCH 10/31] Fix place-holder in allocation decider messages (#27436) Allocation decider messages were using the wrong place-holder, which resulted in output of the form "no allocations are allowed due to {}" when showing diagnostics information in the explain API. --- .../routing/allocation/decider/AllocationDecider.java | 2 +- .../routing/allocation/decider/EnableAllocationDecider.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AllocationDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AllocationDecider.java index 011b6c97737d8..12cac56e11a44 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AllocationDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AllocationDecider.java @@ -118,7 +118,7 @@ public Decision canForceAllocatePrimary(ShardRouting shardRouting, RoutingNode n // On a NO decision, by default, we allow force allocating the primary. return allocation.decision(Decision.YES, decision.label(), - "primary shard [{}] allowed to force allocate on node [{}]", + "primary shard [%s] allowed to force allocate on node [%s]", shardRouting.shardId(), node.nodeId()); } else { // On a THROTTLE/YES decision, we use the same decision instead of forcing allocation diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java index 7bb073a4c4561..50ed065d8a3db 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java @@ -116,20 +116,20 @@ public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, Routing case ALL: return allocation.decision(Decision.YES, NAME, "all allocations are allowed"); case NONE: - return allocation.decision(Decision.NO, NAME, "no allocations are allowed due to {}", setting(enable, usedIndexSetting)); + return allocation.decision(Decision.NO, NAME, "no allocations are allowed due to %s", setting(enable, usedIndexSetting)); case NEW_PRIMARIES: if (shardRouting.primary() && shardRouting.active() == false && shardRouting.recoverySource().getType() != RecoverySource.Type.EXISTING_STORE) { return allocation.decision(Decision.YES, NAME, "new primary allocations are allowed"); } else { - return allocation.decision(Decision.NO, NAME, "non-new primary allocations are forbidden due to {}", + return allocation.decision(Decision.NO, NAME, "non-new primary allocations are forbidden due to %s", setting(enable, usedIndexSetting)); } case PRIMARIES: if (shardRouting.primary()) { return allocation.decision(Decision.YES, NAME, "primary allocations are allowed"); } else { - return allocation.decision(Decision.NO, NAME, "replica allocations are forbidden due to {}", + return allocation.decision(Decision.NO, NAME, "replica allocations are forbidden due to %s", setting(enable, usedIndexSetting)); } default: From d92afa1e0a53ac0eaad47e8430028cd013279c2f Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Fri, 17 Nov 2017 09:38:19 -0700 Subject: [PATCH 11/31] Enforce a minimum task execution and service time of 1 nanosecond Resolves #27371 --- .../elasticsearch/common/util/concurrent/TimedRunnable.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java b/core/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java index 2d8934ba3b30e..ad5519c0a76df 100644 --- a/core/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java +++ b/core/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java @@ -79,7 +79,7 @@ long getTotalNanos() { // There must have been an exception thrown, the total time is unknown (-1) return -1; } - return finishTimeNanos - creationTimeNanos; + return Math.max(finishTimeNanos - creationTimeNanos, 1); } /** @@ -91,7 +91,7 @@ long getTotalExecutionNanos() { // There must have been an exception thrown, the total time is unknown (-1) return -1; } - return finishTimeNanos - startTimeNanos; + return Math.max(finishTimeNanos - startTimeNanos, 1); } } From db688e1a174b8a732b999426048473437411b478 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 17 Nov 2017 11:54:44 -0500 Subject: [PATCH 12/31] Uses TransportMasterNodeAction to update shard snapshot status (#27165) Currently, we are using a plain TransportRequestHandler to post snapshot status messages to the master. However, it doesn't have a robust retry mechanism as TransportMasterNodeAction. This change migrates from TransportRequestHandler to TransportMasterNodeAction for the new versions and keeps the current implementation for the old versions. Closes #27151 --- .../snapshots/SnapshotShardsService.java | 178 +++++++++++++++--- .../snapshots/SnapshotShardsServiceIT.java | 115 +++++++++++ qa/mixed-cluster/build.gradle | 5 + .../elasticsearch/backwards/IndexingIT.java | 54 ++++++ 4 files changed, 331 insertions(+), 21 deletions(-) create mode 100644 core/src/test/java/org/elasticsearch/snapshots/SnapshotShardsServiceIT.java diff --git a/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 259136ca9ccfd..f8a601cc41fd2 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -23,14 +23,24 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.MasterNodeRequest; +import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateApplier; +import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; +import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.cluster.SnapshotsInProgress.ShardSnapshotStatus; import org.elasticsearch.cluster.SnapshotsInProgress.State; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; @@ -85,9 +95,11 @@ * This service runs on data and master nodes and controls currently snapshotted shards on these nodes. It is responsible for * starting and stopping shard level snapshots */ -public class SnapshotShardsService extends AbstractLifecycleComponent implements ClusterStateApplier, IndexEventListener { +public class SnapshotShardsService extends AbstractLifecycleComponent implements ClusterStateListener, IndexEventListener { + + public static final String UPDATE_SNAPSHOT_STATUS_ACTION_NAME_V6 = "internal:cluster/snapshot/update_snapshot"; + public static final String UPDATE_SNAPSHOT_STATUS_ACTION_NAME = "internal:cluster/snapshot/update_snapshot_status"; - public static final String UPDATE_SNAPSHOT_ACTION_NAME = "internal:cluster/snapshot/update_snapshot"; private final ClusterService clusterService; @@ -106,10 +118,12 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements private volatile Map shardSnapshots = emptyMap(); private final SnapshotStateExecutor snapshotStateExecutor = new SnapshotStateExecutor(); + private UpdateSnapshotStatusAction updateSnapshotStatusHandler; @Inject public SnapshotShardsService(Settings settings, ClusterService clusterService, SnapshotsService snapshotsService, ThreadPool threadPool, - TransportService transportService, IndicesService indicesService) { + TransportService transportService, IndicesService indicesService, + ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { super(settings); this.indicesService = indicesService; this.snapshotsService = snapshotsService; @@ -118,20 +132,27 @@ public SnapshotShardsService(Settings settings, ClusterService clusterService, S this.threadPool = threadPool; if (DiscoveryNode.isDataNode(settings)) { // this is only useful on the nodes that can hold data - // addLowPriorityApplier to make sure that Repository will be created before snapshot - clusterService.addLowPriorityApplier(this); + clusterService.addListener(this); } + // The constructor of UpdateSnapshotStatusAction will register itself to the TransportService. + this.updateSnapshotStatusHandler = new UpdateSnapshotStatusAction(settings, UPDATE_SNAPSHOT_STATUS_ACTION_NAME, + transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver); + if (DiscoveryNode.isMasterNode(settings)) { // This needs to run only on nodes that can become masters - transportService.registerRequestHandler(UPDATE_SNAPSHOT_ACTION_NAME, UpdateIndexShardSnapshotStatusRequest::new, ThreadPool.Names.SAME, new UpdateSnapshotStateRequestHandler()); + transportService.registerRequestHandler(UPDATE_SNAPSHOT_STATUS_ACTION_NAME_V6, UpdateSnapshotStatusRequestV6::new, ThreadPool.Names.SAME, new UpdateSnapshotStateRequestHandlerV6()); } } @Override protected void doStart() { - + assert this.updateSnapshotStatusHandler != null; + assert transportService.getRequestHandler(UPDATE_SNAPSHOT_STATUS_ACTION_NAME) != null; + if (DiscoveryNode.isMasterNode(settings)) { + assert transportService.getRequestHandler(UPDATE_SNAPSHOT_STATUS_ACTION_NAME_V6) != null; + } } @Override @@ -151,11 +172,11 @@ protected void doStop() { @Override protected void doClose() { - clusterService.removeApplier(this); + clusterService.removeListener(this); } @Override - public void applyClusterState(ClusterChangedEvent event) { + public void clusterChanged(ClusterChangedEvent event) { try { SnapshotsInProgress prev = event.previousState().custom(SnapshotsInProgress.TYPE); SnapshotsInProgress curr = event.state().custom(SnapshotsInProgress.TYPE); @@ -449,7 +470,7 @@ private SnapshotShards(Map shards) { /** * Internal request that is used to send changes in snapshot status to master */ - public static class UpdateIndexShardSnapshotStatusRequest extends TransportRequest { + public static class UpdateIndexShardSnapshotStatusRequest extends MasterNodeRequest { private Snapshot snapshot; private ShardId shardId; private ShardSnapshotStatus status; @@ -462,6 +483,13 @@ public UpdateIndexShardSnapshotStatusRequest(Snapshot snapshot, ShardId shardId, this.snapshot = snapshot; this.shardId = shardId; this.status = status; + // By default, we keep trying to post snapshot status messages to avoid snapshot processes getting stuck. + this.masterNodeTimeout = TimeValue.timeValueNanos(Long.MAX_VALUE); + } + + @Override + public ActionRequestValidationException validate() { + return null; } @Override @@ -502,11 +530,16 @@ public String toString() { * Updates the shard status */ public void updateIndexShardSnapshotStatus(Snapshot snapshot, ShardId shardId, ShardSnapshotStatus status, DiscoveryNode master) { - UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(snapshot, shardId, status); try { - transportService.sendRequest(master, UPDATE_SNAPSHOT_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME); + if (master.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(snapshot, shardId, status); + transportService.sendRequest(transportService.getLocalNode(), UPDATE_SNAPSHOT_STATUS_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME); + } else { + UpdateSnapshotStatusRequestV6 requestV6 = new UpdateSnapshotStatusRequestV6(snapshot, shardId, status); + transportService.sendRequest(master, UPDATE_SNAPSHOT_STATUS_ACTION_NAME_V6, requestV6, EmptyTransportResponseHandler.INSTANCE_SAME); + } } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("[{}] [{}] failed to update snapshot state", request.snapshot(), request.status()), e); + logger.warn((Supplier) () -> new ParameterizedMessage("[{}] [{}] failed to update snapshot state", snapshot, status), e); } } @@ -515,15 +548,24 @@ public void updateIndexShardSnapshotStatus(Snapshot snapshot, ShardId shardId, S * * @param request update shard status request */ - private void innerUpdateSnapshotState(final UpdateIndexShardSnapshotStatusRequest request) { + private void innerUpdateSnapshotState(final UpdateIndexShardSnapshotStatusRequest request, ActionListener listener) { logger.trace("received updated snapshot restore state [{}]", request); clusterService.submitStateUpdateTask( "update snapshot state", request, ClusterStateTaskConfig.build(Priority.NORMAL), snapshotStateExecutor, - (source, e) -> logger.warn((Supplier) () -> new ParameterizedMessage("[{}][{}] failed to update snapshot status to [{}]", - request.snapshot(), request.shardId(), request.status()), e)); + new ClusterStateTaskListener() { + @Override + public void onFailure(String source, Exception e) { + listener.onFailure(e); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + listener.onResponse(new UpdateIndexShardSnapshotStatusResponse()); + } + }); } class SnapshotStateExecutor implements ClusterStateTaskExecutor { @@ -578,13 +620,107 @@ public ClusterTasksResult execute(Cluster } } + static class UpdateIndexShardSnapshotStatusResponse extends ActionResponse { + + } + + class UpdateSnapshotStatusAction extends TransportMasterNodeAction { + UpdateSnapshotStatusAction(Settings settings, String actionName, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { + super(settings, actionName, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, UpdateIndexShardSnapshotStatusRequest::new); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected UpdateIndexShardSnapshotStatusResponse newResponse() { + return new UpdateIndexShardSnapshotStatusResponse(); + } + + @Override + protected void masterOperation(UpdateIndexShardSnapshotStatusRequest request, ClusterState state, ActionListener listener) throws Exception { + innerUpdateSnapshotState(request, listener); + } + + @Override + protected ClusterBlockException checkBlock(UpdateIndexShardSnapshotStatusRequest request, ClusterState state) { + return null; + } + } + + /** + * A BWC version of {@link UpdateIndexShardSnapshotStatusRequest} + */ + static class UpdateSnapshotStatusRequestV6 extends TransportRequest { + private Snapshot snapshot; + private ShardId shardId; + private ShardSnapshotStatus status; + + UpdateSnapshotStatusRequestV6() { + + } + + UpdateSnapshotStatusRequestV6(Snapshot snapshot, ShardId shardId, ShardSnapshotStatus status) { + this.snapshot = snapshot; + this.shardId = shardId; + this.status = status; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + snapshot = new Snapshot(in); + shardId = ShardId.readShardId(in); + status = new ShardSnapshotStatus(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + snapshot.writeTo(out); + shardId.writeTo(out); + status.writeTo(out); + } + + Snapshot snapshot() { + return snapshot; + } + + ShardId shardId() { + return shardId; + } + + ShardSnapshotStatus status() { + return status; + } + + @Override + public String toString() { + return snapshot + ", shardId [" + shardId + "], status [" + status.state() + "]"; + } + } + /** - * Transport request handler that is used to send changes in snapshot status to master + * A BWC version of {@link UpdateSnapshotStatusAction} */ - class UpdateSnapshotStateRequestHandler implements TransportRequestHandler { + class UpdateSnapshotStateRequestHandlerV6 implements TransportRequestHandler { @Override - public void messageReceived(UpdateIndexShardSnapshotStatusRequest request, final TransportChannel channel) throws Exception { - innerUpdateSnapshotState(request); + public void messageReceived(UpdateSnapshotStatusRequestV6 requestV6, final TransportChannel channel) throws Exception { + final UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(requestV6.snapshot(), requestV6.shardId(), requestV6.status()); + innerUpdateSnapshotState(request, new ActionListener() { + @Override + public void onResponse(UpdateIndexShardSnapshotStatusResponse updateSnapshotStatusResponse) { + + } + + @Override + public void onFailure(Exception e) { + logger.warn("Failed to update snapshot status", e); + } + }); channel.sendResponse(TransportResponse.Empty.INSTANCE); } } diff --git a/core/src/test/java/org/elasticsearch/snapshots/SnapshotShardsServiceIT.java b/core/src/test/java/org/elasticsearch/snapshots/SnapshotShardsServiceIT.java new file mode 100644 index 0000000000000..651cd96776e75 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/snapshots/SnapshotShardsServiceIT.java @@ -0,0 +1,115 @@ +/* + * 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.snapshots; + +import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.snapshots.mockstore.MockRepository; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.disruption.NetworkDisruption; +import org.elasticsearch.test.transport.MockTransportService; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.everyItem; +import static org.hamcrest.Matchers.hasSize; + +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, transportClientRatio = 0) +public class SnapshotShardsServiceIT extends AbstractSnapshotIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(MockRepository.Plugin.class, MockTransportService.TestPlugin.class); + } + + public void testRetryPostingSnapshotStatusMessages() throws Exception { + String masterNode = internalCluster().startMasterOnlyNode(); + String dataNode = internalCluster().startDataOnlyNode(); + + logger.info("--> creating repository"); + assertAcked(client().admin().cluster().preparePutRepository("test-repo") + .setType("mock").setSettings(Settings.builder() + .put("location", randomRepoPath()) + .put("compress", randomBoolean()) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES))); + + final int shards = between(1, 10); + assertAcked(prepareCreate("test-index", 0, Settings.builder().put("number_of_shards", shards).put("number_of_replicas", 0))); + ensureGreen(); + final int numDocs = scaledRandomIntBetween(50, 100); + for (int i = 0; i < numDocs; i++) { + index("test-index", "doc", Integer.toString(i)); + } + + logger.info("--> blocking repository"); + String blockedNode = blockNodeWithIndex("test-repo", "test-index"); + dataNodeClient().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap") + .setWaitForCompletion(false) + .setIndices("test-index") + .get(); + waitForBlock(blockedNode, "test-repo", TimeValue.timeValueSeconds(60)); + + final SnapshotId snapshotId = client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap") + .get().getSnapshots().get(0).snapshotId(); + + logger.info("--> start disrupting cluster"); + final NetworkDisruption networkDisruption = new NetworkDisruption(new NetworkDisruption.TwoPartitions(masterNode, dataNode), + NetworkDisruption.NetworkDelay.random(random())); + internalCluster().setDisruptionScheme(networkDisruption); + networkDisruption.startDisrupting(); + + logger.info("--> unblocking repository"); + unblockNode("test-repo", blockedNode); + + // Retrieve snapshot status from the data node. + SnapshotShardsService snapshotShardsService = internalCluster().getInstance(SnapshotShardsService.class, blockedNode); + assertBusy(() -> { + final Snapshot snapshot = new Snapshot("test-repo", snapshotId); + List stages = snapshotShardsService.currentSnapshotShards(snapshot) + .values().stream().map(IndexShardSnapshotStatus::stage).collect(Collectors.toList()); + assertThat(stages, hasSize(shards)); + assertThat(stages, everyItem(equalTo(IndexShardSnapshotStatus.Stage.DONE))); + }); + + logger.info("--> stop disrupting cluster"); + networkDisruption.stopDisrupting(); + internalCluster().clearDisruptionScheme(true); + + assertBusy(() -> { + GetSnapshotsResponse snapshotsStatusResponse = client().admin().cluster() + .prepareGetSnapshots("test-repo") + .setSnapshots("test-snap").get(); + SnapshotInfo snapshotInfo = snapshotsStatusResponse.getSnapshots().get(0); + logger.info("Snapshot status [{}], successfulShards [{}]", snapshotInfo.state(), snapshotInfo.successfulShards()); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.successfulShards(), equalTo(shards)); + }, 10, TimeUnit.SECONDS); + } +} diff --git a/qa/mixed-cluster/build.gradle b/qa/mixed-cluster/build.gradle index 59a6dfece5278..66cad0c6eb678 100644 --- a/qa/mixed-cluster/build.gradle +++ b/qa/mixed-cluster/build.gradle @@ -57,6 +57,11 @@ for (Version version : wireCompatVersions) { if (project.bwc_tests_enabled) { bwcTest.dependsOn(versionBwcTest) } + + /* To support taking index snapshots, we have to set path.repo setting */ + tasks.getByName("${baseName}#mixedClusterTestRunner").configure { + systemProperty 'tests.path.repo', new File(buildDir, "cluster/shared/repo") + } } test.enabled = false // no unit tests for rolling upgrades, only the rest integration test diff --git a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/IndexingIT.java b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/IndexingIT.java index f744b3029b125..9de8954c531ff 100644 --- a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/IndexingIT.java +++ b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/IndexingIT.java @@ -27,6 +27,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.yaml.ObjectPath; @@ -42,7 +43,9 @@ import static com.carrotsearch.randomizedtesting.RandomizedTest.randomAsciiOfLength; import static java.util.Collections.emptyMap; import static java.util.Collections.singletonMap; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; public class IndexingIT extends ESRestTestCase { @@ -237,6 +240,57 @@ public void testSeqNoCheckpoints() throws Exception { } } + public void testUpdateSnapshotStatus() throws Exception { + Nodes nodes = buildNodeAndVersions(); + assertThat(nodes.getNewNodes(), not(empty())); + logger.info("cluster discovered: {}", nodes.toString()); + + // Create the repository before taking the snapshot. + String repoConfig = JsonXContent.contentBuilder() + .startObject() + .field("type", "fs") + .startObject("settings") + .field("compress", randomBoolean()) + .field("location", System.getProperty("tests.path.repo")) + .endObject() + .endObject() + .string(); + + assertOK( + client().performRequest("PUT", "/_snapshot/repo", emptyMap(), + new StringEntity(repoConfig, ContentType.APPLICATION_JSON)) + ); + + String bwcNames = nodes.getBWCNodes().stream().map(Node::getNodeName).collect(Collectors.joining(",")); + + // Allocating shards on the BWC nodes to makes sure that taking snapshot happens on those nodes. + Settings.Builder settings = Settings.builder() + .put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), between(5, 10)) + .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 1) + .put("index.routing.allocation.include._name", bwcNames); + + final String index = "test-snapshot-index"; + createIndex(index, settings.build()); + indexDocs(index, 0, between(50, 100)); + ensureGreen(); + assertOK(client().performRequest("POST", index + "/_refresh")); + + assertOK( + client().performRequest("PUT", "/_snapshot/repo/bwc-snapshot", singletonMap("wait_for_completion", "true"), + new StringEntity("{\"indices\": \"" + index + "\"}", ContentType.APPLICATION_JSON)) + ); + + // Allocating shards on all nodes, taking snapshots should happen on all nodes. + updateIndexSetting(index, Settings.builder().putNull("index.routing.allocation.include._name")); + ensureGreen(); + assertOK(client().performRequest("POST", index + "/_refresh")); + + assertOK( + client().performRequest("PUT", "/_snapshot/repo/mixed-snapshot", singletonMap("wait_for_completion", "true"), + new StringEntity("{\"indices\": \"" + index + "\"}", ContentType.APPLICATION_JSON)) + ); + } + private void assertCount(final String index, final String preference, final int expectedCount) throws IOException { final Response response = client().performRequest("GET", index + "/_count", Collections.singletonMap("preference", preference)); assertOK(response); From da115151a54c7e5568267813464fbbf7395287bc Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 17 Nov 2017 13:33:58 -0500 Subject: [PATCH 13/31] Log primary-replica resync failures Today we do not fail a replica shard if the primary-replica resync to that replica fails. Yet, we should at least log the failure messages. This commit causes this to be the case. Relates #27421 --- .../resync/TransportResyncReplicationAction.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/core/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java b/core/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java index d217717faeba0..32614c636f128 100644 --- a/core/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java @@ -18,11 +18,13 @@ */ package org.elasticsearch.action.resync; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.action.support.replication.ReplicationOperation; +import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.action.support.replication.TransportWriteAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; @@ -158,6 +160,15 @@ public String executor() { @Override public void handleResponse(ResyncReplicationResponse response) { + final ReplicationResponse.ShardInfo.Failure[] failures = response.getShardInfo().getFailures(); + // noinspection ForLoopReplaceableByForEach + for (int i = 0; i < failures.length; i++) { + final ReplicationResponse.ShardInfo.Failure f = failures[i]; + logger.info( + new ParameterizedMessage( + "{} primary-replica resync to replica on node [{}] failed", f.fullShardId(), f.nodeId()), + f.getCause()); + } listener.onResponse(response); } From a5df2ef5388d8ca8e7723b80365b4836a211ffeb Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 17 Nov 2017 20:51:39 +0100 Subject: [PATCH 14/31] peanut butter hamburgers --- docs/reference/docs/delete.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/docs/delete.asciidoc b/docs/reference/docs/delete.asciidoc index 6a93a2bb42e93..2eb2b88ed132e 100644 --- a/docs/reference/docs/delete.asciidoc +++ b/docs/reference/docs/delete.asciidoc @@ -59,7 +59,7 @@ Example to delete with routing [source,js] -------------------------------------------------- -PUT /twitter/tweet/1?routing=kimhcy +PUT /twitter/tweet/1?routing=kimchy { "test": "test" } From 858b2c7cb89cf8d9dc8922fa8b436416347d9f98 Mon Sep 17 00:00:00 2001 From: Mayya Sharipova Date: Fri, 17 Nov 2017 15:31:52 -0500 Subject: [PATCH 15/31] Standardize underscore requirements in parameters (#27414) Stardardize underscore requirements in parameters across different type of requests: _index, _type, _source, _id keep their underscores params like version and retry_on_conflict will be without underscores Throw an error if older versions of parameters are used BulkRequest, MultiGetRequest, TermVectorcRequest, MoreLikeThisQuery were changed Closes #26886 --- .../org/elasticsearch/client/Request.java | 14 +- .../action/bulk/BulkRequest.java | 42 +++-- .../action/get/MultiGetRequest.java | 42 +++-- .../termvectors/TermVectorsRequest.java | 45 +++-- .../index/query/MoreLikeThisQueryBuilder.java | 161 +++++++++--------- .../action/bulk/BulkRequestTests.java | 4 +- .../query/MoreLikeThisQueryBuilderTests.java | 4 +- .../action/bulk/simple-bulk4.json | 2 +- docs/reference/docs/bulk.asciidoc | 14 +- docs/reference/docs/multi-get.asciidoc | 2 +- docs/reference/migration/migrate_7_0.asciidoc | 4 + .../migration/migrate_7_0/api.asciidoc | 24 +++ .../rest-api-spec/test/bulk/60_deprecated.yml | 26 +++ .../rest-api-spec/test/mget/80_deprecated.yml | 38 +++++ .../test/mtermvectors/20_deprecated.yml | 52 ++++++ 15 files changed, 329 insertions(+), 145 deletions(-) create mode 100644 docs/reference/migration/migrate_7_0/api.asciidoc create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/bulk/60_deprecated.yml create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/mget/80_deprecated.yml create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/mtermvectors/20_deprecated.yml diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/Request.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/Request.java index 4da68e98e2db9..e2a6dcac20b06 100755 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/Request.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/Request.java @@ -191,23 +191,23 @@ static Request bulk(BulkRequest bulkRequest) throws IOException { metadata.field("_id", request.id()); } if (Strings.hasLength(request.routing())) { - metadata.field("_routing", request.routing()); + metadata.field("routing", request.routing()); } if (Strings.hasLength(request.parent())) { - metadata.field("_parent", request.parent()); + metadata.field("parent", request.parent()); } if (request.version() != Versions.MATCH_ANY) { - metadata.field("_version", request.version()); + metadata.field("version", request.version()); } VersionType versionType = request.versionType(); if (versionType != VersionType.INTERNAL) { if (versionType == VersionType.EXTERNAL) { - metadata.field("_version_type", "external"); + metadata.field("version_type", "external"); } else if (versionType == VersionType.EXTERNAL_GTE) { - metadata.field("_version_type", "external_gte"); + metadata.field("version_type", "external_gte"); } else if (versionType == VersionType.FORCE) { - metadata.field("_version_type", "force"); + metadata.field("version_type", "force"); } } @@ -219,7 +219,7 @@ static Request bulk(BulkRequest bulkRequest) throws IOException { } else if (opType == DocWriteRequest.OpType.UPDATE) { UpdateRequest updateRequest = (UpdateRequest) request; if (updateRequest.retryOnConflict() > 0) { - metadata.field("_retry_on_conflict", updateRequest.retryOnConflict()); + metadata.field("retry_on_conflict", updateRequest.retryOnConflict()); } if (updateRequest.fetchSource() != null) { metadata.field("_source", updateRequest.fetchSource()); diff --git a/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java index fbbe6c1bf8a96..d868f0becf88a 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java @@ -30,6 +30,7 @@ import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -68,6 +69,19 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques private static final int REQUEST_OVERHEAD = 50; + private static final ParseField INDEX = new ParseField("_index"); + private static final ParseField TYPE = new ParseField("_type"); + private static final ParseField ID = new ParseField("_id"); + private static final ParseField ROUTING = new ParseField("routing"); + private static final ParseField PARENT = new ParseField("parent"); + private static final ParseField OP_TYPE = new ParseField("op_type"); + private static final ParseField VERSION = new ParseField("version"); + private static final ParseField VERSION_TYPE = new ParseField("version_type"); + private static final ParseField RETRY_ON_CONFLICT = new ParseField("retry_on_conflict"); + private static final ParseField PIPELINE = new ParseField("pipeline"); + private static final ParseField FIELDS = new ParseField("fields"); + private static final ParseField SOURCE = new ParseField("_source"); + /** * Requests that are part of this request. It is only possible to add things that are both {@link ActionRequest}s and * {@link WriteRequest}s to this but java doesn't support syntax to declare that everything in the array has both types so we declare @@ -334,45 +348,45 @@ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Null if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); } else if (token.isValue()) { - if ("_index".equals(currentFieldName)) { + if (INDEX.match(currentFieldName)){ if (!allowExplicitIndex) { throw new IllegalArgumentException("explicit index in bulk is not allowed"); } index = parser.text(); - } else if ("_type".equals(currentFieldName)) { + } else if (TYPE.match(currentFieldName)) { type = parser.text(); - } else if ("_id".equals(currentFieldName)) { + } else if (ID.match(currentFieldName)) { id = parser.text(); - } else if ("_routing".equals(currentFieldName) || "routing".equals(currentFieldName)) { + } else if (ROUTING.match(currentFieldName)) { routing = parser.text(); - } else if ("_parent".equals(currentFieldName) || "parent".equals(currentFieldName)) { + } else if (PARENT.match(currentFieldName)) { parent = parser.text(); - } else if ("op_type".equals(currentFieldName) || "opType".equals(currentFieldName)) { + } else if (OP_TYPE.match(currentFieldName)) { opType = parser.text(); - } else if ("_version".equals(currentFieldName) || "version".equals(currentFieldName)) { + } else if (VERSION.match(currentFieldName)) { version = parser.longValue(); - } else if ("_version_type".equals(currentFieldName) || "_versionType".equals(currentFieldName) || "version_type".equals(currentFieldName) || "versionType".equals(currentFieldName)) { + } else if (VERSION_TYPE.match(currentFieldName)) { versionType = VersionType.fromString(parser.text()); - } else if ("_retry_on_conflict".equals(currentFieldName) || "_retryOnConflict".equals(currentFieldName)) { + } else if (RETRY_ON_CONFLICT.match(currentFieldName)) { retryOnConflict = parser.intValue(); - } else if ("pipeline".equals(currentFieldName)) { + } else if (PIPELINE.match(currentFieldName)) { pipeline = parser.text(); - } else if ("fields".equals(currentFieldName)) { + } else if (FIELDS.match(currentFieldName)) { throw new IllegalArgumentException("Action/metadata line [" + line + "] contains a simple value for parameter [fields] while a list is expected"); - } else if ("_source".equals(currentFieldName)) { + } else if (SOURCE.match(currentFieldName)) { fetchSourceContext = FetchSourceContext.fromXContent(parser); } else { throw new IllegalArgumentException("Action/metadata line [" + line + "] contains an unknown parameter [" + currentFieldName + "]"); } } else if (token == XContentParser.Token.START_ARRAY) { - if ("fields".equals(currentFieldName)) { + if (FIELDS.match(currentFieldName)) { DEPRECATION_LOGGER.deprecated("Deprecated field [fields] used, expected [_source] instead"); List values = parser.list(); fields = values.toArray(new String[values.size()]); } else { throw new IllegalArgumentException("Malformed action/metadata line [" + line + "], expected a simple value for field [" + currentFieldName + "] but found [" + token + "]"); } - } else if (token == XContentParser.Token.START_OBJECT && "_source".equals(currentFieldName)) { + } else if (token == XContentParser.Token.START_OBJECT && SOURCE.match(currentFieldName)) { fetchSourceContext = FetchSourceContext.fromXContent(parser); } else if (token != XContentParser.Token.VALUE_NULL) { throw new IllegalArgumentException("Malformed action/metadata line [" + line + "], expected a simple value for field [" + currentFieldName + "] but found [" + token + "]"); diff --git a/core/src/main/java/org/elasticsearch/action/get/MultiGetRequest.java b/core/src/main/java/org/elasticsearch/action/get/MultiGetRequest.java index 420e0b448b052..48e3f5e81bf6f 100644 --- a/core/src/main/java/org/elasticsearch/action/get/MultiGetRequest.java +++ b/core/src/main/java/org/elasticsearch/action/get/MultiGetRequest.java @@ -28,6 +28,7 @@ import org.elasticsearch.action.ValidateActions; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; @@ -48,6 +49,17 @@ public class MultiGetRequest extends ActionRequest implements Iterable, CompositeIndicesRequest, RealtimeRequest { + private static final ParseField INDEX = new ParseField("_index"); + private static final ParseField TYPE = new ParseField("_type"); + private static final ParseField ID = new ParseField("_id"); + private static final ParseField ROUTING = new ParseField("routing"); + private static final ParseField PARENT = new ParseField("parent"); + private static final ParseField VERSION = new ParseField("version"); + private static final ParseField VERSION_TYPE = new ParseField("version_type"); + private static final ParseField FIELDS = new ParseField("fields"); + private static final ParseField STORED_FIELDS = new ParseField("stored_fields"); + private static final ParseField SOURCE = new ParseField("_source"); + /** * A single get item. */ @@ -379,30 +391,30 @@ public static void parseDocuments(XContentParser parser, List items, @Null if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); } else if (token.isValue()) { - if ("_index".equals(currentFieldName)) { + if (INDEX.match(currentFieldName)) { if (!allowExplicitIndex) { throw new IllegalArgumentException("explicit index in multi get is not allowed"); } index = parser.text(); - } else if ("_type".equals(currentFieldName)) { + } else if (TYPE.match(currentFieldName)) { type = parser.text(); - } else if ("_id".equals(currentFieldName)) { + } else if (ID.match(currentFieldName)) { id = parser.text(); - } else if ("_routing".equals(currentFieldName) || "routing".equals(currentFieldName)) { + } else if (ROUTING.match(currentFieldName)) { routing = parser.text(); - } else if ("_parent".equals(currentFieldName) || "parent".equals(currentFieldName)) { + } else if (PARENT.match(currentFieldName)) { parent = parser.text(); - } else if ("fields".equals(currentFieldName)) { + } else if (FIELDS.match(currentFieldName)) { throw new ParsingException(parser.getTokenLocation(), "Unsupported field [fields] used, expected [stored_fields] instead"); - } else if ("stored_fields".equals(currentFieldName)) { + } else if (STORED_FIELDS.match(currentFieldName)) { storedFields = new ArrayList<>(); storedFields.add(parser.text()); - } else if ("_version".equals(currentFieldName) || "version".equals(currentFieldName)) { + } else if (VERSION.match(currentFieldName)) { version = parser.longValue(); - } else if ("_version_type".equals(currentFieldName) || "_versionType".equals(currentFieldName) || "version_type".equals(currentFieldName) || "versionType".equals(currentFieldName)) { + } else if (VERSION_TYPE.match(currentFieldName)) { versionType = VersionType.fromString(parser.text()); - } else if ("_source".equals(currentFieldName)) { + } else if (SOURCE.match(currentFieldName)) { // check lenient to avoid interpreting the value as string but parse strict in order to provoke an error early on. if (parser.isBooleanValueLenient()) { fetchSourceContext = new FetchSourceContext(parser.booleanValue(), fetchSourceContext.includes(), @@ -413,17 +425,19 @@ public static void parseDocuments(XContentParser parser, List items, @Null } else { throw new ElasticsearchParseException("illegal type for _source: [{}]", token); } + } else { + throw new ElasticsearchParseException("failed to parse multi get request. unknown field [{}]", currentFieldName); } } else if (token == XContentParser.Token.START_ARRAY) { - if ("fields".equals(currentFieldName)) { + if (FIELDS.match(currentFieldName)) { throw new ParsingException(parser.getTokenLocation(), "Unsupported field [fields] used, expected [stored_fields] instead"); - } else if ("stored_fields".equals(currentFieldName)) { + } else if (STORED_FIELDS.match(currentFieldName)) { storedFields = new ArrayList<>(); while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { storedFields.add(parser.text()); } - } else if ("_source".equals(currentFieldName)) { + } else if (SOURCE.match(currentFieldName)) { ArrayList includes = new ArrayList<>(); while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { includes.add(parser.text()); @@ -433,7 +447,7 @@ public static void parseDocuments(XContentParser parser, List items, @Null } } else if (token == XContentParser.Token.START_OBJECT) { - if ("_source".equals(currentFieldName)) { + if (SOURCE.match(currentFieldName)) { List currentList = null, includes = null, excludes = null; while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { diff --git a/core/src/main/java/org/elasticsearch/action/termvectors/TermVectorsRequest.java b/core/src/main/java/org/elasticsearch/action/termvectors/TermVectorsRequest.java index 1886a8c2661ed..0e87de98049d0 100644 --- a/core/src/main/java/org/elasticsearch/action/termvectors/TermVectorsRequest.java +++ b/core/src/main/java/org/elasticsearch/action/termvectors/TermVectorsRequest.java @@ -27,6 +27,7 @@ import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.support.single.shard.SingleShardRequest; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; @@ -60,6 +61,22 @@ */ public class TermVectorsRequest extends SingleShardRequest implements RealtimeRequest { + private static final ParseField INDEX = new ParseField("_index"); + private static final ParseField TYPE = new ParseField("_type"); + private static final ParseField ID = new ParseField("_id"); + private static final ParseField ROUTING = new ParseField("routing"); + private static final ParseField PARENT = new ParseField("parent"); + private static final ParseField VERSION = new ParseField("version"); + private static final ParseField VERSION_TYPE = new ParseField("version_type"); + private static final ParseField FIELDS = new ParseField("fields"); + private static final ParseField OFFSETS = new ParseField("offsets"); + private static final ParseField POSITIONS = new ParseField("positions"); + private static final ParseField PAYLOADS = new ParseField("payloads"); + private static final ParseField DFS = new ParseField("dfs"); + private static final ParseField FILTER = new ParseField("filter"); + private static final ParseField DOC = new ParseField("doc"); + + private String type; private String id; @@ -593,7 +610,7 @@ public static void parseRequest(TermVectorsRequest termVectorsRequest, XContentP if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); } else if (currentFieldName != null) { - if (currentFieldName.equals("fields")) { + if (FIELDS.match(currentFieldName)) { if (token == XContentParser.Token.START_ARRAY) { while (parser.nextToken() != XContentParser.Token.END_ARRAY) { fields.add(parser.text()); @@ -601,43 +618,43 @@ public static void parseRequest(TermVectorsRequest termVectorsRequest, XContentP } else { throw new ElasticsearchParseException("failed to parse term vectors request. field [fields] must be an array"); } - } else if (currentFieldName.equals("offsets")) { + } else if (OFFSETS.match(currentFieldName)) { termVectorsRequest.offsets(parser.booleanValue()); - } else if (currentFieldName.equals("positions")) { + } else if (POSITIONS.match(currentFieldName)) { termVectorsRequest.positions(parser.booleanValue()); - } else if (currentFieldName.equals("payloads")) { + } else if (PAYLOADS.match(currentFieldName)) { termVectorsRequest.payloads(parser.booleanValue()); } else if (currentFieldName.equals("term_statistics") || currentFieldName.equals("termStatistics")) { termVectorsRequest.termStatistics(parser.booleanValue()); } else if (currentFieldName.equals("field_statistics") || currentFieldName.equals("fieldStatistics")) { termVectorsRequest.fieldStatistics(parser.booleanValue()); - } else if (currentFieldName.equals("dfs")) { + } else if (DFS.match(currentFieldName)) { throw new IllegalArgumentException("distributed frequencies is not supported anymore for term vectors"); } else if (currentFieldName.equals("per_field_analyzer") || currentFieldName.equals("perFieldAnalyzer")) { termVectorsRequest.perFieldAnalyzer(readPerFieldAnalyzer(parser.map())); - } else if (currentFieldName.equals("filter")) { + } else if (FILTER.match(currentFieldName)) { termVectorsRequest.filterSettings(readFilterSettings(parser)); - } else if ("_index".equals(currentFieldName)) { // the following is important for multi request parsing. + } else if (INDEX.match(currentFieldName)) { // the following is important for multi request parsing. termVectorsRequest.index = parser.text(); - } else if ("_type".equals(currentFieldName)) { + } else if (TYPE.match(currentFieldName)) { termVectorsRequest.type = parser.text(); - } else if ("_id".equals(currentFieldName)) { + } else if (ID.match(currentFieldName)) { if (termVectorsRequest.doc != null) { throw new ElasticsearchParseException("failed to parse term vectors request. either [id] or [doc] can be specified, but not both!"); } termVectorsRequest.id = parser.text(); - } else if ("doc".equals(currentFieldName)) { + } else if (DOC.match(currentFieldName)) { if (termVectorsRequest.id != null) { throw new ElasticsearchParseException("failed to parse term vectors request. either [id] or [doc] can be specified, but not both!"); } termVectorsRequest.doc(jsonBuilder().copyCurrentStructure(parser)); - } else if ("_routing".equals(currentFieldName) || "routing".equals(currentFieldName)) { + } else if (ROUTING.match(currentFieldName)) { termVectorsRequest.routing = parser.text(); - } else if ("_parent".equals(currentFieldName) || "parent".equals(currentFieldName)) { + } else if (PARENT.match(currentFieldName)) { termVectorsRequest.parent = parser.text(); - } else if ("_version".equals(currentFieldName) || "version".equals(currentFieldName)) { + } else if (VERSION.match(currentFieldName)) { termVectorsRequest.version = parser.longValue(); - } else if ("_version_type".equals(currentFieldName) || "_versionType".equals(currentFieldName) || "version_type".equals(currentFieldName) || "versionType".equals(currentFieldName)) { + } else if (VERSION_TYPE.match(currentFieldName)) { termVectorsRequest.versionType = VersionType.fromString(parser.text()); } else { throw new ElasticsearchParseException("failed to parse term vectors request. unknown field [{}]", currentFieldName); diff --git a/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java index 34411d669ec3b..24a4eef9802d5 100644 --- a/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java @@ -92,23 +92,31 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder> SUPPORTED_FIELD_TYPES = new HashSet<>( Arrays.asList(TextFieldType.class, KeywordFieldType.class)); - private interface Field { - ParseField FIELDS = new ParseField("fields"); - ParseField LIKE = new ParseField("like"); - ParseField UNLIKE = new ParseField("unlike"); - ParseField MAX_QUERY_TERMS = new ParseField("max_query_terms"); - ParseField MIN_TERM_FREQ = new ParseField("min_term_freq"); - ParseField MIN_DOC_FREQ = new ParseField("min_doc_freq"); - ParseField MAX_DOC_FREQ = new ParseField("max_doc_freq"); - ParseField MIN_WORD_LENGTH = new ParseField("min_word_length"); - ParseField MAX_WORD_LENGTH = new ParseField("max_word_length"); - ParseField STOP_WORDS = new ParseField("stop_words"); - ParseField ANALYZER = new ParseField("analyzer"); - ParseField MINIMUM_SHOULD_MATCH = new ParseField("minimum_should_match"); - ParseField BOOST_TERMS = new ParseField("boost_terms"); - ParseField INCLUDE = new ParseField("include"); - ParseField FAIL_ON_UNSUPPORTED_FIELD = new ParseField("fail_on_unsupported_field"); - } + private static final ParseField FIELDS = new ParseField("fields"); + private static final ParseField LIKE = new ParseField("like"); + private static final ParseField UNLIKE = new ParseField("unlike"); + private static final ParseField MAX_QUERY_TERMS = new ParseField("max_query_terms"); + private static final ParseField MIN_TERM_FREQ = new ParseField("min_term_freq"); + private static final ParseField MIN_DOC_FREQ = new ParseField("min_doc_freq"); + private static final ParseField MAX_DOC_FREQ = new ParseField("max_doc_freq"); + private static final ParseField MIN_WORD_LENGTH = new ParseField("min_word_length"); + private static final ParseField MAX_WORD_LENGTH = new ParseField("max_word_length"); + private static final ParseField STOP_WORDS = new ParseField("stop_words"); + private static final ParseField ANALYZER = new ParseField("analyzer"); + private static final ParseField MINIMUM_SHOULD_MATCH = new ParseField("minimum_should_match"); + private static final ParseField BOOST_TERMS = new ParseField("boost_terms"); + private static final ParseField INCLUDE = new ParseField("include"); + private static final ParseField FAIL_ON_UNSUPPORTED_FIELD = new ParseField("fail_on_unsupported_field"); + + private static final ParseField INDEX = new ParseField("_index"); + private static final ParseField TYPE = new ParseField("_type"); + private static final ParseField ID = new ParseField("_id"); + public static final ParseField DOC = new ParseField("doc"); + private static final ParseField PER_FIELD_ANALYZER = new ParseField("per_field_analyzer"); + private static final ParseField ROUTING = new ParseField("routing"); + private static final ParseField VERSION = new ParseField("version"); + private static final ParseField VERSION_TYPE = new ParseField("version_type"); + // document inputs private final String[] fields; @@ -141,18 +149,6 @@ private interface Field { public static final class Item implements ToXContentObject, Writeable { public static final Item[] EMPTY_ARRAY = new Item[0]; - public interface Field { - ParseField INDEX = new ParseField("_index"); - ParseField TYPE = new ParseField("_type"); - ParseField ID = new ParseField("_id"); - ParseField DOC = new ParseField("doc"); - ParseField FIELDS = new ParseField("fields"); - ParseField PER_FIELD_ANALYZER = new ParseField("per_field_analyzer"); - ParseField ROUTING = new ParseField("_routing"); - ParseField VERSION = new ParseField("_version"); - ParseField VERSION_TYPE = new ParseField("_version_type"); - } - private String index; private String type; private String id; @@ -370,16 +366,16 @@ public static Item parse(XContentParser parser, Item item) throws IOException { if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); } else if (currentFieldName != null) { - if (Field.INDEX.match(currentFieldName)) { + if (INDEX.match(currentFieldName)) { item.index = parser.text(); - } else if (Field.TYPE.match(currentFieldName)) { + } else if (TYPE.match(currentFieldName)) { item.type = parser.text(); - } else if (Field.ID.match(currentFieldName)) { + } else if (ID.match(currentFieldName)) { item.id = parser.text(); - } else if (Field.DOC.match(currentFieldName)) { + } else if (DOC.match(currentFieldName)) { item.doc = jsonBuilder().copyCurrentStructure(parser).bytes(); item.xContentType = XContentType.JSON; - } else if (Field.FIELDS.match(currentFieldName)) { + } else if (FIELDS.match(currentFieldName)) { if (token == XContentParser.Token.START_ARRAY) { List fields = new ArrayList<>(); while (parser.nextToken() != XContentParser.Token.END_ARRAY) { @@ -390,14 +386,13 @@ public static Item parse(XContentParser parser, Item item) throws IOException { throw new ElasticsearchParseException( "failed to parse More Like This item. field [fields] must be an array"); } - } else if (Field.PER_FIELD_ANALYZER.match(currentFieldName)) { + } else if (PER_FIELD_ANALYZER.match(currentFieldName)) { item.perFieldAnalyzer(TermVectorsRequest.readPerFieldAnalyzer(parser.map())); - } else if ("_routing".equals(currentFieldName) || "routing".equals(currentFieldName)) { + } else if (ROUTING.match(currentFieldName)) { item.routing = parser.text(); - } else if ("_version".equals(currentFieldName) || "version".equals(currentFieldName)) { + } else if (VERSION.match(currentFieldName)) { item.version = parser.longValue(); - } else if ("_version_type".equals(currentFieldName) || "_versionType".equals(currentFieldName) - || "version_type".equals(currentFieldName) || "versionType".equals(currentFieldName)) { + } else if (VERSION_TYPE.match(currentFieldName)) { item.versionType = VersionType.fromString(parser.text()); } else { throw new ElasticsearchParseException( @@ -420,31 +415,31 @@ public static Item parse(XContentParser parser, Item item) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); if (this.index != null) { - builder.field(Field.INDEX.getPreferredName(), this.index); + builder.field(INDEX.getPreferredName(), this.index); } if (this.type != null) { - builder.field(Field.TYPE.getPreferredName(), this.type); + builder.field(TYPE.getPreferredName(), this.type); } if (this.id != null) { - builder.field(Field.ID.getPreferredName(), this.id); + builder.field(ID.getPreferredName(), this.id); } if (this.doc != null) { - builder.rawField(Field.DOC.getPreferredName(), this.doc, xContentType); + builder.rawField(DOC.getPreferredName(), this.doc, xContentType); } if (this.fields != null) { - builder.array(Field.FIELDS.getPreferredName(), this.fields); + builder.array(FIELDS.getPreferredName(), this.fields); } if (this.perFieldAnalyzer != null) { - builder.field(Field.PER_FIELD_ANALYZER.getPreferredName(), this.perFieldAnalyzer); + builder.field(PER_FIELD_ANALYZER.getPreferredName(), this.perFieldAnalyzer); } if (this.routing != null) { - builder.field(Field.ROUTING.getPreferredName(), this.routing); + builder.field(ROUTING.getPreferredName(), this.routing); } if (this.version != Versions.MATCH_ANY) { - builder.field(Field.VERSION.getPreferredName(), this.version); + builder.field(VERSION.getPreferredName(), this.version); } if (this.versionType != VersionType.INTERNAL) { - builder.field(Field.VERSION_TYPE.getPreferredName(), this.versionType.toString().toLowerCase(Locale.ROOT)); + builder.field(VERSION_TYPE.getPreferredName(), this.versionType.toString().toLowerCase(Locale.ROOT)); } return builder.endObject(); } @@ -781,26 +776,26 @@ public static Item[] ids(String... ids) { protected void doXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(NAME); if (fields != null) { - builder.array(Field.FIELDS.getPreferredName(), fields); - } - buildLikeField(builder, Field.LIKE.getPreferredName(), likeTexts, likeItems); - buildLikeField(builder, Field.UNLIKE.getPreferredName(), unlikeTexts, unlikeItems); - builder.field(Field.MAX_QUERY_TERMS.getPreferredName(), maxQueryTerms); - builder.field(Field.MIN_TERM_FREQ.getPreferredName(), minTermFreq); - builder.field(Field.MIN_DOC_FREQ.getPreferredName(), minDocFreq); - builder.field(Field.MAX_DOC_FREQ.getPreferredName(), maxDocFreq); - builder.field(Field.MIN_WORD_LENGTH.getPreferredName(), minWordLength); - builder.field(Field.MAX_WORD_LENGTH.getPreferredName(), maxWordLength); + builder.array(FIELDS.getPreferredName(), fields); + } + buildLikeField(builder, LIKE.getPreferredName(), likeTexts, likeItems); + buildLikeField(builder, UNLIKE.getPreferredName(), unlikeTexts, unlikeItems); + builder.field(MAX_QUERY_TERMS.getPreferredName(), maxQueryTerms); + builder.field(MIN_TERM_FREQ.getPreferredName(), minTermFreq); + builder.field(MIN_DOC_FREQ.getPreferredName(), minDocFreq); + builder.field(MAX_DOC_FREQ.getPreferredName(), maxDocFreq); + builder.field(MIN_WORD_LENGTH.getPreferredName(), minWordLength); + builder.field(MAX_WORD_LENGTH.getPreferredName(), maxWordLength); if (stopWords != null) { - builder.array(Field.STOP_WORDS.getPreferredName(), stopWords); + builder.array(STOP_WORDS.getPreferredName(), stopWords); } if (analyzer != null) { - builder.field(Field.ANALYZER.getPreferredName(), analyzer); + builder.field(ANALYZER.getPreferredName(), analyzer); } - builder.field(Field.MINIMUM_SHOULD_MATCH.getPreferredName(), minimumShouldMatch); - builder.field(Field.BOOST_TERMS.getPreferredName(), boostTerms); - builder.field(Field.INCLUDE.getPreferredName(), include); - builder.field(Field.FAIL_ON_UNSUPPORTED_FIELD.getPreferredName(), failOnUnsupportedField); + builder.field(MINIMUM_SHOULD_MATCH.getPreferredName(), minimumShouldMatch); + builder.field(BOOST_TERMS.getPreferredName(), boostTerms); + builder.field(INCLUDE.getPreferredName(), include); + builder.field(FAIL_ON_UNSUPPORTED_FIELD.getPreferredName(), failOnUnsupportedField); printBoostAndQueryName(builder); builder.endObject(); } @@ -839,31 +834,31 @@ public static MoreLikeThisQueryBuilder fromXContent(XContentParser parser) throw if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); } else if (token.isValue()) { - if (Field.LIKE.match(currentFieldName)) { + if (LIKE.match(currentFieldName)) { parseLikeField(parser, likeTexts, likeItems); - } else if (Field.UNLIKE.match(currentFieldName)) { + } else if (UNLIKE.match(currentFieldName)) { parseLikeField(parser, unlikeTexts, unlikeItems); - } else if (Field.MAX_QUERY_TERMS.match(currentFieldName)) { + } else if (MAX_QUERY_TERMS.match(currentFieldName)) { maxQueryTerms = parser.intValue(); - } else if (Field.MIN_TERM_FREQ.match(currentFieldName)) { + } else if (MIN_TERM_FREQ.match(currentFieldName)) { minTermFreq =parser.intValue(); - } else if (Field.MIN_DOC_FREQ.match(currentFieldName)) { + } else if (MIN_DOC_FREQ.match(currentFieldName)) { minDocFreq = parser.intValue(); - } else if (Field.MAX_DOC_FREQ.match(currentFieldName)) { + } else if (MAX_DOC_FREQ.match(currentFieldName)) { maxDocFreq = parser.intValue(); - } else if (Field.MIN_WORD_LENGTH.match(currentFieldName)) { + } else if (MIN_WORD_LENGTH.match(currentFieldName)) { minWordLength = parser.intValue(); - } else if (Field.MAX_WORD_LENGTH.match(currentFieldName)) { + } else if (MAX_WORD_LENGTH.match(currentFieldName)) { maxWordLength = parser.intValue(); - } else if (Field.ANALYZER.match(currentFieldName)) { + } else if (ANALYZER.match(currentFieldName)) { analyzer = parser.text(); - } else if (Field.MINIMUM_SHOULD_MATCH.match(currentFieldName)) { + } else if (MINIMUM_SHOULD_MATCH.match(currentFieldName)) { minimumShouldMatch = parser.text(); - } else if (Field.BOOST_TERMS.match(currentFieldName)) { + } else if (BOOST_TERMS.match(currentFieldName)) { boostTerms = parser.floatValue(); - } else if (Field.INCLUDE.match(currentFieldName)) { + } else if (INCLUDE.match(currentFieldName)) { include = parser.booleanValue(); - } else if (Field.FAIL_ON_UNSUPPORTED_FIELD.match(currentFieldName)) { + } else if (FAIL_ON_UNSUPPORTED_FIELD.match(currentFieldName)) { failOnUnsupportedField = parser.booleanValue(); } else if ("boost".equals(currentFieldName)) { boost = parser.floatValue(); @@ -873,20 +868,20 @@ public static MoreLikeThisQueryBuilder fromXContent(XContentParser parser) throw throw new ParsingException(parser.getTokenLocation(), "[mlt] query does not support [" + currentFieldName + "]"); } } else if (token == XContentParser.Token.START_ARRAY) { - if (Field.FIELDS.match(currentFieldName)) { + if (FIELDS.match(currentFieldName)) { fields = new ArrayList<>(); while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { fields.add(parser.text()); } - } else if (Field.LIKE.match(currentFieldName)) { + } else if (LIKE.match(currentFieldName)) { while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { parseLikeField(parser, likeTexts, likeItems); } - } else if (Field.UNLIKE.match(currentFieldName)) { + } else if (UNLIKE.match(currentFieldName)) { while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { parseLikeField(parser, unlikeTexts, unlikeItems); } - } else if (Field.STOP_WORDS.match(currentFieldName)) { + } else if (STOP_WORDS.match(currentFieldName)) { stopWords = new ArrayList<>(); while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { stopWords.add(parser.text()); @@ -895,9 +890,9 @@ public static MoreLikeThisQueryBuilder fromXContent(XContentParser parser) throw throw new ParsingException(parser.getTokenLocation(), "[mlt] query does not support [" + currentFieldName + "]"); } } else if (token == XContentParser.Token.START_OBJECT) { - if (Field.LIKE.match(currentFieldName)) { + if (LIKE.match(currentFieldName)) { parseLikeField(parser, likeTexts, likeItems); - } else if (Field.UNLIKE.match(currentFieldName)) { + } else if (UNLIKE.match(currentFieldName)) { parseLikeField(parser, unlikeTexts, unlikeItems); } else { throw new ParsingException(parser.getTokenLocation(), "[mlt] query does not support [" + currentFieldName + "]"); diff --git a/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java b/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java index d52087db0e7c5..97a1ef2806a3e 100644 --- a/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java @@ -292,7 +292,7 @@ public void testToValidateUpsertRequestAndVersionInBulkRequest() throws IOExcept builder.field("_index", "index"); builder.field("_type", "type"); builder.field("_id", "id"); - builder.field("_version", 1L); + builder.field("version", 1L); builder.endObject(); builder.endObject(); } @@ -301,7 +301,7 @@ public void testToValidateUpsertRequestAndVersionInBulkRequest() throws IOExcept builder.startObject(); builder.field("doc", "{}"); Map values = new HashMap<>(); - values.put("_version", 2L); + values.put("version", 2L); values.put("_index", "index"); values.put("_type", "type"); builder.field("upsert", values); diff --git a/core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java index 4f7e461ecaa63..922aa9a682f45 100644 --- a/core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java @@ -67,7 +67,7 @@ public class MoreLikeThisQueryBuilderTests extends AbstractQueryTestCase { - private static final String[] SHUFFLE_PROTECTED_FIELDS = new String[]{Item.Field.DOC.getPreferredName()}; + private static final String[] SHUFFLE_PROTECTED_FIELDS = new String[]{MoreLikeThisQueryBuilder.DOC.getPreferredName()}; private static String[] randomFields; private static Item[] randomLikeItems; @@ -222,7 +222,7 @@ protected String[] shuffleProtectedFields() { @Override protected Set getObjectsHoldingArbitraryContent() { //doc contains arbitrary content, anything can be added to it and no exception will be thrown - return Collections.singleton(MoreLikeThisQueryBuilder.Item.Field.DOC.getPreferredName()); + return Collections.singleton(MoreLikeThisQueryBuilder.DOC.getPreferredName()); } @Override diff --git a/core/src/test/resources/org/elasticsearch/action/bulk/simple-bulk4.json b/core/src/test/resources/org/elasticsearch/action/bulk/simple-bulk4.json index bbcc0dbd6868d..94d95614568ca 100644 --- a/core/src/test/resources/org/elasticsearch/action/bulk/simple-bulk4.json +++ b/core/src/test/resources/org/elasticsearch/action/bulk/simple-bulk4.json @@ -1,4 +1,4 @@ -{ "update" : {"_id" : "1", "_retry_on_conflict" : 2} } +{ "update" : {"_id" : "1", "retry_on_conflict" : 2} } { "doc" : {"field" : "value"} } { "update" : { "_id" : "0", "_type" : "type1", "_index" : "index1" } } { "script" : { "source" : "counter += param1", "lang" : "javascript", "params" : {"param1" : 1}}, "upsert" : {"counter" : 1}} diff --git a/docs/reference/docs/bulk.asciidoc b/docs/reference/docs/bulk.asciidoc index b932e2466216f..6c90df9de4021 100644 --- a/docs/reference/docs/bulk.asciidoc +++ b/docs/reference/docs/bulk.asciidoc @@ -201,16 +201,16 @@ chunks, as this will slow things down. === Versioning Each bulk item can include the version value using the -`_version`/`version` field. It automatically follows the behavior of the +`version` field. It automatically follows the behavior of the index / delete operation based on the `_version` mapping. It also -support the `version_type`/`_version_type` (see <>) +support the `version_type` (see <>) [float] [[bulk-routing]] === Routing Each bulk item can include the routing value using the -`_routing`/`routing` field. It automatically follows the behavior of the +`routing` field. It automatically follows the behavior of the index / delete operation based on the `_routing` mapping. [float] @@ -234,7 +234,7 @@ Control when the changes made by this request are visible to search. See [[bulk-update]] === Update -When using `update` action `_retry_on_conflict` can be used as field in +When using `update` action `retry_on_conflict` can be used as field in the action itself (not in the extra payload line), to specify how many times an update should be retried in the case of a version conflict. @@ -246,11 +246,11 @@ the options. Example with update actions: [source,js] -------------------------------------------------- POST _bulk -{ "update" : {"_id" : "1", "_type" : "type1", "_index" : "index1", "_retry_on_conflict" : 3} } +{ "update" : {"_id" : "1", "_type" : "type1", "_index" : "index1", "retry_on_conflict" : 3} } { "doc" : {"field" : "value"} } -{ "update" : { "_id" : "0", "_type" : "type1", "_index" : "index1", "_retry_on_conflict" : 3} } +{ "update" : { "_id" : "0", "_type" : "type1", "_index" : "index1", "retry_on_conflict" : 3} } { "script" : { "source": "ctx._source.counter += params.param1", "lang" : "painless", "params" : {"param1" : 1}}, "upsert" : {"counter" : 1}} -{ "update" : {"_id" : "2", "_type" : "type1", "_index" : "index1", "_retry_on_conflict" : 3} } +{ "update" : {"_id" : "2", "_type" : "type1", "_index" : "index1", "retry_on_conflict" : 3} } { "doc" : {"field" : "value"}, "doc_as_upsert" : true } { "update" : {"_id" : "3", "_type" : "type1", "_index" : "index1", "_source" : true} } { "doc" : {"field" : "value"} } diff --git a/docs/reference/docs/multi-get.asciidoc b/docs/reference/docs/multi-get.asciidoc index 6940fe8319cc6..b241607f85369 100644 --- a/docs/reference/docs/multi-get.asciidoc +++ b/docs/reference/docs/multi-get.asciidoc @@ -230,7 +230,7 @@ GET /_mget?routing=key1 "_index" : "test", "_type" : "type", "_id" : "1", - "_routing" : "key2" + "routing" : "key2" }, { "_index" : "test", diff --git a/docs/reference/migration/migrate_7_0.asciidoc b/docs/reference/migration/migrate_7_0.asciidoc index 043d62465be39..53dc8ef9ec824 100644 --- a/docs/reference/migration/migrate_7_0.asciidoc +++ b/docs/reference/migration/migrate_7_0.asciidoc @@ -30,6 +30,8 @@ way to reindex old indices is to use the `reindex` API. * <> * <> * <> +* <> + include::migrate_7_0/aggregations.asciidoc[] include::migrate_7_0/cluster.asciidoc[] @@ -37,3 +39,5 @@ include::migrate_7_0/indices.asciidoc[] include::migrate_7_0/mappings.asciidoc[] include::migrate_7_0/search.asciidoc[] include::migrate_7_0/plugins.asciidoc[] +include::migrate_7_0/api.asciidoc[] + diff --git a/docs/reference/migration/migrate_7_0/api.asciidoc b/docs/reference/migration/migrate_7_0/api.asciidoc new file mode 100644 index 0000000000000..06a0a6af44b08 --- /dev/null +++ b/docs/reference/migration/migrate_7_0/api.asciidoc @@ -0,0 +1,24 @@ +[[breaking_70_api_changes]] +=== Breaking changes in 7.0 + +==== Camel case and underscore parameters deprecated in 6.x have been removed +A number of duplicate parameters deprecated in 6.x have been removed from +Bulk request, Multi Get request, Term Vectors request, and More Like This Query +requests. + +The following camel case parameters have been removed: + +* `opType` +* `versionType`, `_versionType` + +The following parameters starting with underscore have been removed: + +* `_parent` +* `_retry_on_conflict` +* `_routing` +* `_version` +* `_version_type` + +Instead of these removed parameters, use their non camel case equivalents without +starting underscore, e.g. use `version_type` instead of `_version_type` or `versionType`. + diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/60_deprecated.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/60_deprecated.yml new file mode 100644 index 0000000000000..1401fcc086208 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/60_deprecated.yml @@ -0,0 +1,26 @@ + +--- +"Deprecated parameters should fail in Bulk query": + + - skip: + version: " - 6.99.99" + reason: some parameters are removed starting from 7.0, their equivalents without underscore are used instead + features: "warnings" + + - do: + catch: bad_request + bulk: + body: | + { "update": { "_index": "test_index", "_type": "test_type", "_id": "test_id_1", "_version": 1 } } + { "doc": { "f1": "v1" } } + { "update": { "_index": "test_index", "_type": "test_type", "_id": "test_id_2", "_version": 1 } } + { "doc": { "f1": "v2" } } + + - do: + catch: bad_request + bulk: + body: | + { "update": { "_index": "test_index", "_type": "test_type", "_id": "test_id_1", "_routing": "test1" } } + { "doc": { "f1": "v1" } } + { "update": { "_index": "test_index", "_type": "test_type", "_id": "test_id_2", "_routing": "test1" } } + { "doc": { "f1": "v2" } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/mget/80_deprecated.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/mget/80_deprecated.yml new file mode 100644 index 0000000000000..5033f75c79426 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/mget/80_deprecated.yml @@ -0,0 +1,38 @@ + +--- +"Deprecated parameters should fail in Multi Get query": + + - skip: + version: " - 6.99.99" + reason: _version, _routing are removed starting from 7.0, their equivalents without underscore are used instead + features: "warnings" + + - do: + index: + index: test_1 + type: test + id: 1 + body: { foo: bar } + + - do: + index: + index: test_1 + type: test + id: 2 + body: { foo: baz } + + - do: + catch: bad_request + mget: + body: + docs: + - { _index: test_1, _type: test, _id: 1, _routing : test1 } + - { _index: test_1, _type: test, _id: 2, _routing : test1 } + + - do: + catch: bad_request + mget: + body: + docs: + - { _index: test_1, _type: test, _id: 1, _version : 1 } + - { _index: test_1, _type: test, _id: 2, _version : 1 } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/mtermvectors/20_deprecated.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/mtermvectors/20_deprecated.yml new file mode 100644 index 0000000000000..3ee06780a1f65 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/mtermvectors/20_deprecated.yml @@ -0,0 +1,52 @@ + +--- +"Deprecated camel case and _ parameters should fail in Term Vectors query": + + - skip: + version: " - 6.99.99" + reason: camel case and _ parameters (e.g. versionType, _version_type) should fail from 7.0 + features: "warnings" + + - do: + indices.create: + index: testidx + body: + mappings: + testtype: + properties: + text: + type : "text" + term_vector : "with_positions_offsets" + + - do: + index: + index: testidx + type: testtype + id: testing_document + body: {"text" : "The quick brown fox is brown."} + + - do: + catch: bad_request + mtermvectors: + "term_statistics" : true + "body" : + "docs": + - + "_index" : "testidx" + "_type" : "testtype" + "_id" : "testing_document" + "version" : 1 + "versionType" : "external" + + - do: + catch: bad_request + mtermvectors: + "term_statistics" : true + "body" : + "docs": + - + "_index" : "testidx" + "_type" : "testtype" + "_id" : "testing_document" + "version" : 1 + "_version_type" : "external" From 075c77fc81f9c1ebf243e23ca248c3bd863ae28d Mon Sep 17 00:00:00 2001 From: Nicholas Knize Date: Thu, 16 Nov 2017 15:09:23 -0600 Subject: [PATCH 16/31] [GEO] fix pointsOnly bug for MULTIPOINT This commit fixes a bug where geo_shape indexes configured for "points_only" : "true" reject documents containing multipoint shape types. --- .../index/mapper/GeoShapeFieldMapper.java | 32 ++++++++++++----- .../search/geo/GeoShapeQueryTests.java | 35 +++++++++++++++++++ 2 files changed, 58 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/mapper/GeoShapeFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/GeoShapeFieldMapper.java index 495bdf49cb9fc..db40fb228bc57 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/GeoShapeFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/GeoShapeFieldMapper.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.Explicit; import org.elasticsearch.common.geo.GeoUtils; import org.elasticsearch.common.geo.SpatialStrategy; +import org.elasticsearch.common.geo.XShapeCollection; import org.elasticsearch.common.geo.builders.ShapeBuilder; import org.elasticsearch.common.geo.builders.ShapeBuilder.Orientation; import org.elasticsearch.common.geo.parsers.ShapeParser; @@ -463,7 +464,6 @@ public GeoShapeFieldMapper(String simpleName, MappedFieldType fieldType, Explici public GeoShapeFieldType fieldType() { return (GeoShapeFieldType) super.fieldType(); } - @Override public Mapper parse(ParseContext context) throws IOException { try { @@ -475,14 +475,20 @@ public Mapper parse(ParseContext context) throws IOException { } shape = shapeBuilder.build(); } - if (fieldType().pointsOnly() && !(shape instanceof Point)) { - throw new MapperParsingException("[{" + fieldType().name() + "}] is configured for points only but a " + - ((shape instanceof JtsGeometry) ? ((JtsGeometry) shape).getGeom().getGeometryType() : shape.getClass()) + " was found"); - } - List fields = new ArrayList<>(Arrays.asList(fieldType().defaultStrategy().createIndexableFields(shape))); - createFieldNamesField(context, fields); - for (IndexableField field : fields) { - context.doc().add(field); + if (fieldType().pointsOnly() == true) { + // index configured for pointsOnly + if (shape instanceof XShapeCollection && XShapeCollection.class.cast(shape).pointsOnly()) { + // MULTIPOINT data: index each point separately + List shapes = ((XShapeCollection) shape).getShapes(); + for (Shape s : shapes) { + indexShape(context, s); + } + } else if (shape instanceof Point == false) { + throw new MapperParsingException("[{" + fieldType().name() + "}] is configured for points only but a " + + ((shape instanceof JtsGeometry) ? ((JtsGeometry)shape).getGeom().getGeometryType() : shape.getClass()) + " was found"); + } + } else { + indexShape(context, shape); } } catch (Exception e) { if (ignoreMalformed.value() == false) { @@ -492,6 +498,14 @@ public Mapper parse(ParseContext context) throws IOException { return null; } + private void indexShape(ParseContext context, Shape shape) { + List fields = new ArrayList<>(Arrays.asList(fieldType().defaultStrategy().createIndexableFields(shape))); + createFieldNamesField(context, fields); + for (IndexableField field : fields) { + context.doc().add(field); + } + } + @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { } diff --git a/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java b/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java index 47ac551d38caa..5ca4193da46fb 100644 --- a/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java +++ b/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java @@ -48,6 +48,7 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.query.QueryBuilders.geoIntersectionQuery; import static org.elasticsearch.index.query.QueryBuilders.geoShapeQuery; +import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.test.geo.RandomShapeGenerator.createGeometryCollectionWithin; import static org.elasticsearch.test.geo.RandomShapeGenerator.xRandomPoint; import static org.elasticsearch.test.geo.RandomShapeGenerator.xRandomRectangle; @@ -468,4 +469,38 @@ public void testPointsOnly() throws Exception { assertEquals(1, response.getHits().getTotalHits()); } + + public void testPointsOnlyExplicit() throws Exception { + String mapping = XContentFactory.jsonBuilder().startObject().startObject("type1") + .startObject("properties").startObject("location") + .field("type", "geo_shape") + .field("tree", randomBoolean() ? "quadtree" : "geohash") + .field("tree_levels", "6") + .field("distance_error_pct", "0.01") + .field("points_only", true) + .endObject().endObject() + .endObject().endObject().string(); + + client().admin().indices().prepareCreate("geo_points_only").addMapping("type1", mapping, XContentType.JSON).execute().actionGet(); + ensureGreen(); + + // MULTIPOINT + ShapeBuilder shape = RandomShapeGenerator.createShape(random(), RandomShapeGenerator.ShapeType.MULTIPOINT); + client().prepareIndex("geo_points_only", "type1", "1") + .setSource(jsonBuilder().startObject().field("location", shape).endObject()) + .setRefreshPolicy(IMMEDIATE).get(); + + // POINT + shape = RandomShapeGenerator.createShape(random(), RandomShapeGenerator.ShapeType.POINT); + client().prepareIndex("geo_points_only", "type1", "2") + .setSource(jsonBuilder().startObject().field("location", shape).endObject()) + .setRefreshPolicy(IMMEDIATE).get(); + + // test that point was inserted + SearchResponse response = client().prepareSearch("geo_points_only").setTypes("type1") + .setQuery(matchAllQuery()) + .execute().actionGet(); + + assertEquals(2, response.getHits().getTotalHits()); + } } From cc3be6ddda0e58ff36eef0e58e6573242c5d1d65 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 17 Nov 2017 14:53:15 -0700 Subject: [PATCH 17/31] Remove parameters on HandshakeResponseHandler (#27444) This is a followup to #27407. That commit removed the channel type parameter from TcpTransport. This commit removes the parameter from the handshake response handler. --- .../main/java/org/elasticsearch/transport/TcpTransport.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java index d5aed81ae805b..0bf600cb02928 100644 --- a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -278,13 +278,13 @@ public void setTransportService(TransportService service) { this.transportService = service; } - private static class HandshakeResponseHandler implements TransportResponseHandler { + private static class HandshakeResponseHandler implements TransportResponseHandler { final AtomicReference versionRef = new AtomicReference<>(); final CountDownLatch latch = new CountDownLatch(1); final AtomicReference exceptionRef = new AtomicReference<>(); - final Channel channel; + final TcpChannel channel; - HandshakeResponseHandler(Channel channel) { + HandshakeResponseHandler(TcpChannel channel) { this.channel = channel; } From ce45e29be7f978f49d892582f00075b37a2f7f39 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 17 Nov 2017 16:20:09 -0700 Subject: [PATCH 18/31] Remove manual tracking of registered channels (#27445) This is related to #27260. Currently, every ESSelector keeps track of all channels that are registered with it. ESSelector is just an abstraction over a raw java nio selector. The java nio selector already tracks its own selection keys. This commit removes our tracking and relies on the java nio selector tracking. --- .../transport/nio/AcceptingSelector.java | 5 ----- .../transport/nio/ESSelector.java | 19 ++-------------- .../transport/nio/SocketSelector.java | 1 - .../nio/channel/AbstractNioChannel.java | 3 --- .../transport/nio/AcceptingSelectorTests.java | 21 +++++------------- .../transport/nio/ESSelectorTests.java | 7 ------ .../transport/nio/SocketSelectorTests.java | 22 +++++++------------ 7 files changed, 16 insertions(+), 62 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/AcceptingSelector.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/AcceptingSelector.java index e116d6421706d..23775b4bc1665 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/AcceptingSelector.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/AcceptingSelector.java @@ -22,13 +22,9 @@ import org.elasticsearch.transport.nio.channel.NioServerSocketChannel; import java.io.IOException; -import java.nio.channels.CancelledKeyException; import java.nio.channels.ClosedChannelException; -import java.nio.channels.ClosedSelectorException; import java.nio.channels.SelectionKey; import java.nio.channels.Selector; -import java.util.Iterator; -import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -93,7 +89,6 @@ private void setUpNewServerChannels() { newChannel.register(); SelectionKey selectionKey = newChannel.getSelectionKey(); selectionKey.attach(newChannel); - addRegisteredChannel(newChannel); eventHandler.serverChannelRegistered(newChannel); } else { eventHandler.registrationException(newChannel, new ClosedChannelException()); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/ESSelector.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/ESSelector.java index ba0fae3ee3127..667107f7b3eee 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/ESSelector.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/ESSelector.java @@ -28,14 +28,13 @@ import java.nio.channels.ClosedSelectorException; import java.nio.channels.SelectionKey; import java.nio.channels.Selector; -import java.util.Collections; import java.util.Iterator; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; /** * This is a basic selector abstraction used by {@link org.elasticsearch.transport.nio.NioTransport}. This @@ -56,7 +55,6 @@ public abstract class ESSelector implements Closeable { private final CountDownLatch exitedLoop = new CountDownLatch(1); private final AtomicBoolean isClosed = new AtomicBoolean(false); private final PlainActionFuture isRunningFuture = PlainActionFuture.newFuture(); - private final Set registeredChannels = Collections.newSetFromMap(new ConcurrentHashMap()); private volatile Thread thread; ESSelector(EventHandler eventHandler) throws IOException { @@ -134,7 +132,7 @@ void singleLoop() { void cleanupAndCloseChannels() { cleanup(); - channelsToClose.addAll(registeredChannels); + channelsToClose.addAll(selector.keys().stream().map(sk -> (NioChannel) sk.attachment()).collect(Collectors.toList())); closePendingChannels(); } @@ -171,19 +169,6 @@ void wakeup() { selector.wakeup(); } - public Set getRegisteredChannels() { - return registeredChannels; - } - - public void addRegisteredChannel(NioChannel channel) { - assert registeredChannels.contains(channel) == false : "Should only register channel once"; - registeredChannels.add(channel); - } - - public void removeRegisteredChannel(NioChannel channel) { - registeredChannels.remove(channel); - } - @Override public void close() throws IOException { if (isClosed.compareAndSet(false, true)) { diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketSelector.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketSelector.java index 9c90463421a81..5a298b34bb948 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketSelector.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketSelector.java @@ -171,7 +171,6 @@ private void setupChannel(NioSocketChannel newChannel) { try { if (newChannel.isOpen()) { newChannel.register(); - addRegisteredChannel(newChannel); SelectionKey key = newChannel.getSelectionKey(); key.attach(newChannel); eventHandler.handleRegistration(newChannel); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/AbstractNioChannel.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/AbstractNioChannel.java index a7208beb6618f..21f46631c6e83 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/AbstractNioChannel.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/AbstractNioChannel.java @@ -115,9 +115,6 @@ public void closeFromSelector() throws IOException { } catch (IOException e) { closeContext.completeExceptionally(e); throw e; - } finally { - // There is no problem with calling this multiple times - selector.removeRegisteredChannel(this); } } } diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/AcceptingSelectorTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/AcceptingSelectorTests.java index 140c44133d3fd..4e35f6177f5dc 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/AcceptingSelectorTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/AcceptingSelectorTests.java @@ -20,7 +20,6 @@ package org.elasticsearch.transport.nio; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioServerSocketChannel; import org.elasticsearch.transport.nio.utils.TestSelectionKey; import org.junit.Before; @@ -30,8 +29,8 @@ import java.nio.channels.SelectionKey; import java.nio.channels.Selector; import java.security.PrivilegedActionException; +import java.util.Collections; import java.util.HashSet; -import java.util.Set; import static org.mockito.Matchers.any; import static org.mockito.Matchers.same; @@ -46,6 +45,7 @@ public class AcceptingSelectorTests extends ESTestCase { private NioServerSocketChannel serverChannel; private AcceptorEventHandler eventHandler; private TestSelectionKey selectionKey; + private Selector rawSelector; @Before public void setUp() throws Exception { @@ -54,7 +54,7 @@ public void setUp() throws Exception { eventHandler = mock(AcceptorEventHandler.class); serverChannel = mock(NioServerSocketChannel.class); - Selector rawSelector = mock(Selector.class); + rawSelector = mock(Selector.class); selector = new AcceptingSelector(eventHandler, rawSelector); this.selector.setThread(); @@ -71,9 +71,6 @@ public void testRegisteredChannel() throws IOException, PrivilegedActionExceptio selector.preSelect(); verify(eventHandler).serverChannelRegistered(serverChannel); - Set registeredChannels = selector.getRegisteredChannels(); - assertEquals(1, registeredChannels.size()); - assertTrue(registeredChannels.contains(serverChannel)); } public void testClosedChannelWillNotBeRegistered() throws Exception { @@ -83,10 +80,6 @@ public void testClosedChannelWillNotBeRegistered() throws Exception { selector.preSelect(); verify(eventHandler).registrationException(same(serverChannel), any(ClosedChannelException.class)); - - Set registeredChannels = selector.getRegisteredChannels(); - assertEquals(0, registeredChannels.size()); - assertFalse(registeredChannels.contains(serverChannel)); } public void testRegisterChannelFailsDueToException() throws Exception { @@ -98,10 +91,6 @@ public void testRegisterChannelFailsDueToException() throws Exception { selector.preSelect(); verify(eventHandler).registrationException(serverChannel, closedChannelException); - - Set registeredChannels = selector.getRegisteredChannels(); - assertEquals(0, registeredChannels.size()); - assertFalse(registeredChannels.contains(serverChannel)); } public void testAcceptEvent() throws IOException { @@ -128,7 +117,9 @@ public void testCleanup() throws IOException { selector.preSelect(); - assertEquals(1, selector.getRegisteredChannels().size()); + TestSelectionKey key = new TestSelectionKey(0); + key.attach(serverChannel); + when(rawSelector.keys()).thenReturn(new HashSet<>(Collections.singletonList(key))); selector.cleanupAndCloseChannels(); diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/ESSelectorTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/ESSelectorTests.java index afcd42dcb528e..69c2c00489d2a 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/ESSelectorTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/ESSelectorTests.java @@ -51,19 +51,12 @@ public void setUp() throws Exception { public void testQueueChannelForClosed() throws IOException { NioChannel channel = mock(NioChannel.class); when(channel.getSelector()).thenReturn(selector); - selector.addRegisteredChannel(channel); selector.queueChannelClose(channel); - assertEquals(1, selector.getRegisteredChannels().size()); - selector.singleLoop(); verify(handler).handleClose(channel); - // Will be called in the channel close method - selector.removeRegisteredChannel(channel); - - assertEquals(0, selector.getRegisteredChannels().size()); } public void testSelectorClosedExceptionIsNotCaughtWhileRunning() throws IOException { diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketSelectorTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketSelectorTests.java index fdaed26a557f7..670134d9beeab 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketSelectorTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketSelectorTests.java @@ -34,7 +34,8 @@ import java.nio.channels.ClosedSelectorException; import java.nio.channels.SelectionKey; import java.nio.channels.Selector; -import java.util.Set; +import java.util.Collections; +import java.util.HashSet; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; @@ -54,6 +55,7 @@ public class SocketSelectorTests extends ESTestCase { private WriteContext writeContext; private ActionListener listener; private NetworkBytesReference bufferReference = NetworkBytesReference.wrap(new BytesArray(new byte[1])); + private Selector rawSelector; @Before @SuppressWarnings("unchecked") @@ -65,7 +67,7 @@ public void setUp() throws Exception { listener = mock(ActionListener.class); selectionKey = new TestSelectionKey(0); selectionKey.attach(channel); - Selector rawSelector = mock(Selector.class); + rawSelector = mock(Selector.class); this.socketSelector = new SocketSelector(eventHandler, rawSelector); this.socketSelector.setThread(); @@ -83,10 +85,6 @@ public void testRegisterChannel() throws Exception { socketSelector.preSelect(); verify(eventHandler).handleRegistration(channel); - - Set registeredChannels = socketSelector.getRegisteredChannels(); - assertEquals(1, registeredChannels.size()); - assertTrue(registeredChannels.contains(channel)); } public void testClosedChannelWillNotBeRegistered() throws Exception { @@ -97,10 +95,6 @@ public void testClosedChannelWillNotBeRegistered() throws Exception { verify(eventHandler).registrationException(same(channel), any(ClosedChannelException.class)); verify(channel, times(0)).finishConnect(); - - Set registeredChannels = socketSelector.getRegisteredChannels(); - assertEquals(0, registeredChannels.size()); - assertFalse(registeredChannels.contains(channel)); } public void testRegisterChannelFailsDueToException() throws Exception { @@ -113,10 +107,6 @@ public void testRegisterChannelFailsDueToException() throws Exception { verify(eventHandler).registrationException(channel, closedChannelException); verify(channel, times(0)).finishConnect(); - - Set registeredChannels = socketSelector.getRegisteredChannels(); - assertEquals(0, registeredChannels.size()); - assertFalse(registeredChannels.contains(channel)); } public void testSuccessfullyRegisterChannelWillConnect() throws Exception { @@ -309,6 +299,10 @@ public void testCleanup() throws Exception { socketSelector.queueWrite(new WriteOperation(mock(NioSocketChannel.class), networkBuffer, listener)); socketSelector.scheduleForRegistration(unRegisteredChannel); + TestSelectionKey testSelectionKey = new TestSelectionKey(0); + testSelectionKey.attach(channel); + when(rawSelector.keys()).thenReturn(new HashSet<>(Collections.singletonList(testSelectionKey))); + socketSelector.cleanupAndCloseChannels(); verify(listener).onFailure(any(ClosedSelectorException.class)); From 4f711a828bfbdfdc508a1f0938bb296e4c9ef4ba Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 17 Nov 2017 20:13:56 -0500 Subject: [PATCH 19/31] Removes BWC snapshot status handler used in 6.x (#27443) We introduced a new snapshot status update handler in 6.1.0. We will keep the old handler along with this new one in all 6.x. This commit removes the old handler from 7.0. Relates #27151 --- .../snapshots/SnapshotShardsService.java | 99 +------------------ 1 file changed, 2 insertions(+), 97 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index f8a601cc41fd2..15f70e8b2c6fc 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -23,7 +23,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; @@ -68,10 +67,6 @@ import org.elasticsearch.repositories.Repository; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.EmptyTransportResponseHandler; -import org.elasticsearch.transport.TransportChannel; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestHandler; -import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import java.io.IOException; @@ -97,10 +92,8 @@ */ public class SnapshotShardsService extends AbstractLifecycleComponent implements ClusterStateListener, IndexEventListener { - public static final String UPDATE_SNAPSHOT_STATUS_ACTION_NAME_V6 = "internal:cluster/snapshot/update_snapshot"; public static final String UPDATE_SNAPSHOT_STATUS_ACTION_NAME = "internal:cluster/snapshot/update_snapshot_status"; - private final ClusterService clusterService; private final IndicesService indicesService; @@ -138,21 +131,12 @@ public SnapshotShardsService(Settings settings, ClusterService clusterService, S // The constructor of UpdateSnapshotStatusAction will register itself to the TransportService. this.updateSnapshotStatusHandler = new UpdateSnapshotStatusAction(settings, UPDATE_SNAPSHOT_STATUS_ACTION_NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver); - - if (DiscoveryNode.isMasterNode(settings)) { - // This needs to run only on nodes that can become masters - transportService.registerRequestHandler(UPDATE_SNAPSHOT_STATUS_ACTION_NAME_V6, UpdateSnapshotStatusRequestV6::new, ThreadPool.Names.SAME, new UpdateSnapshotStateRequestHandlerV6()); - } - } @Override protected void doStart() { assert this.updateSnapshotStatusHandler != null; assert transportService.getRequestHandler(UPDATE_SNAPSHOT_STATUS_ACTION_NAME) != null; - if (DiscoveryNode.isMasterNode(settings)) { - assert transportService.getRequestHandler(UPDATE_SNAPSHOT_STATUS_ACTION_NAME_V6) != null; - } } @Override @@ -531,13 +515,8 @@ public String toString() { */ public void updateIndexShardSnapshotStatus(Snapshot snapshot, ShardId shardId, ShardSnapshotStatus status, DiscoveryNode master) { try { - if (master.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { - UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(snapshot, shardId, status); - transportService.sendRequest(transportService.getLocalNode(), UPDATE_SNAPSHOT_STATUS_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME); - } else { - UpdateSnapshotStatusRequestV6 requestV6 = new UpdateSnapshotStatusRequestV6(snapshot, shardId, status); - transportService.sendRequest(master, UPDATE_SNAPSHOT_STATUS_ACTION_NAME_V6, requestV6, EmptyTransportResponseHandler.INSTANCE_SAME); - } + UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(snapshot, shardId, status); + transportService.sendRequest(transportService.getLocalNode(), UPDATE_SNAPSHOT_STATUS_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME); } catch (Exception e) { logger.warn((Supplier) () -> new ParameterizedMessage("[{}] [{}] failed to update snapshot state", snapshot, status), e); } @@ -651,78 +630,4 @@ protected ClusterBlockException checkBlock(UpdateIndexShardSnapshotStatusRequest } } - /** - * A BWC version of {@link UpdateIndexShardSnapshotStatusRequest} - */ - static class UpdateSnapshotStatusRequestV6 extends TransportRequest { - private Snapshot snapshot; - private ShardId shardId; - private ShardSnapshotStatus status; - - UpdateSnapshotStatusRequestV6() { - - } - - UpdateSnapshotStatusRequestV6(Snapshot snapshot, ShardId shardId, ShardSnapshotStatus status) { - this.snapshot = snapshot; - this.shardId = shardId; - this.status = status; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - snapshot = new Snapshot(in); - shardId = ShardId.readShardId(in); - status = new ShardSnapshotStatus(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - snapshot.writeTo(out); - shardId.writeTo(out); - status.writeTo(out); - } - - Snapshot snapshot() { - return snapshot; - } - - ShardId shardId() { - return shardId; - } - - ShardSnapshotStatus status() { - return status; - } - - @Override - public String toString() { - return snapshot + ", shardId [" + shardId + "], status [" + status.state() + "]"; - } - } - - /** - * A BWC version of {@link UpdateSnapshotStatusAction} - */ - class UpdateSnapshotStateRequestHandlerV6 implements TransportRequestHandler { - @Override - public void messageReceived(UpdateSnapshotStatusRequestV6 requestV6, final TransportChannel channel) throws Exception { - final UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(requestV6.snapshot(), requestV6.shardId(), requestV6.status()); - innerUpdateSnapshotState(request, new ActionListener() { - @Override - public void onResponse(UpdateIndexShardSnapshotStatusResponse updateSnapshotStatusResponse) { - - } - - @Override - public void onFailure(Exception e) { - logger.warn("Failed to update snapshot status", e); - } - }); - channel.sendResponse(TransportResponse.Empty.INSTANCE); - } - } - } From 56540281a868f6b68be2da11749fe67b58d16dd8 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sat, 18 Nov 2017 07:19:22 -0500 Subject: [PATCH 20/31] Avoid NPE when getting build information When the Elasticsearch code is loaded in an unusual classloading environment (e.g., when using the high-level REST client) in Jetty, the code source can be null and we trip with an NPE. This commit addresses this. Relates #27442 --- core/src/main/java/org/elasticsearch/Build.java | 14 +++++++++----- .../test/java/org/elasticsearch/BuildTests.java | 2 +- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/Build.java b/core/src/main/java/org/elasticsearch/Build.java index 7e46b340dfc01..cc162f35c2fcb 100644 --- a/core/src/main/java/org/elasticsearch/Build.java +++ b/core/src/main/java/org/elasticsearch/Build.java @@ -26,6 +26,7 @@ import java.io.IOException; import java.net.URL; +import java.security.CodeSource; import java.util.jar.JarInputStream; import java.util.jar.Manifest; @@ -45,8 +46,8 @@ public class Build { final boolean isSnapshot; final String esPrefix = "elasticsearch-" + Version.CURRENT; - final URL url = getElasticsearchCodebase(); - final String urlStr = url.toString(); + final URL url = getElasticsearchCodeSourceLocation(); + final String urlStr = url == null ? "" : url.toString(); if (urlStr.startsWith("file:/") && (urlStr.endsWith(esPrefix + ".jar") || urlStr.endsWith(esPrefix + "-SNAPSHOT.jar"))) { try (JarInputStream jar = new JarInputStream(FileSystemUtils.openFileURLStream(url))) { Manifest manifest = jar.getManifest(); @@ -88,10 +89,13 @@ public class Build { private final boolean isSnapshot; /** - * Returns path to elasticsearch codebase path + * The location of the code source for Elasticsearch + * + * @return the location of the code source for Elasticsearch which may be null */ - static URL getElasticsearchCodebase() { - return Build.class.getProtectionDomain().getCodeSource().getLocation(); + static URL getElasticsearchCodeSourceLocation() { + final CodeSource codeSource = Build.class.getProtectionDomain().getCodeSource(); + return codeSource == null ? null : codeSource.getLocation(); } private final String shortHash; diff --git a/core/src/test/java/org/elasticsearch/BuildTests.java b/core/src/test/java/org/elasticsearch/BuildTests.java index ac5496856f92f..a704bc7c3fcb5 100644 --- a/core/src/test/java/org/elasticsearch/BuildTests.java +++ b/core/src/test/java/org/elasticsearch/BuildTests.java @@ -30,7 +30,7 @@ public class BuildTests extends ESTestCase { /** Asking for the jar metadata should not throw exception in tests, no matter how configured */ public void testJarMetadata() throws IOException { - URL url = Build.getElasticsearchCodebase(); + URL url = Build.getElasticsearchCodeSourceLocation(); // throws exception if does not exist, or we cannot access it try (InputStream ignored = FileSystemUtils.openFileURLStream(url)) {} // these should never be null From 365dda874885df29bc725374bcce506ac116f0e4 Mon Sep 17 00:00:00 2001 From: "K. Daniel Newton" Date: Sat, 18 Nov 2017 06:35:35 -0600 Subject: [PATCH 21/31] Correct usage of "an" to "a" in getting started docs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit corrects a word usage error in the getting started docs. Since pronunciation is what determines when to use either "a" or "an" and the word "ubiquitous" is pronounced /yo͞oˈbikwədəs/, it should be preceded by "a." Relates #27420 --- docs/reference/getting-started.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/getting-started.asciidoc b/docs/reference/getting-started.asciidoc index 40354016a638d..e8c617e9be4b8 100755 --- a/docs/reference/getting-started.asciidoc +++ b/docs/reference/getting-started.asciidoc @@ -64,7 +64,7 @@ A type used to be a logical category/partition of your index to allow you to sto [float] === Document -A document is a basic unit of information that can be indexed. For example, you can have a document for a single customer, another document for a single product, and yet another for a single order. This document is expressed in http://json.org/[JSON] (JavaScript Object Notation) which is an ubiquitous internet data interchange format. +A document is a basic unit of information that can be indexed. For example, you can have a document for a single customer, another document for a single product, and yet another for a single order. This document is expressed in http://json.org/[JSON] (JavaScript Object Notation) which is a ubiquitous internet data interchange format. Within an index/type, you can store as many documents as you want. Note that although a document physically resides in an index, a document actually must be indexed/assigned to a type inside an index. From cb3e8f47634784a3e0d1f194d8b965ee880c522c Mon Sep 17 00:00:00 2001 From: Michael Basnight Date: Sat, 18 Nov 2017 21:42:57 -0600 Subject: [PATCH 22/31] Move the CLI into its own subproject (#27114) Projects the depend on the CLI currently depend on core. This should not always be the case. The EnvironmentAwareCommand will remain in :core, but the rest of the CLI components have been moved into their own subproject of :core, :core:cli. --- build.gradle | 1 + core/build.gradle | 8 ++++- core/cli/build.gradle | 36 +++++++++++++++++++ .../licenses/jopt-simple-5.0.2.jar.sha1 | 0 .../licenses/jopt-simple-LICENSE.txt | 0 .../{ => cli}/licenses/jopt-simple-NOTICE.txt | 0 .../java/org/elasticsearch/cli/Command.java | 36 ++++++++----------- .../java/org/elasticsearch/cli/ExitCodes.java | 0 .../org/elasticsearch/cli/MultiCommand.java | 0 .../elasticsearch/cli/SuppressForbidden.java | 34 ++++++++++++++++++ .../java/org/elasticsearch/cli/Terminal.java | 2 -- .../org/elasticsearch/cli/UserException.java | 0 .../cli/EnvironmentAwareCommand.java | 22 ++++++++++++ distribution/tools/plugin-cli/build.gradle | 1 + .../elasticsearch/cli/EvilCommandTests.java | 8 ++--- settings.gradle | 1 + test/framework/build.gradle | 1 + 17 files changed, 121 insertions(+), 29 deletions(-) create mode 100644 core/cli/build.gradle rename core/{ => cli}/licenses/jopt-simple-5.0.2.jar.sha1 (100%) rename core/{ => cli}/licenses/jopt-simple-LICENSE.txt (100%) rename core/{ => cli}/licenses/jopt-simple-NOTICE.txt (100%) rename core/{ => cli}/src/main/java/org/elasticsearch/cli/Command.java (81%) rename core/{ => cli}/src/main/java/org/elasticsearch/cli/ExitCodes.java (100%) rename core/{ => cli}/src/main/java/org/elasticsearch/cli/MultiCommand.java (100%) create mode 100644 core/cli/src/main/java/org/elasticsearch/cli/SuppressForbidden.java rename core/{ => cli}/src/main/java/org/elasticsearch/cli/Terminal.java (99%) rename core/{ => cli}/src/main/java/org/elasticsearch/cli/UserException.java (100%) diff --git a/build.gradle b/build.gradle index e498101a16a78..4484d3fc14c0e 100644 --- a/build.gradle +++ b/build.gradle @@ -231,6 +231,7 @@ subprojects { "org.elasticsearch.gradle:build-tools:${version}": ':build-tools', "org.elasticsearch:rest-api-spec:${version}": ':rest-api-spec', "org.elasticsearch:elasticsearch:${version}": ':core', + "org.elasticsearch:elasticsearch-cli:${version}": ':core:cli', "org.elasticsearch.client:elasticsearch-rest-client:${version}": ':client:rest', "org.elasticsearch.client:elasticsearch-rest-client-sniffer:${version}": ':client:sniffer', "org.elasticsearch.client:elasticsearch-rest-high-level-client:${version}": ':client:rest-high-level', diff --git a/core/build.gradle b/core/build.gradle index fe60cd8b1cf6b..b5ad8eb5c32a6 100644 --- a/core/build.gradle +++ b/core/build.gradle @@ -58,7 +58,7 @@ dependencies { compile 'org.elasticsearch:securesm:1.1' // utilities - compile 'net.sf.jopt-simple:jopt-simple:5.0.2' + compile "org.elasticsearch:elasticsearch-cli:${version}" compile 'com.carrotsearch:hppc:0.7.1' // time handling, remove with java 8 time @@ -265,6 +265,12 @@ if (JavaVersion.current() > JavaVersion.VERSION_1_8) { dependencyLicenses { mapping from: /lucene-.*/, to: 'lucene' mapping from: /jackson-.*/, to: 'jackson' + dependencies = project.configurations.runtime.fileCollection { + it.group.startsWith('org.elasticsearch') == false || + // keep the following org.elasticsearch jars in + (it.name == 'jna' || + it.name == 'securesm') + } } if (isEclipse == false || project.path == ":core-tests") { diff --git a/core/cli/build.gradle b/core/cli/build.gradle new file mode 100644 index 0000000000000..fc93523f6b785 --- /dev/null +++ b/core/cli/build.gradle @@ -0,0 +1,36 @@ +/* + * 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. + */ + +import org.elasticsearch.gradle.precommit.PrecommitTasks + +apply plugin: 'elasticsearch.build' + +archivesBaseName = 'elasticsearch-cli' + +dependencies { + compile 'net.sf.jopt-simple:jopt-simple:5.0.2' +} + +test.enabled = false +// Since CLI does not depend on :core, it cannot run the jarHell task +jarHell.enabled = false + +forbiddenApisMain { + signaturesURLs = [PrecommitTasks.getResource('/forbidden/jdk-signatures.txt')] +} diff --git a/core/licenses/jopt-simple-5.0.2.jar.sha1 b/core/cli/licenses/jopt-simple-5.0.2.jar.sha1 similarity index 100% rename from core/licenses/jopt-simple-5.0.2.jar.sha1 rename to core/cli/licenses/jopt-simple-5.0.2.jar.sha1 diff --git a/core/licenses/jopt-simple-LICENSE.txt b/core/cli/licenses/jopt-simple-LICENSE.txt similarity index 100% rename from core/licenses/jopt-simple-LICENSE.txt rename to core/cli/licenses/jopt-simple-LICENSE.txt diff --git a/core/licenses/jopt-simple-NOTICE.txt b/core/cli/licenses/jopt-simple-NOTICE.txt similarity index 100% rename from core/licenses/jopt-simple-NOTICE.txt rename to core/cli/licenses/jopt-simple-NOTICE.txt diff --git a/core/src/main/java/org/elasticsearch/cli/Command.java b/core/cli/src/main/java/org/elasticsearch/cli/Command.java similarity index 81% rename from core/src/main/java/org/elasticsearch/cli/Command.java rename to core/cli/src/main/java/org/elasticsearch/cli/Command.java index a60dece26113a..78a9f31283d00 100644 --- a/core/src/main/java/org/elasticsearch/cli/Command.java +++ b/core/cli/src/main/java/org/elasticsearch/cli/Command.java @@ -23,11 +23,6 @@ import joptsimple.OptionParser; import joptsimple.OptionSet; import joptsimple.OptionSpec; -import org.apache.logging.log4j.Level; -import org.apache.lucene.util.SetOnce; -import org.elasticsearch.common.SuppressForbidden; -import org.elasticsearch.common.logging.LogConfigurator; -import org.elasticsearch.common.settings.Settings; import java.io.Closeable; import java.io.IOException; @@ -55,12 +50,13 @@ public Command(String description) { this.description = description; } - final SetOnce shutdownHookThread = new SetOnce<>(); + private Thread shutdownHookThread; /** Parses options for this command from args and executes it. */ public final int main(String[] args, Terminal terminal) throws Exception { if (addShutdownHook()) { - shutdownHookThread.set(new Thread(() -> { + + shutdownHookThread = new Thread(() -> { try { this.close(); } catch (final IOException e) { @@ -75,16 +71,11 @@ public final int main(String[] args, Terminal terminal) throws Exception { throw new AssertionError(impossible); } } - })); - Runtime.getRuntime().addShutdownHook(shutdownHookThread.get()); + }); + Runtime.getRuntime().addShutdownHook(shutdownHookThread); } - if (shouldConfigureLoggingWithoutConfig()) { - // initialize default for es.logger.level because we will not read the log4j2.properties - final String loggerLevel = System.getProperty("es.logger.level", Level.INFO.name()); - final Settings settings = Settings.builder().put("logger.level", loggerLevel).build(); - LogConfigurator.configureWithoutConfig(settings); - } + beforeExecute(); try { mainWithoutErrorHandling(args, terminal); @@ -103,14 +94,10 @@ public final int main(String[] args, Terminal terminal) throws Exception { } /** - * Indicate whether or not logging should be configured without reading a log4j2.properties. Most commands should do this because we do - * not configure logging for CLI tools. Only commands that configure logging on their own should not do this. - * - * @return true if logging should be configured without reading a log4j2.properties file + * Setup method to be executed before parsing or execution of the command being run. Any exceptions thrown by the + * method will not be cleanly caught by the parser. */ - protected boolean shouldConfigureLoggingWithoutConfig() { - return true; - } + protected void beforeExecute() {} /** * Executes the command, but all errors are thrown. @@ -166,6 +153,11 @@ protected boolean addShutdownHook() { return true; } + /** Gets the shutdown hook thread if it exists **/ + Thread getShutdownHookThread() { + return shutdownHookThread; + } + @Override public void close() throws IOException { diff --git a/core/src/main/java/org/elasticsearch/cli/ExitCodes.java b/core/cli/src/main/java/org/elasticsearch/cli/ExitCodes.java similarity index 100% rename from core/src/main/java/org/elasticsearch/cli/ExitCodes.java rename to core/cli/src/main/java/org/elasticsearch/cli/ExitCodes.java diff --git a/core/src/main/java/org/elasticsearch/cli/MultiCommand.java b/core/cli/src/main/java/org/elasticsearch/cli/MultiCommand.java similarity index 100% rename from core/src/main/java/org/elasticsearch/cli/MultiCommand.java rename to core/cli/src/main/java/org/elasticsearch/cli/MultiCommand.java diff --git a/core/cli/src/main/java/org/elasticsearch/cli/SuppressForbidden.java b/core/cli/src/main/java/org/elasticsearch/cli/SuppressForbidden.java new file mode 100644 index 0000000000000..882414a0eaad3 --- /dev/null +++ b/core/cli/src/main/java/org/elasticsearch/cli/SuppressForbidden.java @@ -0,0 +1,34 @@ +/* + * 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.cli; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation to suppress forbidden-apis errors inside a whole class, a method, or a field. + */ +@Retention(RetentionPolicy.CLASS) +@Target({ ElementType.CONSTRUCTOR, ElementType.FIELD, ElementType.METHOD, ElementType.TYPE }) +public @interface SuppressForbidden { + String reason(); +} + diff --git a/core/src/main/java/org/elasticsearch/cli/Terminal.java b/core/cli/src/main/java/org/elasticsearch/cli/Terminal.java similarity index 99% rename from core/src/main/java/org/elasticsearch/cli/Terminal.java rename to core/cli/src/main/java/org/elasticsearch/cli/Terminal.java index d42e3475dc491..85abd61677445 100644 --- a/core/src/main/java/org/elasticsearch/cli/Terminal.java +++ b/core/cli/src/main/java/org/elasticsearch/cli/Terminal.java @@ -19,8 +19,6 @@ package org.elasticsearch.cli; -import org.elasticsearch.common.SuppressForbidden; - import java.io.BufferedReader; import java.io.Console; import java.io.IOException; diff --git a/core/src/main/java/org/elasticsearch/cli/UserException.java b/core/cli/src/main/java/org/elasticsearch/cli/UserException.java similarity index 100% rename from core/src/main/java/org/elasticsearch/cli/UserException.java rename to core/cli/src/main/java/org/elasticsearch/cli/UserException.java diff --git a/core/src/main/java/org/elasticsearch/cli/EnvironmentAwareCommand.java b/core/src/main/java/org/elasticsearch/cli/EnvironmentAwareCommand.java index d9d19a56a2f32..44fcb37d0835c 100644 --- a/core/src/main/java/org/elasticsearch/cli/EnvironmentAwareCommand.java +++ b/core/src/main/java/org/elasticsearch/cli/EnvironmentAwareCommand.java @@ -22,7 +22,9 @@ import joptsimple.OptionSet; import joptsimple.OptionSpec; import joptsimple.util.KeyValuePair; +import org.apache.logging.log4j.Level; import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.logging.LogConfigurator; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.node.InternalSettingsPreparer; @@ -102,6 +104,26 @@ private static void putSystemPropertyIfSettingIsMissing(final Map Date: Sun, 19 Nov 2017 22:33:17 -0600 Subject: [PATCH 23/31] Remove config prompting for secrets and text (#27216) This commit removes the ability to use ${prompt.secret} and ${prompt.text} as valid config settings. Secure settings has obsoleted the need for this, and it cleans up some of the code in Bootstrap. --- .../elasticsearch/bootstrap/Bootstrap.java | 4 +- .../cli/EnvironmentAwareCommand.java | 6 +- .../node/InternalSettingsPreparer.java | 105 ++++++------------ .../java/org/elasticsearch/node/Node.java | 2 +- .../settings/AddFileKeyStoreCommandTests.java | 2 +- .../AddStringKeyStoreCommandTests.java | 2 +- .../settings/CreateKeyStoreCommandTests.java | 2 +- .../settings/ListKeyStoreCommandTests.java | 2 +- .../RemoveSettingKeyStoreCommandTests.java | 2 +- .../common/settings/SettingsTests.java | 10 -- .../node/InternalSettingsPreparerTests.java | 63 +---------- .../plugins/ListPluginsCommandTests.java | 2 +- .../plugins/RemovePluginCommandTests.java | 2 +- docs/reference/setup/configuration.asciidoc | 27 ----- .../bootstrap/ESElasticsearchCliTestCase.java | 2 +- .../java/org/elasticsearch/node/MockNode.java | 2 +- .../test/AbstractQueryTestCase.java | 2 +- 17 files changed, 54 insertions(+), 183 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java b/core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java index 30b9fb7e28dd0..ced1dace96e8d 100644 --- a/core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java +++ b/core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java @@ -30,7 +30,6 @@ import org.apache.lucene.util.StringHelper; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; -import org.elasticsearch.cli.Terminal; import org.elasticsearch.cli.UserException; import org.elasticsearch.common.PidFile; import org.elasticsearch.common.SuppressForbidden; @@ -245,7 +244,6 @@ private static Environment createEnvironment( final SecureSettings secureSettings, final Settings initialSettings, final Path configPath) { - Terminal terminal = foreground ? Terminal.DEFAULT : null; Settings.Builder builder = Settings.builder(); if (pidFile != null) { builder.put(Environment.PIDFILE_SETTING.getKey(), pidFile); @@ -254,7 +252,7 @@ private static Environment createEnvironment( if (secureSettings != null) { builder.setSecureSettings(secureSettings); } - return InternalSettingsPreparer.prepareEnvironment(builder.build(), terminal, Collections.emptyMap(), configPath); + return InternalSettingsPreparer.prepareEnvironment(builder.build(), Collections.emptyMap(), configPath); } private void start() throws NodeValidationException { diff --git a/core/src/main/java/org/elasticsearch/cli/EnvironmentAwareCommand.java b/core/src/main/java/org/elasticsearch/cli/EnvironmentAwareCommand.java index 44fcb37d0835c..b2bd887e0f6ec 100644 --- a/core/src/main/java/org/elasticsearch/cli/EnvironmentAwareCommand.java +++ b/core/src/main/java/org/elasticsearch/cli/EnvironmentAwareCommand.java @@ -68,16 +68,16 @@ protected void execute(Terminal terminal, OptionSet options) throws Exception { putSystemPropertyIfSettingIsMissing(settings, "path.home", "es.path.home"); putSystemPropertyIfSettingIsMissing(settings, "path.logs", "es.path.logs"); - execute(terminal, options, createEnv(terminal, settings)); + execute(terminal, options, createEnv(settings)); } /** Create an {@link Environment} for the command to use. Overrideable for tests. */ - protected Environment createEnv(final Terminal terminal, final Map settings) throws UserException { + protected Environment createEnv(final Map settings) throws UserException { final String esPathConf = System.getProperty("es.path.conf"); if (esPathConf == null) { throw new UserException(ExitCodes.CONFIG, "the system property [es.path.conf] must be set"); } - return InternalSettingsPreparer.prepareEnvironment(Settings.EMPTY, terminal, settings, getConfigPath(esPathConf)); + return InternalSettingsPreparer.prepareEnvironment(Settings.EMPTY, settings, getConfigPath(esPathConf)); } @SuppressForbidden(reason = "need path to construct environment") diff --git a/core/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java b/core/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java index a2c7663ec9e15..29ed1e88188cb 100644 --- a/core/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java +++ b/core/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java @@ -28,9 +28,8 @@ import java.util.Map; import java.util.function.Function; -import org.elasticsearch.cli.Terminal; +import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsException; @@ -38,10 +37,8 @@ public class InternalSettingsPreparer { - private static final String[] ALLOWED_SUFFIXES = {".yml", ".yaml", ".json"}; - - public static final String SECRET_PROMPT_VALUE = "${prompt.secret}"; - public static final String TEXT_PROMPT_VALUE = "${prompt.text}"; + private static final String SECRET_PROMPT_VALUE = "${prompt.secret}"; + private static final String TEXT_PROMPT_VALUE = "${prompt.text}"; /** * Prepares the settings by gathering all elasticsearch system properties and setting defaults. @@ -49,36 +46,29 @@ public class InternalSettingsPreparer { public static Settings prepareSettings(Settings input) { Settings.Builder output = Settings.builder(); initializeSettings(output, input, Collections.emptyMap()); - finalizeSettings(output, null); + finalizeSettings(output); return output.build(); } /** - * Prepares the settings by gathering all elasticsearch system properties, optionally loading the configuration settings, - * and then replacing all property placeholders. If a {@link Terminal} is provided and configuration settings are loaded, - * settings with a value of ${prompt.text} or ${prompt.secret} will result in a prompt for - * the setting to the user. + * Prepares the settings by gathering all elasticsearch system properties, optionally loading the configuration settings. + * * @param input The custom settings to use. These are not overwritten by settings in the configuration file. - * @param terminal the Terminal to use for input/output * @return the {@link Settings} and {@link Environment} as a {@link Tuple} */ - public static Environment prepareEnvironment(Settings input, Terminal terminal) { - return prepareEnvironment(input, terminal, Collections.emptyMap(), null); + public static Environment prepareEnvironment(Settings input) { + return prepareEnvironment(input, Collections.emptyMap(), null); } /** - * Prepares the settings by gathering all elasticsearch system properties, optionally loading the configuration settings, - * and then replacing all property placeholders. If a {@link Terminal} is provided and configuration settings are loaded, - * settings with a value of ${prompt.text} or ${prompt.secret} will result in a prompt for - * the setting to the user. + * Prepares the settings by gathering all elasticsearch system properties, optionally loading the configuration settings. * * @param input the custom settings to use; these are not overwritten by settings in the configuration file - * @param terminal the Terminal to use for input/output * @param properties map of properties key/value pairs (usually from the command-line) * @param configPath path to config directory; (use null to indicate the default) * @return the {@link Settings} and {@link Environment} as a {@link Tuple} */ - public static Environment prepareEnvironment(Settings input, Terminal terminal, Map properties, Path configPath) { + public static Environment prepareEnvironment(Settings input, Map properties, Path configPath) { // just create enough settings to build the environment, to get the config dir Settings.Builder output = Settings.builder(); initializeSettings(output, input, properties); @@ -104,7 +94,8 @@ public static Environment prepareEnvironment(Settings input, Terminal terminal, // re-initialize settings now that the config file has been loaded initializeSettings(output, input, properties); - finalizeSettings(output, terminal); + checkSettingsForTerminalDeprecation(output); + finalizeSettings(output); environment = new Environment(output.build(), configPath); @@ -128,10 +119,28 @@ static void initializeSettings(final Settings.Builder output, final Settings inp } /** - * Finish preparing settings by replacing forced settings, prompts, and any defaults that need to be added. - * The provided terminal is used to prompt for settings needing to be replaced. + * Checks all settings values to make sure they do not have the old prompt settings. These were deprecated in 6.0.0. + * This check should be removed in 8.0.0. + */ + private static void checkSettingsForTerminalDeprecation(final Settings.Builder output) throws SettingsException { + // This method to be removed in 8.0.0, as it was deprecated in 6.0 and removed in 7.0 + assert Version.CURRENT.major != 8: "Logic pertaining to config driven prompting should be removed"; + for (String setting : output.keys()) { + switch (output.get(setting)) { + case SECRET_PROMPT_VALUE: + throw new SettingsException("Config driven secret prompting was deprecated in 6.0.0. Use the keystore" + + " for secure settings."); + case TEXT_PROMPT_VALUE: + throw new SettingsException("Config driven text prompting was deprecated in 6.0.0. Use the keystore" + + " for secure settings."); + } + } + } + + /** + * Finish preparing settings by replacing forced settings and any defaults that need to be added. */ - private static void finalizeSettings(Settings.Builder output, Terminal terminal) { + private static void finalizeSettings(Settings.Builder output) { // allow to force set properties based on configuration of the settings provided List forcedSettings = new ArrayList<>(); for (String setting : output.keys()) { @@ -149,53 +158,5 @@ private static void finalizeSettings(Settings.Builder output, Terminal terminal) if (output.get(ClusterName.CLUSTER_NAME_SETTING.getKey()) == null) { output.put(ClusterName.CLUSTER_NAME_SETTING.getKey(), ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY).value()); } - - replacePromptPlaceholders(output, terminal); - } - - private static void replacePromptPlaceholders(Settings.Builder settings, Terminal terminal) { - List secretToPrompt = new ArrayList<>(); - List textToPrompt = new ArrayList<>(); - for (String key : settings.keys()) { - switch (settings.get(key)) { - case SECRET_PROMPT_VALUE: - secretToPrompt.add(key); - break; - case TEXT_PROMPT_VALUE: - textToPrompt.add(key); - break; - } - } - for (String setting : secretToPrompt) { - String secretValue = promptForValue(setting, terminal, true); - if (Strings.hasLength(secretValue)) { - settings.put(setting, secretValue); - } else { - // TODO: why do we remove settings if prompt returns empty?? - settings.remove(setting); - } - } - for (String setting : textToPrompt) { - String textValue = promptForValue(setting, terminal, false); - if (Strings.hasLength(textValue)) { - settings.put(setting, textValue); - } else { - // TODO: why do we remove settings if prompt returns empty?? - settings.remove(setting); - } - } - } - - private static String promptForValue(String key, Terminal terminal, boolean secret) { - if (terminal == null) { - throw new UnsupportedOperationException("found property [" + key + "] with value [" - + (secret ? SECRET_PROMPT_VALUE : TEXT_PROMPT_VALUE) - + "]. prompting for property values is only supported when running elasticsearch in the foreground"); - } - - if (secret) { - return new String(terminal.readSecret("Enter value for [" + key + "]: ")); - } - return terminal.readText("Enter value for [" + key + "]: "); } } diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 0ddc03de8c049..fee6d76ca3d5c 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -238,7 +238,7 @@ public static final Settings addNodeNameIfNeeded(Settings settings, final String * @param preparedSettings Base settings to configure the node with */ public Node(Settings preparedSettings) { - this(InternalSettingsPreparer.prepareEnvironment(preparedSettings, null)); + this(InternalSettingsPreparer.prepareEnvironment(preparedSettings)); } public Node(Environment environment) { diff --git a/core/src/test/java/org/elasticsearch/common/settings/AddFileKeyStoreCommandTests.java b/core/src/test/java/org/elasticsearch/common/settings/AddFileKeyStoreCommandTests.java index 071d394eb1e64..b562c72011b3c 100644 --- a/core/src/test/java/org/elasticsearch/common/settings/AddFileKeyStoreCommandTests.java +++ b/core/src/test/java/org/elasticsearch/common/settings/AddFileKeyStoreCommandTests.java @@ -37,7 +37,7 @@ public class AddFileKeyStoreCommandTests extends KeyStoreCommandTestCase { protected Command newCommand() { return new AddFileKeyStoreCommand() { @Override - protected Environment createEnv(Terminal terminal, Map settings) throws UserException { + protected Environment createEnv(Map settings) throws UserException { return env; } }; diff --git a/core/src/test/java/org/elasticsearch/common/settings/AddStringKeyStoreCommandTests.java b/core/src/test/java/org/elasticsearch/common/settings/AddStringKeyStoreCommandTests.java index d0d8fdb500d3a..733832a500f3c 100644 --- a/core/src/test/java/org/elasticsearch/common/settings/AddStringKeyStoreCommandTests.java +++ b/core/src/test/java/org/elasticsearch/common/settings/AddStringKeyStoreCommandTests.java @@ -39,7 +39,7 @@ public class AddStringKeyStoreCommandTests extends KeyStoreCommandTestCase { protected Command newCommand() { return new AddStringKeyStoreCommand() { @Override - protected Environment createEnv(Terminal terminal, Map settings) throws UserException { + protected Environment createEnv(Map settings) throws UserException { return env; } @Override diff --git a/core/src/test/java/org/elasticsearch/common/settings/CreateKeyStoreCommandTests.java b/core/src/test/java/org/elasticsearch/common/settings/CreateKeyStoreCommandTests.java index 88e058b6444d7..cfa6625b03e36 100644 --- a/core/src/test/java/org/elasticsearch/common/settings/CreateKeyStoreCommandTests.java +++ b/core/src/test/java/org/elasticsearch/common/settings/CreateKeyStoreCommandTests.java @@ -35,7 +35,7 @@ public class CreateKeyStoreCommandTests extends KeyStoreCommandTestCase { protected Command newCommand() { return new CreateKeyStoreCommand() { @Override - protected Environment createEnv(Terminal terminal, Map settings) throws UserException { + protected Environment createEnv(Map settings) throws UserException { return env; } }; diff --git a/core/src/test/java/org/elasticsearch/common/settings/ListKeyStoreCommandTests.java b/core/src/test/java/org/elasticsearch/common/settings/ListKeyStoreCommandTests.java index 406821dafdc34..c5c18abc08cbf 100644 --- a/core/src/test/java/org/elasticsearch/common/settings/ListKeyStoreCommandTests.java +++ b/core/src/test/java/org/elasticsearch/common/settings/ListKeyStoreCommandTests.java @@ -35,7 +35,7 @@ public class ListKeyStoreCommandTests extends KeyStoreCommandTestCase { protected Command newCommand() { return new ListKeyStoreCommand() { @Override - protected Environment createEnv(Terminal terminal, Map settings) throws UserException { + protected Environment createEnv(Map settings) throws UserException { return env; } }; diff --git a/core/src/test/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommandTests.java b/core/src/test/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommandTests.java index 3ad48127b4b8a..3c93441c9ae50 100644 --- a/core/src/test/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommandTests.java +++ b/core/src/test/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommandTests.java @@ -36,7 +36,7 @@ public class RemoveSettingKeyStoreCommandTests extends KeyStoreCommandTestCase { protected Command newCommand() { return new RemoveSettingKeyStoreCommand() { @Override - protected Environment createEnv(Terminal terminal, Map settings) throws UserException { + protected Environment createEnv(Map settings) throws UserException { return env; } }; diff --git a/core/src/test/java/org/elasticsearch/common/settings/SettingsTests.java b/core/src/test/java/org/elasticsearch/common/settings/SettingsTests.java index 42cb0f1e3e7e3..04cd1717e7f06 100644 --- a/core/src/test/java/org/elasticsearch/common/settings/SettingsTests.java +++ b/core/src/test/java/org/elasticsearch/common/settings/SettingsTests.java @@ -87,16 +87,6 @@ public void testReplacePropertiesPlaceholderByEnvironmentVariables() { assertThat(implicitEnvSettings.get("setting1"), equalTo(hostname)); } - public void testReplacePropertiesPlaceholderIgnoresPrompt() { - Settings settings = Settings.builder() - .put("setting1", "${prompt.text}") - .put("setting2", "${prompt.secret}") - .replacePropertyPlaceholders() - .build(); - assertThat(settings.get("setting1"), is("${prompt.text}")); - assertThat(settings.get("setting2"), is("${prompt.secret}")); - } - public void testGetAsSettings() { Settings settings = Settings.builder() .put("bar", "hello world") diff --git a/core/src/test/java/org/elasticsearch/node/InternalSettingsPreparerTests.java b/core/src/test/java/org/elasticsearch/node/InternalSettingsPreparerTests.java index 1ce6ed5779f9c..b8c3c158e4dd0 100644 --- a/core/src/test/java/org/elasticsearch/node/InternalSettingsPreparerTests.java +++ b/core/src/test/java/org/elasticsearch/node/InternalSettingsPreparerTests.java @@ -67,7 +67,7 @@ public void testEmptySettings() { assertNotNull(settings.get(ClusterName.CLUSTER_NAME_SETTING.getKey())); // a cluster name was set int size = settings.names().size(); - Environment env = InternalSettingsPreparer.prepareEnvironment(baseEnvSettings, null); + Environment env = InternalSettingsPreparer.prepareEnvironment(baseEnvSettings); settings = env.settings(); assertNull(settings.get("node.name")); // a name was not set assertNotNull(settings.get(ClusterName.CLUSTER_NAME_SETTING.getKey())); // a cluster name was set @@ -84,57 +84,6 @@ public void testDefaultClusterName() { assertEquals("foobar", settings.get("cluster.name")); } - public void testReplacePromptPlaceholders() { - MockTerminal terminal = new MockTerminal(); - terminal.addTextInput("text"); - terminal.addSecretInput("replaced"); - - Settings.Builder builder = Settings.builder() - .put(baseEnvSettings) - .put("password.replace", InternalSettingsPreparer.SECRET_PROMPT_VALUE) - .put("dont.replace", "prompt:secret") - .put("dont.replace2", "_prompt:secret_") - .put("dont.replace3", "_prompt:text__") - .put("dont.replace4", "__prompt:text_") - .put("dont.replace5", "prompt:secret__") - .put("replace_me", InternalSettingsPreparer.TEXT_PROMPT_VALUE); - Settings settings = InternalSettingsPreparer.prepareEnvironment(builder.build(), terminal).settings(); - - assertThat(settings.get("password.replace"), equalTo("replaced")); - assertThat(settings.get("replace_me"), equalTo("text")); - - // verify other values unchanged - assertThat(settings.get("dont.replace"), equalTo("prompt:secret")); - assertThat(settings.get("dont.replace2"), equalTo("_prompt:secret_")); - assertThat(settings.get("dont.replace3"), equalTo("_prompt:text__")); - assertThat(settings.get("dont.replace4"), equalTo("__prompt:text_")); - assertThat(settings.get("dont.replace5"), equalTo("prompt:secret__")); - } - - public void testReplaceSecretPromptPlaceholderWithNullTerminal() { - Settings.Builder builder = Settings.builder() - .put(baseEnvSettings) - .put("replace_me1", InternalSettingsPreparer.SECRET_PROMPT_VALUE); - try { - InternalSettingsPreparer.prepareEnvironment(builder.build(), null); - fail("an exception should have been thrown since no terminal was provided!"); - } catch (UnsupportedOperationException e) { - assertThat(e.getMessage(), containsString("with value [" + InternalSettingsPreparer.SECRET_PROMPT_VALUE + "]")); - } - } - - public void testReplaceTextPromptPlaceholderWithNullTerminal() { - Settings.Builder builder = Settings.builder() - .put(baseEnvSettings) - .put("replace_me1", InternalSettingsPreparer.TEXT_PROMPT_VALUE); - try { - InternalSettingsPreparer.prepareEnvironment(builder.build(), null); - fail("an exception should have been thrown since no terminal was provided!"); - } catch (UnsupportedOperationException e) { - assertThat(e.getMessage(), containsString("with value [" + InternalSettingsPreparer.TEXT_PROMPT_VALUE + "]")); - } - } - public void testGarbageIsNotSwallowed() throws IOException { try { InputStream garbage = getClass().getResourceAsStream("/config/garbage/garbage.yml"); @@ -144,7 +93,7 @@ public void testGarbageIsNotSwallowed() throws IOException { Files.copy(garbage, config.resolve("elasticsearch.yml")); InternalSettingsPreparer.prepareEnvironment(Settings.builder() .put(baseEnvSettings) - .build(), null); + .build()); } catch (SettingsException e) { assertEquals("Failed to load settings from [elasticsearch.yml]", e.getMessage()); } @@ -156,7 +105,7 @@ public void testYamlNotAllowed() throws IOException { Files.createDirectory(config); Files.copy(yaml, config.resolve("elasticsearch.yaml")); SettingsException e = expectThrows(SettingsException.class, () -> - InternalSettingsPreparer.prepareEnvironment(Settings.builder().put(baseEnvSettings).build(), null)); + InternalSettingsPreparer.prepareEnvironment(Settings.builder().put(baseEnvSettings).build())); assertEquals("elasticsearch.yaml was deprecated in 5.5.0 and must be renamed to elasticsearch.yml", e.getMessage()); } @@ -166,7 +115,7 @@ public void testJsonNotAllowed() throws IOException { Files.createDirectory(config); Files.copy(yaml, config.resolve("elasticsearch.json")); SettingsException e = expectThrows(SettingsException.class, () -> - InternalSettingsPreparer.prepareEnvironment(Settings.builder().put(baseEnvSettings).build(), null)); + InternalSettingsPreparer.prepareEnvironment(Settings.builder().put(baseEnvSettings).build())); assertEquals("elasticsearch.json was deprecated in 5.5.0 and must be converted to elasticsearch.yml", e.getMessage()); } @@ -174,14 +123,14 @@ public void testSecureSettings() { MockSecureSettings secureSettings = new MockSecureSettings(); secureSettings.setString("foo", "secret"); Settings input = Settings.builder().put(baseEnvSettings).setSecureSettings(secureSettings).build(); - Environment env = InternalSettingsPreparer.prepareEnvironment(input, null); + Environment env = InternalSettingsPreparer.prepareEnvironment(input); Setting fakeSetting = SecureSetting.secureString("foo", null); assertEquals("secret", fakeSetting.get(env.settings()).toString()); } public void testDefaultPropertiesDoNothing() throws Exception { Map props = Collections.singletonMap("default.setting", "foo"); - Environment env = InternalSettingsPreparer.prepareEnvironment(baseEnvSettings, null, props, null); + Environment env = InternalSettingsPreparer.prepareEnvironment(baseEnvSettings, props, null); assertEquals("foo", env.settings().get("default.setting")); assertNull(env.settings().get("setting")); } diff --git a/distribution/tools/plugin-cli/src/test/java/org/elasticsearch/plugins/ListPluginsCommandTests.java b/distribution/tools/plugin-cli/src/test/java/org/elasticsearch/plugins/ListPluginsCommandTests.java index 9a1f61c0d889c..0e82d5dd5c80f 100644 --- a/distribution/tools/plugin-cli/src/test/java/org/elasticsearch/plugins/ListPluginsCommandTests.java +++ b/distribution/tools/plugin-cli/src/test/java/org/elasticsearch/plugins/ListPluginsCommandTests.java @@ -69,7 +69,7 @@ static MockTerminal listPlugins(Path home, String[] args) throws Exception { MockTerminal terminal = new MockTerminal(); int status = new ListPluginsCommand() { @Override - protected Environment createEnv(Terminal terminal, Map settings) throws UserException { + protected Environment createEnv(Map settings) throws UserException { Settings.Builder builder = Settings.builder().put("path.home", home); settings.forEach((k,v) -> builder.put(k, v)); final Settings realSettings = builder.build(); diff --git a/distribution/tools/plugin-cli/src/test/java/org/elasticsearch/plugins/RemovePluginCommandTests.java b/distribution/tools/plugin-cli/src/test/java/org/elasticsearch/plugins/RemovePluginCommandTests.java index 6c462d39e5775..356aeff7260e6 100644 --- a/distribution/tools/plugin-cli/src/test/java/org/elasticsearch/plugins/RemovePluginCommandTests.java +++ b/distribution/tools/plugin-cli/src/test/java/org/elasticsearch/plugins/RemovePluginCommandTests.java @@ -57,7 +57,7 @@ private MockRemovePluginCommand(final Environment env) { } @Override - protected Environment createEnv(Terminal terminal, Map settings) throws UserException { + protected Environment createEnv(Map settings) throws UserException { return env; } diff --git a/docs/reference/setup/configuration.asciidoc b/docs/reference/setup/configuration.asciidoc index bad6da3701fb5..83d66f196d634 100644 --- a/docs/reference/setup/configuration.asciidoc +++ b/docs/reference/setup/configuration.asciidoc @@ -78,30 +78,3 @@ variable, for instance: node.name: ${HOSTNAME} network.host: ${ES_NETWORK_HOST} -------------------------------------------------- - -[float] -=== Prompting for settings - -For settings that you do not wish to store in the configuration file, you can -use the value `${prompt.text}` or `${prompt.secret}` and start Elasticsearch -in the foreground. `${prompt.secret}` has echoing disabled so that the value -entered will not be shown in your terminal; `${prompt.text}` will allow you to -see the value as you type it in. For example: - -[source,yaml] --------------------------------------------------- -node: - name: ${prompt.text} --------------------------------------------------- - -When starting Elasticsearch, you will be prompted to enter the actual value -like so: - -[source,sh] --------------------------------------------------- -Enter value for [node.name]: --------------------------------------------------- - -NOTE: Elasticsearch will not start if `${prompt.text}` or `${prompt.secret}` -is used in the settings and the process is run as a service or in the background. - diff --git a/test/framework/src/main/java/org/elasticsearch/bootstrap/ESElasticsearchCliTestCase.java b/test/framework/src/main/java/org/elasticsearch/bootstrap/ESElasticsearchCliTestCase.java index b3ebdb6a69b29..9aabd901d0d49 100644 --- a/test/framework/src/main/java/org/elasticsearch/bootstrap/ESElasticsearchCliTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/bootstrap/ESElasticsearchCliTestCase.java @@ -51,7 +51,7 @@ void runTest( final AtomicBoolean init = new AtomicBoolean(); final int status = Elasticsearch.main(args, new Elasticsearch() { @Override - protected Environment createEnv(final Terminal terminal, final Map settings) throws UserException { + protected Environment createEnv(final Map settings) throws UserException { Settings.Builder builder = Settings.builder().put("path.home", home); settings.forEach((k,v) -> builder.put(k, v)); final Settings realSettings = builder.build(); diff --git a/test/framework/src/main/java/org/elasticsearch/node/MockNode.java b/test/framework/src/main/java/org/elasticsearch/node/MockNode.java index 4255163db7fea..34ed8d337e800 100644 --- a/test/framework/src/main/java/org/elasticsearch/node/MockNode.java +++ b/test/framework/src/main/java/org/elasticsearch/node/MockNode.java @@ -66,7 +66,7 @@ public MockNode(Settings settings, Collection> classpath } public MockNode(Settings settings, Collection> classpathPlugins, Path configPath) { - this(InternalSettingsPreparer.prepareEnvironment(settings, null, Collections.emptyMap(), configPath), classpathPlugins); + this(InternalSettingsPreparer.prepareEnvironment(settings, Collections.emptyMap(), configPath), classpathPlugins); } public MockNode(Environment environment, Collection> classpathPlugins) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java index 43904d1f1f9eb..2e05281356729 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java @@ -1026,7 +1026,7 @@ private static class ServiceHolder implements Closeable { ServiceHolder(Settings nodeSettings, Settings indexSettings, Collection> plugins, AbstractQueryTestCase testCase) throws IOException { - Environment env = InternalSettingsPreparer.prepareEnvironment(nodeSettings, null); + Environment env = InternalSettingsPreparer.prepareEnvironment(nodeSettings); PluginsService pluginsService; pluginsService = new PluginsService(nodeSettings, null, env.modulesFile(), env.pluginsFile(), plugins); From d1093bd2fa6ddd849cdaecc16ec3e053aaa43096 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Mon, 20 Nov 2017 08:41:02 +0100 Subject: [PATCH 24/31] #26800: Fix docs rendering --- .../bucket/composite-aggregation.asciidoc | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/docs/reference/aggregations/bucket/composite-aggregation.asciidoc b/docs/reference/aggregations/bucket/composite-aggregation.asciidoc index 8efe67dfdbbfc..2c595766b7cb6 100644 --- a/docs/reference/aggregations/bucket/composite-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/composite-aggregation.asciidoc @@ -59,23 +59,28 @@ POST /sales/docs/_bulk?refresh For instance the following document: -``` +[source,js] +-------------------------------------------------- { "keyword": ["foo", "bar"], "number": [23, 65, 76] } -``` +-------------------------------------------------- +// NOTCONSOLE + \... creates the following composite buckets when `keyword` and `number` are used as values source for the aggregation: -``` +[source,js] +-------------------------------------------------- { "keyword": "foo", "number": 23 } { "keyword": "foo", "number": 65 } { "keyword": "foo", "number": 76 } { "keyword": "bar", "number": 23 } { "keyword": "bar", "number": 65 } { "keyword": "bar", "number": 76 } -``` +-------------------------------------------------- +// NOTCONSOLE ==== Values source From 682a85b2c16e195fbd8f019492df909a8fdf9318 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= <10398885+cbuescher@users.noreply.github.com> Date: Mon, 20 Nov 2017 09:05:03 +0100 Subject: [PATCH 25/31] Delete some seemingly unused exceptions (#27439) --- .../AsynchronousComputationException.java | 29 ------------------ .../inject/internal/ComputationException.java | 27 ----------------- .../inject/internal/NullOutputException.java | 30 ------------------- 3 files changed, 86 deletions(-) delete mode 100644 core/src/main/java/org/elasticsearch/common/inject/internal/AsynchronousComputationException.java delete mode 100644 core/src/main/java/org/elasticsearch/common/inject/internal/ComputationException.java delete mode 100644 core/src/main/java/org/elasticsearch/common/inject/internal/NullOutputException.java diff --git a/core/src/main/java/org/elasticsearch/common/inject/internal/AsynchronousComputationException.java b/core/src/main/java/org/elasticsearch/common/inject/internal/AsynchronousComputationException.java deleted file mode 100644 index 2f5ee937c53fb..0000000000000 --- a/core/src/main/java/org/elasticsearch/common/inject/internal/AsynchronousComputationException.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Copyright (C) 2009 Google Inc. - * - * Licensed 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.common.inject.internal; - -/** - * Wraps an exception that occurred during a computation in a different thread. - * - * @author Bob Lee - */ -public class AsynchronousComputationException extends ComputationException { - - public AsynchronousComputationException(Throwable cause) { - super(cause); - } -} diff --git a/core/src/main/java/org/elasticsearch/common/inject/internal/ComputationException.java b/core/src/main/java/org/elasticsearch/common/inject/internal/ComputationException.java deleted file mode 100644 index 1244a2fd33d53..0000000000000 --- a/core/src/main/java/org/elasticsearch/common/inject/internal/ComputationException.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Copyright (C) 2009 Google Inc. - * - * Licensed 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.common.inject.internal; - -/** - * Wraps an exception that occurred during a computation. - */ -public class ComputationException extends RuntimeException { - - public ComputationException(Throwable cause) { - super(cause); - } -} diff --git a/core/src/main/java/org/elasticsearch/common/inject/internal/NullOutputException.java b/core/src/main/java/org/elasticsearch/common/inject/internal/NullOutputException.java deleted file mode 100644 index 0fec6b5bac281..0000000000000 --- a/core/src/main/java/org/elasticsearch/common/inject/internal/NullOutputException.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Copyright (C) 2009 Google Inc. - * - * Licensed 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.common.inject.internal; - -/** - * Thrown when a computer function returns null. This subclass exists so - * that our ReferenceCache adapter can differentiate null output from null - * keys, but we don't want to make this public otherwise. - * - * @author Bob Lee - */ -class NullOutputException extends NullPointerException { - NullOutputException(String s) { - super(s); - } -} From d3e3bc8656de6c28efc2d4416b1aeb018fd5029a Mon Sep 17 00:00:00 2001 From: Dimitrios Liappis Date: Mon, 20 Nov 2017 16:59:00 +0200 Subject: [PATCH 26/31] Tests: Add Fedora-27 to packaging tests Replace Fedora-25 with Fedora-27 and fix old references in `TESTING.asciidoc`. Relates #27434 --- TESTING.asciidoc | 12 ++++++------ Vagrantfile | 8 ++++---- .../gradle/vagrant/VagrantTestPlugin.groovy | 2 +- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/TESTING.asciidoc b/TESTING.asciidoc index 9f64d1dd0afb8..1cb05792b8298 100644 --- a/TESTING.asciidoc +++ b/TESTING.asciidoc @@ -351,8 +351,8 @@ These are the linux flavors the Vagrantfile currently supports: * debian-9 aka stretch, the current debian stable distribution * centos-6 * centos-7 -* fedora-25 * fedora-26 +* fedora-27 * oel-6 aka Oracle Enterprise Linux 6 * oel-7 aka Oracle Enterprise Linux 7 * sles-12 @@ -428,23 +428,23 @@ sudo -E bats $BATS_TESTS/*.bats You can also use Gradle to prepare the test environment and then starts a single VM: ------------------------------------------------- -gradle vagrantFedora25#up +gradle vagrantFedora27#up ------------------------------------------------- Or any of vagrantCentos6#up, vagrantCentos7#up, vagrantDebian8#up, -vagrantFedora25#up, vagrantOel6#up, vagrantOel7#up, vagrantOpensuse13#up, -vagrantSles12#up, vagrantUbuntu1404#up, vagrantUbuntu1604#up. +vagrantDebian9#up, vagrantFedora26#up, vagrantFedora27#up, vagrantOel6#up, vagrantOel7#up, +vagrantOpensuse42#up,vagrantSles12#up, vagrantUbuntu1404#up, vagrantUbuntu1604#up. Once up, you can then connect to the VM using SSH from the elasticsearch directory: ------------------------------------------------- -vagrant ssh fedora-25 +vagrant ssh fedora-27 ------------------------------------------------- Or from another directory: ------------------------------------------------- -VAGRANT_CWD=/path/to/elasticsearch vagrant ssh fedora-25 +VAGRANT_CWD=/path/to/elasticsearch vagrant ssh fedora-27 ------------------------------------------------- Note: Starting vagrant VM outside of the elasticsearch folder requires to diff --git a/Vagrantfile b/Vagrantfile index 487594bba8a1a..021b4d630a1e6 100644 --- a/Vagrantfile +++ b/Vagrantfile @@ -60,14 +60,14 @@ Vagrant.configure(2) do |config| config.vm.box = "elastic/oraclelinux-7-x86_64" rpm_common config end - config.vm.define "fedora-25" do |config| - config.vm.box = "elastic/fedora-25-x86_64" - dnf_common config - end config.vm.define "fedora-26" do |config| config.vm.box = "elastic/fedora-26-x86_64" dnf_common config end + config.vm.define "fedora-27" do |config| + config.vm.box = "elastic/fedora-27-x86_64" + dnf_common config + end config.vm.define "opensuse-42" do |config| config.vm.box = "elastic/opensuse-42-x86_64" opensuse_common config diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy index 8c94d48fcc43a..59a7359a6391b 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy @@ -19,8 +19,8 @@ class VagrantTestPlugin implements Plugin { 'centos-7', 'debian-8', 'debian-9', - 'fedora-25', 'fedora-26', + 'fedora-27', 'oel-6', 'oel-7', 'opensuse-42', From 720e96e2887dfa5f6d4c0de461da0c6861cea99f Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 20 Nov 2017 16:50:08 +0100 Subject: [PATCH 27/31] Ensure nested documents have consistent version and seq_ids (#27455) Today we index dummy values for seq_ids and version on nested documents. This is on the one hand trappy since users can request these values via inner hits and on the other hand not necessarily good for compression since the dummy value will likely not compress well when seqIDs are lowish. This change ensures that we share the same field values for all documents in a nested block. This won't have any overhead, in-fact it might be more efficient since we even reduce the work needed slightly. --- .../index/mapper/SeqNoFieldMapper.java | 15 +++-- .../index/mapper/VersionFieldMapper.java | 4 +- .../test/search.inner_hits/10_basic.yml | 57 +++++++++++++++++++ 3 files changed, 69 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java index 7d74f9e52aa4a..55fc1333d2030 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java @@ -245,15 +245,18 @@ public Mapper parse(ParseContext context) throws IOException { @Override public void postParse(ParseContext context) throws IOException { - // In the case of nested docs, let's fill nested docs with seqNo=1 and - // primaryTerm=0 so that Lucene doesn't write a Bitset for documents - // that don't have the field. This is consistent with the default value + // In the case of nested docs, let's fill nested docs with the original + // so that Lucene doesn't write a Bitset for documents that + // don't have the field. This is consistent with the default value // for efficiency. + // we share the parent docs fields to ensure good compression + SequenceIDFields seqID = context.seqID(); + assert seqID != null; for (int i = 1; i < context.docs().size(); i++) { final Document doc = context.docs().get(i); - doc.add(new LongPoint(NAME, 1)); - doc.add(new NumericDocValuesField(NAME, 1L)); - doc.add(new NumericDocValuesField(PRIMARY_TERM_NAME, 0L)); + doc.add(seqID.seqNo); + doc.add(seqID.seqNoDocValue); + doc.add(seqID.primaryTerm); } } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/VersionFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/VersionFieldMapper.java index 90ea85024c119..c5ead1327cc9b 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/VersionFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/VersionFieldMapper.java @@ -126,9 +126,11 @@ public Mapper parse(ParseContext context) throws IOException { public void postParse(ParseContext context) throws IOException { // In the case of nested docs, let's fill nested docs with version=1 so that Lucene doesn't write a Bitset for documents // that don't have the field. This is consistent with the default value for efficiency. + Field version = context.version(); + assert version != null; for (int i = 1; i < context.docs().size(); i++) { final Document doc = context.docs().get(i); - doc.add(new NumericDocValuesField(NAME, 1L)); + doc.add(version); } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.inner_hits/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.inner_hits/10_basic.yml index 524c1c593965d..e211aeeedc5fd 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.inner_hits/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.inner_hits/10_basic.yml @@ -41,3 +41,60 @@ setup: - match: { hits.hits.0.inner_hits.nested_field.hits.hits.0._nested.offset: 0 } - is_false: hits.hits.0.inner_hits.nested_field.hits.hits.0._nested.child + +--- +"Nested doc version and seqIDs": + + - skip: + # fixed in 7.0 + version: " - 6.99.99" + reason: "version and seq IDs where not accurate in previous versions" + + - do: + index: + index: test + type: type_1 + id: 1 + body: + "nested_field" : [ { "foo": "bar" } ] + - do: + indices.refresh: {} + + - do: + search: + body: { "query" : { "nested" : { "path" : "nested_field", "query" : { "match_all" : {} }, "inner_hits" : { version: true, "docvalue_fields": ["_seq_no"]} }}, "version": true, "docvalue_fields" : ["_seq_no"] } + + - match: { hits.total: 1 } + - match: { hits.hits.0._index: "test" } + - match: { hits.hits.0._type: "type_1" } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.0._version: 1 } + - match: { hits.hits.0.fields._seq_no: [0] } + - match: { hits.hits.0.inner_hits.nested_field.hits.hits.0.fields._seq_no: [0] } + + + - do: + index: + index: test + type: type_1 + id: 1 + body: + "nested_field" : [ { "foo": "baz" } ] + - do: + indices.refresh: {} + + - do: + search: + body: { "query" : { "nested" : { "path" : "nested_field", "query" : { "match_all" : {} }, "inner_hits" : { version: true, "docvalue_fields": ["_seq_no"]} }}, "version": true, "docvalue_fields" : ["_seq_no"] } + + - match: { hits.total: 1 } + - match: { hits.hits.0._index: "test" } + - match: { hits.hits.0._type: "type_1" } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.0._version: 2 } + - match: { hits.hits.0.fields._seq_no: [1] } + - match: { hits.hits.0.inner_hits.nested_field.hits.hits.0._version: 2 } + - match: { hits.hits.0.inner_hits.nested_field.hits.hits.0.fields._seq_no: [1] } + + + From 28e5cf933f82f5c4009d2bb9192aee454e0ef0bc Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 20 Nov 2017 16:52:19 +0100 Subject: [PATCH 28/31] Bump test version after backport Relates to #27455 --- .../rest-api-spec/test/search.inner_hits/10_basic.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.inner_hits/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.inner_hits/10_basic.yml index e211aeeedc5fd..4264083c13fb5 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.inner_hits/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.inner_hits/10_basic.yml @@ -46,8 +46,8 @@ setup: "Nested doc version and seqIDs": - skip: - # fixed in 7.0 - version: " - 6.99.99" + # fixed in 6.0.1 + version: " - 6.0.0" reason: "version and seq IDs where not accurate in previous versions" - do: From d02f45f694420400b55039fb61cbf81adef5502b Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 20 Nov 2017 17:16:36 +0100 Subject: [PATCH 29/31] AwaitsFix GeoShapeQueryTests#testPointsOnly #27454 --- .../java/org/elasticsearch/search/geo/GeoShapeQueryTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java b/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java index 5ca4193da46fb..ebd39530dd5cc 100644 --- a/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java +++ b/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java @@ -437,6 +437,7 @@ public void testShapeFilterWithDefinedGeoCollection() throws Exception { assertHitCount(result, 0); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/27454") public void testPointsOnly() throws Exception { String mapping = XContentFactory.jsonBuilder().startObject().startObject("type1") .startObject("properties").startObject("location") From 0a8f48d5921a7d3a1ddab6c382e23d594e42dc90 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 20 Nov 2017 10:47:47 -0700 Subject: [PATCH 30/31] Transition transport apis to use void listeners (#27440) Currently we use ActionListener for connect, close, and send message listeners in TcpTransport. However, all of the listeners have to capture a reference to a channel in the case of the exception api being called. This commit changes these listeners to be type as passing the channel to onResponse is not necessary. Additionally, this change makes it easier to integrate with low level transports (which use different implementations of TcpChannel). --- .../elasticsearch/transport/TcpChannel.java | 20 ++++++------ .../elasticsearch/transport/TcpTransport.java | 20 ++++++------ .../transport/TcpTransportTests.java | 8 ++--- .../transport/netty4/Netty4Transport.java | 4 +-- .../transport/netty4/NettyTcpChannel.java | 10 +++--- .../transport/MockTcpTransport.java | 14 ++++----- .../transport/nio/NioTransport.java | 17 ++-------- .../transport/nio/WriteOperation.java | 7 ++--- .../nio/channel/AbstractNioChannel.java | 7 ++--- .../nio/channel/NioServerSocketChannel.java | 4 +-- .../nio/channel/NioSocketChannel.java | 22 +++---------- .../nio/channel/TcpWriteContext.java | 4 +-- .../transport/nio/channel/WriteContext.java | 2 +- .../transport/nio/SocketSelectorTests.java | 2 +- .../transport/nio/WriteOperationTests.java | 2 +- .../channel/NioServerSocketChannelTests.java | 30 +++++++++++++----- .../nio/channel/NioSocketChannelTests.java | 31 +++++++++++++------ .../nio/channel/TcpWriteContextTests.java | 10 +++--- 18 files changed, 104 insertions(+), 110 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/transport/TcpChannel.java b/core/src/main/java/org/elasticsearch/transport/TcpChannel.java index ee2be4ed736c9..22453ac43b4ea 100644 --- a/core/src/main/java/org/elasticsearch/transport/TcpChannel.java +++ b/core/src/main/java/org/elasticsearch/transport/TcpChannel.java @@ -30,7 +30,6 @@ import java.io.IOException; import java.net.InetSocketAddress; -import java.net.SocketAddress; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -61,7 +60,7 @@ public interface TcpChannel extends Releasable { * * @param listener to be executed */ - void addCloseListener(ActionListener listener); + void addCloseListener(ActionListener listener); /** @@ -94,7 +93,7 @@ public interface TcpChannel extends Releasable { * @param reference to send to channel * @param listener to execute upon send completion */ - void sendMessage(BytesReference reference, ActionListener listener); + void sendMessage(BytesReference reference, ActionListener listener); /** * Closes the channel. @@ -114,10 +113,10 @@ static void closeChannel(C channel, boolean blocking) { */ static void closeChannels(List channels, boolean blocking) { if (blocking) { - ArrayList> futures = new ArrayList<>(channels.size()); + ArrayList> futures = new ArrayList<>(channels.size()); for (final C channel : channels) { if (channel.isOpen()) { - PlainActionFuture closeFuture = PlainActionFuture.newFuture(); + PlainActionFuture closeFuture = PlainActionFuture.newFuture(); channel.addCloseListener(closeFuture); channel.close(); futures.add(closeFuture); @@ -136,15 +135,14 @@ static void closeChannels(List channels, boolean block * @param discoveryNode the node for the pending connections * @param connectionFutures representing the pending connections * @param connectTimeout to wait for a connection - * @param the type of channel * @throws ConnectTransportException if one of the connections fails */ - static void awaitConnected(DiscoveryNode discoveryNode, List> connectionFutures, - TimeValue connectTimeout) throws ConnectTransportException { + static void awaitConnected(DiscoveryNode discoveryNode, List> connectionFutures, TimeValue connectTimeout) + throws ConnectTransportException { Exception connectionException = null; boolean allConnected = true; - for (ActionFuture connectionFuture : connectionFutures) { + for (ActionFuture connectionFuture : connectionFutures) { try { connectionFuture.get(connectTimeout.getMillis(), TimeUnit.MILLISECONDS); } catch (TimeoutException e) { @@ -169,8 +167,8 @@ static void awaitConnected(DiscoveryNode discoveryNode, L } } - static void blockOnFutures(List> futures) { - for (ActionFuture future : futures) { + static void blockOnFutures(List> futures) { + for (ActionFuture future : futures) { try { future.get(); } catch (ExecutionException e) { diff --git a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java index 0bf600cb02928..bfcd3dff5ab8d 100644 --- a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -343,7 +343,7 @@ protected void doRunInLifecycle() throws Exception { for (TcpChannel channel : channels.getChannels()) { internalSendMessage(channel, pingHeader, new SendMetricListener(pingHeader.length()) { @Override - protected void innerInnerOnResponse(TcpChannel channel) { + protected void innerInnerOnResponse(Void v) { successfulPings.inc(); } @@ -595,10 +595,10 @@ public final NodeChannels openConnection(DiscoveryNode node, ConnectionProfile c int numConnections = connectionProfile.getNumConnections(); assert numConnections > 0 : "A connection profile must be configured with at least one connection"; List channels = new ArrayList<>(numConnections); - List> connectionFutures = new ArrayList<>(numConnections); + List> connectionFutures = new ArrayList<>(numConnections); for (int i = 0; i < numConnections; ++i) { try { - PlainActionFuture connectFuture = PlainActionFuture.newFuture(); + PlainActionFuture connectFuture = PlainActionFuture.newFuture(); connectionFutures.add(connectFuture); TcpChannel channel = initiateChannel(node, connectionProfile.getConnectTimeout(), connectFuture); channels.add(channel); @@ -940,7 +940,7 @@ protected final void doStop() { for (Map.Entry> entry : serverChannels.entrySet()) { String profile = entry.getKey(); List channels = entry.getValue(); - ActionListener closeFailLogger = ActionListener.wrap(c -> {}, + ActionListener closeFailLogger = ActionListener.wrap(c -> {}, e -> logger.warn(() -> new ParameterizedMessage("Error closing serverChannel for profile [{}]", profile), e)); channels.forEach(c -> c.addCloseListener(closeFailLogger)); TcpChannel.closeChannels(channels, true); @@ -1016,7 +1016,7 @@ protected void onException(TcpChannel channel, Exception e) { BytesArray message = new BytesArray(e.getMessage().getBytes(StandardCharsets.UTF_8)); final SendMetricListener closeChannel = new SendMetricListener(message.length()) { @Override - protected void innerInnerOnResponse(TcpChannel channel) { + protected void innerInnerOnResponse(Void v) { TcpChannel.closeChannel(channel, false); } @@ -1060,7 +1060,7 @@ protected void serverAcceptedChannel(TcpChannel channel) { * @return the pending connection * @throws IOException if an I/O exception occurs while opening the channel */ - protected abstract TcpChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) + protected abstract TcpChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) throws IOException; /** @@ -1686,7 +1686,7 @@ protected final void ensureOpen() { /** * This listener increments the transmitted bytes metric on success. */ - private abstract class SendMetricListener extends NotifyOnceListener { + private abstract class SendMetricListener extends NotifyOnceListener { private final long messageSize; private SendMetricListener(long messageSize) { @@ -1694,12 +1694,12 @@ private SendMetricListener(long messageSize) { } @Override - protected final void innerOnResponse(org.elasticsearch.transport.TcpChannel object) { + protected final void innerOnResponse(Void object) { transmittedBytesMetric.inc(messageSize); innerInnerOnResponse(object); } - protected abstract void innerInnerOnResponse(org.elasticsearch.transport.TcpChannel object); + protected abstract void innerInnerOnResponse(Void object); } private final class SendListener extends SendMetricListener { @@ -1715,7 +1715,7 @@ private SendListener(TcpChannel channel, Releasable optionalReleasable, Runnable } @Override - protected void innerInnerOnResponse(TcpChannel channel) { + protected void innerInnerOnResponse(Void v) { release(); } diff --git a/core/src/test/java/org/elasticsearch/transport/TcpTransportTests.java b/core/src/test/java/org/elasticsearch/transport/TcpTransportTests.java index 275c6dbaeb291..5182951a0fd53 100644 --- a/core/src/test/java/org/elasticsearch/transport/TcpTransportTests.java +++ b/core/src/test/java/org/elasticsearch/transport/TcpTransportTests.java @@ -185,8 +185,8 @@ protected FakeChannel bind(String name, InetSocketAddress address) throws IOExce } @Override - protected FakeChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, - ActionListener connectListener) throws IOException { + protected FakeChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) + throws IOException { return new FakeChannel(messageCaptor); } @@ -251,7 +251,7 @@ public void close() { } @Override - public void addCloseListener(ActionListener listener) { + public void addCloseListener(ActionListener listener) { } @Override @@ -269,7 +269,7 @@ public InetSocketAddress getLocalAddress() { } @Override - public void sendMessage(BytesReference reference, ActionListener listener) { + public void sendMessage(BytesReference reference, ActionListener listener) { messageCaptor.set(reference); } } diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java index 29ff3967d6dea..6343a24184370 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java @@ -249,7 +249,7 @@ public long getNumOpenServerConnections() { } @Override - protected NettyTcpChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener listener) + protected NettyTcpChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener listener) throws IOException { ChannelFuture channelFuture = bootstrap.connect(node.getAddress().address()); Channel channel = channelFuture.channel(); @@ -264,7 +264,7 @@ protected NettyTcpChannel initiateChannel(DiscoveryNode node, TimeValue connectT channelFuture.addListener(f -> { if (f.isSuccess()) { - listener.onResponse(nettyChannel); + listener.onResponse(null); } else { Throwable cause = f.cause(); if (cause instanceof Error) { diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyTcpChannel.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyTcpChannel.java index 12ab34a32af75..fa9989f7270c4 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyTcpChannel.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyTcpChannel.java @@ -34,13 +34,13 @@ public class NettyTcpChannel implements TcpChannel { private final Channel channel; - private final CompletableFuture closeContext = new CompletableFuture<>(); + private final CompletableFuture closeContext = new CompletableFuture<>(); NettyTcpChannel(Channel channel) { this.channel = channel; this.channel.closeFuture().addListener(f -> { if (f.isSuccess()) { - closeContext.complete(this); + closeContext.complete(null); } else { Throwable cause = f.cause(); if (cause instanceof Error) { @@ -59,7 +59,7 @@ public void close() { } @Override - public void addCloseListener(ActionListener listener) { + public void addCloseListener(ActionListener listener) { closeContext.whenComplete(ActionListener.toBiConsumer(listener)); } @@ -79,11 +79,11 @@ public InetSocketAddress getLocalAddress() { } @Override - public void sendMessage(BytesReference reference, ActionListener listener) { + public void sendMessage(BytesReference reference, ActionListener listener) { final ChannelFuture future = channel.writeAndFlush(Netty4Utils.toByteBuf(reference)); future.addListener(f -> { if (f.isSuccess()) { - listener.onResponse(this); + listener.onResponse(null); } else { final Throwable cause = f.cause(); Netty4Utils.maybeDie(cause); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java index 5d5e14b406177..f6ec96d13df81 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java @@ -171,7 +171,7 @@ private void readMessage(MockChannel mockChannel, StreamInput input) throws IOEx } @Override - protected MockChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) + protected MockChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) throws IOException { InetSocketAddress address = node.getAddress().address(); final MockSocket socket = new MockSocket(); @@ -186,7 +186,7 @@ protected MockChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeo MockChannel channel = new MockChannel(socket, address, "none", (c) -> {}); channel.loopRead(executor); success = true; - connectListener.onResponse(channel); + connectListener.onResponse(null); return channel; } finally { if (success == false) { @@ -231,7 +231,7 @@ public final class MockChannel implements Closeable, TcpChannel { private final String profile; private final CancellableThreads cancellableThreads = new CancellableThreads(); private final Closeable onClose; - private final CompletableFuture closeFuture = new CompletableFuture<>(); + private final CompletableFuture closeFuture = new CompletableFuture<>(); /** * Constructs a new MockChannel instance intended for handling the actual incoming / outgoing traffic. @@ -356,14 +356,14 @@ public String toString() { public void close() { try { close0(); - closeFuture.complete(this); + closeFuture.complete(null); } catch (IOException e) { closeFuture.completeExceptionally(e); } } @Override - public void addCloseListener(ActionListener listener) { + public void addCloseListener(ActionListener listener) { closeFuture.whenComplete(ActionListener.toBiConsumer(listener)); } @@ -386,14 +386,14 @@ public InetSocketAddress getLocalAddress() { } @Override - public void sendMessage(BytesReference reference, ActionListener listener) { + public void sendMessage(BytesReference reference, ActionListener listener) { try { synchronized (this) { OutputStream outputStream = new BufferedOutputStream(activeChannel.getOutputStream()); reference.writeTo(outputStream); outputStream.flush(); } - listener.onResponse(this); + listener.onResponse(null); } catch (IOException e) { listener.onFailure(e); onException(this, e); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java index 2902b0bccfd9d..822c0181ae7e9 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java @@ -20,7 +20,6 @@ package org.elasticsearch.transport.nio; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; @@ -32,7 +31,6 @@ import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TcpChannel; import org.elasticsearch.transport.TcpTransport; import org.elasticsearch.transport.Transports; import org.elasticsearch.transport.nio.channel.ChannelFactory; @@ -95,22 +93,11 @@ protected NioServerSocketChannel bind(String name, InetSocketAddress address) th } @Override - protected NioChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) + protected NioChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeout, ActionListener connectListener) throws IOException { NioSocketChannel channel = clientChannelFactory.openNioChannel(node.getAddress().address(), clientSelectorSupplier.get()); openChannels.clientChannelOpened(channel); - // TODO: Temporary conversion due to types - channel.addConnectListener(new ActionListener() { - @Override - public void onResponse(NioChannel nioChannel) { - connectListener.onResponse(nioChannel); - } - - @Override - public void onFailure(Exception e) { - connectListener.onFailure(e); - } - }); + channel.addConnectListener(connectListener); return channel; } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java index 67ed2447f6383..f91acc5bbea3a 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java @@ -24,7 +24,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; import java.io.IOException; @@ -33,10 +32,10 @@ public class WriteOperation { private final NioSocketChannel channel; - private final ActionListener listener; + private final ActionListener listener; private final NetworkBytesReference[] references; - public WriteOperation(NioSocketChannel channel, BytesReference bytesReference, ActionListener listener) { + public WriteOperation(NioSocketChannel channel, BytesReference bytesReference, ActionListener listener) { this.channel = channel; this.listener = listener; this.references = toArray(bytesReference); @@ -46,7 +45,7 @@ public NetworkBytesReference[] getByteReferences() { return references; } - public ActionListener getListener() { + public ActionListener getListener() { return listener; } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/AbstractNioChannel.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/AbstractNioChannel.java index 21f46631c6e83..a2924eff56bcb 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/AbstractNioChannel.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/AbstractNioChannel.java @@ -20,7 +20,6 @@ package org.elasticsearch.transport.nio.channel; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.transport.TcpChannel; import org.elasticsearch.transport.nio.ESSelector; import java.io.IOException; @@ -58,7 +57,7 @@ public abstract class AbstractNioChannel closeContext = new CompletableFuture<>(); + private final CompletableFuture closeContext = new CompletableFuture<>(); private final ESSelector selector; private SelectionKey selectionKey; @@ -111,7 +110,7 @@ public void closeFromSelector() throws IOException { if (closeContext.isDone() == false) { try { closeRawChannel(); - closeContext.complete(this); + closeContext.complete(null); } catch (IOException e) { closeContext.completeExceptionally(e); throw e; @@ -156,7 +155,7 @@ void closeRawChannel() throws IOException { } @Override - public void addCloseListener(ActionListener listener) { + public void addCloseListener(ActionListener listener) { closeContext.whenComplete(ActionListener.toBiConsumer(listener)); } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannel.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannel.java index 5b365fbd36c46..643496ac39b4b 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannel.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannel.java @@ -21,12 +21,10 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.transport.TcpChannel; import org.elasticsearch.transport.nio.AcceptingSelector; import java.io.IOException; import java.nio.channels.ServerSocketChannel; -import java.util.concurrent.Future; public class NioServerSocketChannel extends AbstractNioChannel { @@ -43,7 +41,7 @@ public ChannelFactory getChannelFactory() { } @Override - public void sendMessage(BytesReference reference, ActionListener listener) { + public void sendMessage(BytesReference reference, ActionListener listener) { throw new UnsupportedOperationException("Cannot send a message to a server channel."); } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioSocketChannel.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioSocketChannel.java index 520cefd27f9d7..fb2d940348a16 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioSocketChannel.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/NioSocketChannel.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.transport.TcpChannel; import org.elasticsearch.transport.nio.NetworkBytesReference; import org.elasticsearch.transport.nio.SocketSelector; @@ -36,7 +35,7 @@ public class NioSocketChannel extends AbstractNioChannel { private final InetSocketAddress remoteAddress; - private final CompletableFuture connectContext = new CompletableFuture<>(); + private final CompletableFuture connectContext = new CompletableFuture<>(); private final SocketSelector socketSelector; private WriteContext writeContext; private ReadContext readContext; @@ -49,19 +48,8 @@ public NioSocketChannel(String profile, SocketChannel socketChannel, SocketSelec } @Override - public void sendMessage(BytesReference reference, ActionListener listener) { - // TODO: Temporary conversion due to types - writeContext.sendMessage(reference, new ActionListener() { - @Override - public void onResponse(NioChannel nioChannel) { - listener.onResponse(nioChannel); - } - - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - }); + public void sendMessage(BytesReference reference, ActionListener listener) { + writeContext.sendMessage(reference, listener); } @Override @@ -169,12 +157,12 @@ public boolean finishConnect() throws IOException { isConnected = internalFinish(); } if (isConnected) { - connectContext.complete(this); + connectContext.complete(null); } return isConnected; } - public void addConnectListener(ActionListener listener) { + public void addConnectListener(ActionListener listener) { connectContext.whenComplete(ActionListener.toBiConsumer(listener)); } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java index 03eb652e1aeee..d38cd1320d194 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java @@ -38,7 +38,7 @@ public TcpWriteContext(NioSocketChannel channel) { } @Override - public void sendMessage(BytesReference reference, ActionListener listener) { + public void sendMessage(BytesReference reference, ActionListener listener) { if (channel.isWritable() == false) { listener.onFailure(new ClosedChannelException()); return; @@ -96,7 +96,7 @@ private void singleFlush(WriteOperation headOp) throws IOException { } if (headOp.isFullyFlushed()) { - headOp.getListener().onResponse(channel); + headOp.getListener().onResponse(null); } else { queued.push(headOp); } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java index 1a14d279dd2fa..718b7daf8c628 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java @@ -27,7 +27,7 @@ public interface WriteContext { - void sendMessage(BytesReference reference, ActionListener listener); + void sendMessage(BytesReference reference, ActionListener listener); void queueWriteOperations(WriteOperation writeOperation); diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketSelectorTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketSelectorTests.java index 670134d9beeab..0de1bb72063ba 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketSelectorTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketSelectorTests.java @@ -53,7 +53,7 @@ public class SocketSelectorTests extends ESTestCase { private NioSocketChannel channel; private TestSelectionKey selectionKey; private WriteContext writeContext; - private ActionListener listener; + private ActionListener listener; private NetworkBytesReference bufferReference = NetworkBytesReference.wrap(new BytesArray(new byte[1])); private Selector rawSelector; diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/WriteOperationTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/WriteOperationTests.java index d7284491d6421..1f6f95e62af3e 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/WriteOperationTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/WriteOperationTests.java @@ -36,7 +36,7 @@ public class WriteOperationTests extends ESTestCase { private NioSocketChannel channel; - private ActionListener listener; + private ActionListener listener; @Before @SuppressWarnings("unchecked") diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannelTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannelTests.java index 62f87d4f57473..93d905c806810 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannelTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannelTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.transport.nio.channel; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.TcpChannel; import org.elasticsearch.transport.nio.AcceptingSelector; @@ -61,26 +62,39 @@ public void stopSelector() throws IOException, InterruptedException { } public void testClose() throws Exception { - AtomicReference ref = new AtomicReference<>(); + AtomicBoolean isClosed = new AtomicBoolean(false); CountDownLatch latch = new CountDownLatch(1); NioChannel channel = new DoNotCloseServerChannel("nio", mock(ServerSocketChannel.class), mock(ChannelFactory.class), selector); - Consumer listener = (c) -> { - ref.set(c); - latch.countDown(); - }; - channel.addCloseListener(ActionListener.wrap(listener::accept, (e) -> listener.accept(channel))); + + channel.addCloseListener(new ActionListener() { + @Override + public void onResponse(Void o) { + isClosed.set(true); + latch.countDown(); + } + + @Override + public void onFailure(Exception e) { + isClosed.set(true); + latch.countDown(); + } + }); assertTrue(channel.isOpen()); assertFalse(closedRawChannel.get()); + assertFalse(isClosed.get()); - TcpChannel.closeChannel(channel, true); + PlainActionFuture closeFuture = PlainActionFuture.newFuture(); + channel.addCloseListener(closeFuture); + channel.close(); + closeFuture.actionGet(); assertTrue(closedRawChannel.get()); assertFalse(channel.isOpen()); latch.await(); - assertSame(channel, ref.get()); + assertTrue(isClosed.get()); } private class DoNotCloseServerChannel extends DoNotRegisterServerChannel { diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioSocketChannelTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioSocketChannelTests.java index d8d4b41df7038..1fb32d0f6e164 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioSocketChannelTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioSocketChannelTests.java @@ -68,29 +68,40 @@ public void stopSelector() throws IOException, InterruptedException { } public void testClose() throws Exception { - AtomicReference ref = new AtomicReference<>(); + AtomicBoolean isClosed = new AtomicBoolean(false); CountDownLatch latch = new CountDownLatch(1); NioSocketChannel socketChannel = new DoNotCloseChannel(NioChannel.CLIENT, mock(SocketChannel.class), selector); openChannels.clientChannelOpened(socketChannel); socketChannel.setContexts(mock(ReadContext.class), mock(WriteContext.class)); - Consumer listener = (c) -> { - ref.set(c); - latch.countDown(); - }; - socketChannel.addCloseListener(ActionListener.wrap(listener::accept, (e) -> listener.accept(socketChannel))); + socketChannel.addCloseListener(new ActionListener() { + @Override + public void onResponse(Void o) { + isClosed.set(true); + latch.countDown(); + } + @Override + public void onFailure(Exception e) { + isClosed.set(true); + latch.countDown(); + } + }); assertTrue(socketChannel.isOpen()); assertFalse(closedRawChannel.get()); + assertFalse(isClosed.get()); assertTrue(openChannels.getClientChannels().containsKey(socketChannel)); - TcpChannel.closeChannel(socketChannel, true); + PlainActionFuture closeFuture = PlainActionFuture.newFuture(); + socketChannel.addCloseListener(closeFuture); + socketChannel.close(); + closeFuture.actionGet(); assertTrue(closedRawChannel.get()); assertFalse(socketChannel.isOpen()); assertFalse(openChannels.getClientChannels().containsKey(socketChannel)); latch.await(); - assertSame(socketChannel, ref.get()); + assertTrue(isClosed.get()); } public void testConnectSucceeds() throws Exception { @@ -100,7 +111,7 @@ public void testConnectSucceeds() throws Exception { socketChannel.setContexts(mock(ReadContext.class), mock(WriteContext.class)); selector.scheduleForRegistration(socketChannel); - PlainActionFuture connectFuture = PlainActionFuture.newFuture(); + PlainActionFuture connectFuture = PlainActionFuture.newFuture(); socketChannel.addConnectListener(connectFuture); connectFuture.get(100, TimeUnit.SECONDS); @@ -116,7 +127,7 @@ public void testConnectFails() throws Exception { socketChannel.setContexts(mock(ReadContext.class), mock(WriteContext.class)); selector.scheduleForRegistration(socketChannel); - PlainActionFuture connectFuture = PlainActionFuture.newFuture(); + PlainActionFuture connectFuture = PlainActionFuture.newFuture(); socketChannel.addConnectListener(connectFuture); ExecutionException e = expectThrows(ExecutionException.class, () -> connectFuture.get(100, TimeUnit.SECONDS)); assertTrue(e.getCause() instanceof IOException); diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpWriteContextTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpWriteContextTests.java index b4fab855403b9..7e6410b6c6164 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpWriteContextTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpWriteContextTests.java @@ -40,7 +40,7 @@ public class TcpWriteContextTests extends ESTestCase { private SocketSelector selector; - private ActionListener listener; + private ActionListener listener; private TcpWriteContext writeContext; private NioSocketChannel channel; @@ -136,7 +136,7 @@ public void testQueuedWriteIsFlushedInFlushCall() throws Exception { writeContext.flushChannel(); verify(writeOperation).flush(); - verify(listener).onResponse(channel); + verify(listener).onResponse(null); assertFalse(writeContext.hasQueuedWriteOps()); } @@ -151,7 +151,7 @@ public void testPartialFlush() throws IOException { when(writeOperation.isFullyFlushed()).thenReturn(false); writeContext.flushChannel(); - verify(listener, times(0)).onResponse(channel); + verify(listener, times(0)).onResponse(null); assertTrue(writeContext.hasQueuedWriteOps()); } @@ -173,7 +173,7 @@ public void testMultipleWritesPartialFlushes() throws IOException { when(writeOperation2.isFullyFlushed()).thenReturn(false); writeContext.flushChannel(); - verify(listener).onResponse(channel); + verify(listener).onResponse(null); verify(listener2, times(0)).onResponse(channel); assertTrue(writeContext.hasQueuedWriteOps()); @@ -181,7 +181,7 @@ public void testMultipleWritesPartialFlushes() throws IOException { writeContext.flushChannel(); - verify(listener2).onResponse(channel); + verify(listener2).onResponse(null); assertFalse(writeContext.hasQueuedWriteOps()); } From 093218e052acc84338e90e1ab77ed413843a6244 Mon Sep 17 00:00:00 2001 From: Nicholas Knize Date: Mon, 20 Nov 2017 12:09:30 -0600 Subject: [PATCH 31/31] [TEST] Fix `GeoShapeQueryTests#testPointsOnly` failure Changes unnecessary geoIntersection query to a matchAll query. closes #27454 --- .../java/org/elasticsearch/search/geo/GeoShapeQueryTests.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java b/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java index ebd39530dd5cc..4b88a5d381c1e 100644 --- a/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java +++ b/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java @@ -437,7 +437,6 @@ public void testShapeFilterWithDefinedGeoCollection() throws Exception { assertHitCount(result, 0); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/27454") public void testPointsOnly() throws Exception { String mapping = XContentFactory.jsonBuilder().startObject().startObject("type1") .startObject("properties").startObject("location") @@ -465,7 +464,7 @@ public void testPointsOnly() throws Exception { // test that point was inserted SearchResponse response = client().prepareSearch("geo_points_only").setTypes("type1") - .setQuery(geoIntersectionQuery("location", shape)) + .setQuery(matchAllQuery()) .execute().actionGet(); assertEquals(1, response.getHits().getTotalHits());