Merge branch 'master' into enhancement/use_shard_bulk_for_single_ops

This commit is contained in:
Areek Zillur 2016-12-21 16:33:30 -05:00
commit 5b2393d8b4
121 changed files with 2061 additions and 1143 deletions

View File

@ -590,9 +590,6 @@
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]termvectors[/\\]TermVectorsUnitTests.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]aliases[/\\]IndexAliasesIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]bootstrap[/\\]JarHellTests.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]bwcompat[/\\]BasicAnalysisBackwardCompatibilityIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]bwcompat[/\\]BasicBackwardsCompatibilityIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]bwcompat[/\\]GetIndexBackwardsCompatibilityIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]bwcompat[/\\]RestoreBackwardsCompatIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]client[/\\]AbstractClientHeadersTestCase.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]cluster[/\\]ClusterHealthIT.java" checks="LineLength" />
@ -704,7 +701,6 @@
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]gateway[/\\]PriorityComparatorTests.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]gateway[/\\]QuorumGatewayIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]gateway[/\\]RecoverAfterNodesIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]gateway[/\\]RecoveryBackwardsCompatibilityIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]gateway[/\\]RecoveryFromGatewayIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]gateway[/\\]ReplicaShardAllocatorTests.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]gateway[/\\]ReusePeerRecoverySharedTest.java" checks="LineLength" />
@ -923,7 +919,6 @@
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]snapshots[/\\]DedicatedClusterSnapshotRestoreIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]snapshots[/\\]RepositoriesIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]snapshots[/\\]SharedClusterSnapshotRestoreIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]snapshots[/\\]SnapshotBackwardsCompatibilityIT.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]snapshots[/\\]SnapshotUtilsTests.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]ESBlobStoreRepositoryIntegTestCase.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]geo[/\\]RandomShapeGenerator.java" checks="LineLength" />
@ -993,13 +988,8 @@
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]common[/\\]util[/\\]MockBigArrays.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]bucket[/\\]script[/\\]NativeSignificanceScoreScriptWithParams.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]BackgroundIndexer.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]CompositeTestCluster.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]CorruptionUtils.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]ESBackcompatTestCase.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]ESIntegTestCase.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]ESSingleNodeTestCase.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]ESTestCase.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]ExternalNode.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]IndexSettingsModule.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]InternalTestCluster.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]test[/\\]MockIndexEventListener.java" checks="LineLength" />

View File

@ -22,6 +22,7 @@ package org.elasticsearch.transport.client;
import io.netty.util.ThreadDeathWatcher;
import io.netty.util.concurrent.GlobalEventExecutor;
import org.elasticsearch.client.transport.TransportClient;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.network.NetworkModule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.reindex.ReindexPlugin;
@ -47,6 +48,32 @@ import java.util.concurrent.TimeUnit;
@SuppressWarnings({"unchecked","varargs"})
public class PreBuiltTransportClient extends TransportClient {
static {
// initialize Netty system properties before triggering any Netty class loads
initializeNetty();
}
/**
* Netty wants to do some unsafe things like use unsafe and replace a private field. This method disables these things by default, but
* can be overridden by setting the corresponding system properties.
*/
@SuppressForbidden(reason = "set system properties to configure Netty")
private static void initializeNetty() {
final String noUnsafeKey = "io.netty.noUnsafe";
final String noUnsafe = System.getProperty(noUnsafeKey);
if (noUnsafe == null) {
// disable Netty from using unsafe
System.setProperty(noUnsafeKey, Boolean.toString(true));
}
final String noKeySetOptimizationKey = "io.netty.noKeySetOptimization";
final String noKeySetOptimization = System.getProperty(noKeySetOptimizationKey);
if (noKeySetOptimization == null) {
// disable Netty from replacing the selector key set
System.setProperty(noKeySetOptimizationKey, Boolean.toString(true));
}
}
private static final Collection<Class<? extends Plugin>> PRE_INSTALLED_PLUGINS =
Collections.unmodifiableList(
Arrays.asList(

View File

@ -41,7 +41,7 @@ public class PreBuiltTransportClientTests extends RandomizedTest {
@Test
public void testPluginInstalled() {
// TODO: remove when Netty 4.1.5 is upgraded to Netty 4.1.6 including https://github.com/netty/netty/pull/5778
// TODO: remove when Netty 4.1.6 is upgraded to Netty 4.1.7 including https://github.com/netty/netty/pull/6068
assumeFalse(Constants.JRE_IS_MINIMUM_JAVA9);
try (TransportClient client = new PreBuiltTransportClient(Settings.EMPTY)) {
Settings settings = client.settings();

View File

@ -101,6 +101,21 @@ final class TransportClientNodesService extends AbstractComponent implements Clo
private final TransportClient.HostFailureListener hostFailureListener;
// TODO: migrate this to use low level connections and single type channels
/** {@link ConnectionProfile} to use when to connecting to the listed nodes and doing a liveness check */
private static final ConnectionProfile LISTED_NODES_PROFILE;
static {
ConnectionProfile.Builder builder = new ConnectionProfile.Builder();
builder.addConnections(1,
TransportRequestOptions.Type.BULK,
TransportRequestOptions.Type.PING,
TransportRequestOptions.Type.RECOVERY,
TransportRequestOptions.Type.REG,
TransportRequestOptions.Type.STATE);
LISTED_NODES_PROFILE = builder.build();
}
TransportClientNodesService(Settings settings, TransportService transportService,
ThreadPool threadPool, TransportClient.HostFailureListener hostFailureListener) {
super(settings);
@ -389,8 +404,8 @@ final class TransportClientNodesService extends AbstractComponent implements Clo
if (!transportService.nodeConnected(listedNode)) {
try {
// its a listed node, light connect to it...
logger.trace("connecting to listed node (light) [{}]", listedNode);
transportService.connectToNode(listedNode, ConnectionProfile.LIGHT_PROFILE);
logger.trace("connecting to listed node [{}]", listedNode);
transportService.connectToNode(listedNode, LISTED_NODES_PROFILE);
} catch (Exception e) {
logger.info(
(Supplier<?>)
@ -470,7 +485,7 @@ final class TransportClientNodesService extends AbstractComponent implements Clo
} else {
// its a listed node, light connect to it...
logger.trace("connecting to listed node (light) [{}]", listedNode);
transportService.connectToNode(listedNode, ConnectionProfile.LIGHT_PROFILE);
transportService.connectToNode(listedNode, LISTED_NODES_PROFILE);
}
} catch (Exception e) {
logger.debug(

View File

@ -22,6 +22,7 @@ package org.elasticsearch.common.xcontent;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import java.io.IOException;
import java.util.Locale;
import java.util.function.Supplier;
@ -34,6 +35,19 @@ public final class XContentParserUtils {
private XContentParserUtils() {
}
/**
* Makes sure that current token is of type {@link XContentParser.Token#FIELD_NAME} and the the field name is equal to the provided one
* @throws ParsingException if the token is not of type {@link XContentParser.Token#FIELD_NAME} or is not equal to the given field name
*/
public static void ensureFieldName(XContentParser parser, Token token, String fieldName) throws IOException {
ensureExpectedToken(Token.FIELD_NAME, token, parser::getTokenLocation);
String currentName = parser.currentName();
if (currentName.equals(fieldName) == false) {
String message = "Failed to parse object: expecting field with name [%s] but found [%s]";
throw new ParsingException(parser.getTokenLocation(), String.format(Locale.ROOT, message, fieldName, currentName));
}
}
/**
* @throws ParsingException with a "unknown field found" reason
*/

View File

@ -24,7 +24,6 @@ import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
@ -174,7 +173,7 @@ public class ElectMasterService extends AbstractComponent {
* Returns the given nodes sorted by likelihood of being elected as master, most likely first.
* Non-master nodes are not removed but are rather put in the end
*/
public static List<DiscoveryNode> sortByMasterLikelihood(Iterable<DiscoveryNode> nodes) {
static List<DiscoveryNode> sortByMasterLikelihood(Iterable<DiscoveryNode> nodes) {
ArrayList<DiscoveryNode> sortedNodes = CollectionUtils.iterableAsArrayList(nodes);
CollectionUtil.introSort(sortedNodes, ElectMasterService::compareNodes);
return sortedNodes;

View File

@ -23,13 +23,12 @@ import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.ElasticsearchException;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -44,10 +43,14 @@ import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor;
import org.elasticsearch.common.util.concurrent.KeyedLock;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.ConnectionProfile;
import org.elasticsearch.transport.NodeNotConnectedException;
import org.elasticsearch.transport.RemoteTransportException;
import org.elasticsearch.transport.Transport.Connection;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequest;
@ -60,8 +63,8 @@ import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
@ -70,18 +73,17 @@ import java.util.Objects;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptyMap;
@ -116,22 +118,19 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
private volatile PingContextProvider contextProvider;
private final AtomicInteger pingHandlerIdGenerator = new AtomicInteger();
private final AtomicInteger pingingRoundIdGenerator = new AtomicInteger();
// used to generate unique ids for nodes/address we temporarily connect to
private final AtomicInteger unicastNodeIdGenerator = new AtomicInteger();
// used as a node id prefix for nodes/address we temporarily connect to
// used as a node id prefix for configured unicast host nodes/address
private static final String UNICAST_NODE_PREFIX = "#zen_unicast_";
private final Map<Integer, SendPingsHandler> receivedResponses = newConcurrentMap();
private final Map<Integer, PingingRound> activePingingRounds = newConcurrentMap();
// a list of temporal responses a node will return for a request (holds responses from other nodes)
private final Queue<PingResponse> temporalResponses = ConcurrentCollections.newQueue();
private final UnicastHostsProvider hostsProvider;
private final ExecutorService unicastZenPingExecutorService;
protected final EsThreadPoolExecutor unicastZenPingExecutorService;
private final TimeValue resolveTimeout;
@ -146,15 +145,14 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
this.hostsProvider = unicastHostsProvider;
this.concurrentConnects = DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.get(settings);
final List<String> hosts = DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.get(settings);
if (hosts.isEmpty()) {
if (DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.exists(settings)) {
configuredHosts = DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.get(settings);
// we only limit to 1 addresses, makes no sense to ping 100 ports
limitPortCounts = LIMIT_FOREIGN_PORTS_COUNT;
} else {
// if unicast hosts are not specified, fill with simple defaults on the local machine
configuredHosts = transportService.getLocalAddresses();
limitPortCounts = LIMIT_LOCAL_PORTS_COUNT;
} else {
configuredHosts = hosts;
// we only limit to 1 addresses, makes no sense to ping 100 ports
limitPortCounts = LIMIT_FOREIGN_PORTS_COUNT;
}
resolveTimeout = DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT.get(settings);
logger.debug(
@ -164,7 +162,7 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
resolveTimeout);
transportService.registerRequestHandler(ACTION_NAME, UnicastPingRequest::new, ThreadPool.Names.SAME,
new UnicastPingRequestHandler());
new UnicastPingRequestHandler());
final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(settings, "[unicast_connect]");
unicastZenPingExecutorService = EsExecutors.newScaling(
@ -186,23 +184,23 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
* @param hosts the hosts to resolve
* @param limitPortCounts the number of ports to resolve (should be 1 for non-local transport)
* @param transportService the transport service
* @param idGenerator the generator to supply unique ids for each discovery node
* @param nodeId_prefix a prefix to use for node ids
* @param resolveTimeout the timeout before returning from hostname lookups
* @return a list of discovery nodes with resolved transport addresses
*/
public static List<DiscoveryNode> resolveDiscoveryNodes(
public static List<DiscoveryNode> resolveHostsLists(
final ExecutorService executorService,
final Logger logger,
final List<String> hosts,
final int limitPortCounts,
final TransportService transportService,
final Supplier<String> idGenerator,
final String nodeId_prefix,
final TimeValue resolveTimeout) throws InterruptedException {
Objects.requireNonNull(executorService);
Objects.requireNonNull(logger);
Objects.requireNonNull(hosts);
Objects.requireNonNull(transportService);
Objects.requireNonNull(idGenerator);
Objects.requireNonNull(nodeId_prefix);
Objects.requireNonNull(resolveTimeout);
if (resolveTimeout.nanos() < 0) {
throw new IllegalArgumentException("resolve timeout must be non-negative but was [" + resolveTimeout + "]");
@ -211,7 +209,7 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
final List<Callable<TransportAddress[]>> callables =
hosts
.stream()
.map(hn -> (Callable<TransportAddress[]>)() -> transportService.addressesFromString(hn, limitPortCounts))
.map(hn -> (Callable<TransportAddress[]>) () -> transportService.addressesFromString(hn, limitPortCounts))
.collect(Collectors.toList());
final List<Future<TransportAddress[]>> futures =
executorService.invokeAll(callables, resolveTimeout.nanos(), TimeUnit.NANOSECONDS);
@ -226,11 +224,11 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
try {
final TransportAddress[] addresses = future.get();
logger.trace("resolved host [{}] to {}", hostname, addresses);
for (final TransportAddress address : addresses) {
for (int addressId = 0; addressId < addresses.length; addressId++) {
discoveryNodes.add(
new DiscoveryNode(
idGenerator.get(),
address,
nodeId_prefix + hostname + "_" + addressId + "#",
addresses[addressId],
emptyMap(),
emptySet(),
Version.CURRENT.minimumCompatibilityVersion()));
@ -249,8 +247,8 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
@Override
public void close() {
ThreadPool.terminate(unicastZenPingExecutorService, 0, TimeUnit.SECONDS);
Releasables.close(receivedResponses.values());
ThreadPool.terminate(unicastZenPingExecutorService, 10, TimeUnit.SECONDS);
Releasables.close(activePingingRounds.values());
closed = true;
}
@ -266,106 +264,106 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
temporalResponses.clear();
}
// test only
Collection<PingResponse> pingAndWait(TimeValue duration) {
final AtomicReference<Collection<PingResponse>> response = new AtomicReference<>();
final CountDownLatch latch = new CountDownLatch(1);
ping(pings -> {
response.set(pings);
latch.countDown();
}, duration);
try {
latch.await();
return response.get();
} catch (InterruptedException e) {
return null;
}
}
/**
* Sends three rounds of pings notifying the specified {@link PingListener} when pinging is complete. Pings are sent after resolving
* Sends three rounds of pings notifying the specified {@link Consumer} when pinging is complete. Pings are sent after resolving
* configured unicast hosts to their IP address (subject to DNS caching within the JVM). A batch of pings is sent, then another batch
* of pings is sent at half the specified {@link TimeValue}, and then another batch of pings is sent at the specified {@link TimeValue}.
* The pings that are sent carry a timeout of 1.25 times the specified {@link TimeValue}. When pinging each node, a connection and
* handshake is performed, with a connection timeout of the specified {@link TimeValue}.
*
* @param listener the callback when pinging is complete
* @param duration the timeout for various components of the pings
* @param resultsConsumer the callback when pinging is complete
* @param duration the timeout for various components of the pings
*/
@Override
public void ping(final PingListener listener, final TimeValue duration) {
final List<DiscoveryNode> resolvedDiscoveryNodes;
public void ping(final Consumer<PingCollection> resultsConsumer, final TimeValue duration) {
ping(resultsConsumer, duration, duration);
}
/**
* a variant of {@link #ping(Consumer, TimeValue)}, but allows separating the scheduling duration
* from the duration used for request level time outs. This is useful for testing
*/
protected void ping(final Consumer<PingCollection> resultsConsumer,
final TimeValue scheduleDuration,
final TimeValue requestDuration) {
final List<DiscoveryNode> seedNodes;
try {
resolvedDiscoveryNodes = resolveDiscoveryNodes(
seedNodes = resolveHostsLists(
unicastZenPingExecutorService,
logger,
configuredHosts,
limitPortCounts,
transportService,
() -> UNICAST_NODE_PREFIX + unicastNodeIdGenerator.incrementAndGet() + "#",
UNICAST_NODE_PREFIX,
resolveTimeout);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
final SendPingsHandler sendPingsHandler = new SendPingsHandler(pingHandlerIdGenerator.incrementAndGet());
try {
receivedResponses.put(sendPingsHandler.id(), sendPingsHandler);
try {
sendPings(duration, null, sendPingsHandler, resolvedDiscoveryNodes);
} catch (RejectedExecutionException e) {
logger.debug("Ping execution rejected", e);
// The RejectedExecutionException can come from the fact unicastZenPingExecutorService is at its max down in sendPings
// But don't bail here, we can retry later on after the send ping has been scheduled.
seedNodes.addAll(hostsProvider.buildDynamicNodes());
final DiscoveryNodes nodes = contextProvider.nodes();
// add all possible master nodes that were active in the last known cluster configuration
for (ObjectCursor<DiscoveryNode> masterNode : nodes.getMasterNodes().values()) {
seedNodes.add(masterNode.value);
}
final ConnectionProfile connectionProfile =
ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG, requestDuration, requestDuration);
final PingingRound pingingRound = new PingingRound(pingingRoundIdGenerator.incrementAndGet(), seedNodes, resultsConsumer,
nodes.getLocalNode(), connectionProfile);
activePingingRounds.put(pingingRound.id(), pingingRound);
final AbstractRunnable pingSender = new AbstractRunnable() {
@Override
public void onFailure(Exception e) {
if (e instanceof AlreadyClosedException == false) {
logger.warn("unexpected error while pinging", e);
}
}
threadPool.schedule(TimeValue.timeValueMillis(duration.millis() / 2), ThreadPool.Names.GENERIC, new AbstractRunnable() {
@Override
protected void doRun() {
sendPings(duration, null, sendPingsHandler, resolvedDiscoveryNodes);
threadPool.schedule(TimeValue.timeValueMillis(duration.millis() / 2), ThreadPool.Names.GENERIC, new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
sendPings(duration, TimeValue.timeValueMillis(duration.millis() / 2), sendPingsHandler, resolvedDiscoveryNodes);
sendPingsHandler.close();
listener.onPing(sendPingsHandler.pingCollection().toList());
for (DiscoveryNode node : sendPingsHandler.nodeToDisconnect) {
logger.trace("[{}] disconnecting from {}", sendPingsHandler.id(), node);
transportService.disconnectFromNode(node);
}
}
@Override
protected void doRun() throws Exception {
sendPings(requestDuration, pingingRound);
}
};
threadPool.generic().execute(pingSender);
threadPool.schedule(TimeValue.timeValueMillis(scheduleDuration.millis() / 3), ThreadPool.Names.GENERIC, pingSender);
threadPool.schedule(TimeValue.timeValueMillis(scheduleDuration.millis() / 3 * 2), ThreadPool.Names.GENERIC, pingSender);
threadPool.schedule(scheduleDuration, ThreadPool.Names.GENERIC, new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
finishPingingRound(pingingRound);
}
@Override
public void onFailure(Exception e) {
logger.debug("Ping execution failed", e);
sendPingsHandler.close();
}
});
}
@Override
public void onFailure(Exception e) {
logger.debug("Ping execution failed", e);
sendPingsHandler.close();
}
});
} catch (EsRejectedExecutionException ex) { // TODO: remove this once ScheduledExecutor has support for AbstractRunnable
sendPingsHandler.close();
// we are shutting down
} catch (Exception e) {
sendPingsHandler.close();
throw new ElasticsearchException("Ping execution failed", e);
}
@Override
public void onFailure(Exception e) {
logger.warn("unexpected error while finishing pinging round", e);
}
});
}
class SendPingsHandler implements Releasable {
// for testing
protected void finishPingingRound(PingingRound pingingRound) {
pingingRound.close();
}
protected class PingingRound implements Releasable {
private final int id;
private final Set<DiscoveryNode> nodeToDisconnect = ConcurrentCollections.newConcurrentSet();
private final Map<TransportAddress, Connection> tempConnections = new HashMap<>();
private final KeyedLock<TransportAddress> connectionLock = new KeyedLock<>(true);
private final PingCollection pingCollection;
private final List<DiscoveryNode> seedNodes;
private final Consumer<PingCollection> pingListener;
private final DiscoveryNode localNode;
private final ConnectionProfile connectionProfile;
private AtomicBoolean closed = new AtomicBoolean(false);
SendPingsHandler(int id) {
PingingRound(int id, List<DiscoveryNode> seedNodes, Consumer<PingCollection> resultsConsumer, DiscoveryNode localNode,
ConnectionProfile connectionProfile) {
this.id = id;
this.seedNodes = Collections.unmodifiableList(new ArrayList<>(seedNodes));
this.pingListener = resultsConsumer;
this.localNode = localNode;
this.connectionProfile = connectionProfile;
this.pingCollection = new PingCollection();
}
@ -377,154 +375,170 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
return this.closed.get();
}
public PingCollection pingCollection() {
return pingCollection;
public List<DiscoveryNode> getSeedNodes() {
ensureOpen();
return seedNodes;
}
public Connection getOrConnect(DiscoveryNode node) throws IOException {
Connection result;
try (Releasable ignore = connectionLock.acquire(node.getAddress())) {
result = tempConnections.get(node.getAddress());
if (result == null) {
boolean success = false;
result = transportService.openConnection(node, connectionProfile);
try {
transportService.handshake(result, connectionProfile.getHandshakeTimeout().millis());
synchronized (this) {
// acquire lock to prevent concurrent closing
Connection existing = tempConnections.put(node.getAddress(), result);
assert existing == null;
success = true;
}
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(result);
}
}
}
}
return result;
}
private void ensureOpen() {
if (isClosed()) {
throw new AlreadyClosedException("pinging round [" + id + "] is finished");
}
}
public void addPingResponseToCollection(PingResponse pingResponse) {
if (localNode.equals(pingResponse.node()) == false) {
pingCollection.addPing(pingResponse);
}
}
@Override
public void close() {
if (closed.compareAndSet(false, true)) {
receivedResponses.remove(id);
List<Connection> toClose = null;
synchronized (this) {
if (closed.compareAndSet(false, true)) {
activePingingRounds.remove(id);
toClose = new ArrayList<>(tempConnections.values());
tempConnections.clear();
}
}
if (toClose != null) {
// we actually closed
try {
pingListener.accept(pingCollection);
} finally {
IOUtils.closeWhileHandlingException(toClose);
}
}
}
public ConnectionProfile getConnectionProfile() {
return connectionProfile;
}
}
void sendPings(
final TimeValue timeout,
@Nullable TimeValue waitTime,
final SendPingsHandler sendPingsHandler,
final List<DiscoveryNode> resolvedDiscoveryNodes) {
protected void sendPings(final TimeValue timeout, final PingingRound pingingRound) {
final UnicastPingRequest pingRequest = new UnicastPingRequest();
pingRequest.id = sendPingsHandler.id();
pingRequest.id = pingingRound.id();
pingRequest.timeout = timeout;
DiscoveryNodes discoNodes = contextProvider.nodes();
pingRequest.pingResponse = createPingResponse(discoNodes);
HashSet<DiscoveryNode> nodesToPingSet = new HashSet<>();
for (PingResponse temporalResponse : temporalResponses) {
// Only send pings to nodes that have the same cluster name.
if (clusterName.equals(temporalResponse.clusterName())) {
nodesToPingSet.add(temporalResponse.node());
}
}
nodesToPingSet.addAll(hostsProvider.buildDynamicNodes());
Set<DiscoveryNode> nodesFromResponses = temporalResponses.stream().map(pingResponse -> {
assert clusterName.equals(pingResponse.clusterName()) :
"got a ping request from a different cluster. expected " + clusterName + " got " + pingResponse.clusterName();
return pingResponse.node();
}).collect(Collectors.toSet());
// add all possible master nodes that were active in the last known cluster configuration
for (ObjectCursor<DiscoveryNode> masterNode : discoNodes.getMasterNodes().values()) {
nodesToPingSet.add(masterNode.value);
}
// dedup by address
final Map<TransportAddress, DiscoveryNode> uniqueNodesByAddress =
Stream.concat(pingingRound.getSeedNodes().stream(), nodesFromResponses.stream())
.collect(Collectors.toMap(DiscoveryNode::getAddress, Function.identity(), (n1, n2) -> n1));
// sort the nodes by likelihood of being an active master
List<DiscoveryNode> sortedNodesToPing = ElectMasterService.sortByMasterLikelihood(nodesToPingSet);
// add the configured hosts first
final List<DiscoveryNode> nodesToPing = new ArrayList<>(resolvedDiscoveryNodes.size() + sortedNodesToPing.size());
nodesToPing.addAll(resolvedDiscoveryNodes);
nodesToPing.addAll(sortedNodesToPing);
final CountDownLatch latch = new CountDownLatch(nodesToPing.size());
for (final DiscoveryNode node : nodesToPing) {
// make sure we are connected
final boolean nodeFoundByAddress;
DiscoveryNode nodeToSend = discoNodes.findByAddress(node.getAddress());
if (nodeToSend != null) {
nodeFoundByAddress = true;
} else {
nodeToSend = node;
nodeFoundByAddress = false;
}
if (!transportService.nodeConnected(nodeToSend)) {
if (sendPingsHandler.isClosed()) {
return;
// resolve what we can via the latest cluster state
final Set<DiscoveryNode> nodesToPing = uniqueNodesByAddress.values().stream()
.map(node -> {
DiscoveryNode foundNode = discoNodes.findByAddress(node.getAddress());
if (foundNode == null) {
return node;
} else {
return foundNode;
}
// if we find on the disco nodes a matching node by address, we are going to restore the connection
// anyhow down the line if its not connected...
// if we can't resolve the node, we don't know and we have to clean up after pinging. We do have
// to make sure we don't disconnect a true node which was temporarily removed from the DiscoveryNodes
// but will be added again during the pinging. We therefore create a new temporary node
if (!nodeFoundByAddress) {
if (!nodeToSend.getId().startsWith(UNICAST_NODE_PREFIX)) {
DiscoveryNode tempNode = new DiscoveryNode("",
UNICAST_NODE_PREFIX + unicastNodeIdGenerator.incrementAndGet() + "_" + nodeToSend.getId() + "#",
UUIDs.randomBase64UUID(), nodeToSend.getHostName(), nodeToSend.getHostAddress(), nodeToSend.getAddress(),
nodeToSend.getAttributes(), nodeToSend.getRoles(), nodeToSend.getVersion());
}).collect(Collectors.toSet());
logger.trace("replacing {} with temp node {}", nodeToSend, tempNode);
nodeToSend = tempNode;
}
sendPingsHandler.nodeToDisconnect.add(nodeToSend);
}
// fork the connection to another thread
final DiscoveryNode finalNodeToSend = nodeToSend;
unicastZenPingExecutorService.execute(new Runnable() {
@Override
public void run() {
if (sendPingsHandler.isClosed()) {
return;
}
boolean success = false;
try {
// connect to the node, see if we manage to do it, if not, bail
if (!nodeFoundByAddress) {
logger.trace("[{}] connecting (light) to {}", sendPingsHandler.id(), finalNodeToSend);
transportService.connectToNodeAndHandshake(finalNodeToSend, timeout.getMillis());
} else {
logger.trace("[{}] connecting to {}", sendPingsHandler.id(), finalNodeToSend);
transportService.connectToNode(finalNodeToSend);
}
logger.trace("[{}] connected to {}", sendPingsHandler.id(), node);
if (receivedResponses.containsKey(sendPingsHandler.id())) {
// we are connected and still in progress, send the ping request
sendPingRequestToNode(sendPingsHandler.id(), timeout, pingRequest, latch, node, finalNodeToSend);
} else {
// connect took too long, just log it and bail
latch.countDown();
logger.trace("[{}] connect to {} was too long outside of ping window, bailing",
sendPingsHandler.id(), node);
}
success = true;
} catch (ConnectTransportException e) {
// can't connect to the node - this is a more common path!
logger.trace(
(Supplier<?>) () -> new ParameterizedMessage(
"[{}] failed to connect to {}", sendPingsHandler.id(), finalNodeToSend), e);
} catch (RemoteTransportException e) {
// something went wrong on the other side
logger.debug(
(Supplier<?>) () -> new ParameterizedMessage(
"[{}] received a remote error as a response to ping {}", sendPingsHandler.id(), finalNodeToSend), e);
} catch (Exception e) {
logger.warn(
(Supplier<?>) () -> new ParameterizedMessage(
"[{}] failed send ping to {}", sendPingsHandler.id(), finalNodeToSend), e);
} finally {
if (!success) {
latch.countDown();
}
}
}
});
} else {
sendPingRequestToNode(sendPingsHandler.id(), timeout, pingRequest, latch, node, nodeToSend);
}
}
if (waitTime != null) {
try {
latch.await(waitTime.millis(), TimeUnit.MILLISECONDS);
} catch (InterruptedException e) {
// ignore
}
}
nodesToPing.forEach(node -> sendPingRequestToNode(node, timeout, pingingRound, pingRequest));
}
private void sendPingRequestToNode(final int id, final TimeValue timeout, final UnicastPingRequest pingRequest,
final CountDownLatch latch, final DiscoveryNode node, final DiscoveryNode nodeToSend) {
logger.trace("[{}] sending to {}", id, nodeToSend);
transportService.sendRequest(nodeToSend, ACTION_NAME, pingRequest, TransportRequestOptions.builder()
.withTimeout((long) (timeout.millis() * 1.25)).build(), new TransportResponseHandler<UnicastPingResponse>() {
private void sendPingRequestToNode(final DiscoveryNode node, TimeValue timeout, final PingingRound pingingRound,
final UnicastPingRequest pingRequest) {
submitToExecutor(new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
Connection connection = null;
if (transportService.nodeConnected(node)) {
try {
// concurrency can still cause disconnects
connection = transportService.getConnection(node);
} catch (NodeNotConnectedException e) {
logger.trace("[{}] node [{}] just disconnected, will create a temp connection", pingingRound.id(), node);
}
}
if (connection == null) {
connection = pingingRound.getOrConnect(node);
}
logger.trace("[{}] sending to {}", pingingRound.id(), node);
transportService.sendRequest(connection, ACTION_NAME, pingRequest,
TransportRequestOptions.builder().withTimeout((long) (timeout.millis() * 1.25)).build(),
getPingResponseHandler(pingingRound, node));
}
@Override
public void onFailure(Exception e) {
if (e instanceof ConnectTransportException || e instanceof AlreadyClosedException) {
// can't connect to the node - this is more common path!
logger.trace(
(Supplier<?>) () -> new ParameterizedMessage(
"[{}] failed to ping {}", pingingRound.id(), node), e);
} else if (e instanceof RemoteTransportException) {
// something went wrong on the other side
logger.debug(
(Supplier<?>) () -> new ParameterizedMessage(
"[{}] received a remote error as a response to ping {}", pingingRound.id(), node), e);
} else {
logger.warn(
(Supplier<?>) () -> new ParameterizedMessage(
"[{}] failed send ping to {}", pingingRound.id(), node), e);
}
}
@Override
public void onRejection(Exception e) {
// The RejectedExecutionException can come from the fact unicastZenPingExecutorService is at its max down in sendPings
// But don't bail here, we can retry later on after the send ping has been scheduled.
logger.debug("Ping execution rejected", e);
}
});
}
// for testing
protected void submitToExecutor(AbstractRunnable abstractRunnable) {
unicastZenPingExecutorService.execute(abstractRunnable);
}
// for testing
protected TransportResponseHandler<UnicastPingResponse> getPingResponseHandler(final PingingRound pingingRound,
final DiscoveryNode node) {
return new TransportResponseHandler<UnicastPingResponse>() {
@Override
public UnicastPingResponse newInstance() {
@ -538,50 +552,36 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
@Override
public void handleResponse(UnicastPingResponse response) {
logger.trace("[{}] received response from {}: {}", id, nodeToSend, Arrays.toString(response.pingResponses));
try {
DiscoveryNodes discoveryNodes = contextProvider.nodes();
for (PingResponse pingResponse : response.pingResponses) {
if (pingResponse.node().equals(discoveryNodes.getLocalNode())) {
// that's us, ignore
continue;
}
SendPingsHandler sendPingsHandler = receivedResponses.get(response.id);
if (sendPingsHandler == null) {
if (!closed) {
// Only log when we're not closing the node. Having no send ping handler is then expected
logger.warn("received ping response {} with no matching handler id [{}]", pingResponse, response.id);
}
} else {
sendPingsHandler.pingCollection().addPing(pingResponse);
}
logger.trace("[{}] received response from {}: {}", pingingRound.id(), node, Arrays.toString(response.pingResponses));
if (pingingRound.isClosed()) {
if (logger.isTraceEnabled()) {
logger.trace("[{}] skipping received response from {}. already closed", pingingRound.id(), node);
}
} finally {
latch.countDown();
} else {
Stream.of(response.pingResponses).forEach(pingingRound::addPingResponseToCollection);
}
}
@Override
public void handleException(TransportException exp) {
latch.countDown();
if (exp instanceof ConnectTransportException) {
if (exp instanceof ConnectTransportException || exp.getCause() instanceof ConnectTransportException) {
// ok, not connected...
logger.trace((Supplier<?>) () -> new ParameterizedMessage("failed to connect to {}", nodeToSend), exp);
} else {
logger.trace((Supplier<?>) () -> new ParameterizedMessage("failed to connect to {}", node), exp);
} else if (closed == false) {
logger.warn((Supplier<?>) () -> new ParameterizedMessage("failed to send ping to [{}]", node), exp);
}
}
});
};
}
private UnicastPingResponse handlePingRequest(final UnicastPingRequest request) {
assert clusterName.equals(request.pingResponse.clusterName()) :
"got a ping request from a different cluster. expected " + clusterName + " got " + request.pingResponse.clusterName();
temporalResponses.add(request.pingResponse);
threadPool.schedule(TimeValue.timeValueMillis(request.timeout.millis() * 2), ThreadPool.Names.SAME, new Runnable() {
@Override
public void run() {
temporalResponses.remove(request.pingResponse);
}
});
// add to any ongoing pinging
activePingingRounds.values().forEach(p -> p.addPingResponseToCollection(request.pingResponse));
threadPool.schedule(TimeValue.timeValueMillis(request.timeout.millis() * 2), ThreadPool.Names.SAME,
() -> temporalResponses.remove(request.pingResponse));
List<PingResponse> pingResponses = CollectionUtils.iterableAsArrayList(temporalResponses);
pingResponses.add(createPingResponse(contextProvider.nodes()));
@ -601,11 +601,11 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
channel.sendResponse(handlePingRequest(request));
} else {
throw new IllegalStateException(
String.format(
Locale.ROOT,
"mismatched cluster names; request: [%s], local: [%s]",
request.pingResponse.clusterName().value(),
clusterName.value()));
String.format(
Locale.ROOT,
"mismatched cluster names; request: [%s], local: [%s]",
request.pingResponse.clusterName().value(),
clusterName.value()));
}
}

View File

@ -67,11 +67,11 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiFunction;
import java.util.function.Consumer;
import java.util.stream.Collectors;
@ -1021,24 +1021,22 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
}
private ZenPing.PingCollection pingAndWait(TimeValue timeout) {
final ZenPing.PingCollection response = new ZenPing.PingCollection();
final CountDownLatch latch = new CountDownLatch(1);
final CompletableFuture<ZenPing.PingCollection> response = new CompletableFuture<>();
try {
zenPing.ping(pings -> {
response.addPings(pings);
latch.countDown();
}, timeout);
zenPing.ping(response::complete, timeout);
} catch (Exception ex) {
logger.warn("Ping execution failed", ex);
latch.countDown();
// logged later
response.completeExceptionally(ex);
}
try {
latch.await();
return response;
return response.get();
} catch (InterruptedException e) {
logger.trace("pingAndWait interrupted");
return response;
return new ZenPing.PingCollection();
} catch (ExecutionException e) {
logger.warn("Ping execution failed", e);
return new ZenPing.PingCollection();
}
}

View File

@ -30,11 +30,11 @@ import org.elasticsearch.common.unit.TimeValue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Consumer;
import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK;
@ -42,17 +42,7 @@ public interface ZenPing extends Releasable {
void start(PingContextProvider contextProvider);
void ping(PingListener listener, TimeValue timeout);
interface PingListener {
/**
* called when pinging is done.
*
* @param pings ping result *must
*/
void onPing(Collection<PingResponse> pings);
}
void ping(Consumer<PingCollection> resultsConsumer, TimeValue timeout);
class PingResponse implements Streamable {
@ -191,13 +181,6 @@ public interface ZenPing extends Releasable {
return false;
}
/** adds multiple pings if newer than previous pings from the same node */
public synchronized void addPings(Iterable<PingResponse> pings) {
for (PingResponse ping : pings) {
addPing(ping);
}
}
/** serialize current pings to a list. It is guaranteed that the list contains one ping response per node */
public synchronized List<PingResponse> toList() {
return new ArrayList<>(pings.values());

View File

@ -48,7 +48,6 @@ import org.elasticsearch.indices.IndicesQueryCache;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.indices.mapper.MapperRegistry;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.threadpool.ThreadPool;
@ -327,7 +326,6 @@ public final class IndexModule {
BigArrays bigArrays,
ThreadPool threadPool,
ScriptService scriptService,
IndicesQueriesRegistry indicesQueriesRegistry,
ClusterService clusterService,
Client client,
IndicesQueryCache indicesQueryCache,
@ -366,7 +364,7 @@ public final class IndexModule {
}
return new IndexService(indexSettings, environment, xContentRegistry, new SimilarityService(indexSettings, similarities),
shardStoreDeleter, analysisRegistry, engineFactory.get(), circuitBreakerService, bigArrays, threadPool, scriptService,
indicesQueriesRegistry, clusterService, client, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry,
clusterService, client, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry,
indicesFieldDataCache, globalCheckpointSyncer, searchOperationListeners, indexOperationListeners);
}

View File

@ -68,7 +68,6 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.cluster.IndicesClusterStateService;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.indices.mapper.MapperRegistry;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.threadpool.ThreadPool;
@ -120,7 +119,6 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
private final BigArrays bigArrays;
private final AsyncGlobalCheckpointTask globalCheckpointTask;
private final ScriptService scriptService;
private final IndicesQueriesRegistry queryRegistry;
private final ClusterService clusterService;
private final Client client;
@ -134,7 +132,6 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
BigArrays bigArrays,
ThreadPool threadPool,
ScriptService scriptService,
IndicesQueriesRegistry queryRegistry,
ClusterService clusterService,
Client client,
QueryCache queryCache,
@ -162,7 +159,6 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
this.bigArrays = bigArrays;
this.threadPool = threadPool;
this.scriptService = scriptService;
this.queryRegistry = queryRegistry;
this.clusterService = clusterService;
this.client = client;
this.eventListener = eventListener;
@ -478,7 +474,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
public QueryShardContext newQueryShardContext(int shardId, IndexReader indexReader, LongSupplier nowInMillis) {
return new QueryShardContext(
shardId, indexSettings, indexCache.bitsetFilterCache(), indexFieldData, mapperService(),
similarityService(), scriptService, xContentRegistry, queryRegistry,
similarityService(), scriptService, xContentRegistry,
client, indexReader,
nowInMillis);
}

View File

@ -152,7 +152,7 @@ public class DynamicTemplate implements ToXContent {
return v;
}
}
throw new IllegalArgumentException("No xcontent type matched on [" + value + "], possible values are "
throw new IllegalArgumentException("No field type matched on [" + value + "], possible values are "
+ Arrays.toString(values()));
}
@ -208,6 +208,8 @@ public class DynamicTemplate implements ToXContent {
if (indexVersionCreated.onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
throw e;
} else {
DEPRECATION_LOGGER.deprecated("match_mapping_type [" + matchMappingType + "] is invalid and will be ignored: "
+ e.getMessage());
// this template is on an unknown type so it will never match anything
// null indicates that the template should be ignored
return null;

View File

@ -186,6 +186,30 @@ public class NumberFieldMapper extends FieldMapper {
return HalfFloatPoint.newSetQuery(field, v);
}
private float nextDown(float f) {
// HalfFloatPoint.nextDown considers that -0 is the same as +0
// while point ranges are consistent with Float.compare, so
// they consider that -0 < +0, so we explicitly make sure
// that nextDown(+0) returns -0
if (Float.floatToIntBits(f) == Float.floatToIntBits(0f)) {
return -0f;
} else {
return HalfFloatPoint.nextDown(f);
}
}
private float nextUp(float f) {
// HalfFloatPoint.nextUp considers that -0 is the same as +0
// while point ranges are consistent with Float.compare, so
// they consider that -0 < +0, so we explicitly make sure
// that nextUp(-0) returns +0
if (Float.floatToIntBits(f) == Float.floatToIntBits(-0f)) {
return +0f;
} else {
return HalfFloatPoint.nextUp(f);
}
}
@Override
Query rangeQuery(String field, Object lowerTerm, Object upperTerm,
boolean includeLower, boolean includeUpper) {
@ -194,16 +218,16 @@ public class NumberFieldMapper extends FieldMapper {
if (lowerTerm != null) {
l = parse(lowerTerm);
if (includeLower) {
l = Math.nextDown(l);
l = nextDown(l);
}
l = HalfFloatPoint.nextUp(l);
}
if (upperTerm != null) {
u = parse(upperTerm);
if (includeUpper) {
u = Math.nextUp(u);
u = nextUp(u);
}
u = HalfFloatPoint.nextDown(u);
u = nextDown(u);
}
return HalfFloatPoint.newRangeQuery(field, l, u);
}
@ -276,6 +300,30 @@ public class NumberFieldMapper extends FieldMapper {
return FloatPoint.newSetQuery(field, v);
}
private float nextDown(float f) {
// Math.nextDown considers that -0 is the same as +0
// while point ranges are consistent with Float.compare, so
// they consider that -0 < +0, so we explicitly make sure
// that nextDown(+0) returns -0
if (Float.floatToIntBits(f) == Float.floatToIntBits(0f)) {
return -0f;
} else {
return Math.nextDown(f);
}
}
private float nextUp(float f) {
// Math.nextUp considers that -0 is the same as +0
// while point ranges are consistent with Float.compare, so
// they consider that -0 < +0, so we explicitly make sure
// that nextUp(-0) returns +0
if (Float.floatToIntBits(f) == Float.floatToIntBits(-0f)) {
return +0f;
} else {
return Math.nextUp(f);
}
}
@Override
Query rangeQuery(String field, Object lowerTerm, Object upperTerm,
boolean includeLower, boolean includeUpper) {
@ -284,13 +332,13 @@ public class NumberFieldMapper extends FieldMapper {
if (lowerTerm != null) {
l = parse(lowerTerm);
if (includeLower == false) {
l = Math.nextUp(l);
l = nextUp(l);
}
}
if (upperTerm != null) {
u = parse(upperTerm);
if (includeUpper == false) {
u = Math.nextDown(u);
u = nextDown(u);
}
}
return FloatPoint.newRangeQuery(field, l, u);
@ -364,6 +412,30 @@ public class NumberFieldMapper extends FieldMapper {
return DoublePoint.newSetQuery(field, v);
}
private double nextDown(double d) {
// Math.nextDown considers that -0 is the same as +0
// while point ranges are consistent with Double.compare, so
// they consider that -0 < +0, so we explicitly make sure
// that nextDown(+0) returns -0
if (Double.doubleToLongBits(d) == Double.doubleToLongBits(0d)) {
return -0d;
} else {
return Math.nextDown(d);
}
}
private double nextUp(double d) {
// Math.nextUp considers that -0 is the same as +0
// while point ranges are consistent with Double.compare, so
// they consider that -0 < +0, so we explicitly make sure
// that nextUp(-0) returns +0
if (Double.doubleToLongBits(d) == Double.doubleToLongBits(-0d)) {
return +0d;
} else {
return Math.nextUp(d);
}
}
@Override
Query rangeQuery(String field, Object lowerTerm, Object upperTerm,
boolean includeLower, boolean includeUpper) {
@ -372,13 +444,13 @@ public class NumberFieldMapper extends FieldMapper {
if (lowerTerm != null) {
l = parse(lowerTerm);
if (includeLower == false) {
l = Math.nextUp(l);
l = nextUp(l);
}
}
if (upperTerm != null) {
u = parse(upperTerm);
if (includeUpper == false) {
u = Math.nextDown(u);
u = nextDown(u);
}
}
return DoublePoint.newRangeQuery(field, l, u);

View File

@ -58,7 +58,7 @@ public class MatchAllQueryBuilder extends AbstractQueryBuilder<MatchAllQueryBuil
builder.endObject();
}
private static ObjectParser<MatchAllQueryBuilder, QueryParseContext> PARSER = new ObjectParser<>(NAME, MatchAllQueryBuilder::new);
private static final ObjectParser<MatchAllQueryBuilder, QueryParseContext> PARSER = new ObjectParser<>(NAME, MatchAllQueryBuilder::new);
static {
declareStandardFields(PARSER);

View File

@ -23,8 +23,9 @@ import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParseFieldMatcherSupplier;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.NamedXContentRegistry.UnknownNamedObjectException;
import org.elasticsearch.common.xcontent.XContentLocation;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.Script;
import java.io.IOException;
@ -36,17 +37,14 @@ public class QueryParseContext implements ParseFieldMatcherSupplier {
private static final ParseField CACHE_KEY = new ParseField("_cache_key").withAllDeprecated("Filters are always used as cache keys");
private final XContentParser parser;
private final IndicesQueriesRegistry indicesQueriesRegistry;
private final ParseFieldMatcher parseFieldMatcher;
private final String defaultScriptLanguage;
public QueryParseContext(IndicesQueriesRegistry registry, XContentParser parser, ParseFieldMatcher parseFieldMatcher) {
this(Script.DEFAULT_SCRIPT_LANG, registry, parser, parseFieldMatcher);
public QueryParseContext(XContentParser parser, ParseFieldMatcher parseFieldMatcher) {
this(Script.DEFAULT_SCRIPT_LANG, parser, parseFieldMatcher);
}
public QueryParseContext(String defaultScriptLanguage, IndicesQueriesRegistry registry, XContentParser parser,
ParseFieldMatcher parseFieldMatcher) {
this.indicesQueriesRegistry = Objects.requireNonNull(registry, "indices queries registry cannot be null");
public QueryParseContext(String defaultScriptLanguage, XContentParser parser, ParseFieldMatcher parseFieldMatcher) {
this.parser = Objects.requireNonNull(parser, "parser cannot be null");
this.parseFieldMatcher = Objects.requireNonNull(parseFieldMatcher, "parse field matcher cannot be null");
this.defaultScriptLanguage = defaultScriptLanguage;
@ -105,7 +103,15 @@ public class QueryParseContext implements ParseFieldMatcherSupplier {
if (parser.nextToken() != XContentParser.Token.START_OBJECT) {
throw new ParsingException(parser.getTokenLocation(), "[" + queryName + "] query malformed, no start_object after query name");
}
QueryBuilder result = indicesQueriesRegistry.lookup(queryName, parseFieldMatcher, parser.getTokenLocation()).fromXContent(this);
QueryBuilder result;
try {
result = parser.namedObject(QueryBuilder.class, queryName, this);
} catch (UnknownNamedObjectException e) {
// Preserve the error message from 5.0 until we have a compellingly better message so we don't break BWC.
// This intentionally doesn't include the causing exception because that'd change the "root_cause" of any unknown query errors
throw new ParsingException(new XContentLocation(e.getLineNumber(), e.getColumnNumber()),
"no [query] registered for [" + e.getName() + "]");
}
//end_object of the specific query (e.g. match, multi_match etc.) element
if (parser.currentToken() != XContentParser.Token.END_OBJECT) {
throw new ParsingException(parser.getTokenLocation(),

View File

@ -27,7 +27,6 @@ import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptContext;
@ -43,19 +42,17 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier {
protected final ScriptService scriptService;
protected final IndexSettings indexSettings;
private final NamedXContentRegistry xContentRegistry;
protected final IndicesQueriesRegistry indicesQueriesRegistry;
protected final Client client;
protected final IndexReader reader;
protected final LongSupplier nowInMillis;
public QueryRewriteContext(IndexSettings indexSettings, MapperService mapperService, ScriptService scriptService,
NamedXContentRegistry xContentRegistry, IndicesQueriesRegistry indicesQueriesRegistry, Client client, IndexReader reader,
NamedXContentRegistry xContentRegistry, Client client, IndexReader reader,
LongSupplier nowInMillis) {
this.mapperService = mapperService;
this.scriptService = scriptService;
this.indexSettings = indexSettings;
this.xContentRegistry = xContentRegistry;
this.indicesQueriesRegistry = indicesQueriesRegistry;
this.client = client;
this.reader = reader;
this.nowInMillis = nowInMillis;
@ -107,7 +104,7 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier {
* are configured in the index settings. The default script language will always default to Painless.
*/
public QueryParseContext newParseContext(XContentParser parser) {
return new QueryParseContext(indicesQueriesRegistry, parser, indexSettings.getParseFieldMatcher());
return new QueryParseContext(parser, indexSettings.getParseFieldMatcher());
}
public long nowInMillis() {

View File

@ -48,7 +48,6 @@ import org.elasticsearch.index.mapper.ObjectMapper;
import org.elasticsearch.index.mapper.TextFieldMapper;
import org.elasticsearch.index.query.support.NestedScope;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.CompiledScript;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.Script;
@ -92,7 +91,6 @@ public class QueryShardContext extends QueryRewriteContext {
private final Map<String, Query> namedQueries = new HashMap<>();
private final MapperQueryParser queryParser = new MapperQueryParser(this);
private final IndicesQueriesRegistry indicesQueriesRegistry;
private boolean allowUnmappedFields;
private boolean mapUnmappedFieldAsString;
private NestedScope nestedScope;
@ -100,9 +98,9 @@ public class QueryShardContext extends QueryRewriteContext {
public QueryShardContext(int shardId, IndexSettings indexSettings, BitsetFilterCache bitsetFilterCache,
IndexFieldDataService indexFieldDataService, MapperService mapperService, SimilarityService similarityService,
ScriptService scriptService, NamedXContentRegistry xContentRegistry, IndicesQueriesRegistry indicesQueriesRegistry,
ScriptService scriptService, NamedXContentRegistry xContentRegistry,
Client client, IndexReader reader, LongSupplier nowInMillis) {
super(indexSettings, mapperService, scriptService, xContentRegistry, indicesQueriesRegistry, client, reader, nowInMillis);
super(indexSettings, mapperService, scriptService, xContentRegistry, client, reader, nowInMillis);
this.shardId = shardId;
this.indexSettings = indexSettings;
this.similarityService = similarityService;
@ -110,14 +108,13 @@ public class QueryShardContext extends QueryRewriteContext {
this.bitsetFilterCache = bitsetFilterCache;
this.indexFieldDataService = indexFieldDataService;
this.allowUnmappedFields = indexSettings.isDefaultAllowUnmappedFields();
this.indicesQueriesRegistry = indicesQueriesRegistry;
this.nestedScope = new NestedScope();
}
public QueryShardContext(QueryShardContext source) {
this(source.shardId, source.indexSettings, source.bitsetFilterCache, source.indexFieldDataService, source.mapperService,
source.similarityService, source.scriptService, source.getXContentRegistry(), source.indicesQueriesRegistry, source.client,
source.similarityService, source.scriptService, source.getXContentRegistry(), source.client,
source.reader, source.nowInMillis);
this.types = source.getTypes();
}

View File

@ -19,8 +19,6 @@
package org.elasticsearch.indices;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.index.DirectoryReader;
@ -41,7 +39,6 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.routing.RecoverySource;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService;
@ -106,7 +103,6 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.cluster.IndicesClusterStateService;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.indices.mapper.MapperRegistry;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.plugins.PluginsService;
@ -156,7 +152,6 @@ public class IndicesService extends AbstractLifecycleComponent
private final NamedXContentRegistry xContentRegistry;
private final TimeValue shardsClosedTimeout;
private final AnalysisRegistry analysisRegistry;
private final IndicesQueriesRegistry indicesQueriesRegistry;
private final IndexNameExpressionResolver indexNameExpressionResolver;
private final IndexScopedSettings indexScopeSetting;
private final IndicesFieldDataCache indicesFieldDataCache;
@ -187,7 +182,7 @@ public class IndicesService extends AbstractLifecycleComponent
public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvironment nodeEnv, NamedXContentRegistry xContentRegistry,
ClusterSettings clusterSettings, AnalysisRegistry analysisRegistry,
IndicesQueriesRegistry indicesQueriesRegistry, IndexNameExpressionResolver indexNameExpressionResolver,
IndexNameExpressionResolver indexNameExpressionResolver,
MapperRegistry mapperRegistry, NamedWriteableRegistry namedWriteableRegistry,
ThreadPool threadPool, IndexScopedSettings indexScopedSettings, CircuitBreakerService circuitBreakerService,
BigArrays bigArrays, ScriptService scriptService, ClusterService clusterService, Client client,
@ -199,7 +194,6 @@ public class IndicesService extends AbstractLifecycleComponent
this.xContentRegistry = xContentRegistry;
this.shardsClosedTimeout = settings.getAsTime(INDICES_SHARDS_CLOSED_TIMEOUT, new TimeValue(1, TimeUnit.DAYS));
this.analysisRegistry = analysisRegistry;
this.indicesQueriesRegistry = indicesQueriesRegistry;
this.indexNameExpressionResolver = indexNameExpressionResolver;
this.indicesRequestCache = new IndicesRequestCache(settings);
this.indicesQueryCache = new IndicesQueryCache(settings);
@ -451,7 +445,6 @@ public class IndicesService extends AbstractLifecycleComponent
bigArrays,
threadPool,
scriptService,
indicesQueriesRegistry,
clusterService,
client,
indicesQueryCache,
@ -1017,13 +1010,6 @@ public class IndicesService extends AbstractLifecycleComponent
return numUncompletedDeletes.get() > 0;
}
/**
* Returns this nodes {@link IndicesQueriesRegistry}
*/
public IndicesQueriesRegistry getIndicesQueryRegistry() {
return indicesQueriesRegistry;
}
public AnalysisRegistry getAnalysis() {
return analysisRegistry;
}
@ -1268,7 +1254,7 @@ public class IndicesService extends AbstractLifecycleComponent
* of dependencies we pass in a function that can perform the parsing. */
ShardSearchRequest.FilterParser filterParser = bytes -> {
try (XContentParser parser = XContentFactory.xContent(bytes).createParser(xContentRegistry, bytes)) {
return new QueryParseContext(indicesQueriesRegistry, parser, new ParseFieldMatcher(settings)).parseInnerQueryBuilder();
return new QueryParseContext(parser, new ParseFieldMatcher(settings)).parseInnerQueryBuilder();
}
};
String[] aliases = indexNameExpressionResolver.filteringAliases(state, index, expressions);

View File

@ -1,32 +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.indices.query;
import org.elasticsearch.common.xcontent.ParseFieldRegistry;
import org.elasticsearch.index.query.QueryParser;
/**
* Extensions to ParseFieldRegistry to make Guice happy.
*/
public class IndicesQueriesRegistry extends ParseFieldRegistry<QueryParser<?>> {
public IndicesQueriesRegistry() {
super("query");
}
}

View File

@ -94,7 +94,6 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.indices.cluster.IndicesClusterStateService;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.indices.recovery.PeerRecoverySourceService;
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
import org.elasticsearch.indices.recovery.RecoverySettings;
@ -372,7 +371,7 @@ public class Node implements Closeable {
).flatMap(Function.identity()).collect(toList()));
final MetaStateService metaStateService = new MetaStateService(settings, nodeEnvironment);
final IndicesService indicesService = new IndicesService(settings, pluginsService, nodeEnvironment, xContentRegistry,
settingsModule.getClusterSettings(), analysisModule.getAnalysisRegistry(), searchModule.getQueryParserRegistry(),
settingsModule.getClusterSettings(), analysisModule.getAnalysisRegistry(),
clusterModule.getIndexNameExpressionResolver(), indicesModule.getMapperRegistry(), namedWriteableRegistry,
threadPool, settingsModule.getIndexScopedSettings(), circuitBreakerService, bigArrays, scriptModule.getScriptService(),
clusterService, client, metaStateService);
@ -410,7 +409,6 @@ public class Node implements Closeable {
final DiscoveryModule discoveryModule = new DiscoveryModule(this.settings, threadPool, transportService,
networkService, clusterService, pluginsService.filterPlugins(DiscoveryPlugin.class));
modules.add(b -> {
b.bind(IndicesQueriesRegistry.class).toInstance(searchModule.getQueryParserRegistry());
b.bind(SearchRequestParsers.class).toInstance(searchModule.getSearchRequestParsers());
b.bind(SearchExtRegistry.class).toInstance(searchModule.getSearchExtRegistry());
b.bind(NamedXContentRegistry.class).toInstance(xContentRegistry);

View File

@ -36,7 +36,6 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryStringQueryBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.rest.BytesRestResponse;
import org.elasticsearch.rest.RestChannel;
import org.elasticsearch.rest.RestRequest;
@ -195,9 +194,8 @@ public class RestActions {
return queryBuilder;
}
public static QueryBuilder getQueryContent(XContentParser requestParser, IndicesQueriesRegistry indicesQueriesRegistry,
ParseFieldMatcher parseFieldMatcher) {
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, requestParser, parseFieldMatcher);
public static QueryBuilder getQueryContent(XContentParser requestParser, ParseFieldMatcher parseFieldMatcher) {
QueryParseContext context = new QueryParseContext(requestParser, parseFieldMatcher);
return context.parseTopLevelQueryBuilder();
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.rest.action.admin.indices;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.action.admin.indices.validate.query.QueryExplanation;
import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryRequest;
import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryResponse;
@ -30,9 +29,6 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.BytesRestResponse;
import org.elasticsearch.rest.RestChannel;
@ -50,11 +46,8 @@ import static org.elasticsearch.rest.RestStatus.OK;
import static org.elasticsearch.rest.action.RestActions.buildBroadcastShardsHeader;
public class RestValidateQueryAction extends BaseRestHandler {
private final IndicesQueriesRegistry indicesQueriesRegistry;
@Inject
public RestValidateQueryAction(Settings settings, RestController controller, IndicesQueriesRegistry indicesQueriesRegistry) {
public RestValidateQueryAction(Settings settings, RestController controller) {
super(settings);
controller.registerHandler(GET, "/_validate/query", this);
controller.registerHandler(POST, "/_validate/query", this);
@ -62,7 +55,6 @@ public class RestValidateQueryAction extends BaseRestHandler {
controller.registerHandler(POST, "/{index}/_validate/query", this);
controller.registerHandler(GET, "/{index}/{type}/_validate/query", this);
controller.registerHandler(POST, "/{index}/{type}/_validate/query", this);
this.indicesQueriesRegistry = indicesQueriesRegistry;
}
@Override
@ -77,7 +69,7 @@ public class RestValidateQueryAction extends BaseRestHandler {
try {
request.withContentOrSourceParamParserOrNull(parser -> {
if (parser != null) {
validateQueryRequest.query(RestActions.getQueryContent(parser, indicesQueriesRegistry, parseFieldMatcher));
validateQueryRequest.query(RestActions.getQueryContent(parser, parseFieldMatcher));
} else if (request.hasParam("q")) {
validateQueryRequest.query(RestActions.urlParamsToQueryBuilder(request));
}

View File

@ -28,7 +28,6 @@ import org.elasticsearch.common.Table;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.RestResponse;
@ -41,15 +40,11 @@ import java.io.IOException;
import static org.elasticsearch.rest.RestRequest.Method.GET;
public class RestCountAction extends AbstractCatAction {
private final IndicesQueriesRegistry indicesQueriesRegistry;
@Inject
public RestCountAction(Settings settings, RestController restController, RestController controller, IndicesQueriesRegistry indicesQueriesRegistry) {
public RestCountAction(Settings settings, RestController restController, RestController controller) {
super(settings);
restController.registerHandler(GET, "/_cat/count", this);
restController.registerHandler(GET, "/_cat/count/{index}", this);
this.indicesQueriesRegistry = indicesQueriesRegistry;
}
@Override
@ -72,7 +67,7 @@ public class RestCountAction extends AbstractCatAction {
searchSourceBuilder.query(queryBuilder);
}
} else {
searchSourceBuilder.query(RestActions.getQueryContent(parser, indicesQueriesRegistry, parseFieldMatcher));
searchSourceBuilder.query(RestActions.getQueryContent(parser, parseFieldMatcher));
}
});
} catch (IOException e) {

View File

@ -19,7 +19,6 @@
package org.elasticsearch.rest.action.document;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.IndicesOptions;
@ -28,9 +27,7 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.BytesRestResponse;
import org.elasticsearch.rest.RestController;
@ -48,11 +45,8 @@ import static org.elasticsearch.rest.action.RestActions.buildBroadcastShardsHead
import static org.elasticsearch.search.internal.SearchContext.DEFAULT_TERMINATE_AFTER;
public class RestCountAction extends BaseRestHandler {
private final IndicesQueriesRegistry indicesQueriesRegistry;
@Inject
public RestCountAction(Settings settings, RestController controller, IndicesQueriesRegistry indicesQueriesRegistry) {
public RestCountAction(Settings settings, RestController controller) {
super(settings);
controller.registerHandler(POST, "/_count", this);
controller.registerHandler(GET, "/_count", this);
@ -60,7 +54,6 @@ public class RestCountAction extends BaseRestHandler {
controller.registerHandler(GET, "/{index}/_count", this);
controller.registerHandler(POST, "/{index}/{type}/_count", this);
controller.registerHandler(GET, "/{index}/{type}/_count", this);
this.indicesQueriesRegistry = indicesQueriesRegistry;
}
@Override
@ -76,7 +69,7 @@ public class RestCountAction extends BaseRestHandler {
searchSourceBuilder.query(queryBuilder);
}
} else {
searchSourceBuilder.query(RestActions.getQueryContent(parser, indicesQueriesRegistry, parseFieldMatcher));
searchSourceBuilder.query(RestActions.getQueryContent(parser, parseFieldMatcher));
}
});
countRequest.routing(request.param("routing"));

View File

@ -29,7 +29,6 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.get.GetResult;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.BytesRestResponse;
import org.elasticsearch.rest.RestController;
@ -50,13 +49,9 @@ import static org.elasticsearch.rest.RestStatus.OK;
* Rest action for computing a score explanation for specific documents.
*/
public class RestExplainAction extends BaseRestHandler {
private final IndicesQueriesRegistry indicesQueriesRegistry;
@Inject
public RestExplainAction(Settings settings, RestController controller, IndicesQueriesRegistry indicesQueriesRegistry) {
public RestExplainAction(Settings settings, RestController controller) {
super(settings);
this.indicesQueriesRegistry = indicesQueriesRegistry;
controller.registerHandler(GET, "/{index}/{type}/{id}/_explain", this);
controller.registerHandler(POST, "/{index}/{type}/{id}/_explain", this);
}
@ -70,7 +65,7 @@ public class RestExplainAction extends BaseRestHandler {
String queryString = request.param("q");
request.withContentOrSourceParamParserOrNull(parser -> {
if (parser != null) {
explainRequest.query(RestActions.getQueryContent(parser, indicesQueriesRegistry, parseFieldMatcher));
explainRequest.query(RestActions.getQueryContent(parser, parseFieldMatcher));
} else if (queryString != null) {
QueryBuilder query = RestActions.urlParamsToQueryBuilder(request);
explainRequest.query(query);

View File

@ -89,8 +89,7 @@ public class RestMultiSearchAction extends BaseRestHandler {
parseMultiLineRequest(restRequest, multiRequest.indicesOptions(), allowExplicitIndex, (searchRequest, parser) -> {
try {
final QueryParseContext queryParseContext = new QueryParseContext(searchRequestParsers.queryParsers, parser,
parseFieldMatcher);
final QueryParseContext queryParseContext = new QueryParseContext(parser, parseFieldMatcher);
searchRequest.source(SearchSourceBuilder.fromXContent(queryParseContext,
searchRequestParsers.aggParsers, searchRequestParsers.suggesters, searchRequestParsers.searchExtParsers));
multiRequest.add(searchRequest);

View File

@ -92,7 +92,7 @@ public class RestSearchAction extends BaseRestHandler {
}
searchRequest.indices(Strings.splitStringByCommaToArray(request.param("index")));
if (requestContentParser != null) {
QueryParseContext context = new QueryParseContext(searchRequestParsers.queryParsers, requestContentParser, parseFieldMatcher);
QueryParseContext context = new QueryParseContext(requestContentParser, parseFieldMatcher);
searchRequest.source().parseXContent(context, searchRequestParsers.aggParsers, searchRequestParsers.suggesters,
searchRequestParsers.searchExtParsers);
}

View File

@ -85,7 +85,6 @@ import org.elasticsearch.index.query.functionscore.RandomScoreFunctionBuilder;
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
import org.elasticsearch.index.query.functionscore.ScriptScoreFunctionBuilder;
import org.elasticsearch.index.query.functionscore.WeightBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.plugins.SearchPlugin;
import org.elasticsearch.plugins.SearchPlugin.AggregationSpec;
import org.elasticsearch.plugins.SearchPlugin.FetchPhaseConstructionContext;
@ -269,7 +268,6 @@ public class SearchModule {
private final boolean transportClient;
private final Map<String, Highlighter> highlighters;
private final Map<String, Suggester<?>> suggesters;
private final IndicesQueriesRegistry queryParserRegistry = new IndicesQueriesRegistry();
private final ParseFieldRegistry<Aggregator.Parser> aggregationParserRegistry = new ParseFieldRegistry<>("aggregation");
private final ParseFieldRegistry<PipelineAggregator.Parser> pipelineAggregationParserRegistry = new ParseFieldRegistry<>(
"pipline_aggregation");
@ -304,7 +302,7 @@ public class SearchModule {
registerFetchSubPhases(plugins);
registerSearchExts(plugins);
registerShapes();
searchRequestParsers = new SearchRequestParsers(queryParserRegistry, aggregatorParsers, getSuggesters(), searchExtParserRegistry);
searchRequestParsers = new SearchRequestParsers(aggregatorParsers, getSuggesters(), searchExtParserRegistry);
}
public List<NamedWriteableRegistry.Entry> getNamedWriteables() {
@ -319,10 +317,6 @@ public class SearchModule {
return new Suggesters(suggesters);
}
public IndicesQueriesRegistry getQueryParserRegistry() {
return queryParserRegistry;
}
public SearchRequestParsers getSearchRequestParsers() {
return searchRequestParsers;
}
@ -442,11 +436,12 @@ public class SearchModule {
if (false == transportClient) {
aggregationParserRegistry.register(spec.getParser(), spec.getName());
}
namedWriteables.add(new Entry(AggregationBuilder.class, spec.getName().getPreferredName(), spec.getReader()));
namedWriteables.add(
new NamedWriteableRegistry.Entry(AggregationBuilder.class, spec.getName().getPreferredName(), spec.getReader()));
for (Map.Entry<String, Writeable.Reader<? extends InternalAggregation>> t : spec.getResultReaders().entrySet()) {
String writeableName = t.getKey();
Writeable.Reader<? extends InternalAggregation> internalReader = t.getValue();
namedWriteables.add(new Entry(InternalAggregation.class, writeableName, internalReader));
namedWriteables.add(new NamedWriteableRegistry.Entry(InternalAggregation.class, writeableName, internalReader));
}
}
@ -535,10 +530,13 @@ public class SearchModule {
if (false == transportClient) {
pipelineAggregationParserRegistry.register(spec.getParser(), spec.getName());
}
namedWriteables.add(new Entry(PipelineAggregationBuilder.class, spec.getName().getPreferredName(), spec.getReader()));
namedWriteables.add(new Entry(PipelineAggregator.class, spec.getName().getPreferredName(), spec.getAggregatorReader()));
namedWriteables.add(
new NamedWriteableRegistry.Entry(PipelineAggregationBuilder.class, spec.getName().getPreferredName(), spec.getReader()));
namedWriteables.add(
new NamedWriteableRegistry.Entry(PipelineAggregator.class, spec.getName().getPreferredName(), spec.getAggregatorReader()));
for (Map.Entry<String, Writeable.Reader<? extends InternalAggregation>> resultReader : spec.getResultReaders().entrySet()) {
namedWriteables.add(new Entry(InternalAggregation.class, resultReader.getKey(), resultReader.getValue()));
namedWriteables
.add(new NamedWriteableRegistry.Entry(InternalAggregation.class, resultReader.getKey(), resultReader.getValue()));
}
}
@ -549,14 +547,14 @@ public class SearchModule {
}
private void registerRescorers() {
namedWriteables.add(new Entry(RescoreBuilder.class, QueryRescorerBuilder.NAME, QueryRescorerBuilder::new));
namedWriteables.add(new NamedWriteableRegistry.Entry(RescoreBuilder.class, QueryRescorerBuilder.NAME, QueryRescorerBuilder::new));
}
private void registerSorts() {
namedWriteables.add(new Entry(SortBuilder.class, GeoDistanceSortBuilder.NAME, GeoDistanceSortBuilder::new));
namedWriteables.add(new Entry(SortBuilder.class, ScoreSortBuilder.NAME, ScoreSortBuilder::new));
namedWriteables.add(new Entry(SortBuilder.class, ScriptSortBuilder.NAME, ScriptSortBuilder::new));
namedWriteables.add(new Entry(SortBuilder.class, FieldSortBuilder.NAME, FieldSortBuilder::new));
namedWriteables.add(new NamedWriteableRegistry.Entry(SortBuilder.class, GeoDistanceSortBuilder.NAME, GeoDistanceSortBuilder::new));
namedWriteables.add(new NamedWriteableRegistry.Entry(SortBuilder.class, ScoreSortBuilder.NAME, ScoreSortBuilder::new));
namedWriteables.add(new NamedWriteableRegistry.Entry(SortBuilder.class, ScriptSortBuilder.NAME, ScriptSortBuilder::new));
namedWriteables.add(new NamedWriteableRegistry.Entry(SortBuilder.class, FieldSortBuilder.NAME, FieldSortBuilder::new));
}
private <T> void registerFromPlugin(List<SearchPlugin> plugins, Function<SearchPlugin, List<T>> producer, Consumer<T> consumer) {
@ -568,9 +566,9 @@ public class SearchModule {
}
public static void registerSmoothingModels(List<Entry> namedWriteables) {
namedWriteables.add(new Entry(SmoothingModel.class, Laplace.NAME, Laplace::new));
namedWriteables.add(new Entry(SmoothingModel.class, LinearInterpolation.NAME, LinearInterpolation::new));
namedWriteables.add(new Entry(SmoothingModel.class, StupidBackoff.NAME, StupidBackoff::new));
namedWriteables.add(new NamedWriteableRegistry.Entry(SmoothingModel.class, Laplace.NAME, Laplace::new));
namedWriteables.add(new NamedWriteableRegistry.Entry(SmoothingModel.class, LinearInterpolation.NAME, LinearInterpolation::new));
namedWriteables.add(new NamedWriteableRegistry.Entry(SmoothingModel.class, StupidBackoff.NAME, StupidBackoff::new));
}
private Map<String, Suggester<?>> setupSuggesters(List<SearchPlugin> plugins) {
@ -581,7 +579,7 @@ public class SearchModule {
@Override
public void register(String name, Suggester<?> t) {
super.register(name, t);
namedWriteables.add(new Entry(SuggestionBuilder.class, name, t));
namedWriteables.add(new NamedWriteableRegistry.Entry(SuggestionBuilder.class, name, t));
}
};
suggesters.register("phrase", PhraseSuggester.INSTANCE);
@ -619,7 +617,7 @@ public class SearchModule {
//weight doesn't have its own parser, so every function supports it out of the box.
//Can be a single function too when not associated to any other function, which is why it needs to be registered manually here.
namedWriteables.add(new Entry(ScoreFunctionBuilder.class, WeightBuilder.NAME, WeightBuilder::new));
namedWriteables.add(new NamedWriteableRegistry.Entry(ScoreFunctionBuilder.class, WeightBuilder.NAME, WeightBuilder::new));
registerFromPlugin(plugins, SearchPlugin::getScoreFunctions, this::registerScoreFunction);
}
@ -646,7 +644,7 @@ public class SearchModule {
* Register a new ValueFormat.
*/
private void registerValueFormat(String name, Writeable.Reader<? extends DocValueFormat> reader) {
namedWriteables.add(new Entry(DocValueFormat.class, name, reader));
namedWriteables.add(new NamedWriteableRegistry.Entry(DocValueFormat.class, name, reader));
}
private void registerSignificanceHeuristics(List<SearchPlugin> plugins) {
@ -662,7 +660,8 @@ public class SearchModule {
private void registerSignificanceHeuristic(SearchExtensionSpec<SignificanceHeuristic, SignificanceHeuristicParser> heuristic) {
significanceHeuristicParserRegistry.register(heuristic.getParser(), heuristic.getName());
namedWriteables.add(new Entry(SignificanceHeuristic.class, heuristic.getName().getPreferredName(), heuristic.getReader()));
namedWriteables.add(new NamedWriteableRegistry.Entry(SignificanceHeuristic.class, heuristic.getName().getPreferredName(),
heuristic.getReader()));
}
private void registerMovingAverageModels(List<SearchPlugin> plugins) {
@ -677,7 +676,8 @@ public class SearchModule {
private void registerMovingAverageModel(SearchExtensionSpec<MovAvgModel, MovAvgModel.AbstractModelParser> movAvgModel) {
movingAverageModelParserRegistry.register(movAvgModel.getParser(), movAvgModel.getName());
namedWriteables.add(new Entry(MovAvgModel.class, movAvgModel.getName().getPreferredName(), movAvgModel.getReader()));
namedWriteables.add(
new NamedWriteableRegistry.Entry(MovAvgModel.class, movAvgModel.getName().getPreferredName(), movAvgModel.getReader()));
}
private void registerFetchSubPhases(List<SearchPlugin> plugins) {
@ -700,7 +700,7 @@ public class SearchModule {
private void registerSearchExt(SearchExtSpec<?> spec) {
searchExtParserRegistry.register(spec.getParser(), spec.getName());
namedWriteables.add(new Entry(SearchExtBuilder.class, spec.getName().getPreferredName(), spec.getReader()));
namedWriteables.add(new NamedWriteableRegistry.Entry(SearchExtBuilder.class, spec.getName().getPreferredName(), spec.getReader()));
}
private void registerFetchSubPhase(FetchSubPhase subPhase) {
@ -774,8 +774,9 @@ public class SearchModule {
}
private void registerQuery(QuerySpec<?> spec) {
queryParserRegistry.register(spec.getParser(), spec.getName());
namedWriteables.add(new Entry(QueryBuilder.class, spec.getName().getPreferredName(), spec.getReader()));
namedWriteables.add(new NamedWriteableRegistry.Entry(QueryBuilder.class, spec.getName().getPreferredName(), spec.getReader()));
namedXContents.add(new NamedXContentRegistry.Entry(QueryBuilder.class, spec.getName(),
(p, c) -> spec.getParser().fromXContent((QueryParseContext) c)));
}
public FetchPhase getFetchPhase() {

View File

@ -20,7 +20,6 @@
package org.elasticsearch.search;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.aggregations.AggregatorParsers;
import org.elasticsearch.search.suggest.Suggesters;
@ -33,15 +32,6 @@ public class SearchRequestParsers {
// methods split across RestSearchAction and SearchSourceBuilder should be moved here
// TODO: make all members private once parsing functions are moved here
// TODO: IndicesQueriesRegistry should be removed and just have the map of query parsers here
/**
* Query parsers that may be used in search requests.
* @see org.elasticsearch.index.query.QueryParseContext
* @see org.elasticsearch.search.builder.SearchSourceBuilder#fromXContent(QueryParseContext, AggregatorParsers,
* Suggesters, SearchExtRegistry)
*/
public final IndicesQueriesRegistry queryParsers;
// TODO: AggregatorParsers should be removed and the underlying maps of agg
// and pipeline agg parsers should be here
/**
@ -64,9 +54,7 @@ public class SearchRequestParsers {
*/
public final SearchExtRegistry searchExtParsers;
public SearchRequestParsers(IndicesQueriesRegistry queryParsers, AggregatorParsers aggParsers, Suggesters suggesters,
SearchExtRegistry searchExtParsers) {
this.queryParsers = queryParsers;
public SearchRequestParsers(AggregatorParsers aggParsers, Suggesters suggesters, SearchExtRegistry searchExtParsers) {
this.aggParsers = aggParsers;
this.suggesters = suggesters;
this.searchExtParsers = searchExtParsers;

View File

@ -50,15 +50,10 @@ public class ScriptLongValues extends SortingNumericDocValues implements ScorerA
resize(0);
}
else if (value instanceof Number) {
resize(1);
values[0] = ((Number) value).longValue();
}
else if (value.getClass().isArray()) {
resize(Array.getLength(value));
for (int i = 0; i < count(); ++i) {
values[i] = ((Number) Array.get(value, i)).longValue();
values[i] = toLongValue(Array.get(value, i));
}
}
@ -66,18 +61,33 @@ public class ScriptLongValues extends SortingNumericDocValues implements ScorerA
resize(((Collection<?>) value).size());
int i = 0;
for (Iterator<?> it = ((Collection<?>) value).iterator(); it.hasNext(); ++i) {
values[i] = ((Number) it.next()).longValue();
values[i] = toLongValue(it.next());
}
assert i == count();
}
else {
throw new AggregationExecutionException("Unsupported script value [" + value + "]");
resize(1);
values[0] = toLongValue(value);
}
sort();
}
private static long toLongValue(Object o) {
if (o instanceof Number) {
return ((Number) o).longValue();
} else if (o instanceof Boolean) {
// We do expose boolean fields as boolean in scripts, however aggregations still expect
// that scripts return the same internal representation as regular fields, so boolean
// values in scripts need to be converted to a number, and the value formatter will
// make sure of using true/false in the key_as_string field
return ((Boolean) o).booleanValue() ? 1L : 0L;
} else {
throw new AggregationExecutionException("Unsupported script value [" + o + "], expected a number");
}
}
@Override
public void setScorer(Scorer scorer) {
script.setScorer(scorer);

View File

@ -20,7 +20,6 @@
package org.elasticsearch.search.internal;
import org.apache.lucene.search.Explanation;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField;
@ -48,7 +47,6 @@ import org.elasticsearch.search.lookup.SourceLookup;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
@ -67,8 +65,6 @@ import static org.elasticsearch.search.internal.InternalSearchHitField.readSearc
public class InternalSearchHit implements SearchHit {
private static final Object[] EMPTY_SORT_VALUES = new Object[0];
private transient int docId;
private float score = Float.NEGATIVE_INFINITY;
@ -86,7 +82,7 @@ public class InternalSearchHit implements SearchHit {
private Map<String, HighlightField> highlightFields = null;
private Object[] sortValues = EMPTY_SORT_VALUES;
private SearchSortValues sortValues = SearchSortValues.EMPTY;
private String[] matchedQueries = Strings.EMPTY_ARRAY;
@ -343,17 +339,12 @@ public class InternalSearchHit implements SearchHit {
}
public void sortValues(Object[] sortValues, DocValueFormat[] sortValueFormats) {
this.sortValues = Arrays.copyOf(sortValues, sortValues.length);
for (int i = 0; i < sortValues.length; ++i) {
if (this.sortValues[i] instanceof BytesRef) {
this.sortValues[i] = sortValueFormats[i].format((BytesRef) sortValues[i]);
}
}
this.sortValues = new SearchSortValues(sortValues, sortValueFormats);
}
@Override
public Object[] sortValues() {
return sortValues;
return sortValues.sortValues();
}
@Override
@ -499,13 +490,7 @@ public class InternalSearchHit implements SearchHit {
}
builder.endObject();
}
if (sortValues != null && sortValues.length > 0) {
builder.startArray(Fields.SORT);
for (Object sortValue : sortValues) {
builder.value(sortValue);
}
builder.endArray();
}
sortValues.toXContent(builder, params);
if (matchedQueries.length > 0) {
builder.startArray(Fields.MATCHED_QUERIES);
for (String matchedFilter : matchedQueries) {
@ -603,34 +588,7 @@ public class InternalSearchHit implements SearchHit {
this.highlightFields = unmodifiableMap(highlightFields);
}
size = in.readVInt();
if (size > 0) {
sortValues = new Object[size];
for (int i = 0; i < sortValues.length; i++) {
byte type = in.readByte();
if (type == 0) {
sortValues[i] = null;
} else if (type == 1) {
sortValues[i] = in.readString();
} else if (type == 2) {
sortValues[i] = in.readInt();
} else if (type == 3) {
sortValues[i] = in.readLong();
} else if (type == 4) {
sortValues[i] = in.readFloat();
} else if (type == 5) {
sortValues[i] = in.readDouble();
} else if (type == 6) {
sortValues[i] = in.readByte();
} else if (type == 7) {
sortValues[i] = in.readShort();
} else if (type == 8) {
sortValues[i] = in.readBoolean();
} else {
throw new IOException("Can't match type [" + type + "]");
}
}
}
sortValues = new SearchSortValues(in);
size = in.readVInt();
if (size > 0) {
@ -681,46 +639,7 @@ public class InternalSearchHit implements SearchHit {
highlightField.writeTo(out);
}
}
if (sortValues.length == 0) {
out.writeVInt(0);
} else {
out.writeVInt(sortValues.length);
for (Object sortValue : sortValues) {
if (sortValue == null) {
out.writeByte((byte) 0);
} else {
Class type = sortValue.getClass();
if (type == String.class) {
out.writeByte((byte) 1);
out.writeString((String) sortValue);
} else if (type == Integer.class) {
out.writeByte((byte) 2);
out.writeInt((Integer) sortValue);
} else if (type == Long.class) {
out.writeByte((byte) 3);
out.writeLong((Long) sortValue);
} else if (type == Float.class) {
out.writeByte((byte) 4);
out.writeFloat((Float) sortValue);
} else if (type == Double.class) {
out.writeByte((byte) 5);
out.writeDouble((Double) sortValue);
} else if (type == Byte.class) {
out.writeByte((byte) 6);
out.writeByte((Byte) sortValue);
} else if (type == Short.class) {
out.writeByte((byte) 7);
out.writeShort((Short) sortValue);
} else if (type == Boolean.class) {
out.writeByte((byte) 8);
out.writeBoolean((Boolean) sortValue);
} else {
throw new IOException("Can't handle sort field value of type [" + type + "]");
}
}
}
}
sortValues.writeTo(out);
if (matchedQueries.length == 0) {
out.writeVInt(0);

View File

@ -0,0 +1,168 @@
/*
* 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.search.internal;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParserUtils;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.internal.InternalSearchHit.Fields;
import java.io.IOException;
import java.util.Arrays;
import java.util.Objects;
public class SearchSortValues implements ToXContent, Writeable {
static final SearchSortValues EMPTY = new SearchSortValues(new Object[0]);
private final Object[] sortValues;
SearchSortValues(Object[] sortValues) {
this.sortValues = Objects.requireNonNull(sortValues, "sort values must not be empty");
}
public SearchSortValues(Object[] sortValues, DocValueFormat[] sortValueFormats) {
Objects.requireNonNull(sortValues);
Objects.requireNonNull(sortValueFormats);
this.sortValues = Arrays.copyOf(sortValues, sortValues.length);
for (int i = 0; i < sortValues.length; ++i) {
if (this.sortValues[i] instanceof BytesRef) {
this.sortValues[i] = sortValueFormats[i].format((BytesRef) sortValues[i]);
}
}
}
public SearchSortValues(StreamInput in) throws IOException {
int size = in.readVInt();
if (size > 0) {
sortValues = new Object[size];
for (int i = 0; i < sortValues.length; i++) {
byte type = in.readByte();
if (type == 0) {
sortValues[i] = null;
} else if (type == 1) {
sortValues[i] = in.readString();
} else if (type == 2) {
sortValues[i] = in.readInt();
} else if (type == 3) {
sortValues[i] = in.readLong();
} else if (type == 4) {
sortValues[i] = in.readFloat();
} else if (type == 5) {
sortValues[i] = in.readDouble();
} else if (type == 6) {
sortValues[i] = in.readByte();
} else if (type == 7) {
sortValues[i] = in.readShort();
} else if (type == 8) {
sortValues[i] = in.readBoolean();
} else {
throw new IOException("Can't match type [" + type + "]");
}
}
} else {
sortValues = new Object[0];
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(sortValues.length);
for (Object sortValue : sortValues) {
if (sortValue == null) {
out.writeByte((byte) 0);
} else {
Class type = sortValue.getClass();
if (type == String.class) {
out.writeByte((byte) 1);
out.writeString((String) sortValue);
} else if (type == Integer.class) {
out.writeByte((byte) 2);
out.writeInt((Integer) sortValue);
} else if (type == Long.class) {
out.writeByte((byte) 3);
out.writeLong((Long) sortValue);
} else if (type == Float.class) {
out.writeByte((byte) 4);
out.writeFloat((Float) sortValue);
} else if (type == Double.class) {
out.writeByte((byte) 5);
out.writeDouble((Double) sortValue);
} else if (type == Byte.class) {
out.writeByte((byte) 6);
out.writeByte((Byte) sortValue);
} else if (type == Short.class) {
out.writeByte((byte) 7);
out.writeShort((Short) sortValue);
} else if (type == Boolean.class) {
out.writeByte((byte) 8);
out.writeBoolean((Boolean) sortValue);
} else {
throw new IOException("Can't handle sort field value of type [" + type + "]");
}
}
}
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (sortValues.length > 0) {
builder.startArray(Fields.SORT);
for (Object sortValue : sortValues) {
builder.value(sortValue);
}
builder.endArray();
}
return builder;
}
public static SearchSortValues fromXContent(XContentParser parser) throws IOException {
XContentParserUtils.ensureFieldName(parser, parser.currentToken(), Fields.SORT);
XContentParser.Token token = parser.nextToken();
XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_ARRAY, token, parser::getTokenLocation);
return new SearchSortValues(parser.list().toArray());
}
public Object[] sortValues() {
return sortValues;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
SearchSortValues other = (SearchSortValues) obj;
return Arrays.equals(sortValues, other.sortValues);
}
@Override
public int hashCode() {
return Arrays.hashCode(sortValues);
}
}

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.transport;
import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.unit.TimeValue;
import java.util.ArrayList;
@ -35,16 +36,25 @@ import java.util.concurrent.atomic.AtomicInteger;
public final class ConnectionProfile {
/**
* A pre-built light connection profile that shares a single connection across all
* types.
* Builds a connection profile that is dedicated to a single channel type. Use this
* when opening single use connections
*/
public static final ConnectionProfile LIGHT_PROFILE = new ConnectionProfile(
Collections.singletonList(new ConnectionTypeHandle(0, 1, EnumSet.of(
TransportRequestOptions.Type.BULK,
TransportRequestOptions.Type.PING,
TransportRequestOptions.Type.RECOVERY,
TransportRequestOptions.Type.REG,
TransportRequestOptions.Type.STATE))), 1, null, null);
public static ConnectionProfile buildSingleChannelProfile(TransportRequestOptions.Type channelType,
@Nullable TimeValue connectTimeout,
@Nullable TimeValue handshakeTimeout) {
Builder builder = new Builder();
builder.addConnections(1, channelType);
final EnumSet<TransportRequestOptions.Type> otherTypes = EnumSet.allOf(TransportRequestOptions.Type.class);
otherTypes.remove(channelType);
builder.addConnections(0, otherTypes.stream().toArray(TransportRequestOptions.Type[]::new));
if (connectTimeout != null) {
builder.setConnectTimeout(connectTimeout);
}
if (handshakeTimeout != null) {
builder.setHandshakeTimeout(handshakeTimeout);
}
return builder.build();
}
private final List<ConnectionTypeHandle> handles;
private final int numConnections;

View File

@ -63,8 +63,7 @@ public interface Transport extends LifecycleComponent {
boolean nodeConnected(DiscoveryNode node);
/**
* Connects to a node with the given connection profile. Use {@link ConnectionProfile#LIGHT_PROFILE} when just connecting for ping
* and then disconnecting. If the node is already connected this method has no effect
* Connects to a node with the given connection profile. If the node is already connected this method has no effect
*/
void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile) throws ConnectTransportException;

View File

@ -62,7 +62,6 @@ import java.util.Objects;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.function.Supplier;
@ -328,32 +327,6 @@ public class TransportService extends AbstractLifecycleComponent {
}
}
/**
* Lightly connect to the specified node, returning updated node
* information. The handshake will fail if the cluster name on the
* target node mismatches the local cluster name and
* {@code checkClusterName} is {@code true}.
*
* @param node the node to connect to
* @param handshakeTimeout handshake timeout
* @return the connected node
* @throws ConnectTransportException if the connection failed
* @throws IllegalStateException if the handshake failed
*/
public DiscoveryNode connectToNodeAndHandshake(
final DiscoveryNode node,
final long handshakeTimeout) throws IOException {
if (node.equals(localNode)) {
return localNode;
}
DiscoveryNode handshakeNode;
try (Transport.Connection connection = transport.openConnection(node, ConnectionProfile.LIGHT_PROFILE)) {
handshakeNode = handshake(connection, handshakeTimeout);
}
connectToNode(node, ConnectionProfile.LIGHT_PROFILE);
return handshakeNode;
}
/**
* Executes a high-level handshake using the given connection
* and returns the discovery node of the node the connection

View File

@ -20,15 +20,18 @@
package org.elasticsearch.action.search;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.MatchAllQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryParser;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.search.RestMultiSearchAction;
import org.elasticsearch.search.SearchRequestParsers;
@ -38,6 +41,7 @@ import org.elasticsearch.test.rest.FakeRestRequest;
import java.io.IOException;
import static java.util.Collections.singletonList;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.nullValue;
@ -168,13 +172,12 @@ public class MultiSearchRequestTests extends ESTestCase {
private MultiSearchRequest parseMultiSearchRequest(String sample) throws IOException {
byte[] data = StreamsUtils.copyToBytesFromClasspath(sample);
RestRequest restRequest = new FakeRestRequest.Builder(xContentRegistry()).withContent(new BytesArray(data)).build();
return RestMultiSearchAction.parseRequest(restRequest, true, parsers(), ParseFieldMatcher.EMPTY);
return RestMultiSearchAction.parseRequest(restRequest, true, new SearchRequestParsers(null, null, null), ParseFieldMatcher.EMPTY);
}
private SearchRequestParsers parsers() {
IndicesQueriesRegistry registry = new IndicesQueriesRegistry();
QueryParser<MatchAllQueryBuilder> parser = MatchAllQueryBuilder::fromXContent;
registry.register(parser, MatchAllQueryBuilder.NAME);
return new SearchRequestParsers(registry, null, null, null);
@Override
protected NamedXContentRegistry xContentRegistry() {
return new NamedXContentRegistry(singletonList(new NamedXContentRegistry.Entry(QueryBuilder.class,
new ParseField(MatchAllQueryBuilder.NAME), (p, c) -> MatchAllQueryBuilder.fromXContent((QueryParseContext) c))));
}
}

View File

@ -26,6 +26,7 @@ import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNode.Role;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.network.NetworkAddress;
@ -34,18 +35,22 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.VersionUtils;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.ConnectionProfile;
import org.elasticsearch.transport.MockTcpTransport;
import org.elasticsearch.transport.Transport;
import org.elasticsearch.transport.TransportConnectionListener;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.TransportSettings;
import org.junit.After;
@ -60,12 +65,14 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.Stack;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
@ -82,7 +89,6 @@ import static java.util.Collections.emptySet;
import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.hasSize;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.mock;
@ -124,8 +130,7 @@ public class UnicastZenPingTests extends ESTestCase {
private static final UnicastHostsProvider EMPTY_HOSTS_PROVIDER = Collections::emptyList;
@TestLogging("org.elasticsearch.transport:TRACE,org.elasticsearch.discovery.zen.UnicastZenPing:TRACE")
public void testSimplePings() throws IOException, InterruptedException {
public void testSimplePings() throws IOException, InterruptedException, ExecutionException {
// use ephemeral ports
final Settings settings = Settings.builder().put("cluster.name", "test").put(TransportSettings.PORT.getKey(), 0).build();
final Settings settingsMismatch =
@ -140,7 +145,12 @@ public class UnicastZenPingTests extends ESTestCase {
new NoneCircuitBreakerService(),
new NamedWriteableRegistry(Collections.emptyList()),
networkService,
v);
v) {
@Override
public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile) {
throw new AssertionError("zen pings should never connect to node (got [" + node + "])");
}
};
NetworkHandle handleA = startServices(settings, threadPool, "UZP_A", Version.CURRENT, supplier);
closeables.push(handleA.transportService);
@ -148,25 +158,30 @@ public class UnicastZenPingTests extends ESTestCase {
closeables.push(handleB.transportService);
NetworkHandle handleC = startServices(settingsMismatch, threadPool, "UZP_C", Version.CURRENT, supplier);
closeables.push(handleC.transportService);
// just fake that no versions are compatible with this node
Version previousVersion = VersionUtils.getPreviousVersion(Version.CURRENT.minimumCompatibilityVersion());
Version versionD = VersionUtils.randomVersionBetween(random(), previousVersion.minimumCompatibilityVersion(), previousVersion);
final Version versionD;
if (randomBoolean()) {
versionD = VersionUtils.randomVersionBetween(random(), Version.CURRENT.minimumCompatibilityVersion(), Version.CURRENT);
} else {
versionD = Version.CURRENT;
}
logger.info("UZP_D version set to [{}]", versionD);
NetworkHandle handleD = startServices(settingsMismatch, threadPool, "UZP_D", versionD, supplier);
closeables.push(handleD.transportService);
final ClusterState state = ClusterState.builder(new ClusterName("test")).version(randomPositiveLong()).build();
final ClusterState stateMismatch = ClusterState.builder(new ClusterName("mismatch")).version(randomPositiveLong()).build();
Settings hostsSettings = Settings.builder()
.putArray("discovery.zen.ping.unicast.hosts",
.putArray("discovery.zen.ping.unicast.hosts",
NetworkAddress.format(new InetSocketAddress(handleA.address.address().getAddress(), handleA.address.address().getPort())),
NetworkAddress.format(new InetSocketAddress(handleB.address.address().getAddress(), handleB.address.address().getPort())),
NetworkAddress.format(new InetSocketAddress(handleC.address.address().getAddress(), handleC.address.address().getPort())),
NetworkAddress.format(new InetSocketAddress(handleD.address.address().getAddress(), handleD.address.address().getPort())))
.put("cluster.name", "test")
.build();
.put("cluster.name", "test")
.build();
Settings hostsSettingsMismatch = Settings.builder().put(hostsSettings).put(settingsMismatch).build();
UnicastZenPing zenPingA = new UnicastZenPing(hostsSettings, threadPool, handleA.transportService, EMPTY_HOSTS_PROVIDER);
TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, EMPTY_HOSTS_PROVIDER);
zenPingA.start(new PingContextProvider() {
@Override
public DiscoveryNodes nodes() {
@ -180,7 +195,7 @@ public class UnicastZenPingTests extends ESTestCase {
});
closeables.push(zenPingA);
UnicastZenPing zenPingB = new UnicastZenPing(hostsSettings, threadPool, handleB.transportService, EMPTY_HOSTS_PROVIDER);
TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, EMPTY_HOSTS_PROVIDER);
zenPingB.start(new PingContextProvider() {
@Override
public DiscoveryNodes nodes() {
@ -194,7 +209,8 @@ public class UnicastZenPingTests extends ESTestCase {
});
closeables.push(zenPingB);
UnicastZenPing zenPingC = new UnicastZenPing(hostsSettingsMismatch, threadPool, handleC.transportService, EMPTY_HOSTS_PROVIDER) {
TestUnicastZenPing zenPingC = new TestUnicastZenPing(hostsSettingsMismatch, threadPool, handleC,
EMPTY_HOSTS_PROVIDER) {
@Override
protected Version getVersion() {
return versionD;
@ -208,12 +224,13 @@ public class UnicastZenPingTests extends ESTestCase {
@Override
public ClusterState clusterState() {
return state;
return stateMismatch;
}
});
closeables.push(zenPingC);
UnicastZenPing zenPingD = new UnicastZenPing(hostsSettingsMismatch, threadPool, handleD.transportService, EMPTY_HOSTS_PROVIDER);
TestUnicastZenPing zenPingD = new TestUnicastZenPing(hostsSettingsMismatch, threadPool, handleD,
EMPTY_HOSTS_PROVIDER);
zenPingD.start(new PingContextProvider() {
@Override
public DiscoveryNodes nodes() {
@ -222,40 +239,48 @@ public class UnicastZenPingTests extends ESTestCase {
@Override
public ClusterState clusterState() {
return state;
return stateMismatch;
}
});
closeables.push(zenPingD);
logger.info("ping from UZP_A");
Collection<ZenPing.PingResponse> pingResponses = zenPingA.pingAndWait(TimeValue.timeValueMillis(500));
Collection<ZenPing.PingResponse> pingResponses = zenPingA.pingAndWait().toList();
assertThat(pingResponses.size(), equalTo(1));
ZenPing.PingResponse ping = pingResponses.iterator().next();
assertThat(ping.node().getId(), equalTo("UZP_B"));
assertThat(ping.getClusterStateVersion(), equalTo(state.version()));
assertCountersMoreThan(handleA, handleB, handleC, handleD);
assertPingCount(handleA, handleB, 3);
assertPingCount(handleA, handleC, 0); // mismatch, shouldn't ping
assertPingCount(handleA, handleD, 0); // mismatch, shouldn't ping
// ping again, this time from B,
logger.info("ping from UZP_B");
pingResponses = zenPingB.pingAndWait(TimeValue.timeValueMillis(500));
pingResponses = zenPingB.pingAndWait().toList();
assertThat(pingResponses.size(), equalTo(1));
ping = pingResponses.iterator().next();
assertThat(ping.node().getId(), equalTo("UZP_A"));
assertThat(ping.getClusterStateVersion(), equalTo(ElectMasterService.MasterCandidate.UNRECOVERED_CLUSTER_VERSION));
assertCountersMoreThan(handleB, handleA, handleC, handleD);
assertPingCount(handleB, handleA, 3);
assertPingCount(handleB, handleC, 0); // mismatch, shouldn't ping
assertPingCount(handleB, handleD, 0); // mismatch, shouldn't ping
logger.info("ping from UZP_C");
pingResponses = zenPingC.pingAndWait(TimeValue.timeValueMillis(500));
assertThat(pingResponses.size(), equalTo(0));
assertCountersMoreThan(handleC, handleA, handleB, handleD);
pingResponses = zenPingC.pingAndWait().toList();
assertThat(pingResponses.size(), equalTo(1));
assertPingCount(handleC, handleA, 0);
assertPingCount(handleC, handleB, 0);
assertPingCount(handleC, handleD, 3);
logger.info("ping from UZP_D");
pingResponses = zenPingD.pingAndWait(TimeValue.timeValueMillis(500));
assertThat(pingResponses.size(), equalTo(0));
assertCountersMoreThan(handleD, handleA, handleB, handleC);
pingResponses = zenPingD.pingAndWait().toList();
assertThat(pingResponses.size(), equalTo(1));
assertPingCount(handleD, handleA, 0);
assertPingCount(handleD, handleB, 0);
assertPingCount(handleD, handleC, 3);
}
public void testUnknownHostNotCached() {
public void testUnknownHostNotCached() throws ExecutionException, InterruptedException {
// use ephemeral ports
final Settings settings = Settings.builder().put("cluster.name", "test").put(TransportSettings.PORT.getKey(), 0).build();
@ -306,7 +331,7 @@ public class UnicastZenPingTests extends ESTestCase {
final ClusterState state = ClusterState.builder(new ClusterName("test")).version(randomPositiveLong()).build();
final UnicastZenPing zenPingA = new UnicastZenPing(hostsSettings, threadPool, handleA.transportService, EMPTY_HOSTS_PROVIDER);
final TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, EMPTY_HOSTS_PROVIDER);
zenPingA.start(new PingContextProvider() {
@Override
public DiscoveryNodes nodes() {
@ -320,7 +345,7 @@ public class UnicastZenPingTests extends ESTestCase {
});
closeables.push(zenPingA);
UnicastZenPing zenPingB = new UnicastZenPing(hostsSettings, threadPool, handleB.transportService, EMPTY_HOSTS_PROVIDER);
TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, EMPTY_HOSTS_PROVIDER);
zenPingB.start(new PingContextProvider() {
@Override
public DiscoveryNodes nodes() {
@ -334,7 +359,7 @@ public class UnicastZenPingTests extends ESTestCase {
});
closeables.push(zenPingB);
UnicastZenPing zenPingC = new UnicastZenPing(hostsSettings, threadPool, handleC.transportService, EMPTY_HOSTS_PROVIDER);
TestUnicastZenPing zenPingC = new TestUnicastZenPing(hostsSettings, threadPool, handleC, EMPTY_HOSTS_PROVIDER);
zenPingC.start(new PingContextProvider() {
@Override
public DiscoveryNodes nodes() {
@ -350,12 +375,13 @@ public class UnicastZenPingTests extends ESTestCase {
// the presence of an unresolvable host should not prevent resolvable hosts from being pinged
{
final Collection<ZenPing.PingResponse> pingResponses = zenPingA.pingAndWait(TimeValue.timeValueMillis(500));
final Collection<ZenPing.PingResponse> pingResponses = zenPingA.pingAndWait().toList();
assertThat(pingResponses.size(), equalTo(1));
ZenPing.PingResponse ping = pingResponses.iterator().next();
assertThat(ping.node().getId(), equalTo("UZP_C"));
assertThat(ping.getClusterStateVersion(), equalTo(state.version()));
assertCountersMoreThan(handleA, handleC);
assertPingCount(handleA, handleB, 0);
assertPingCount(handleA, handleC, 3);
assertNull(handleA.counters.get(handleB.address));
}
@ -373,11 +399,13 @@ public class UnicastZenPingTests extends ESTestCase {
// now we should see pings to UZP_B; this establishes that host resolutions are not cached
{
final Collection<ZenPing.PingResponse> secondPingResponses = zenPingA.pingAndWait(TimeValue.timeValueMillis(500));
handleA.counters.clear();
final Collection<ZenPing.PingResponse> secondPingResponses = zenPingA.pingAndWait().toList();
assertThat(secondPingResponses.size(), equalTo(2));
final Set<String> ids = new HashSet<>(secondPingResponses.stream().map(p -> p.node().getId()).collect(Collectors.toList()));
assertThat(ids, equalTo(new HashSet<>(Arrays.asList("UZP_B", "UZP_C"))));
assertCountersMoreThan(moreThan, handleA, handleB, handleC);
assertPingCount(handleA, handleB, 3);
assertPingCount(handleA, handleC, 3);
}
}
@ -395,15 +423,14 @@ public class UnicastZenPingTests extends ESTestCase {
final TransportService transportService =
new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, null);
closeables.push(transportService);
final AtomicInteger idGenerator = new AtomicInteger();
final int limitPortCounts = randomIntBetween(1, 10);
final List<DiscoveryNode> discoveryNodes = UnicastZenPing.resolveDiscoveryNodes(
final List<DiscoveryNode> discoveryNodes = TestUnicastZenPing.resolveHostsLists(
executorService,
logger,
Collections.singletonList("127.0.0.1"),
limitPortCounts,
transportService,
() -> Integer.toString(idGenerator.incrementAndGet()),
"test_",
TimeValue.timeValueSeconds(1));
assertThat(discoveryNodes, hasSize(limitPortCounts));
final Set<Integer> ports = new HashSet<>();
@ -439,15 +466,14 @@ public class UnicastZenPingTests extends ESTestCase {
final TransportService transportService =
new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, null);
closeables.push(transportService);
final AtomicInteger idGenerator = new AtomicInteger();
final List<DiscoveryNode> discoveryNodes = UnicastZenPing.resolveDiscoveryNodes(
final List<DiscoveryNode> discoveryNodes = TestUnicastZenPing.resolveHostsLists(
executorService,
logger,
Arrays.asList(hostname),
1,
transportService,
() -> Integer.toString(idGenerator.incrementAndGet()),
"test_",
TimeValue.timeValueSeconds(1)
);
@ -490,16 +516,15 @@ public class UnicastZenPingTests extends ESTestCase {
final TransportService transportService =
new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, null);
closeables.push(transportService);
final AtomicInteger idGenerator = new AtomicInteger();
final TimeValue resolveTimeout = TimeValue.timeValueSeconds(randomIntBetween(1, 3));
try {
final List<DiscoveryNode> discoveryNodes = UnicastZenPing.resolveDiscoveryNodes(
final List<DiscoveryNode> discoveryNodes = TestUnicastZenPing.resolveHostsLists(
executorService,
logger,
Arrays.asList("hostname1", "hostname2"),
1,
transportService,
() -> Integer.toString(idGenerator.incrementAndGet()),
"test+",
resolveTimeout);
assertThat(discoveryNodes, hasSize(1));
@ -513,6 +538,156 @@ public class UnicastZenPingTests extends ESTestCase {
}
}
public void testResolveReuseExistingNodeConnections() throws ExecutionException, InterruptedException {
final Settings settings = Settings.builder().put("cluster.name", "test").put(TransportSettings.PORT.getKey(), 0).build();
NetworkService networkService = new NetworkService(settings, Collections.emptyList());
final BiFunction<Settings, Version, Transport> supplier = (s, v) -> new MockTcpTransport(
s,
threadPool,
BigArrays.NON_RECYCLING_INSTANCE,
new NoneCircuitBreakerService(),
new NamedWriteableRegistry(Collections.emptyList()),
networkService,
v);
NetworkHandle handleA = startServices(settings, threadPool, "UZP_A", Version.CURRENT, supplier, EnumSet.allOf(Role.class));
closeables.push(handleA.transportService);
NetworkHandle handleB = startServices(settings, threadPool, "UZP_B", Version.CURRENT, supplier, EnumSet.allOf(Role.class));
closeables.push(handleB.transportService);
final boolean useHosts = randomBoolean();
final Settings.Builder hostsSettingsBuilder = Settings.builder().put("cluster.name", "test");
if (useHosts) {
hostsSettingsBuilder.putArray("discovery.zen.ping.unicast.hosts",
NetworkAddress.format(new InetSocketAddress(handleB.address.address().getAddress(), handleB.address.address().getPort()))
);
} else {
hostsSettingsBuilder.put("discovery.zen.ping.unicast.hosts", (String) null);
}
final Settings hostsSettings = hostsSettingsBuilder.build();
final ClusterState state = ClusterState.builder(new ClusterName("test")).version(randomPositiveLong()).build();
// connection to reuse
handleA.transportService.connectToNode(handleB.node);
// install a listener to check that no new connections are made
handleA.transportService.addConnectionListener(new TransportConnectionListener() {
@Override
public void onConnectionOpened(DiscoveryNode node) {
fail("should not open any connections. got [" + node + "]");
}
});
final TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, EMPTY_HOSTS_PROVIDER);
zenPingA.start(new PingContextProvider() {
@Override
public DiscoveryNodes nodes() {
return DiscoveryNodes.builder().add(handleA.node).add(handleB.node).localNodeId("UZP_A").build();
}
@Override
public ClusterState clusterState() {
return ClusterState.builder(state).blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)).build();
}
});
closeables.push(zenPingA);
TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, EMPTY_HOSTS_PROVIDER);
zenPingB.start(new PingContextProvider() {
@Override
public DiscoveryNodes nodes() {
return DiscoveryNodes.builder().add(handleB.node).localNodeId("UZP_B").build();
}
@Override
public ClusterState clusterState() {
return state;
}
});
closeables.push(zenPingB);
Collection<ZenPing.PingResponse> pingResponses = zenPingA.pingAndWait().toList();
assertThat(pingResponses.size(), equalTo(1));
ZenPing.PingResponse ping = pingResponses.iterator().next();
assertThat(ping.node().getId(), equalTo("UZP_B"));
assertThat(ping.getClusterStateVersion(), equalTo(state.version()));
}
public void testPingingTemporalPings() throws ExecutionException, InterruptedException {
final Settings settings = Settings.builder().put("cluster.name", "test").put(TransportSettings.PORT.getKey(), 0).build();
NetworkService networkService = new NetworkService(settings, Collections.emptyList());
final BiFunction<Settings, Version, Transport> supplier = (s, v) -> new MockTcpTransport(
s,
threadPool,
BigArrays.NON_RECYCLING_INSTANCE,
new NoneCircuitBreakerService(),
new NamedWriteableRegistry(Collections.emptyList()),
networkService,
v);
NetworkHandle handleA = startServices(settings, threadPool, "UZP_A", Version.CURRENT, supplier, EnumSet.allOf(Role.class));
closeables.push(handleA.transportService);
NetworkHandle handleB = startServices(settings, threadPool, "UZP_B", Version.CURRENT, supplier, EnumSet.allOf(Role.class));
closeables.push(handleB.transportService);
final Settings hostsSettings = Settings.builder()
.put("cluster.name", "test")
.put("discovery.zen.ping.unicast.hosts", (String) null) // use nodes for simplicity
.build();
final ClusterState state = ClusterState.builder(new ClusterName("test")).version(randomPositiveLong()).build();
final TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, EMPTY_HOSTS_PROVIDER);
zenPingA.start(new PingContextProvider() {
@Override
public DiscoveryNodes nodes() {
return DiscoveryNodes.builder().add(handleA.node).add(handleB.node).localNodeId("UZP_A").build();
}
@Override
public ClusterState clusterState() {
return ClusterState.builder(state).blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)).build();
}
});
closeables.push(zenPingA);
// Node B doesn't know about A!
TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, EMPTY_HOSTS_PROVIDER);
zenPingB.start(new PingContextProvider() {
@Override
public DiscoveryNodes nodes() {
return DiscoveryNodes.builder().add(handleB.node).localNodeId("UZP_B").build();
}
@Override
public ClusterState clusterState() {
return state;
}
});
closeables.push(zenPingB);
{
logger.info("pinging from UZP_A so UZP_B will learn about it");
Collection<ZenPing.PingResponse> pingResponses = zenPingA.pingAndWait().toList();
assertThat(pingResponses.size(), equalTo(1));
ZenPing.PingResponse ping = pingResponses.iterator().next();
assertThat(ping.node().getId(), equalTo("UZP_B"));
assertThat(ping.getClusterStateVersion(), equalTo(state.version()));
}
{
logger.info("pinging from UZP_B");
Collection<ZenPing.PingResponse> pingResponses = zenPingB.pingAndWait().toList();
assertThat(pingResponses.size(), equalTo(1));
ZenPing.PingResponse ping = pingResponses.iterator().next();
assertThat(ping.node().getId(), equalTo("UZP_A"));
assertThat(ping.getClusterStateVersion(), equalTo(-1L)); // A has a block
}
}
public void testInvalidHosts() throws InterruptedException {
final Logger logger = mock(Logger.class);
final NetworkService networkService = new NetworkService(Settings.EMPTY, Collections.emptyList());
@ -529,14 +704,13 @@ public class UnicastZenPingTests extends ESTestCase {
final TransportService transportService =
new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, null);
closeables.push(transportService);
final AtomicInteger idGenerator = new AtomicInteger();
final List<DiscoveryNode> discoveryNodes = UnicastZenPing.resolveDiscoveryNodes(
final List<DiscoveryNode> discoveryNodes = TestUnicastZenPing.resolveHostsLists(
executorService,
logger,
Arrays.asList("127.0.0.1:9300:9300", "127.0.0.1:9301"),
1,
transportService,
() -> Integer.toString(idGenerator.incrementAndGet()),
"test_",
TimeValue.timeValueSeconds(1));
assertThat(discoveryNodes, hasSize(1)); // only one of the two is valid and will be used
assertThat(discoveryNodes.get(0).getAddress().getAddress(), equalTo("127.0.0.1"));
@ -544,24 +718,13 @@ public class UnicastZenPingTests extends ESTestCase {
verify(logger).warn(eq("failed to resolve host [127.0.0.1:9300:9300]"), Matchers.any(ExecutionException.class));
}
// assert that we tried to ping each of the configured nodes at least once
private void assertCountersMoreThan(final NetworkHandle that, final NetworkHandle...handles) {
final HashMap<TransportAddress, Integer> moreThan = new HashMap<>();
for (final NetworkHandle handle : handles) {
assert handle != that;
moreThan.put(handle.address, 0);
}
assertCountersMoreThan(moreThan, that, handles);
}
private void assertCountersMoreThan(
final Map<TransportAddress, Integer> moreThan,
final NetworkHandle that,
final NetworkHandle... handles) {
for (final NetworkHandle handle : handles) {
assert handle != that;
assertThat(that.counters.get(handle.address).get(), greaterThan(moreThan.get(handle.address)));
}
private void assertPingCount(final NetworkHandle fromNode, final NetworkHandle toNode, int expectedCount) {
final AtomicInteger counter = fromNode.counters.getOrDefault(toNode.address, new AtomicInteger());
final String onNodeName = fromNode.node.getName();
assertNotNull("handle for [" + onNodeName + "] has no 'expected' counter", counter);
final String forNodeName = toNode.node.getName();
assertThat("node [" + onNodeName + "] ping count to [" + forNodeName + "] is unexpected",
counter.get(), equalTo(expectedCount));
}
private NetworkHandle startServices(
@ -570,31 +733,36 @@ public class UnicastZenPingTests extends ESTestCase {
final String nodeId,
final Version version,
final BiFunction<Settings, Version, Transport> supplier) {
final Transport transport = supplier.apply(settings, version);
final TransportService transportService =
new MockTransportService(settings, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, null);
return startServices(settings, threadPool, nodeId, version, supplier, emptySet());
}
private NetworkHandle startServices(
final Settings settings,
final ThreadPool threadPool,
final String nodeId,
final Version version,
final BiFunction<Settings, Version, Transport> supplier,
final Set<Role> nodeRoles) {
final Settings nodeSettings = Settings.builder().put(settings)
.put("node.name", nodeId)
.put(TransportService.TRACE_LOG_INCLUDE_SETTING.getKey(), "internal:discovery/zen/unicast")
.build();
final Transport transport = supplier.apply(nodeSettings, version);
final MockTransportService transportService =
new MockTransportService(nodeSettings, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, null);
transportService.start();
transportService.acceptIncomingRequests();
final ConcurrentMap<TransportAddress, AtomicInteger> counters = ConcurrentCollections.newConcurrentMap();
transportService.addConnectionListener(new TransportConnectionListener() {
transportService.addTracer(new MockTransportService.Tracer() {
@Override
public void onNodeConnected(DiscoveryNode node) {
}
@Override
public void onConnectionOpened(DiscoveryNode node) {
public void requestSent(DiscoveryNode node, long requestId, String action, TransportRequestOptions options) {
counters.computeIfAbsent(node.getAddress(), k -> new AtomicInteger());
counters.get(node.getAddress()).incrementAndGet();
}
@Override
public void onNodeDisconnected(DiscoveryNode node) {
}
});
final DiscoveryNode node =
new DiscoveryNode(nodeId, transportService.boundAddress().publishAddress(), emptyMap(), emptySet(), version);
new DiscoveryNode(nodeId, nodeId, transportService.boundAddress().publishAddress(), emptyMap(), nodeRoles, version);
transportService.setLocalNode(node);
return new NetworkHandle(transport.boundAddress().publishAddress(), transportService, node, counters);
}
@ -616,7 +784,123 @@ public class UnicastZenPingTests extends ESTestCase {
this.node = discoveryNode;
this.counters = counters;
}
}
private static class TestUnicastZenPing extends UnicastZenPing {
public TestUnicastZenPing(Settings settings, ThreadPool threadPool, NetworkHandle networkHandle,
UnicastHostsProvider unicastHostsProvider) {
super(Settings.builder().put("node.name", networkHandle.node.getName()).put(settings).build(),
threadPool, networkHandle.transportService, unicastHostsProvider);
}
volatile CountDownLatch allTasksCompleted;
volatile AtomicInteger pendingTasks;
PingCollection pingAndWait() throws ExecutionException, InterruptedException {
allTasksCompleted = new CountDownLatch(1);
pendingTasks = new AtomicInteger();
// make the three sending rounds to come as started
markTaskAsStarted("send pings");
markTaskAsStarted("send pings");
markTaskAsStarted("send pings");
final CompletableFuture<PingCollection> response = new CompletableFuture<>();
try {
ping(response::complete, TimeValue.timeValueMillis(1), TimeValue.timeValueSeconds(1));
} catch (Exception ex) {
response.completeExceptionally(ex);
}
return response.get();
}
@Override
protected void finishPingingRound(PingingRound pingingRound) {
// wait for all activity to finish before closing
try {
allTasksCompleted.await();
} catch (InterruptedException e) {
// ok, finish anyway
}
super.finishPingingRound(pingingRound);
}
@Override
protected void sendPings(TimeValue timeout, PingingRound pingingRound) {
super.sendPings(timeout, pingingRound);
markTaskAsCompleted("send pings");
}
@Override
protected void submitToExecutor(AbstractRunnable abstractRunnable) {
markTaskAsStarted("executor runnable");
super.submitToExecutor(new AbstractRunnable() {
@Override
public void onRejection(Exception e) {
try {
super.onRejection(e);
} finally {
markTaskAsCompleted("executor runnable (rejected)");
}
}
@Override
public void onAfter() {
markTaskAsCompleted("executor runnable");
}
@Override
protected void doRun() throws Exception {
abstractRunnable.run();
}
@Override
public void onFailure(Exception e) {
// we shouldn't really end up here.
throw new AssertionError("unexpected error", e);
}
});
}
private void markTaskAsStarted(String task) {
logger.trace("task [{}] started. count [{}]", task, pendingTasks.incrementAndGet());
}
private void markTaskAsCompleted(String task) {
final int left = pendingTasks.decrementAndGet();
logger.trace("task [{}] completed. count [{}]", task, left);
if (left == 0) {
allTasksCompleted.countDown();
}
}
@Override
protected TransportResponseHandler<UnicastPingResponse> getPingResponseHandler(PingingRound pingingRound, DiscoveryNode node) {
markTaskAsStarted("ping [" + node + "]");
TransportResponseHandler<UnicastPingResponse> original = super.getPingResponseHandler(pingingRound, node);
return new TransportResponseHandler<UnicastPingResponse>() {
@Override
public UnicastPingResponse newInstance() {
return original.newInstance();
}
@Override
public void handleResponse(UnicastPingResponse response) {
original.handleResponse(response);
markTaskAsCompleted("ping [" + node + "]");
}
@Override
public void handleException(TransportException exp) {
original.handleException(exp);
markTaskAsCompleted("ping [" + node + "] (error)");
}
@Override
public String executor() {
return original.executor();
}
};
}
}
}

View File

@ -68,7 +68,7 @@ public class ZenPingTests extends ESTestCase {
Collections.shuffle(pings, random());
ZenPing.PingCollection collection = new ZenPing.PingCollection();
collection.addPings(pings);
pings.forEach(collection::addPing);
List<ZenPing.PingResponse> aggregate = collection.toList();

View File

@ -64,7 +64,6 @@ import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.indices.mapper.MapperRegistry;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.ScriptContextRegistry;
import org.elasticsearch.script.ScriptEngineRegistry;
import org.elasticsearch.script.ScriptService;
@ -110,7 +109,6 @@ public class IndexModuleTests extends ESTestCase {
private CircuitBreakerService circuitBreakerService;
private BigArrays bigArrays;
private ScriptService scriptService;
private IndicesQueriesRegistry indicesQueriesRegistry;
private ClusterService clusterService;
@Override
@ -130,7 +128,6 @@ public class IndexModuleTests extends ESTestCase {
ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry);
scriptService = new ScriptService(settings, environment, new ResourceWatcherService(settings, threadPool), scriptEngineRegistry,
scriptContextRegistry, scriptSettings);
indicesQueriesRegistry = new IndicesQueriesRegistry();
clusterService = ClusterServiceUtils.createClusterService(threadPool);
nodeEnvironment = new NodeEnvironment(settings, environment);
mapperRegistry = new IndicesModule(Collections.emptyList()).getMapperRegistry();
@ -145,7 +142,7 @@ public class IndexModuleTests extends ESTestCase {
private IndexService newIndexService(IndexModule module) throws IOException {
return module.newIndexService(nodeEnvironment, xContentRegistry(), deleter, circuitBreakerService, bigArrays, threadPool,
scriptService, indicesQueriesRegistry, clusterService, null, indicesQueryCache, mapperRegistry, shardId -> {},
scriptService, clusterService, null, indicesQueryCache, mapperRegistry, shardId -> {},
new IndicesFieldDataCache(settings, listener));
}

View File

@ -71,7 +71,7 @@ public class DateFieldTypeTests extends FieldTypeTestCase {
}
public void testIsFieldWithinQueryEmptyReader() throws IOException {
QueryRewriteContext context = new QueryRewriteContext(null, null, null, xContentRegistry(), null, null, null,
QueryRewriteContext context = new QueryRewriteContext(null, null, null, xContentRegistry(), null, null,
() -> nowInMillis);
IndexReader reader = new MultiReader();
DateFieldType ft = new DateFieldType();
@ -82,7 +82,7 @@ public class DateFieldTypeTests extends FieldTypeTestCase {
private void doTestIsFieldWithinQuery(DateFieldType ft, DirectoryReader reader,
DateTimeZone zone, DateMathParser alternateFormat) throws IOException {
QueryRewriteContext context = new QueryRewriteContext(null, null, null, xContentRegistry(), null, null, null,
QueryRewriteContext context = new QueryRewriteContext(null, null, null, xContentRegistry(), null, null,
() -> nowInMillis);
assertEquals(Relation.INTERSECTS, ft.isFieldWithinQuery(reader, "2015-10-09", "2016-01-02",
randomBoolean(), randomBoolean(), null, null, context));
@ -130,7 +130,7 @@ public class DateFieldTypeTests extends FieldTypeTestCase {
DateFieldType ft2 = new DateFieldType();
ft2.setName("my_date2");
QueryRewriteContext context = new QueryRewriteContext(null, null, null, xContentRegistry(), null, null, null,
QueryRewriteContext context = new QueryRewriteContext(null, null, null, xContentRegistry(), null, null,
() -> nowInMillis);
assertEquals(Relation.DISJOINT, ft2.isFieldWithinQuery(reader, "2015-10-09", "2016-01-02", false, false, null, null, context));
IOUtils.close(reader, w, dir);
@ -166,7 +166,7 @@ public class DateFieldTypeTests extends FieldTypeTestCase {
QueryShardContext context = new QueryShardContext(0,
new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(),
indexSettings),
null, null, null, null, null, xContentRegistry(), null, null, null, () -> nowInMillis);
null, null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis);
MappedFieldType ft = createDefaultFieldType();
ft.setName("field");
String date = "2015-10-12T14:10:55";
@ -185,7 +185,7 @@ public class DateFieldTypeTests extends FieldTypeTestCase {
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).build();
QueryShardContext context = new QueryShardContext(0,
new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings),
null, null, null, null, null, xContentRegistry(), null, null, null, () -> nowInMillis);
null, null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis);
MappedFieldType ft = createDefaultFieldType();
ft.setName("field");
String date1 = "2015-10-12T14:10:55";

View File

@ -50,14 +50,15 @@ public class DynamicTemplateTests extends ESTestCase {
templateDef.put("mapping", Collections.singletonMap("store", true));
// if a wrong match type is specified, we ignore the template
assertNull(DynamicTemplate.parse("my_template", templateDef, Version.V_5_0_0_alpha5));
assertWarnings("match_mapping_type [short] is invalid and will be ignored: No field type matched on [short], " +
"possible values are [object, string, long, double, boolean, date, binary]");
Map<String, Object> templateDef2 = new HashMap<>();
templateDef2.put("match_mapping_type", "text");
templateDef2.put("mapping", Collections.singletonMap("store", true));
// if a wrong match type is specified, we ignore the template
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
() -> DynamicTemplate.parse("my_template", templateDef2, Version.V_6_0_0_alpha1_UNRELEASED));
assertEquals("No xcontent type matched on [text], possible values are [object, string, long, double, boolean, date, binary]",
assertEquals("No field type matched on [text], possible values are [object, string, long, double, boolean, date, binary]",
e.getMessage());
}

View File

@ -149,4 +149,20 @@ public class NumberFieldTypeTests extends FieldTypeTestCase {
}
IOUtils.close(reader, dir);
}
public void testNegativeZero() {
assertEquals(
NumberType.DOUBLE.rangeQuery("field", null, -0d, true, true),
NumberType.DOUBLE.rangeQuery("field", null, +0d, true, false));
assertEquals(
NumberType.FLOAT.rangeQuery("field", null, -0f, true, true),
NumberType.FLOAT.rangeQuery("field", null, +0f, true, false));
assertEquals(
NumberType.HALF_FLOAT.rangeQuery("field", null, -0f, true, true),
NumberType.HALF_FLOAT.rangeQuery("field", null, +0f, true, false));
assertFalse(NumberType.DOUBLE.termQuery("field", -0d).equals(NumberType.DOUBLE.termQuery("field", +0d)));
assertFalse(NumberType.FLOAT.termQuery("field", -0f).equals(NumberType.FLOAT.termQuery("field", +0f)));
assertFalse(NumberType.HALF_FLOAT.termQuery("field", -0f).equals(NumberType.HALF_FLOAT.termQuery("field", +0f)));
}
}

View File

@ -74,7 +74,7 @@ public class RangeFieldTypeTests extends FieldTypeTestCase {
Settings indexSettings = Settings.builder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAsciiOfLengthBetween(1, 10), indexSettings);
QueryShardContext context = new QueryShardContext(0, idxSettings, null, null, null, null, null, xContentRegistry(), null,
QueryShardContext context = new QueryShardContext(0, idxSettings, null, null, null, null, null, xContentRegistry(),
null, null, () -> nowInMillis);
RangeFieldMapper.RangeFieldType ft = new RangeFieldMapper.RangeFieldType(type);
ft.setName(FIELDNAME);

View File

@ -30,7 +30,6 @@ import org.elasticsearch.common.xcontent.XContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.test.AbstractQueryTestCase;
import org.hamcrest.Matchers;

View File

@ -22,13 +22,13 @@ import org.apache.lucene.search.join.ScoreMode;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.query.functionscore.FunctionScoreQueryBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptType;
import org.elasticsearch.search.SearchModule;
@ -62,19 +62,24 @@ public class InnerHitBuilderTests extends ESTestCase {
private static final int NUMBER_OF_TESTBUILDERS = 20;
private static NamedWriteableRegistry namedWriteableRegistry;
private static IndicesQueriesRegistry indicesQueriesRegistry;
private static NamedXContentRegistry xContentRegistry;
@BeforeClass
public static void init() {
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList());
namedWriteableRegistry = new NamedWriteableRegistry(searchModule.getNamedWriteables());
indicesQueriesRegistry = searchModule.getQueryParserRegistry();
xContentRegistry = new NamedXContentRegistry(searchModule.getNamedXContents());
}
@AfterClass
public static void afterClass() throws Exception {
namedWriteableRegistry = null;
indicesQueriesRegistry = null;
xContentRegistry = null;
}
@Override
protected NamedXContentRegistry xContentRegistry() {
return xContentRegistry;
}
public void testSerialization() throws Exception {
@ -98,7 +103,7 @@ public class InnerHitBuilderTests extends ESTestCase {
}
XContentParser parser = createParser(shuffled);
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.EMPTY);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.EMPTY);
InnerHitBuilder secondInnerHits = InnerHitBuilder.fromXContent(context);
assertThat(innerHit, not(sameInstance(secondInnerHits)));
assertThat(innerHit, equalTo(secondInnerHits));

View File

@ -24,12 +24,13 @@ import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.test.ESTestCase;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
@ -38,12 +39,16 @@ import java.io.IOException;
import static java.util.Collections.emptyList;
public class QueryParseContextTests extends ESTestCase {
private static IndicesQueriesRegistry indicesQueriesRegistry;
private static NamedXContentRegistry xContentRegistry;
@BeforeClass
public static void init() {
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, false, emptyList()).getQueryParserRegistry();
xContentRegistry = new NamedXContentRegistry(new SearchModule(Settings.EMPTY, false, emptyList()).getNamedXContents());
}
@AfterClass
public static void cleanup() {
xContentRegistry = null;
}
private ThreadContext threadContext;
@ -64,7 +69,7 @@ public class QueryParseContextTests extends ESTestCase {
QueryBuilder query = new MatchQueryBuilder("foo", "bar");
String requestBody = "{ \"query\" : " + query.toString() + "}";
try (XContentParser parser = createParser(JsonXContent.jsonXContent, requestBody)) {
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
QueryBuilder actual = context.parseTopLevelQueryBuilder();
assertEquals(query, actual);
}
@ -73,7 +78,7 @@ public class QueryParseContextTests extends ESTestCase {
public void testParseTopLevelBuilderEmptyObject() throws IOException {
String requestBody = "{}";
try (XContentParser parser = createParser(JsonXContent.jsonXContent, requestBody)) {
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
QueryBuilder query = context.parseTopLevelQueryBuilder();
assertNull(query);
}
@ -82,7 +87,7 @@ public class QueryParseContextTests extends ESTestCase {
public void testParseTopLevelBuilderUnknownParameter() throws IOException {
String requestBody = "{ \"foo\" : \"bar\"}";
try (XContentParser parser = createParser(JsonXContent.jsonXContent, requestBody)) {
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
ParsingException exception = expectThrows(ParsingException.class, () -> context.parseTopLevelQueryBuilder());
assertEquals("request does not support [foo]", exception.getMessage());
}
@ -92,7 +97,7 @@ public class QueryParseContextTests extends ESTestCase {
QueryBuilder query = new MatchQueryBuilder("foo", "bar");
String source = query.toString();
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
QueryBuilder actual = context.parseInnerQueryBuilder();
assertEquals(query, actual);
}
@ -103,30 +108,35 @@ public class QueryParseContextTests extends ESTestCase {
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
parser.nextToken();
parser.nextToken(); // don't start with START_OBJECT to provoke exception
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
ParsingException exception = expectThrows(ParsingException.class, () -> context.parseInnerQueryBuilder());
assertEquals("[_na] query malformed, must start with start_object", exception.getMessage());
}
source = "{}";
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.EMPTY);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.EMPTY);
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> context.parseInnerQueryBuilder());
assertEquals("query malformed, empty clause found at [1:2]", exception.getMessage());
}
source = "{ \"foo\" : \"bar\" }";
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
ParsingException exception = expectThrows(ParsingException.class, () -> context.parseInnerQueryBuilder());
assertEquals("[foo] query malformed, no start_object after query name", exception.getMessage());
}
source = "{ \"foo\" : {} }";
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
ParsingException exception = expectThrows(ParsingException.class, () -> context.parseInnerQueryBuilder());
assertEquals("no [query] registered for [foo]", exception.getMessage());
}
}
@Override
protected NamedXContentRegistry xContentRegistry() {
return xContentRegistry;
}
}

View File

@ -48,7 +48,7 @@ public class QueryShardContextTests extends ESTestCase {
when(mapperService.getIndexSettings()).thenReturn(indexSettings);
final long nowInMillis = randomPositiveLong();
QueryShardContext context = new QueryShardContext(
0, indexSettings, null, null, mapperService, null, null, xContentRegistry(), null, null, null,
0, indexSettings, null, null, mapperService, null, null, xContentRegistry(), null, null,
() -> nowInMillis);
context.setAllowUnmappedFields(false);

View File

@ -37,7 +37,7 @@ public class RangeQueryRewriteTests extends ESSingleNodeTestCase {
IndexService indexService = createIndex("test");
IndexReader reader = new MultiReader();
QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, indexService.mapperService(),
null, null, xContentRegistry(), null, null, reader, null);
null, null, xContentRegistry(), null, reader, null);
RangeQueryBuilder range = new RangeQueryBuilder("foo");
assertEquals(Relation.DISJOINT, range.getRelation(context));
}
@ -54,7 +54,7 @@ public class RangeQueryRewriteTests extends ESSingleNodeTestCase {
indexService.mapperService().merge("type",
new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE, false);
QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, indexService.mapperService(),
null, null, xContentRegistry(), null, null, null, null);
null, null, xContentRegistry(), null, null, null);
RangeQueryBuilder range = new RangeQueryBuilder("foo");
// can't make assumptions on a missing reader, so it must return INTERSECT
assertEquals(Relation.INTERSECTS, range.getRelation(context));
@ -73,7 +73,7 @@ public class RangeQueryRewriteTests extends ESSingleNodeTestCase {
new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE, false);
IndexReader reader = new MultiReader();
QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, indexService.mapperService(),
null, null, xContentRegistry(), null, null, reader, null);
null, null, xContentRegistry(), null, reader, null);
RangeQueryBuilder range = new RangeQueryBuilder("foo");
// no values -> DISJOINT
assertEquals(Relation.DISJOINT, range.getRelation(context));

View File

@ -34,37 +34,15 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MockFieldMapper;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.test.ESTestCase;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import static java.util.Collections.emptyList;
import static org.hamcrest.Matchers.equalTo;
public class SimpleQueryParserTests extends ESTestCase {
private static IndicesQueriesRegistry indicesQueriesRegistry;
/**
* setup for the whole base test class
*/
@BeforeClass
public static void init() {
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList());
indicesQueriesRegistry = searchModule.getQueryParserRegistry();
}
@AfterClass
public static void afterClass() throws Exception {
indicesQueriesRegistry = null;
}
private static class MockSimpleQueryParser extends SimpleQueryParser {
public MockSimpleQueryParser(Analyzer analyzer, Map<String, Float> weights, int flags, Settings settings) {
super(analyzer, weights, flags, settings, null);
@ -148,7 +126,7 @@ public class SimpleQueryParserTests extends ESTestCase {
IndexMetaData indexState = IndexMetaData.builder("index").settings(indexSettings).build();
IndexSettings settings = new IndexSettings(indexState, Settings.EMPTY);
QueryShardContext mockShardContext = new QueryShardContext(0, settings, null, null, null, null, null, xContentRegistry(),
indicesQueriesRegistry, null, null, System::currentTimeMillis) {
null, null, System::currentTimeMillis) {
@Override
public MappedFieldType fieldMapper(String name) {
return new MockFieldMapper.FakeFieldType();
@ -161,7 +139,7 @@ public class SimpleQueryParserTests extends ESTestCase {
assertEquals(new TermQuery(new Term("foo.quote", "bar")), parser.parse("\"bar\""));
// Now check what happens if foo.quote does not exist
mockShardContext = new QueryShardContext(0, settings, null, null, null, null, null, xContentRegistry(), indicesQueriesRegistry,
mockShardContext = new QueryShardContext(0, settings, null, null, null, null, null, xContentRegistry(),
null, null, System::currentTimeMillis) {
@Override
public MappedFieldType fieldMapper(String name) {

View File

@ -26,7 +26,6 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.RoutingNode;
@ -45,7 +44,7 @@ import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.recovery.PeerRecoverySourceService;
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
import org.elasticsearch.node.Node;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase;
@ -56,8 +55,6 @@ import org.elasticsearch.test.disruption.BlockClusterStateProcessing;
import org.elasticsearch.test.disruption.SingleNodeDisruption;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.Transport;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportService;
@ -480,18 +477,13 @@ public class IndicesStoreIntegrationIT extends ESIntegTestCase {
@Override
public void receivedRequest(long requestId, String action) {
if (action.equals(IndicesStore.ACTION_SHARD_EXISTS)) {
if (action.equals(PeerRecoveryTargetService.Actions.FILES_INFO)) {
logger.info("received: {}, relocation starts", action);
beginRelocationLatch.countDown();
} else if (action.equals(IndicesStore.ACTION_SHARD_EXISTS)) {
receivedShardExistsRequestLatch.countDown();
logger.info("received: {}, relocation done", action);
}
}
@Override
public void requestSent(DiscoveryNode node, long requestId, String action, TransportRequestOptions options) {
if (action.equals(PeerRecoverySourceService.Actions.START_RECOVERY)) {
logger.info("sent: {}, relocation starts", action);
beginRelocationLatch.countDown();
}
}
}
}

View File

@ -25,10 +25,10 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.indices.IndicesModule;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.plugins.SearchPlugin;
import org.elasticsearch.search.builder.SearchSourceBuilder;
@ -54,7 +54,7 @@ public abstract class AbstractSearchTestCase extends ESTestCase {
protected NamedWriteableRegistry namedWriteableRegistry;
protected SearchRequestParsers searchRequestParsers;
private TestSearchExtPlugin searchExtPlugin;
protected IndicesQueriesRegistry queriesRegistry;
private NamedXContentRegistry xContentRegistry;
public void setUp() throws Exception {
super.setUp();
@ -65,8 +65,13 @@ public abstract class AbstractSearchTestCase extends ESTestCase {
entries.addAll(indicesModule.getNamedWriteables());
entries.addAll(searchModule.getNamedWriteables());
namedWriteableRegistry = new NamedWriteableRegistry(entries);
xContentRegistry = new NamedXContentRegistry(searchModule.getNamedXContents());
searchRequestParsers = searchModule.getSearchRequestParsers();
queriesRegistry = searchModule.getQueryParserRegistry();
}
@Override
protected NamedXContentRegistry xContentRegistry() {
return xContentRegistry;
}
protected SearchSourceBuilder createSearchSourceBuilder() {

View File

@ -19,20 +19,16 @@
package org.elasticsearch.search;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.inject.ModuleTestCase;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryParser;
import org.elasticsearch.index.query.TermQueryBuilder;
import org.elasticsearch.index.query.functionscore.GaussDecayFunctionBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.plugins.SearchPlugin;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@ -70,16 +66,17 @@ import org.elasticsearch.search.suggest.term.TermSuggester;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
import static java.util.Collections.singletonMap;
import static java.util.stream.Collectors.toSet;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.notNullValue;
public class SearchModuleTests extends ModuleTestCase {
@ -109,8 +106,8 @@ public class SearchModuleTests extends ModuleTestCase {
GaussDecayFunctionBuilder.PARSER));
}
};
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, singletonList(registersDupeScoreFunction));
expectThrows(IllegalArgumentException.class, () -> new NamedXContentRegistry(searchModule.getNamedXContents()));
expectThrows(IllegalArgumentException.class, () -> new NamedXContentRegistry(
new SearchModule(Settings.EMPTY, false, singletonList(registersDupeScoreFunction)).getNamedXContents()));
SearchPlugin registersDupeSignificanceHeuristic = new SearchPlugin() {
@Override
@ -144,8 +141,8 @@ public class SearchModuleTests extends ModuleTestCase {
return singletonList(new QuerySpec<>(TermQueryBuilder.NAME, TermQueryBuilder::new, TermQueryBuilder::fromXContent));
}
};
expectThrows(IllegalArgumentException.class, () -> new SearchModule(Settings.EMPTY, false,
singletonList(registersDupeQuery)));
expectThrows(IllegalArgumentException.class, () -> new NamedXContentRegistry(
new SearchModule(Settings.EMPTY, false, singletonList(registersDupeQuery)).getNamedXContents()));
SearchPlugin registersDupeAggregation = new SearchPlugin() {
public List<AggregationSpec> getAggregations() {
@ -200,31 +197,22 @@ public class SearchModuleTests extends ModuleTestCase {
}
public void testRegisteredQueries() throws IOException {
SearchModule module = new SearchModule(Settings.EMPTY, false, emptyList());
List<String> allSupportedQueries = new ArrayList<>();
Collections.addAll(allSupportedQueries, NON_DEPRECATED_QUERIES);
Collections.addAll(allSupportedQueries, DEPRECATED_QUERIES);
String[] supportedQueries = allSupportedQueries.toArray(new String[allSupportedQueries.size()]);
assertThat(module.getQueryParserRegistry().getNames(), containsInAnyOrder(supportedQueries));
SearchModule module = new SearchModule(Settings.EMPTY, false, emptyList());
IndicesQueriesRegistry indicesQueriesRegistry = module.getQueryParserRegistry();
XContentParser dummyParser = createParser(JsonXContent.jsonXContent, new BytesArray("{}"));
for (String queryName : supportedQueries) {
indicesQueriesRegistry.lookup(queryName, ParseFieldMatcher.EMPTY, dummyParser.getTokenLocation());
}
Set<String> registeredNonDeprecated = module.getNamedXContents().stream()
.filter(e -> e.categoryClass.equals(QueryBuilder.class))
.map(e -> e.name.getPreferredName())
.collect(toSet());
Set<String> registeredAll = module.getNamedXContents().stream()
.filter(e -> e.categoryClass.equals(QueryBuilder.class))
.flatMap(e -> Arrays.stream(e.name.getAllNamesIncludedDeprecated()))
.collect(toSet());
for (String queryName : NON_DEPRECATED_QUERIES) {
QueryParser<?> queryParser = indicesQueriesRegistry.lookup(queryName, ParseFieldMatcher.STRICT, dummyParser.getTokenLocation());
assertThat(queryParser, notNullValue());
}
for (String queryName : DEPRECATED_QUERIES) {
try {
indicesQueriesRegistry.lookup(queryName, ParseFieldMatcher.STRICT, dummyParser.getTokenLocation());
fail("query is deprecated, getQueryParser should have failed in strict mode");
} catch(IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("Deprecated field [" + queryName + "] used"));
}
}
assertThat(registeredNonDeprecated, containsInAnyOrder(NON_DEPRECATED_QUERIES));
assertThat(registeredAll, containsInAnyOrder(allSupportedQueries.toArray(new String[0])));
}
public void testRegisterAggregation() {

View File

@ -24,7 +24,6 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchRequestParsers;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.test.ESSingleNodeTestCase;
@ -61,9 +60,8 @@ public class AggregationCollectorTests extends ESSingleNodeTestCase {
private boolean needsScores(IndexService index, String agg) throws IOException {
AggregatorParsers parser = getInstanceFromNode(SearchRequestParsers.class).aggParsers;
IndicesQueriesRegistry queriesRegistry = getInstanceFromNode(IndicesQueriesRegistry.class);
XContentParser aggParser = createParser(JsonXContent.jsonXContent, agg);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, aggParser, ParseFieldMatcher.STRICT);
QueryParseContext parseContext = new QueryParseContext(aggParser, ParseFieldMatcher.STRICT);
aggParser.nextToken();
SearchContext context = createSearchContext(index);
final AggregatorFactories factories = parser.parseAggregators(parseContext).build(context, null);

View File

@ -22,16 +22,16 @@ package org.elasticsearch.search.aggregations;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.env.Environment;
import org.elasticsearch.test.AbstractQueryTestCase;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.test.AbstractQueryTestCase;
import org.elasticsearch.test.ESTestCase;
import java.util.Random;
@ -50,7 +50,7 @@ public class AggregatorParsingTests extends ESTestCase {
}
protected AggregatorParsers aggParsers;
protected IndicesQueriesRegistry queriesRegistry;
private NamedXContentRegistry xContentRegistry;
protected ParseFieldMatcher parseFieldMatcher;
/**
@ -73,7 +73,7 @@ public class AggregatorParsingTests extends ESTestCase {
String type = randomAsciiOfLengthBetween(1, 10);
currentTypes[i] = type;
}
queriesRegistry = searchModule.getQueryParserRegistry();
xContentRegistry = new NamedXContentRegistry(searchModule.getNamedXContents());
parseFieldMatcher = ParseFieldMatcher.STRICT;
}
@ -95,7 +95,7 @@ public class AggregatorParsingTests extends ESTestCase {
.endObject();
try {
XContentParser parser = createParser(source);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
aggParsers.parseAggregators(parseContext);
fail();
@ -132,7 +132,7 @@ public class AggregatorParsingTests extends ESTestCase {
.endObject();
try {
XContentParser parser = createParser(source);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
aggParsers.parseAggregators(parseContext);
fail();
@ -171,7 +171,7 @@ public class AggregatorParsingTests extends ESTestCase {
.endObject();
try {
XContentParser parser = createParser(source);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
aggParsers.parseAggregators(parseContext);
fail();
@ -199,7 +199,7 @@ public class AggregatorParsingTests extends ESTestCase {
.endObject();
try {
XContentParser parser = createParser(source);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
aggParsers.parseAggregators(parseContext);
fail();
@ -228,7 +228,7 @@ public class AggregatorParsingTests extends ESTestCase {
.endObject();
try {
XContentParser parser = createParser(source);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
aggParsers.parseAggregators(parseContext);
fail();
@ -257,7 +257,7 @@ public class AggregatorParsingTests extends ESTestCase {
.endObject();
try {
XContentParser parser = createParser(source);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
aggParsers.parseAggregators(parseContext);
fail();
@ -265,4 +265,9 @@ public class AggregatorParsingTests extends ESTestCase {
// All Good
}
}
@Override
protected NamedXContentRegistry xContentRegistry() {
return xContentRegistry;
}
}

View File

@ -0,0 +1,117 @@
/*
* 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.search.aggregations;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.QueryCachingPolicy;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.MockBigArrays;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.cache.query.DisabledQueryCache;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.fetch.subphase.DocValueFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.subphase.FetchSourceSubPhase;
import org.elasticsearch.search.internal.ContextIndexSearcher;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.lookup.SearchLookup;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import java.util.Arrays;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
/**
* Base class for testing {@link Aggregator} implementations.
* Provides a helper constructing the {@link Aggregator} implementation based on a provided {@link AggregationBuilder} instance.
*/
public abstract class AggregatorTestCase extends ESTestCase {
protected <A extends Aggregator, B extends AggregationBuilder> A createAggregator(B aggregationBuilder,
MappedFieldType fieldType,
IndexSearcher indexSearcher) throws IOException {
IndexSettings indexSettings = new IndexSettings(
IndexMetaData.builder("_index").settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT))
.numberOfShards(1)
.numberOfReplicas(0)
.creationDate(System.currentTimeMillis())
.build(),
Settings.EMPTY
);
Engine.Searcher searcher = new Engine.Searcher("aggregator_test", indexSearcher);
QueryCache queryCache = new DisabledQueryCache(indexSettings);
QueryCachingPolicy queryCachingPolicy = new QueryCachingPolicy() {
@Override
public void onUse(Query query) {
}
@Override
public boolean shouldCache(Query query) throws IOException {
// never cache a query
return false;
}
};
ContextIndexSearcher contextIndexSearcher = new ContextIndexSearcher(searcher, queryCache, queryCachingPolicy);
CircuitBreakerService circuitBreakerService = new NoneCircuitBreakerService();
SearchContext searchContext = mock(SearchContext.class);
when(searchContext.searcher()).thenReturn(contextIndexSearcher);
when(searchContext.parseFieldMatcher()).thenReturn(ParseFieldMatcher.STRICT);
when(searchContext.bigArrays()).thenReturn(new MockBigArrays(Settings.EMPTY, circuitBreakerService));
when(searchContext.fetchPhase())
.thenReturn(new FetchPhase(Arrays.asList(new FetchSourceSubPhase(), new DocValueFieldsFetchSubPhase())));
// TODO: now just needed for top_hits, this will need to be revised for other agg unit tests:
MapperService mapperService = mock(MapperService.class);
when(mapperService.hasNested()).thenReturn(false);
when(searchContext.mapperService()).thenReturn(mapperService);
SearchLookup searchLookup = new SearchLookup(mapperService, mock(IndexFieldDataService.class), new String[]{"type"});
when(searchContext.lookup()).thenReturn(searchLookup);
QueryShardContext queryShardContext = mock(QueryShardContext.class);
IndexFieldData<?> fieldData = fieldType.fielddataBuilder().build(indexSettings, fieldType, null, circuitBreakerService,
mock(MapperService.class));
when(queryShardContext.fieldMapper(anyString())).thenReturn(fieldType);
when(queryShardContext.getForField(any())).thenReturn(fieldData);
when(searchContext.getQueryShardContext()).thenReturn(queryShardContext);
@SuppressWarnings("unchecked")
A aggregator = (A) aggregationBuilder.build(searchContext, null).create(null, true);
return aggregator;
}
}

View File

@ -25,6 +25,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -33,7 +34,6 @@ import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.IndicesModule;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.test.AbstractQueryTestCase;
import org.elasticsearch.test.ESTestCase;
@ -64,7 +64,7 @@ public abstract class BaseAggregationTestCase<AB extends AbstractAggregationBuil
private NamedWriteableRegistry namedWriteableRegistry;
protected AggregatorParsers aggParsers;
protected IndicesQueriesRegistry queriesRegistry;
private NamedXContentRegistry xContentRegistry;
protected ParseFieldMatcher parseFieldMatcher;
protected abstract AB createTestAggregatorBuilder();
@ -85,7 +85,7 @@ public abstract class BaseAggregationTestCase<AB extends AbstractAggregationBuil
entries.addAll(indicesModule.getNamedWriteables());
entries.addAll(searchModule.getNamedWriteables());
namedWriteableRegistry = new NamedWriteableRegistry(entries);
queriesRegistry = searchModule.getQueryParserRegistry();
xContentRegistry = new NamedXContentRegistry(searchModule.getNamedXContents());
aggParsers = searchModule.getSearchRequestParsers().aggParsers;
//create some random type with some default field, those types will stick around for all of the subclasses
currentTypes = new String[randomIntBetween(0, 5)];
@ -96,6 +96,11 @@ public abstract class BaseAggregationTestCase<AB extends AbstractAggregationBuil
parseFieldMatcher = ParseFieldMatcher.STRICT;
}
@Override
protected NamedXContentRegistry xContentRegistry() {
return xContentRegistry;
}
/**
* Generic test that creates new AggregatorFactory from the test
* AggregatorFactory and checks both for equality and asserts equality on
@ -111,7 +116,7 @@ public abstract class BaseAggregationTestCase<AB extends AbstractAggregationBuil
factoriesBuilder.toXContent(builder, ToXContent.EMPTY_PARAMS);
XContentBuilder shuffled = shuffleXContent(builder);
XContentParser parser = createParser(shuffled);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
assertSame(XContentParser.Token.FIELD_NAME, parser.nextToken());
assertEquals(testAgg.name, parser.currentName());

View File

@ -34,7 +34,6 @@ import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.IndicesModule;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder;
import org.elasticsearch.test.AbstractQueryTestCase;
@ -65,7 +64,6 @@ public abstract class BasePipelineAggregationTestCase<AF extends AbstractPipelin
private NamedWriteableRegistry namedWriteableRegistry;
protected AggregatorParsers aggParsers;
protected IndicesQueriesRegistry queriesRegistry;
protected ParseFieldMatcher parseFieldMatcher;
protected abstract AF createTestAggregatorFactory();
@ -86,7 +84,6 @@ public abstract class BasePipelineAggregationTestCase<AF extends AbstractPipelin
entries.addAll(indicesModule.getNamedWriteables());
entries.addAll(searchModule.getNamedWriteables());
namedWriteableRegistry = new NamedWriteableRegistry(entries);
queriesRegistry = searchModule.getQueryParserRegistry();
aggParsers = searchModule.getSearchRequestParsers().aggParsers;
//create some random type with some default field, those types will stick around for all of the subclasses
currentTypes = new String[randomIntBetween(0, 5)];
@ -113,7 +110,7 @@ public abstract class BasePipelineAggregationTestCase<AF extends AbstractPipelin
factoriesBuilder.toXContent(builder, ToXContent.EMPTY_PARAMS);
XContentBuilder shuffled = shuffleXContent(builder);
XContentParser parser = createParser(shuffled);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
String contentString = factoriesBuilder.toString();
logger.info("Content string: {}", contentString);
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());

View File

@ -23,20 +23,16 @@ import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.test.ESTestCase;
import static org.hamcrest.Matchers.instanceOf;
public class GeoHashGridParserTests extends ESTestCase {
private static final IndicesQueriesRegistry mockRegistry = new IndicesQueriesRegistry();
public void testParseValidFromInts() throws Exception {
int precision = randomIntBetween(1, 12);
XContentParser stParser = createParser(JsonXContent.jsonXContent,
"{\"field\":\"my_loc\", \"precision\":" + precision + ", \"size\": 500, \"shard_size\": 550}");
QueryParseContext parseContext = new QueryParseContext(mockRegistry,
stParser, ParseFieldMatcher.STRICT);
QueryParseContext parseContext = new QueryParseContext(stParser, ParseFieldMatcher.STRICT);
XContentParser.Token token = stParser.nextToken();
assertSame(XContentParser.Token.START_OBJECT, token);
// can create a factory
@ -47,7 +43,7 @@ public class GeoHashGridParserTests extends ESTestCase {
int precision = randomIntBetween(1, 12);
XContentParser stParser = createParser(JsonXContent.jsonXContent,
"{\"field\":\"my_loc\", \"precision\":\"" + precision + "\", \"size\": \"500\", \"shard_size\": \"550\"}");
QueryParseContext parseContext = new QueryParseContext(mockRegistry, stParser, ParseFieldMatcher.STRICT);
QueryParseContext parseContext = new QueryParseContext(stParser, ParseFieldMatcher.STRICT);
XContentParser.Token token = stParser.nextToken();
assertSame(XContentParser.Token.START_OBJECT, token);
// can create a factory
@ -56,7 +52,7 @@ public class GeoHashGridParserTests extends ESTestCase {
public void testParseErrorOnNonIntPrecision() throws Exception {
XContentParser stParser = createParser(JsonXContent.jsonXContent, "{\"field\":\"my_loc\", \"precision\":\"2.0\"}");
QueryParseContext parseContext = new QueryParseContext(mockRegistry, stParser, ParseFieldMatcher.STRICT);
QueryParseContext parseContext = new QueryParseContext(stParser, ParseFieldMatcher.STRICT);
XContentParser.Token token = stParser.nextToken();
assertSame(XContentParser.Token.START_OBJECT, token);
try {
@ -70,7 +66,7 @@ public class GeoHashGridParserTests extends ESTestCase {
public void testParseErrorOnBooleanPrecision() throws Exception {
XContentParser stParser = createParser(JsonXContent.jsonXContent, "{\"field\":\"my_loc\", \"precision\":false}");
QueryParseContext parseContext = new QueryParseContext(mockRegistry, stParser, ParseFieldMatcher.STRICT);
QueryParseContext parseContext = new QueryParseContext(stParser, ParseFieldMatcher.STRICT);
XContentParser.Token token = stParser.nextToken();
assertSame(XContentParser.Token.START_OBJECT, token);
try {
@ -83,7 +79,7 @@ public class GeoHashGridParserTests extends ESTestCase {
public void testParseErrorOnPrecisionOutOfRange() throws Exception {
XContentParser stParser = createParser(JsonXContent.jsonXContent, "{\"field\":\"my_loc\", \"precision\":\"13\"}");
QueryParseContext parseContext = new QueryParseContext(mockRegistry, stParser, ParseFieldMatcher.STRICT);
QueryParseContext parseContext = new QueryParseContext(stParser, ParseFieldMatcher.STRICT);
XContentParser.Token token = stParser.nextToken();
assertSame(XContentParser.Token.START_OBJECT, token);
try {

View File

@ -100,7 +100,7 @@ public class ExtendedBoundsTests extends ESTestCase {
SearchContext context = mock(SearchContext.class);
QueryShardContext qsc = new QueryShardContext(0,
new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings), null, null, null, null,
null, xContentRegistry(), null, null, null, () -> now);
null, xContentRegistry(), null, null, () -> now);
when(context.getQueryShardContext()).thenReturn(qsc);
FormatDateTimeFormatter formatter = Joda.forPattern("dateOptionalTime");
DocValueFormat format = new DocValueFormat.DateTime(formatter, DateTimeZone.UTC);

View File

@ -35,7 +35,6 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.SearchShardTarget;
@ -267,11 +266,10 @@ public class SignificanceHeuristicTests extends ESTestCase {
protected void checkParseException(ParseFieldRegistry<SignificanceHeuristicParser> significanceHeuristicParserRegistry,
SearchContext searchContext, String faultyHeuristicDefinition, String expectedError) throws IOException {
IndicesQueriesRegistry registry = new IndicesQueriesRegistry();
try {
XContentParser stParser = createParser(JsonXContent.jsonXContent,
"{\"field\":\"text\", " + faultyHeuristicDefinition + ",\"min_doc_count\":200}");
QueryParseContext parseContext = new QueryParseContext(registry, stParser, ParseFieldMatcher.STRICT);
QueryParseContext parseContext = new QueryParseContext(stParser, ParseFieldMatcher.STRICT);
stParser.nextToken();
SignificantTermsAggregationBuilder.getParser(significanceHeuristicParserRegistry).parse("testagg", parseContext);
fail();
@ -293,8 +291,7 @@ public class SignificanceHeuristicTests extends ESTestCase {
private SignificanceHeuristic parseSignificanceHeuristic(
ParseFieldRegistry<SignificanceHeuristicParser> significanceHeuristicParserRegistry, SearchContext searchContext,
XContentParser stParser) throws IOException {
IndicesQueriesRegistry registry = new IndicesQueriesRegistry();
QueryParseContext parseContext = new QueryParseContext(registry, stParser, ParseFieldMatcher.STRICT);
QueryParseContext parseContext = new QueryParseContext(stParser, ParseFieldMatcher.STRICT);
stParser.nextToken();
SignificantTermsAggregationBuilder aggregatorFactory =
(SignificantTermsAggregationBuilder) SignificantTermsAggregationBuilder.getParser(

View File

@ -0,0 +1,85 @@
/*
* 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.search.aggregations.bucket.terms;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.index.mapper.KeywordFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.ValueType;
public class TermsAggregatorTests extends AggregatorTestCase {
public void testTermsAggregator() throws Exception {
Directory directory = newDirectory();
RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
Document document = new Document();
document.add(new SortedSetDocValuesField("string", new BytesRef("a")));
document.add(new SortedSetDocValuesField("string", new BytesRef("b")));
indexWriter.addDocument(document);
document = new Document();
document.add(new SortedSetDocValuesField("string", new BytesRef("c")));
document.add(new SortedSetDocValuesField("string", new BytesRef("a")));
indexWriter.addDocument(document);
document = new Document();
document.add(new SortedSetDocValuesField("string", new BytesRef("b")));
document.add(new SortedSetDocValuesField("string", new BytesRef("d")));
indexWriter.addDocument(document);
indexWriter.close();
IndexReader indexReader = DirectoryReader.open(directory);
IndexSearcher indexSearcher = newSearcher(indexReader, true, true);
for (TermsAggregatorFactory.ExecutionMode executionMode : TermsAggregatorFactory.ExecutionMode.values()) {
TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name", ValueType.STRING)
.executionHint(executionMode.toString())
.field("string")
.order(Terms.Order.term(true));
MappedFieldType fieldType = new KeywordFieldMapper.KeywordFieldType();
fieldType.setName("string");
fieldType.setHasDocValues(true );
try (TermsAggregator aggregator = createAggregator(aggregationBuilder, fieldType, indexSearcher)) {
aggregator.preCollection();
indexSearcher.search(new MatchAllDocsQuery(), aggregator);
aggregator.postCollection();
Terms result = (Terms) aggregator.buildAggregation(0L);
assertEquals(4, result.getBuckets().size());
assertEquals("a", result.getBuckets().get(0).getKeyAsString());
assertEquals(2L, result.getBuckets().get(0).getDocCount());
assertEquals("b", result.getBuckets().get(1).getKeyAsString());
assertEquals(2L, result.getBuckets().get(1).getDocCount());
assertEquals("c", result.getBuckets().get(2).getKeyAsString());
assertEquals(1L, result.getBuckets().get(2).getDocCount());
assertEquals("d", result.getBuckets().get(3).getKeyAsString());
assertEquals(1L, result.getBuckets().get(3).getDocCount());
}
}
indexReader.close();
directory.close();
}
}

View File

@ -28,7 +28,6 @@ import org.elasticsearch.index.query.MatchNoneQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator.KeyedFilter;
@ -89,8 +88,7 @@ public class FiltersTests extends BaseAggregationTestCase<FiltersAggregationBuil
builder.endObject();
XContentParser parser = createParser(shuffleXContent(builder));
parser.nextToken();
QueryParseContext context = new QueryParseContext(new IndicesQueriesRegistry(), parser,
ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
FiltersAggregationBuilder filters = FiltersAggregationBuilder.parse("agg_name", context);
// The other bucket is disabled by default
assertFalse(filters.otherBucket());
@ -102,7 +100,7 @@ public class FiltersTests extends BaseAggregationTestCase<FiltersAggregationBuil
builder.endObject();
parser = createParser(shuffleXContent(builder));
parser.nextToken();
context = new QueryParseContext(new IndicesQueriesRegistry(), parser, ParseFieldMatcher.STRICT);
context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
filters = FiltersAggregationBuilder.parse("agg_name", context);
// but setting a key enables it automatically
assertTrue(filters.otherBucket());
@ -115,7 +113,7 @@ public class FiltersTests extends BaseAggregationTestCase<FiltersAggregationBuil
builder.endObject();
parser = createParser(shuffleXContent(builder));
parser.nextToken();
context = new QueryParseContext(new IndicesQueriesRegistry(), parser, ParseFieldMatcher.STRICT);
context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
filters = FiltersAggregationBuilder.parse("agg_name", context);
// unless the other bucket is explicitly disabled
assertFalse(filters.otherBucket());

View File

@ -19,10 +19,8 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.test.AbstractQueryTestCase;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.AggregationInitializationException;
@ -33,6 +31,7 @@ import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilderTests;
import org.elasticsearch.search.sort.ScriptSortBuilder.ScriptSortType;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.AbstractQueryTestCase;
import java.util.ArrayList;
import java.util.Collections;
@ -188,7 +187,7 @@ public class TopHitsTests extends BaseAggregationTestCase<TopHitsAggregationBuil
"}";
try {
XContentParser parser = createParser(JsonXContent.jsonXContent, source);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
aggParsers.parseAggregators(parseContext);
fail();

View File

@ -0,0 +1,163 @@
/*
* 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.search.aggregations.metrics.min;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.store.Directory;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
public class MinAggregatorTests extends AggregatorTestCase {
public void testMinAggregator_numericDv() throws Exception {
Directory directory = newDirectory();
RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
Document document = new Document();
document.add(new NumericDocValuesField("number", 9));
indexWriter.addDocument(document);
document = new Document();
document.add(new NumericDocValuesField("number", 7));
indexWriter.addDocument(document);
document = new Document();
document.add(new NumericDocValuesField("number", 5));
indexWriter.addDocument(document);
document = new Document();
document.add(new NumericDocValuesField("number", 3));
indexWriter.addDocument(document);
document = new Document();
document.add(new NumericDocValuesField("number", 1));
indexWriter.addDocument(document);
document = new Document();
document.add(new NumericDocValuesField("number", -1));
indexWriter.addDocument(document);
indexWriter.close();
IndexReader indexReader = DirectoryReader.open(directory);
IndexSearcher indexSearcher = newSearcher(indexReader, true, true);
MinAggregationBuilder aggregationBuilder = new MinAggregationBuilder("_name").field("number");
MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG);
fieldType.setName("number");
try (MinAggregator aggregator = createAggregator(aggregationBuilder, fieldType, indexSearcher)) {
aggregator.preCollection();
indexSearcher.search(new MatchAllDocsQuery(), aggregator);
aggregator.postCollection();
InternalMin result = (InternalMin) aggregator.buildAggregation(0L);
assertEquals(-1.0, result.getValue(), 0);
}
indexReader.close();
directory.close();
}
public void testMinAggregator_sortedNumericDv() throws Exception {
Directory directory = newDirectory();
RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
Document document = new Document();
document.add(new SortedNumericDocValuesField("number", 9));
document.add(new SortedNumericDocValuesField("number", 7));
indexWriter.addDocument(document);
document = new Document();
document.add(new SortedNumericDocValuesField("number", 5));
document.add(new SortedNumericDocValuesField("number", 3));
indexWriter.addDocument(document);
document = new Document();
document.add(new SortedNumericDocValuesField("number", 1));
document.add(new SortedNumericDocValuesField("number", -1));
indexWriter.addDocument(document);
indexWriter.close();
IndexReader indexReader = DirectoryReader.open(directory);
IndexSearcher indexSearcher = newSearcher(indexReader, true, true);
MinAggregationBuilder aggregationBuilder = new MinAggregationBuilder("_name").field("number");
MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG);
fieldType.setName("number");
try (MinAggregator aggregator = createAggregator(aggregationBuilder, fieldType, indexSearcher)) {
aggregator.preCollection();
indexSearcher.search(new MatchAllDocsQuery(), aggregator);
aggregator.postCollection();
InternalMin result = (InternalMin) aggregator.buildAggregation(0L);
assertEquals(-1.0, result.getValue(), 0);
}
indexReader.close();
directory.close();
}
public void testMinAggregator_noValue() throws Exception {
Directory directory = newDirectory();
RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
Document document = new Document();
document.add(new SortedNumericDocValuesField("number1", 7));
indexWriter.addDocument(document);
document = new Document();
document.add(new SortedNumericDocValuesField("number1", 3));
indexWriter.addDocument(document);
document = new Document();
document.add(new SortedNumericDocValuesField("number1", 1));
indexWriter.addDocument(document);
indexWriter.close();
IndexReader indexReader = DirectoryReader.open(directory);
IndexSearcher indexSearcher = newSearcher(indexReader, true, true);
MinAggregationBuilder aggregationBuilder = new MinAggregationBuilder("_name").field("number2");
MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG);
fieldType.setName("number2");
try (MinAggregator aggregator = createAggregator(aggregationBuilder, fieldType, indexSearcher)) {
aggregator.preCollection();
indexSearcher.search(new MatchAllDocsQuery(), aggregator);
aggregator.postCollection();
InternalMin result = (InternalMin) aggregator.buildAggregation(0L);
assertEquals(Double.POSITIVE_INFINITY, result.getValue(), 0);
}
indexReader.close();
directory.close();
}
public void testMinAggregator_noDocs() throws Exception {
Directory directory = newDirectory();
RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
indexWriter.close();
IndexReader indexReader = DirectoryReader.open(directory);
IndexSearcher indexSearcher = newSearcher(indexReader, true, true);
MinAggregationBuilder aggregationBuilder = new MinAggregationBuilder("_name").field("number");
MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG);
fieldType.setName("number");
try (MinAggregator aggregator = createAggregator(aggregationBuilder, fieldType, indexSearcher)) {
aggregator.preCollection();
indexSearcher.search(new MatchAllDocsQuery(), aggregator);
aggregator.postCollection();
InternalMin result = (InternalMin) aggregator.buildAggregation(0L);
assertEquals(Double.POSITIVE_INFINITY, result.getValue(), 0);
}
indexReader.close();
directory.close();
}
}

View File

@ -0,0 +1,87 @@
/*
* 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.search.aggregations.metrics.tophits;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.index.mapper.KeywordFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.sort.SortOrder;
public class TopHitsAggregatorTests extends AggregatorTestCase {
public void testTermsAggregator() throws Exception {
Directory directory = newDirectory();
RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
Document document = new Document();
document.add(new Field(UidFieldMapper.NAME, Uid.createUid("type", "1"), UidFieldMapper.Defaults.FIELD_TYPE));
document.add(new SortedSetDocValuesField("string", new BytesRef("a")));
document.add(new SortedSetDocValuesField("string", new BytesRef("b")));
indexWriter.addDocument(document);
document = new Document();
document.add(new Field(UidFieldMapper.NAME, Uid.createUid("type", "2"), UidFieldMapper.Defaults.FIELD_TYPE));
document.add(new SortedSetDocValuesField("string", new BytesRef("c")));
document.add(new SortedSetDocValuesField("string", new BytesRef("a")));
indexWriter.addDocument(document);
document = new Document();
document.add(new Field(UidFieldMapper.NAME, Uid.createUid("type", "3"), UidFieldMapper.Defaults.FIELD_TYPE));
document.add(new SortedSetDocValuesField("string", new BytesRef("b")));
document.add(new SortedSetDocValuesField("string", new BytesRef("d")));
indexWriter.addDocument(document);
indexWriter.close();
IndexReader indexReader = DirectoryReader.open(directory);
IndexSearcher indexSearcher = newSearcher(indexReader, true, true);
MappedFieldType fieldType = new KeywordFieldMapper.KeywordFieldType();
fieldType.setName("string");
fieldType.setHasDocValues(true );
TopHitsAggregationBuilder aggregationBuilder = new TopHitsAggregationBuilder("_name");
aggregationBuilder.sort("string", SortOrder.DESC);
try (TopHitsAggregator aggregator = createAggregator(aggregationBuilder, fieldType, indexSearcher)){
aggregator.preCollection();
indexSearcher.search(new MatchAllDocsQuery(), aggregator);
aggregator.postCollection();
TopHits topHits = (TopHits) aggregator.buildAggregation(0L);
SearchHits searchHits = topHits.getHits();
assertEquals(3L, searchHits.getTotalHits());
assertEquals("3", searchHits.getAt(0).getId());
assertEquals("type", searchHits.getAt(0).getType());
assertEquals("2", searchHits.getAt(1).getId());
assertEquals("type", searchHits.getAt(1).getType());
assertEquals("1", searchHits.getAt(2).getId());
assertEquals("type", searchHits.getAt(2).getType());
}
indexReader.close();
directory.close();
}
}

View File

@ -47,7 +47,7 @@ public class ExtendedStatsBucketTests extends AbstractBucketMetricsTestCase<Exte
.string();
XContentParser parser = createParser(JsonXContent.jsonXContent, content);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
parser.nextToken(); // skip object start
ExtendedStatsBucketPipelineAggregationBuilder builder = (ExtendedStatsBucketPipelineAggregationBuilder) aggParsers

View File

@ -52,7 +52,7 @@ public class PercentilesBucketTests extends AbstractBucketMetricsTestCase<Percen
.string();
XContentParser parser = createParser(JsonXContent.jsonXContent, content);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
parser.nextToken(); // skip object start
PercentilesBucketPipelineAggregationBuilder builder = (PercentilesBucketPipelineAggregationBuilder) aggParsers

View File

@ -107,7 +107,7 @@ public class MovAvgTests extends BasePipelineAggregationTestCase<MovAvgPipelineA
" }" +
"}";
XContentParser parser = createParser(JsonXContent.jsonXContent, json);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
assertSame(XContentParser.Token.FIELD_NAME, parser.nextToken());
assertEquals(expected.getName(), parser.currentName());

View File

@ -32,7 +32,6 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude.OrdinalsFilter;
@ -43,9 +42,7 @@ import java.util.Collections;
import java.util.TreeSet;
public class IncludeExcludeTests extends ESTestCase {
private final ParseFieldMatcher parseFieldMatcher = ParseFieldMatcher.STRICT;
private final IndicesQueriesRegistry queriesRegistry = new IndicesQueriesRegistry();
public void testEmptyTermsWithOrds() throws IOException {
IncludeExclude inexcl = new IncludeExclude(
@ -237,7 +234,7 @@ public class IncludeExcludeTests extends ESTestCase {
assertEquals(field.getPreferredName(), parser.currentName());
token = parser.nextToken();
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext parseContext = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
if (field.getPreferredName().equalsIgnoreCase("include")) {
return IncludeExclude.parseInclude(parser, parseContext);
} else if (field.getPreferredName().equalsIgnoreCase("exclude")) {
@ -277,7 +274,7 @@ public class IncludeExcludeTests extends ESTestCase {
builder.endObject();
XContentParser parser = createParser(builder);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
XContentParser.Token token = parser.nextToken();
assertEquals(token, XContentParser.Token.START_OBJECT);

View File

@ -108,6 +108,27 @@ public class ScriptValuesTests extends ESTestCase {
}
}
public void testBooleans() {
final Object[][] values = new Boolean[randomInt(10)][];
for (int i = 0; i < values.length; ++i) {
Boolean[] booleans = new Boolean[randomInt(8)];
for (int j = 0; j < booleans.length; ++j) {
booleans[j] = randomBoolean();
}
Arrays.sort(booleans);
values[i] = booleans;
}
FakeSearchScript script = new FakeSearchScript(values);
ScriptLongValues scriptValues = new ScriptLongValues(script);
for (int i = 0; i < values.length; ++i) {
scriptValues.setDocument(i);
assertEquals(values[i].length, scriptValues.count());
for (int j = 0; j < values[i].length; ++j) {
assertEquals(values[i][j], scriptValues.valueAt(j) == 1L);
}
}
}
public void testDoubles() {
final Object[][] values = new Double[randomInt(10)][];
for (int i = 0; i < values.length; ++i) {

View File

@ -70,7 +70,7 @@ public class SearchSourceBuilderTests extends AbstractSearchTestCase {
private void assertParseSearchSource(SearchSourceBuilder testBuilder, XContentParser parser, ParseFieldMatcher pfm)
throws IOException {
QueryParseContext parseContext = new QueryParseContext(searchRequestParsers.queryParsers, parser, pfm);
QueryParseContext parseContext = new QueryParseContext(parser, pfm);
if (randomBoolean()) {
parser.nextToken(); // sometimes we move it on the START_OBJECT to
// test the embedded case
@ -83,7 +83,7 @@ public class SearchSourceBuilderTests extends AbstractSearchTestCase {
}
private QueryParseContext createParseContext(XContentParser parser) {
return new QueryParseContext(searchRequestParsers.queryParsers, parser, ParseFieldMatcher.STRICT);
return new QueryParseContext(parser, ParseFieldMatcher.STRICT);
}
public void testSerialization() throws IOException {

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -46,7 +47,6 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.TermQueryBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder.Field;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder.Order;
@ -75,7 +75,7 @@ public class HighlightBuilderTests extends ESTestCase {
private static final int NUMBER_OF_TESTBUILDERS = 20;
private static NamedWriteableRegistry namedWriteableRegistry;
private static IndicesQueriesRegistry indicesQueriesRegistry;
private static NamedXContentRegistry xContentRegistry;
/**
* setup for the whole base test class
@ -84,13 +84,13 @@ public class HighlightBuilderTests extends ESTestCase {
public static void init() {
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList());
namedWriteableRegistry = new NamedWriteableRegistry(searchModule.getNamedWriteables());
indicesQueriesRegistry = searchModule.getQueryParserRegistry();
xContentRegistry = new NamedXContentRegistry(searchModule.getNamedXContents());
}
@AfterClass
public static void afterClass() throws Exception {
namedWriteableRegistry = null;
indicesQueriesRegistry = null;
xContentRegistry = null;
}
/**
@ -129,7 +129,7 @@ public class HighlightBuilderTests extends ESTestCase {
XContentBuilder shuffled = shuffleXContent(builder);
XContentParser parser = createParser(shuffled);
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.EMPTY);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.EMPTY);
parser.nextToken();
HighlightBuilder secondHighlightBuilder;
try {
@ -170,7 +170,7 @@ public class HighlightBuilderTests extends ESTestCase {
private <T extends Throwable> T expectParseThrows(Class<T> exceptionClass, String highlightElement) throws IOException {
XContentParser parser = createParser(JsonXContent.jsonXContent, highlightElement);
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
return expectThrows(exceptionClass, () -> HighlightBuilder.fromXContent(context));
}
@ -266,7 +266,7 @@ public class HighlightBuilderTests extends ESTestCase {
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, indexSettings);
// shard context will only need indicesQueriesRegistry for building Query objects nested in highlighter
QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, null, null, null, null, null, xContentRegistry(),
indicesQueriesRegistry, null, null, System::currentTimeMillis) {
null, null, System::currentTimeMillis) {
@Override
public MappedFieldType fieldMapper(String name) {
TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name);
@ -380,7 +380,7 @@ public class HighlightBuilderTests extends ESTestCase {
"}\n";
XContentParser parser = createParser(JsonXContent.jsonXContent, highlightElement);
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.EMPTY);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.EMPTY);
HighlightBuilder highlightBuilder = HighlightBuilder.fromXContent(context);
assertArrayEquals("setting tags_schema 'styled' should alter pre_tags", HighlightBuilder.DEFAULT_STYLED_PRE_TAG,
highlightBuilder.preTags());
@ -392,7 +392,7 @@ public class HighlightBuilderTests extends ESTestCase {
"}\n";
parser = createParser(JsonXContent.jsonXContent, highlightElement);
context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.EMPTY);
context = new QueryParseContext(parser, ParseFieldMatcher.EMPTY);
highlightBuilder = HighlightBuilder.fromXContent(context);
assertArrayEquals("setting tags_schema 'default' should alter pre_tags", HighlightBuilder.DEFAULT_PRE_TAGS,
highlightBuilder.preTags());
@ -413,21 +413,21 @@ public class HighlightBuilderTests extends ESTestCase {
String highlightElement = "{ }";
XContentParser parser = createParser(JsonXContent.jsonXContent, highlightElement);
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.EMPTY);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.EMPTY);
HighlightBuilder highlightBuilder = HighlightBuilder.fromXContent(context);
assertEquals("expected plain HighlightBuilder", new HighlightBuilder(), highlightBuilder);
highlightElement = "{ \"fields\" : { } }";
parser = createParser(JsonXContent.jsonXContent, highlightElement);
context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.EMPTY);
context = new QueryParseContext(parser, ParseFieldMatcher.EMPTY);
highlightBuilder = HighlightBuilder.fromXContent(context);
assertEquals("defining no field should return plain HighlightBuilder", new HighlightBuilder(), highlightBuilder);
highlightElement = "{ \"fields\" : { \"foo\" : { } } }";
parser = createParser(JsonXContent.jsonXContent, highlightElement);
context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.EMPTY);
context = new QueryParseContext(parser, ParseFieldMatcher.EMPTY);
highlightBuilder = HighlightBuilder.fromXContent(context);
assertEquals("expected HighlightBuilder with field", new HighlightBuilder().field(new Field("foo")), highlightBuilder);
}
@ -715,4 +715,9 @@ public class HighlightBuilderTests extends ESTestCase {
private static HighlightBuilder serializedCopy(HighlightBuilder original) throws IOException {
return ESTestCase.copyWriteable(original, namedWriteableRegistry, HighlightBuilder::new);
}
@Override
protected NamedXContentRegistry xContentRegistry() {
return xContentRegistry;
}
}

View File

@ -0,0 +1,129 @@
/*
* 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.search.internal;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.function.Supplier;
import static org.elasticsearch.common.xcontent.XContentHelper.toXContent;
import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent;
public class SearchSortValuesTests extends ESTestCase {
public static SearchSortValues createTestItem() {
List<Supplier<Object>> valueSuppliers = new ArrayList<>();
// this should reflect all values that are allowed to go through the transport layer
valueSuppliers.add(() -> null);
valueSuppliers.add(() -> randomInt());
valueSuppliers.add(() -> randomLong());
valueSuppliers.add(() -> randomDouble());
valueSuppliers.add(() -> randomFloat());
valueSuppliers.add(() -> randomByte());
valueSuppliers.add(() -> randomShort());
valueSuppliers.add(() -> randomBoolean());
valueSuppliers.add(() -> frequently() ? randomAsciiOfLengthBetween(1, 30) : randomRealisticUnicodeOfCodepointLength(30));
int size = randomInt(20);
Object[] values = new Object[size];
for (int i = 0; i < size; i++) {
Supplier<Object> supplier = randomFrom(valueSuppliers);
values[i] = supplier.get();
}
return new SearchSortValues(values);
}
public void testFromXContent() throws IOException {
SearchSortValues sortValues = createTestItem();
XContentType xcontentType = randomFrom(XContentType.values());
XContentBuilder builder = XContentFactory.contentBuilder(xcontentType);
if (randomBoolean()) {
builder.prettyPrint();
}
builder.startObject(); // we need to wrap xContent output in proper object to create a parser for it
builder = sortValues.toXContent(builder, ToXContent.EMPTY_PARAMS);
builder.endObject();
XContentParser parser = createParser(builder);
parser.nextToken(); // skip to the elements field name token, fromXContent advances from there if called from ourside
parser.nextToken();
if (sortValues.sortValues().length > 0) {
SearchSortValues parsed = SearchSortValues.fromXContent(parser);
assertToXContentEquivalent(builder.bytes(), toXContent(parsed, xcontentType, true), xcontentType);
parser.nextToken();
}
assertEquals(XContentParser.Token.END_OBJECT, parser.currentToken());
assertNull(parser.nextToken());
}
public void testToXContent() throws IOException {
SearchSortValues sortValues = new SearchSortValues(new Object[]{ 1, "foo", 3.0});
XContentBuilder builder = JsonXContent.contentBuilder();
builder.startObject();
sortValues.toXContent(builder, ToXContent.EMPTY_PARAMS);
builder.endObject();
assertEquals("{\"sort\":[1,\"foo\",3.0]}", builder.string());
}
/**
* Test equality and hashCode properties
*/
public void testEqualsAndHashcode() {
checkEqualsAndHashCode(createTestItem(), SearchSortValuesTests::copy, SearchSortValuesTests::mutate);
}
public void testSerialization() throws IOException {
SearchSortValues sortValues = createTestItem();
try (BytesStreamOutput output = new BytesStreamOutput()) {
sortValues.writeTo(output);
try (StreamInput in = output.bytes().streamInput()) {
SearchSortValues deserializedCopy = new SearchSortValues(in);
assertEquals(sortValues, deserializedCopy);
assertEquals(sortValues.hashCode(), deserializedCopy.hashCode());
assertNotSame(sortValues, deserializedCopy);
}
}
}
private static SearchSortValues mutate(SearchSortValues original) {
Object[] sortValues = original.sortValues();
if (sortValues.length == 0) {
return new SearchSortValues(new Object[] { 1 });
}
return new SearchSortValues(Arrays.copyOf(sortValues, sortValues.length + 1));
}
private static SearchSortValues copy(SearchSortValues original) {
return new SearchSortValues(Arrays.copyOf(original.sortValues(), original.sortValues().length));
}
}

View File

@ -164,7 +164,7 @@ public class ShardSearchTransportRequestTests extends AbstractSearchTestCase {
public QueryBuilder aliasFilter(IndexMetaData indexMetaData, String... aliasNames) {
ShardSearchRequest.FilterParser filterParser = bytes -> {
try (XContentParser parser = XContentFactory.xContent(bytes).createParser(xContentRegistry(), bytes)) {
return new QueryParseContext(queriesRegistry, parser, new ParseFieldMatcher(Settings.EMPTY)).parseInnerQueryBuilder();
return new QueryParseContext(parser, new ParseFieldMatcher(Settings.EMPTY)).parseInnerQueryBuilder();
}
};
return ShardSearchRequest.parseAliasFilter(filterParser, indexMetaData, aliasNames);
@ -200,7 +200,7 @@ public class ShardSearchTransportRequestTests extends AbstractSearchTestCase {
IndexSettings indexSettings = new IndexSettings(indexMetadata.build(), Settings.EMPTY);
final long nowInMillis = randomPositiveLong();
QueryShardContext context = new QueryShardContext(
0, indexSettings, null, null, null, null, null, xContentRegistry(), queriesRegistry, null, null, () -> nowInMillis);
0, indexSettings, null, null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis);
readRequest.rewrite(context);
QueryBuilder queryBuilder = readRequest.filteringAliases();
assertEquals(queryBuilder, QueryBuilders.boolQuery()

View File

@ -27,6 +27,7 @@ import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -42,7 +43,6 @@ import org.elasticsearch.index.query.MatchAllQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.rescore.QueryRescorer.QueryRescoreContext;
import org.elasticsearch.test.ESTestCase;
@ -59,7 +59,7 @@ public class QueryRescoreBuilderTests extends ESTestCase {
private static final int NUMBER_OF_TESTBUILDERS = 20;
private static NamedWriteableRegistry namedWriteableRegistry;
private static IndicesQueriesRegistry indicesQueriesRegistry;
private static NamedXContentRegistry xContentRegistry;
/**
* setup for the whole base test class
@ -68,13 +68,13 @@ public class QueryRescoreBuilderTests extends ESTestCase {
public static void init() {
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList());
namedWriteableRegistry = new NamedWriteableRegistry(searchModule.getNamedWriteables());
indicesQueriesRegistry = searchModule.getQueryParserRegistry();
xContentRegistry = new NamedXContentRegistry(searchModule.getNamedXContents());
}
@AfterClass
public static void afterClass() throws Exception {
namedWriteableRegistry = null;
indicesQueriesRegistry = null;
xContentRegistry = null;
}
/**
@ -119,7 +119,7 @@ public class QueryRescoreBuilderTests extends ESTestCase {
XContentParser parser = createParser(shuffled);
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
parser.nextToken();
RescoreBuilder<?> secondRescoreBuilder = RescoreBuilder.parseFromXContent(context);
assertNotSame(rescoreBuilder, secondRescoreBuilder);
@ -139,7 +139,7 @@ public class QueryRescoreBuilderTests extends ESTestCase {
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAsciiOfLengthBetween(1, 10), indexSettings);
// shard context will only need indicesQueriesRegistry for building Query objects nested in query rescorer
QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, null, null, null, null, null, xContentRegistry(),
indicesQueriesRegistry, null, null, () -> nowInMillis) {
null, null, () -> nowInMillis) {
@Override
public MappedFieldType fieldMapper(String name) {
TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name);
@ -252,12 +252,17 @@ public class QueryRescoreBuilderTests extends ESTestCase {
*/
private QueryParseContext createContext(String rescoreElement) throws IOException {
XContentParser parser = createParser(JsonXContent.jsonXContent, rescoreElement);
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
// move to first token, this is where the internal fromXContent
assertTrue(parser.nextToken() == XContentParser.Token.START_OBJECT);
return context;
}
@Override
protected NamedXContentRegistry xContentRegistry() {
return xContentRegistry;
}
private static RescoreBuilder<?> mutate(RescoreBuilder<?> original) throws IOException {
RescoreBuilder<?> mutation = ESTestCase.copyWriteable(original, namedWriteableRegistry, QueryRescorerBuilder::new);
if (randomBoolean()) {

View File

@ -28,14 +28,9 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.query.MatchAllQueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryParser;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.test.ESTestCase;
import org.hamcrest.Matchers;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import java.io.IOException;
import java.util.Collections;
@ -44,22 +39,6 @@ import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashC
public class SearchAfterBuilderTests extends ESTestCase {
private static final int NUMBER_OF_TESTBUILDERS = 20;
private static IndicesQueriesRegistry indicesQueriesRegistry;
/**
* setup for the whole base test class
*/
@BeforeClass
public static void init() {
indicesQueriesRegistry = new IndicesQueriesRegistry();
QueryParser<MatchAllQueryBuilder> parser = MatchAllQueryBuilder::fromXContent;
indicesQueriesRegistry.register(parser, MatchAllQueryBuilder.NAME);
}
@AfterClass
public static void afterClass() throws Exception {
indicesQueriesRegistry = null;
}
private static SearchAfterBuilder randomSearchAfterBuilder() throws IOException {
int numSearchFrom = randomIntBetween(1, 10);
@ -189,7 +168,7 @@ public class SearchAfterBuilderTests extends ESTestCase {
searchAfterBuilder.innerToXContent(builder);
builder.endObject();
XContentParser parser = createParser(shuffleXContent(builder));
new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
new QueryParseContext(parser, ParseFieldMatcher.STRICT);
parser.nextToken();
parser.nextToken();
parser.nextToken();

View File

@ -39,14 +39,9 @@ import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.query.MatchAllQueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.test.ESTestCase;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import java.io.IOException;
import java.util.ArrayList;
@ -66,22 +61,6 @@ import static org.mockito.Mockito.when;
public class SliceBuilderTests extends ESTestCase {
private static final int MAX_SLICE = 20;
private static IndicesQueriesRegistry indicesQueriesRegistry;
/**
* setup for the whole base test class
*/
@BeforeClass
public static void init() {
indicesQueriesRegistry = new IndicesQueriesRegistry();
QueryParser<MatchAllQueryBuilder> parser = MatchAllQueryBuilder::fromXContent;
indicesQueriesRegistry.register(parser, MatchAllQueryBuilder.NAME);
}
@AfterClass
public static void afterClass() throws Exception {
indicesQueriesRegistry = null;
}
private static SliceBuilder randomSliceBuilder() throws IOException {
int max = randomIntBetween(2, MAX_SLICE);
@ -125,8 +104,7 @@ public class SliceBuilderTests extends ESTestCase {
sliceBuilder.innerToXContent(builder);
builder.endObject();
XContentParser parser = createParser(shuffleXContent(builder));
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser,
ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
SliceBuilder secondSliceBuilder = SliceBuilder.fromXContent(context);
assertNotSame(sliceBuilder, secondSliceBuilder);
assertEquals(sliceBuilder, secondSliceBuilder);

View File

@ -26,6 +26,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -50,7 +51,6 @@ import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.TermQueryBuilder;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.CompiledScript;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptContext;
@ -77,11 +77,11 @@ import static java.util.Collections.emptyList;
import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode;
public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends ESTestCase {
private static final int NUMBER_OF_TESTBUILDERS = 20;
protected static NamedWriteableRegistry namedWriteableRegistry;
private static final int NUMBER_OF_TESTBUILDERS = 20;
static IndicesQueriesRegistry indicesQueriesRegistry;
private static NamedXContentRegistry xContentRegistry;
private static ScriptService scriptService;
@BeforeClass
@ -105,13 +105,13 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList());
namedWriteableRegistry = new NamedWriteableRegistry(searchModule.getNamedWriteables());
indicesQueriesRegistry = searchModule.getQueryParserRegistry();
xContentRegistry = new NamedXContentRegistry(searchModule.getNamedXContents());
}
@AfterClass
public static void afterClass() throws Exception {
namedWriteableRegistry = null;
indicesQueriesRegistry = null;
xContentRegistry = null;
}
/** Returns random sort that is put under test */
@ -146,7 +146,7 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
String elementName = itemParser.currentName();
itemParser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, itemParser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(itemParser, ParseFieldMatcher.STRICT);
T parsedItem = fromXContent(context, elementName);
assertNotSame(testItem, parsedItem);
assertEquals(testItem, parsedItem);
@ -210,7 +210,7 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
});
long nowInMillis = randomPositiveLong();
return new QueryShardContext(0, idxSettings, bitsetFilterCache, ifds, null, null, scriptService,
xContentRegistry(), indicesQueriesRegistry, null, null, () -> nowInMillis) {
xContentRegistry(), null, null, () -> nowInMillis) {
@Override
public MappedFieldType fieldMapper(String name) {
return provideMappedFieldType(name);
@ -235,6 +235,11 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
return doubleFieldType;
}
@Override
protected NamedXContentRegistry xContentRegistry() {
return xContentRegistry;
}
protected static QueryBuilder randomNestedFilter() {
int id = randomIntBetween(0, 2);
switch(id) {

View File

@ -134,7 +134,7 @@ public class FieldSortBuilderTests extends AbstractSortTestCase<FieldSortBuilder
parser.nextToken();
parser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
try {
FieldSortBuilder.fromXContent(context, "");

View File

@ -37,7 +37,6 @@ import org.elasticsearch.index.query.GeoValidationMethod;
import org.elasticsearch.index.query.MatchAllQueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.test.geo.RandomGeoGenerator;
@ -208,7 +207,7 @@ public class GeoDistanceSortBuilderTests extends AbstractSortTestCase<GeoDistanc
XContentParser itemParser = createParser(JsonXContent.jsonXContent, json);
itemParser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, itemParser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(itemParser, ParseFieldMatcher.STRICT);
try {
GeoDistanceSortBuilder.fromXContent(context, "");
@ -229,7 +228,7 @@ public class GeoDistanceSortBuilderTests extends AbstractSortTestCase<GeoDistanc
XContentParser itemParser = createParser(JsonXContent.jsonXContent, json);
itemParser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, itemParser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(itemParser, ParseFieldMatcher.STRICT);
try {
GeoDistanceSortBuilder.fromXContent(context, "");
@ -246,7 +245,7 @@ public class GeoDistanceSortBuilderTests extends AbstractSortTestCase<GeoDistanc
XContentParser itemParser = createParser(JsonXContent.jsonXContent, json);
itemParser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, itemParser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(itemParser, ParseFieldMatcher.STRICT);
try {
GeoDistanceSortBuilder item = GeoDistanceSortBuilder.fromXContent(context, "");
@ -273,7 +272,7 @@ public class GeoDistanceSortBuilderTests extends AbstractSortTestCase<GeoDistanc
XContentParser itemParser = createParser(JsonXContent.jsonXContent, json);
itemParser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, itemParser, ParseFieldMatcher.EMPTY);
QueryParseContext context = new QueryParseContext(itemParser, ParseFieldMatcher.EMPTY);
GeoDistanceSortBuilder.fromXContent(context, "");
assertWarnings("Deprecated field [coerce] used, replaced by [validation_method]");
}
@ -292,7 +291,7 @@ public class GeoDistanceSortBuilderTests extends AbstractSortTestCase<GeoDistanc
XContentParser itemParser = createParser(JsonXContent.jsonXContent, json);
itemParser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, itemParser, ParseFieldMatcher.EMPTY);
QueryParseContext context = new QueryParseContext(itemParser, ParseFieldMatcher.EMPTY);
GeoDistanceSortBuilder.fromXContent(context, "");
assertWarnings("Deprecated field [ignore_malformed] used, replaced by [validation_method]");
}
@ -310,7 +309,7 @@ public class GeoDistanceSortBuilderTests extends AbstractSortTestCase<GeoDistanc
XContentParser itemParser = createParser(JsonXContent.jsonXContent, json);
itemParser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, itemParser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(itemParser, ParseFieldMatcher.STRICT);
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> GeoDistanceSortBuilder.fromXContent(context, ""));
assertEquals("sort_mode [sum] isn't supported for sorting by geo distance", e.getMessage());
@ -335,7 +334,7 @@ public class GeoDistanceSortBuilderTests extends AbstractSortTestCase<GeoDistanc
XContentParser itemParser = createParser(JsonXContent.jsonXContent, json);
itemParser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, itemParser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(itemParser, ParseFieldMatcher.STRICT);
GeoDistanceSortBuilder result = GeoDistanceSortBuilder.fromXContent(context, json);
assertEquals("[-19.700583312660456, -2.8225036337971687, "
@ -457,7 +456,7 @@ public class GeoDistanceSortBuilderTests extends AbstractSortTestCase<GeoDistanc
private GeoDistanceSortBuilder parse(XContentBuilder sortBuilder) throws Exception {
XContentParser parser = createParser(sortBuilder);
QueryParseContext parseContext = new QueryParseContext(new IndicesQueriesRegistry(), parser, ParseFieldMatcher.STRICT);
QueryParseContext parseContext = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
parser.nextToken();
return GeoDistanceSortBuilder.fromXContent(parseContext, null);
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.search.sort;
import org.apache.lucene.search.SortField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.query.QueryParseContext;
@ -69,7 +68,7 @@ public class ScoreSortBuilderTests extends AbstractSortTestCase<ScoreSortBuilder
parser.nextToken();
parser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
ScoreSortBuilder scoreSort = ScoreSortBuilder.fromXContent(context, "_score");
assertEquals(order, scoreSort.order());
}
@ -82,7 +81,7 @@ public class ScoreSortBuilderTests extends AbstractSortTestCase<ScoreSortBuilder
parser.nextToken();
parser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.EMPTY);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.EMPTY);
try {
ScoreSortBuilder.fromXContent(context, "_score");

View File

@ -172,7 +172,7 @@ public class ScriptSortBuilderTests extends AbstractSortTestCase<ScriptSortBuild
parser.nextToken();
parser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
ScriptSortBuilder builder = ScriptSortBuilder.fromXContent(context, null);
assertEquals("doc['field_name'].value * factor", builder.script().getIdOrCode());
assertEquals(Script.DEFAULT_SCRIPT_LANG, builder.script().getLang());
@ -198,7 +198,7 @@ public class ScriptSortBuilderTests extends AbstractSortTestCase<ScriptSortBuild
parser.nextToken();
parser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
ScriptSortBuilder builder = ScriptSortBuilder.fromXContent(context, null);
assertEquals("doc['field_name'].value", builder.script().getIdOrCode());
assertEquals(Script.DEFAULT_SCRIPT_LANG, builder.script().getLang());
@ -218,7 +218,7 @@ public class ScriptSortBuilderTests extends AbstractSortTestCase<ScriptSortBuild
parser.nextToken();
parser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
Exception e = expectThrows(IllegalArgumentException.class, () -> ScriptSortBuilder.fromXContent(context, null));
assertEquals("[_script] unknown field [bad_field], parser not found", e.getMessage());
}
@ -231,7 +231,7 @@ public class ScriptSortBuilderTests extends AbstractSortTestCase<ScriptSortBuild
parser.nextToken();
parser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
Exception e = expectThrows(IllegalArgumentException.class, () -> ScriptSortBuilder.fromXContent(context, null));
assertEquals("[_script] unknown field [bad_field], parser not found", e.getMessage());
}
@ -243,7 +243,7 @@ public class ScriptSortBuilderTests extends AbstractSortTestCase<ScriptSortBuild
parser.nextToken();
parser.nextToken();
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
Exception e = expectThrows(IllegalArgumentException.class, () -> ScriptSortBuilder.fromXContent(context, null));
assertEquals("[_script] script doesn't support values of type: START_ARRAY", e.getMessage());
}

View File

@ -21,15 +21,14 @@ package org.elasticsearch.search.sort;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.test.ESTestCase;
import org.junit.AfterClass;
@ -43,24 +42,19 @@ import java.util.List;
import static java.util.Collections.emptyList;
public class SortBuilderTests extends ESTestCase {
private static final int NUMBER_OF_RUNS = 20;
protected static NamedWriteableRegistry namedWriteableRegistry;
static IndicesQueriesRegistry indicesQueriesRegistry;
private static NamedXContentRegistry xContentRegistry;
@BeforeClass
public static void init() {
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList());
namedWriteableRegistry = new NamedWriteableRegistry(searchModule.getNamedWriteables());
indicesQueriesRegistry = searchModule.getQueryParserRegistry();
xContentRegistry = new NamedXContentRegistry(searchModule.getNamedXContents());
}
@AfterClass
public static void afterClass() throws Exception {
namedWriteableRegistry = null;
indicesQueriesRegistry = null;
xContentRegistry = null;
}
/**
@ -237,9 +231,14 @@ public class SortBuilderTests extends ESTestCase {
assertEquals(new ScoreSortBuilder(), result.get(5));
}
@Override
protected NamedXContentRegistry xContentRegistry() {
return xContentRegistry;
}
private List<SortBuilder<?>> parseSort(String jsonString) throws IOException {
XContentParser itemParser = createParser(JsonXContent.jsonXContent, jsonString);
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, itemParser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(itemParser, ParseFieldMatcher.STRICT);
assertEquals(XContentParser.Token.START_OBJECT, itemParser.nextToken());
assertEquals(XContentParser.Token.FIELD_NAME, itemParser.nextToken());

View File

@ -29,7 +29,6 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.test.ESTestCase;
import org.junit.AfterClass;
@ -44,7 +43,6 @@ public abstract class AbstractSuggestionBuilderTestCase<SB extends SuggestionBui
private static final int NUMBER_OF_TESTBUILDERS = 20;
protected static NamedWriteableRegistry namedWriteableRegistry;
protected static IndicesQueriesRegistry queriesRegistry;
protected static ParseFieldMatcher parseFieldMatcher;
protected static Suggesters suggesters;
@ -55,7 +53,6 @@ public abstract class AbstractSuggestionBuilderTestCase<SB extends SuggestionBui
public static void init() throws IOException {
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList());
namedWriteableRegistry = new NamedWriteableRegistry(searchModule.getNamedWriteables());
queriesRegistry = searchModule.getQueryParserRegistry();
suggesters = searchModule.getSuggesters();
parseFieldMatcher = ParseFieldMatcher.STRICT;
}
@ -64,7 +61,6 @@ public abstract class AbstractSuggestionBuilderTestCase<SB extends SuggestionBui
public static void afterClass() throws Exception {
namedWriteableRegistry = null;
suggesters = null;
queriesRegistry = null;
}
/**
@ -128,7 +124,7 @@ public abstract class AbstractSuggestionBuilderTestCase<SB extends SuggestionBui
XContentBuilder shuffled = shuffleXContent(xContentBuilder, shuffleProtectedFields());
XContentParser parser = createParser(shuffled);
QueryParseContext context = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
QueryParseContext context = new QueryParseContext(parser, parseFieldMatcher);
// we need to skip the start object and the name, those will be parsed by outer SuggestBuilder
parser.nextToken();
@ -191,7 +187,7 @@ public abstract class AbstractSuggestionBuilderTestCase<SB extends SuggestionBui
}
protected static QueryParseContext newParseContext(XContentParser parser) throws IOException {
final QueryParseContext parseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
final QueryParseContext parseContext = new QueryParseContext(parser, parseFieldMatcher);
return parseContext;
}
}

View File

@ -25,11 +25,9 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.suggest.completion.CompletionSuggesterBuilderTests;
import org.elasticsearch.search.suggest.phrase.PhraseSuggestionBuilderTests;
@ -78,7 +76,7 @@ public class SuggestBuilderTests extends ESTestCase {
}
suggestBuilder.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS);
XContentParser parser = createParser(xContentBuilder);
QueryParseContext context = new QueryParseContext(new IndicesQueriesRegistry(), parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
SuggestBuilder secondSuggestBuilder = SuggestBuilder.fromXContent(context, suggesters);
assertNotSame(suggestBuilder, secondSuggestBuilder);
assertEquals(suggestBuilder, secondSuggestBuilder);

View File

@ -34,7 +34,6 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.suggest.completion.context.CategoryContextMapping;
import org.elasticsearch.search.suggest.completion.context.ContextBuilder;
import org.elasticsearch.search.suggest.completion.context.ContextMapping;
@ -258,7 +257,7 @@ public class CategoryContextMappingTests extends ESSingleNodeTestCase {
}
private static QueryParseContext createParseContext(XContentParser parser) {
return new QueryParseContext(new IndicesQueriesRegistry(), parser, ParseFieldMatcher.STRICT);
return new QueryParseContext(parser, ParseFieldMatcher.STRICT);
}
public void testQueryContextParsingMixed() throws Exception {

View File

@ -30,7 +30,6 @@ import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.suggest.completion.context.ContextBuilder;
import org.elasticsearch.search.suggest.completion.context.ContextMapping;
import org.elasticsearch.search.suggest.completion.context.GeoContextMapping;
@ -352,6 +351,6 @@ public class GeoContextMappingTests extends ESSingleNodeTestCase {
}
private static QueryParseContext createParseContext(XContentParser parser) {
return new QueryParseContext(new IndicesQueriesRegistry(), parser, ParseFieldMatcher.STRICT);
return new QueryParseContext(parser, ParseFieldMatcher.STRICT);
};
}

View File

@ -20,13 +20,11 @@
package org.elasticsearch.search.suggest.completion;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
@ -52,7 +50,7 @@ public abstract class QueryContextTestCase<QC extends ToXContent> extends ESTest
toXContent.toXContent(builder, ToXContent.EMPTY_PARAMS);
XContentParser parser = createParser(builder);
parser.nextToken();
QC fromXContext = fromXContent(new QueryParseContext(new IndicesQueriesRegistry(), parser, ParseFieldMatcher.STRICT));
QC fromXContext = fromXContent(new QueryParseContext(parser, ParseFieldMatcher.STRICT));
assertEquals(toXContent, fromXContext);
assertEquals(toXContent.hashCode(), fromXContext.hashCode());
}

View File

@ -30,7 +30,6 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.suggest.phrase.PhraseSuggestionContext.DirectCandidateGenerator;
import org.elasticsearch.test.ESTestCase;
@ -42,13 +41,9 @@ import java.util.function.Supplier;
import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode;
public class DirectCandidateGeneratorTests extends ESTestCase{
private static final IndicesQueriesRegistry mockRegistry = new IndicesQueriesRegistry();
public class DirectCandidateGeneratorTests extends ESTestCase {
private static final int NUMBER_OF_RUNS = 20;
/**
* Test serialization and deserialization of the generator
*/
@ -113,7 +108,7 @@ public class DirectCandidateGeneratorTests extends ESTestCase{
}
generator.toXContent(builder, ToXContent.EMPTY_PARAMS);
XContentParser parser = createParser(shuffleXContent(builder));
QueryParseContext context = new QueryParseContext(mockRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
parser.nextToken();
DirectCandidateGeneratorBuilder secondGenerator = DirectCandidateGeneratorBuilder.fromXContent(context);
assertNotSame(generator, secondGenerator);
@ -177,7 +172,7 @@ public class DirectCandidateGeneratorTests extends ESTestCase{
private void assertIllegalXContent(String directGenerator, Class<? extends Exception> exceptionClass, String exceptionMsg)
throws IOException {
XContentParser parser = createParser(JsonXContent.jsonXContent, directGenerator);
QueryParseContext context = new QueryParseContext(mockRegistry, parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
Exception e = expectThrows(exceptionClass, () -> DirectCandidateGeneratorBuilder.fromXContent(context));
assertEquals(exceptionMsg, e.getMessage());
}

View File

@ -39,7 +39,6 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.test.ESTestCase;
import org.junit.AfterClass;
@ -99,7 +98,7 @@ public abstract class SmoothingModelTestCase extends ESTestCase {
testModel.innerToXContent(contentBuilder, ToXContent.EMPTY_PARAMS);
contentBuilder.endObject();
XContentParser parser = createParser(shuffleXContent(contentBuilder));
QueryParseContext context = new QueryParseContext(new IndicesQueriesRegistry(), parser, ParseFieldMatcher.STRICT);
QueryParseContext context = new QueryParseContext(parser, ParseFieldMatcher.STRICT);
parser.nextToken(); // go to start token, real parsing would do that in the outer element parser
SmoothingModel parsedModel = fromXContent(context);
assertNotSame(testModel, parsedModel);

View File

@ -208,8 +208,8 @@ public class TCPTransportTests extends ESTestCase {
@Override
public NodeChannels getConnection(DiscoveryNode node) {
return new NodeChannels(node, new Object[ConnectionProfile.LIGHT_PROFILE.getNumConnections()],
ConnectionProfile.LIGHT_PROFILE);
return new NodeChannels(node, new Object[MockTcpTransport.LIGHT_PROFILE.getNumConnections()],
MockTcpTransport.LIGHT_PROFILE);
}
};
DiscoveryNode node = new DiscoveryNode("foo", buildNewFakeTransportAddress(), Version.CURRENT);

View File

@ -113,7 +113,7 @@ public class TransportServiceHandshakeTests extends ESTestCase {
emptyMap(),
emptySet(),
Version.CURRENT.minimumCompatibilityVersion());
try (Transport.Connection connection = handleA.transportService.openConnection(discoveryNode, ConnectionProfile.LIGHT_PROFILE)){
try (Transport.Connection connection = handleA.transportService.openConnection(discoveryNode, MockTcpTransport.LIGHT_PROFILE)){
DiscoveryNode connectedNode = handleA.transportService.handshake(connection, timeout);
assertNotNull(connectedNode);
// the name and version should be updated
@ -121,16 +121,6 @@ public class TransportServiceHandshakeTests extends ESTestCase {
assertEquals(connectedNode.getVersion(), handleB.discoveryNode.getVersion());
assertFalse(handleA.transportService.nodeConnected(discoveryNode));
}
DiscoveryNode connectedNode =
handleA.transportService.connectToNodeAndHandshake(discoveryNode, timeout);
assertNotNull(connectedNode);
// the name and version should be updated
assertEquals(connectedNode.getName(), "TS_B");
assertEquals(connectedNode.getVersion(), handleB.discoveryNode.getVersion());
assertTrue(handleA.transportService.nodeConnected(discoveryNode));
}
public void testMismatchedClusterName() {
@ -145,7 +135,7 @@ public class TransportServiceHandshakeTests extends ESTestCase {
Version.CURRENT.minimumCompatibilityVersion());
IllegalStateException ex = expectThrows(IllegalStateException.class, () -> {
try (Transport.Connection connection = handleA.transportService.openConnection(discoveryNode,
ConnectionProfile.LIGHT_PROFILE)) {
MockTcpTransport.LIGHT_PROFILE)) {
handleA.transportService.handshake(connection, timeout);
}
});
@ -166,7 +156,7 @@ public class TransportServiceHandshakeTests extends ESTestCase {
Version.CURRENT.minimumCompatibilityVersion());
IllegalStateException ex = expectThrows(IllegalStateException.class, () -> {
try (Transport.Connection connection = handleA.transportService.openConnection(discoveryNode,
ConnectionProfile.LIGHT_PROFILE)) {
MockTcpTransport.LIGHT_PROFILE)) {
handleA.transportService.handshake(connection, timeout);
}
});

View File

@ -8,7 +8,7 @@
# the most important settings you may want to configure for a production cluster.
#
# Please see the documentation for further information on configuration options:
# <http://www.elastic.co/guide/en/elasticsearch/reference/current/setup-configuration.html>
# <https://www.elastic.co/guide/en/elasticsearch/reference/current/settings.html>
#
# ---------------------------------- Cluster -----------------------------------
#

View File

@ -156,8 +156,8 @@ String _type = response.getType();
String _id = response.getId();
// Version (if it's the first time you index this document, you will get: 1)
long _version = response.getVersion();
// isCreated() is true if the document is a new one, false if it has been updated
boolean created = response.isCreated();
// status has stored current instance statement.
RestStatus status = response.status();
--------------------------------------------------
For more information on the index operation, check out the REST

View File

@ -1,3 +1,10 @@
[[java-query-percolate-query]]
==== Percolate Query
See:
* {ref}/query-dsl-percolate-query.html[Percolate Query]
[source,java]
--------------------------------------------------
Settings settings = Settings.builder().put("cluster.name", "elasticsearch").build();

View File

@ -120,6 +120,7 @@ The available values are:
* `ap-northeast` (`ap-northeast-1`) for Asia Pacific (Tokyo)
* `ap-northeast-2` (`ap-northeast-2`) for Asia Pacific (Seoul)
* `eu-west` (`eu-west-1`) for EU (Ireland)
* `eu-west-2` (`eu-west-2`) for EU (London)
* `eu-central` (`eu-central-1`) for EU (Frankfurt)
* `sa-east` (`sa-east-1`) for South America (São Paulo)
* `cn-north` (`cn-north-1`) for China (Beijing)

View File

@ -105,3 +105,5 @@ Returns this:
NOTE: Extracting contents from binary data is a resource intensive operation and
consumes a lot of resources. It is highly recommended to run pipelines
using this processor in a dedicated ingest node.
NOTE: To process an array of attachments the {ref}/foreach-processor.html[foreach processor] is required.

View File

@ -125,6 +125,7 @@ The available values are:
* `ap-northeast` (`ap-northeast-1`) for Asia Pacific (Tokyo)
* `ap-northeast-2` (`ap-northeast-2`) for Asia Pacific (Seoul)
* `eu-west` (`eu-west-1`) for EU (Ireland)
* `eu-west-2` (`eu-west-2`) for EU (London)
* `eu-central` (`eu-central-1`) for EU (Frankfurt)
* `sa-east` (`sa-east-1`) for South America (São Paulo)
* `cn-north` (`cn-north-1`) for China (Beijing)

View File

@ -55,9 +55,9 @@ functionality is implemented by adding the
with the `keywords` set to the value of the `stem_exclusion` parameter.
The following analyzers support setting custom `stem_exclusion` list:
`arabic`, `armenian`, `basque`, `catalan`, `bulgarian`, `catalan`,
`czech`, `finnish`, `dutch`, `english`, `finnish`, `french`, `galician`,
`german`, `irish`, `hindi`, `hungarian`, `indonesian`, `italian`, `latvian`,
`arabic`, `armenian`, `basque`, `bulgarian`, `catalan`, `czech`,
`dutch`, `english`, `finnish`, `french`, `galician`,
`german`, `hindi`, `hungarian`, `indonesian`, `irish`, `italian`, `latvian`,
`lithuanian`, `norwegian`, `portuguese`, `romanian`, `russian`, `sorani`,
`spanish`, `swedish`, `turkish`.

View File

@ -39,6 +39,12 @@ PUT my_index
--------------------------------------------------
// CONSOLE
NOTE: The `double`, `float` and `half_float` types consider that `-0.0` and
`+0.0` are different values. As a consequence, doing a `term` query on
`-0.0` will not match `+0.0` and vice-versa. Same is true for range queries:
if the upper bound is `-0.0` then `+0.0` will not match, and if the lower
bound is `+0.0` then `-0.0` will not match.
==== Which type should I use?
As far as integer types (`byte`, `short`, `integer` and `long`) are concerned,

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