Merge remote-tracking branch 'origin/master' into index-lifecycle

This commit is contained in:
Lee Hinman 2018-10-25 11:00:36 -06:00
commit 3e7042832a
547 changed files with 7627 additions and 3984 deletions

View File

@ -159,6 +159,7 @@ Please follow these formatting guidelines:
* Java indent is 4 spaces
* Line width is 140 characters
* Line width for code snippets that are included in the documentation (the ones surrounded by `// tag` and `// end` comments) is 76 characters
* The rest is left to Java coding standards
* Disable “auto-format on save” to prevent unnecessary format changes. This makes reviews much harder as it generates unnecessary formatting changes. If your IDE supports formatting only modified chunks that is fine to do.
* Wildcard imports (`import foo.bar.baz.*`) are forbidden and will cause the build to fail. This can be done automatically by your IDE:

View File

@ -31,22 +31,12 @@ if (GradleVersion.current() < GradleVersion.version(minimumGradleVersion)) {
throw new GradleException("Gradle ${minimumGradleVersion}+ is required to build elasticsearch")
}
if (JavaVersion.current() < JavaVersion.VERSION_1_8) {
throw new GradleException('Java 1.8 is required to build elasticsearch gradle tools')
}
if (project == rootProject) {
// change the build dir used during build init, so that doing a clean
// won't wipe out the buildscript jar
buildDir = 'build-bootstrap'
}
// Make sure :buildSrc: doesn't generate classes incompatible with RUNTIME_JAVA_HOME
// We can't use BuildPlugin here, so read from file
String minimumRuntimeVersion = file('src/main/resources/minimumRuntimeVersion').text.trim()
targetCompatibility = minimumRuntimeVersion
sourceCompatibility = minimumRuntimeVersion
/*****************************************************************************
* Propagating version.properties to the rest of the build *
*****************************************************************************/
@ -82,6 +72,45 @@ processResources {
from tempPropertiesFile
}
if (JavaVersion.current() < JavaVersion.VERSION_1_10) {
throw new GradleException('At least Java 10 is required to build elasticsearch gradle tools')
}
/*****************************************************************************
* Java version *
*****************************************************************************/
// Gradle 4.10 does not support setting this to 11 yet
targetCompatibility = "10"
sourceCompatibility = "10"
// We have a few classes that need to be compiled for older java versions because these are used to run checks against
// those
sourceSets {
minimumRuntime {
// We only want Java here, but the Groovy doesn't configure javadoc correctly if we don't define this as groovy
groovy {
srcDirs = ['src/main/minimumRuntime']
}
}
}
compileMinimumRuntimeGroovy {
// We can't use BuildPlugin here, so read from file
String minimumRuntimeVersion = file('src/main/resources/minimumRuntimeVersion').text.trim()
targetCompatibility = minimumRuntimeVersion
sourceCompatibility = minimumRuntimeVersion
}
dependencies {
compile sourceSets.minimumRuntime.output
minimumRuntimeCompile "junit:junit:${props.getProperty('junit')}"
minimumRuntimeCompile localGroovy()
}
jar {
from sourceSets.minimumRuntime.output
}
/*****************************************************************************
* Dependencies used by the entire build *
*****************************************************************************/
@ -94,10 +123,7 @@ dependencies {
compile localGroovy()
compile "com.carrotsearch.randomizedtesting:junit4-ant:${props.getProperty('randomizedrunner')}"
compile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${props.getProperty('randomizedrunner')}"
compile("junit:junit:${props.getProperty('junit')}") {
transitive = false
}
compile 'com.netflix.nebula:gradle-extra-configurations-plugin:3.0.3'
compile 'com.netflix.nebula:nebula-publishing-plugin:4.4.4'
compile 'com.netflix.nebula:gradle-info-plugin:3.0.3'
@ -156,6 +182,7 @@ if (project != rootProject) {
dependenciesInfo.enabled = false
forbiddenApisMain.enabled = false
forbiddenApisTest.enabled = false
forbiddenApisMinimumRuntime.enabled = false
jarHell.enabled = false
thirdPartyAudit.enabled = false

View File

@ -99,12 +99,14 @@ class BuildPlugin implements Plugin<Project> {
configureSourcesJar(project)
configurePomGeneration(project)
applyCommonTestConfig(project)
configureTest(project)
configurePrecommit(project)
configureDependenciesInfo(project)
}
/** Performs checks on the build environment and prints information about the build environment. */
static void globalBuildInfo(Project project) {
if (project.rootProject.ext.has('buildChecksDone') == false) {
@ -776,9 +778,8 @@ class BuildPlugin implements Plugin<Project> {
}
}
/** Returns a closure of common configuration shared by unit and integration tests. */
static Closure commonTestConfig(Project project) {
return {
static void applyCommonTestConfig(Project project) {
project.tasks.withType(RandomizedTestingTask) {
jvm "${project.runtimeJavaHome}/bin/java"
parallelism System.getProperty('tests.jvms', 'auto')
ifNoTests System.getProperty('tests.ifNoTests', 'fail')
@ -873,6 +874,8 @@ class BuildPlugin implements Plugin<Project> {
exclude '**/*$*.class'
dependsOn(project.tasks.testClasses)
project.plugins.withType(ShadowPlugin).whenPluginAdded {
// Test against a shadow jar if we made one
classpath -= project.tasks.compileJava.outputs.files
@ -884,23 +887,9 @@ class BuildPlugin implements Plugin<Project> {
/** Configures the test task */
static Task configureTest(Project project) {
RandomizedTestingTask test = project.tasks.getByName('test')
test.configure(commonTestConfig(project))
test.configure {
project.tasks.getByName('test') {
include '**/*Tests.class'
}
// Add a method to create additional unit tests for a project, which will share the same
// randomized testing setup, but by default run no tests.
project.extensions.add('additionalTest', { String name, Closure config ->
RandomizedTestingTask additionalTest = project.tasks.create(name, RandomizedTestingTask.class)
additionalTest.classpath = test.classpath
additionalTest.testClassesDirs = test.testClassesDirs
additionalTest.configure(commonTestConfig(project))
additionalTest.configure(config)
additionalTest.dependsOn(project.tasks.testClasses)
project.check.dependsOn(additionalTest)
});
}
private static configurePrecommit(Project project) {

View File

@ -68,7 +68,9 @@ class ClusterConfiguration {
* In case of more than one node, this defaults to the number of nodes
*/
@Input
Closure<Integer> minimumMasterNodes = { getNumNodes() > 1 ? getNumNodes() : -1 }
Closure<Integer> minimumMasterNodes = {
return getNumNodes() > 1 ? getNumNodes() : -1
}
@Input
String jvmArgs = "-Xms" + System.getProperty('tests.heap.size', '512m') +

View File

@ -122,8 +122,31 @@ class ClusterFormationTasks {
}
NodeInfo node = new NodeInfo(config, i, project, prefix, elasticsearchVersion, sharedDir)
nodes.add(node)
Object dependsOn = startTasks.empty ? startDependencies : startTasks.get(0)
startTasks.add(configureNode(project, prefix, runner, dependsOn, node, config, distro, nodes.get(0)))
Closure<Map> writeConfigSetup
Object dependsOn
if (node.nodeVersion.onOrAfter("6.5.0-SNAPSHOT")) {
writeConfigSetup = { Map esConfig ->
// Don't force discovery provider if one is set by the test cluster specs already
if (esConfig.containsKey('discovery.zen.hosts_provider') == false) {
esConfig['discovery.zen.hosts_provider'] = 'file'
}
esConfig['discovery.zen.ping.unicast.hosts'] = []
esConfig
}
dependsOn = startDependencies
} else {
dependsOn = startTasks.empty ? startDependencies : startTasks.get(0)
writeConfigSetup = { Map esConfig ->
String unicastTransportUri = node.config.unicastTransportUri(nodes.get(0), node, project.ant)
if (unicastTransportUri == null) {
esConfig['discovery.zen.ping.unicast.hosts'] = []
} else {
esConfig['discovery.zen.ping.unicast.hosts'] = "\"${unicastTransportUri}\""
}
esConfig
}
}
startTasks.add(configureNode(project, prefix, runner, dependsOn, node, config, distro, writeConfigSetup))
}
Task wait = configureWaitTask("${prefix}#wait", project, nodes, startTasks, config.nodeStartupWaitSeconds)
@ -182,7 +205,7 @@ class ClusterFormationTasks {
* @return a task which starts the node.
*/
static Task configureNode(Project project, String prefix, Task runner, Object dependsOn, NodeInfo node, ClusterConfiguration config,
Configuration distribution, NodeInfo seedNode) {
Configuration distribution, Closure<Map> writeConfig) {
// tasks are chained so their execution order is maintained
Task setup = project.tasks.create(name: taskName(prefix, node, 'clean'), type: Delete, dependsOn: dependsOn) {
@ -198,7 +221,7 @@ class ClusterFormationTasks {
setup = configureCheckPreviousTask(taskName(prefix, node, 'checkPrevious'), project, setup, node)
setup = configureStopTask(taskName(prefix, node, 'stopPrevious'), project, setup, node)
setup = configureExtractTask(taskName(prefix, node, 'extract'), project, setup, node, distribution)
setup = configureWriteConfigTask(taskName(prefix, node, 'configure'), project, setup, node, seedNode)
setup = configureWriteConfigTask(taskName(prefix, node, 'configure'), project, setup, node, writeConfig)
setup = configureCreateKeystoreTask(taskName(prefix, node, 'createKeystore'), project, setup, node)
setup = configureAddKeystoreSettingTasks(prefix, project, setup, node)
setup = configureAddKeystoreFileTasks(prefix, project, setup, node)
@ -301,7 +324,7 @@ class ClusterFormationTasks {
}
/** Adds a task to write elasticsearch.yml for the given node configuration */
static Task configureWriteConfigTask(String name, Project project, Task setup, NodeInfo node, NodeInfo seedNode) {
static Task configureWriteConfigTask(String name, Project project, Task setup, NodeInfo node, Closure<Map> configFilter) {
Map esConfig = [
'cluster.name' : node.clusterName,
'node.name' : "node-" + node.nodeNum,
@ -347,10 +370,7 @@ class ClusterFormationTasks {
Task writeConfig = project.tasks.create(name: name, type: DefaultTask, dependsOn: setup)
writeConfig.doFirst {
String unicastTransportUri = node.config.unicastTransportUri(seedNode, node, project.ant)
if (unicastTransportUri != null) {
esConfig['discovery.zen.ping.unicast.hosts'] = "\"${unicastTransportUri}\""
}
esConfig = configFilter.call(esConfig)
File configFile = new File(node.pathConf, 'elasticsearch.yml')
logger.info("Configuring ${configFile}")
configFile.setText(esConfig.collect { key, value -> "${key}: ${value}" }.join('\n'), 'UTF-8')
@ -681,6 +701,19 @@ class ClusterFormationTasks {
static Task configureWaitTask(String name, Project project, List<NodeInfo> nodes, List<Task> startTasks, int waitSeconds) {
Task wait = project.tasks.create(name: name, dependsOn: startTasks)
wait.doLast {
Collection<String> unicastHosts = new HashSet<>()
nodes.forEach { otherNode ->
String unicastHost = otherNode.config.unicastTransportUri(otherNode, null, project.ant)
if (unicastHost != null) {
unicastHosts.addAll(Arrays.asList(unicastHost.split(",")))
}
}
String unicastHostsTxt = String.join("\n", unicastHosts)
nodes.forEach { node ->
node.pathConf.toPath().resolve("unicast_hosts.txt").setText(unicastHostsTxt)
}
ant.waitfor(maxwait: "${waitSeconds}", maxwaitunit: 'second', checkevery: '500', checkeveryunit: 'millisecond', timeoutproperty: "failed${name}") {
or {
for (NodeInfo node : nodes) {

View File

@ -64,8 +64,6 @@ public class RestIntegTestTask extends DefaultTask {
runner.testClassesDirs = project.sourceSets.test.output.classesDirs
clusterConfig = project.extensions.create("${name}Cluster", ClusterConfiguration.class, project)
// start with the common test configuration
runner.configure(BuildPlugin.commonTestConfig(project))
// override/add more for rest tests
runner.parallelism = '1'
runner.include('**/*IT.class')

View File

@ -50,6 +50,7 @@ public class StandaloneRestTestPlugin implements Plugin<Project> {
project.getTasks().create("buildResources", ExportElasticsearchBuildResourcesTask)
BuildPlugin.globalBuildInfo(project)
BuildPlugin.configureRepositories(project)
BuildPlugin.applyCommonTestConfig(project)
// only setup tests to build
project.sourceSets.create('test')

View File

@ -24,7 +24,6 @@ import org.elasticsearch.gradle.BuildPlugin
import org.gradle.api.Plugin
import org.gradle.api.Project
import org.gradle.api.plugins.JavaBasePlugin
import org.gradle.api.tasks.compile.JavaCompile
/**
* Configures the build to compile against Elasticsearch's test framework and
@ -44,7 +43,6 @@ public class StandaloneTestPlugin implements Plugin<Project> {
description: 'Runs unit tests that are separate'
]
RandomizedTestingTask test = project.tasks.create(testOptions)
test.configure(BuildPlugin.commonTestConfig(project))
BuildPlugin.configureCompile(project)
test.classpath = project.sourceSets.test.runtimeClasspath
test.testClassesDirs = project.sourceSets.test.output.classesDirs

View File

@ -39,8 +39,8 @@ import java.util.Map;
public class TestClustersPlugin implements Plugin<Project> {
public static final String LIST_TASK_NAME = "listElasticSearchClusters";
public static final String EXTENSION_NAME = "elasticSearchClusters";
private static final String LIST_TASK_NAME = "listTestClusters";
private static final String NODE_EXTENSION_NAME = "testClusters";
private final Logger logger = Logging.getLogger(TestClustersPlugin.class);
@ -50,7 +50,7 @@ public class TestClustersPlugin implements Plugin<Project> {
ElasticsearchNode.class,
(name) -> new ElasticsearchNode(name, GradleServicesAdapter.getInstance(project))
);
project.getExtensions().add(EXTENSION_NAME, container);
project.getExtensions().add(NODE_EXTENSION_NAME, container);
Task listTask = project.getTasks().create(LIST_TASK_NAME);
listTask.setGroup("ES cluster formation");

View File

@ -614,9 +614,6 @@
<suppress files="server[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]bucket[/\\]TermsDocCountErrorIT.java" checks="LineLength" />
<suppress files="server[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]bucket[/\\]TermsShardMinDocCountIT.java" checks="LineLength" />
<suppress files="server[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]bucket[/\\]nested[/\\]NestedAggregatorTests.java" checks="LineLength" />
<suppress files="server[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]pipeline[/\\]ExtendedStatsBucketIT.java" checks="LineLength" />
<suppress files="server[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]pipeline[/\\]moving[/\\]avg[/\\]MovAvgIT.java" checks="LineLength" />
<suppress files="server[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]pipeline[/\\]serialdiff[/\\]SerialDiffIT.java" checks="LineLength" />
<suppress files="server[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]basic[/\\]SearchWhileCreatingIndexIT.java" checks="LineLength" />
<suppress files="server[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]basic[/\\]SearchWhileRelocatingIT.java" checks="LineLength" />
<suppress files="server[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]basic[/\\]SearchWithRandomExceptionsIT.java" checks="LineLength" />

View File

@ -31,11 +31,11 @@ public class TestClustersPluginIT extends GradleIntegrationTestCase {
public void testListClusters() {
BuildResult result = GradleRunner.create()
.withProjectDir(getProjectDir("testclusters"))
.withArguments("listElasticSearchClusters", "-s")
.withArguments("listTestClusters", "-s")
.withPluginClasspath()
.build();
assertEquals(TaskOutcome.SUCCESS, result.task(":listElasticSearchClusters").getOutcome());
assertEquals(TaskOutcome.SUCCESS, result.task(":listTestClusters").getOutcome());
assertOutputContains(
result.getOutput(),
" * myTestCluster:"

View File

@ -2,40 +2,40 @@ plugins {
id 'elasticsearch.testclusters'
}
elasticSearchClusters {
testClusters {
myTestCluster {
distribution = 'ZIP'
}
}
task user1 {
useCluster elasticSearchClusters.myTestCluster
useCluster testClusters.myTestCluster
doLast {
println "user1 executing"
}
}
task user2 {
useCluster elasticSearchClusters.myTestCluster
useCluster testClusters.myTestCluster
doLast {
println "user2 executing"
}
}
task upToDate1 {
useCluster elasticSearchClusters.myTestCluster
useCluster testClusters.myTestCluster
}
task upToDate2 {
useCluster elasticSearchClusters.myTestCluster
useCluster testClusters.myTestCluster
}
task skipped1 {
enabled = false
useCluster elasticSearchClusters.myTestCluster
useCluster testClusters.myTestCluster
}
task skipped2 {
enabled = false
useCluster elasticSearchClusters.myTestCluster
useCluster testClusters.myTestCluster
}

View File

@ -31,6 +31,8 @@ import java.io.IOException;
final class ClusterRequestConverters {
private ClusterRequestConverters() {}
static Request clusterPutSettings(ClusterUpdateSettingsRequest clusterUpdateSettingsRequest) throws IOException {
Request request = new Request(HttpPut.METHOD_NAME, "/_cluster/settings");

View File

@ -24,7 +24,9 @@ import org.elasticsearch.protocol.xpack.graph.GraphExploreRequest;
import java.io.IOException;
public class GraphRequestConverters {
final class GraphRequestConverters {
private GraphRequestConverters() {}
static Request explore(GraphExploreRequest exploreRequest) throws IOException {
String endpoint = RequestConverters.endpoint(exploreRequest.indices(), exploreRequest.types(), "_xpack/graph/_explore");

View File

@ -53,7 +53,10 @@ import org.elasticsearch.common.Strings;
import java.io.IOException;
import java.util.Locale;
public class IndicesRequestConverters {
final class IndicesRequestConverters {
private IndicesRequestConverters() {}
static Request deleteIndex(DeleteIndexRequest deleteIndexRequest) {
String endpoint = RequestConverters.endpoint(deleteIndexRequest.indices());
Request request = new Request(HttpDelete.METHOD_NAME, endpoint);

View File

@ -30,7 +30,9 @@ import org.elasticsearch.action.ingest.SimulatePipelineRequest;
import java.io.IOException;
public class IngestRequestConverters {
final class IngestRequestConverters {
private IngestRequestConverters() {}
static Request getPipeline(GetPipelineRequest getPipelineRequest) {
String endpoint = new RequestConverters.EndpointBuilder()

View File

@ -29,7 +29,10 @@ import org.elasticsearch.client.license.DeleteLicenseRequest;
import org.elasticsearch.client.license.GetLicenseRequest;
import org.elasticsearch.client.license.PutLicenseRequest;
public class LicenseRequestConverters {
final class LicenseRequestConverters {
private LicenseRequestConverters() {}
static Request putLicense(PutLicenseRequest putLicenseRequest) {
String endpoint = new RequestConverters.EndpointBuilder().addPathPartAsIs("_xpack", "license").build();
Request request = new Request(HttpPut.METHOD_NAME, endpoint);

View File

@ -22,7 +22,9 @@ package org.elasticsearch.client;
import org.apache.http.client.methods.HttpGet;
import org.elasticsearch.protocol.xpack.migration.IndexUpgradeInfoRequest;
public class MigrationRequestConverters {
final class MigrationRequestConverters {
private MigrationRequestConverters() {}
static Request getMigrationAssistance(IndexUpgradeInfoRequest indexUpgradeInfoRequest) {
RequestConverters.EndpointBuilder endpointBuilder = new RequestConverters.EndpointBuilder()

View File

@ -77,6 +77,7 @@ import org.elasticsearch.script.mustache.MultiSearchTemplateRequest;
import org.elasticsearch.script.mustache.SearchTemplateRequest;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.client.core.TermVectorsRequest;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
@ -579,6 +580,19 @@ final class RequestConverters {
return req;
}
static Request termVectors(TermVectorsRequest tvrequest) throws IOException {
String endpoint = new EndpointBuilder().addPathPart(
tvrequest.getIndex(), tvrequest.getType(), tvrequest.getId()).addPathPartAsIs("_termvectors").build();
Request request = new Request(HttpGet.METHOD_NAME, endpoint);
Params params = new Params(request);
params.withRouting(tvrequest.getRouting());
params.withPreference(tvrequest.getPreference());
params.withFields(tvrequest.getFields());
params.withRealtime(tvrequest.getRealtime());
request.setEntity(createEntity(tvrequest, REQUEST_BODY_CONTENT_TYPE));
return request;
}
static Request getScript(GetStoredScriptRequest getStoredScriptRequest) {
String endpoint = new EndpointBuilder().addPathPartAsIs("_scripts").addPathPart(getStoredScriptRequest.id()).build();
Request request = new Request(HttpGet.METHOD_NAME, endpoint);

View File

@ -56,6 +56,8 @@ import org.elasticsearch.action.search.SearchScrollRequest;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.client.core.TermVectorsResponse;
import org.elasticsearch.client.core.TermVectorsRequest;
import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.common.CheckedFunction;
import org.elasticsearch.common.ParseField;
@ -156,16 +158,16 @@ import org.elasticsearch.search.aggregations.metrics.TopHitsAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue;
import org.elasticsearch.search.aggregations.pipeline.ParsedSimpleValue;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.InternalBucketMetricValue;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.ParsedBucketMetricValue;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile.ParsedPercentilesBucket;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile.PercentilesBucketPipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.ParsedStatsBucket;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.StatsBucketPipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.extended.ExtendedStatsBucketPipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.extended.ParsedExtendedStatsBucket;
import org.elasticsearch.search.aggregations.pipeline.derivative.DerivativePipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.derivative.ParsedDerivative;
import org.elasticsearch.search.aggregations.pipeline.InternalBucketMetricValue;
import org.elasticsearch.search.aggregations.pipeline.ParsedBucketMetricValue;
import org.elasticsearch.search.aggregations.pipeline.ParsedPercentilesBucket;
import org.elasticsearch.search.aggregations.pipeline.PercentilesBucketPipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.ParsedStatsBucket;
import org.elasticsearch.search.aggregations.pipeline.StatsBucketPipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.ExtendedStatsBucketPipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.ParsedExtendedStatsBucket;
import org.elasticsearch.search.aggregations.pipeline.DerivativePipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.ParsedDerivative;
import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.search.suggest.completion.CompletionSuggestion;
import org.elasticsearch.search.suggest.completion.CompletionSuggestionBuilder;
@ -1041,6 +1043,36 @@ public class RestHighLevelClient implements Closeable {
listener, singleton(404));
}
/**
* Calls the Term Vectors API
*
* See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-termvectors.html">Term Vectors API on
* elastic.co</a>
*
* @param request the request
* @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
*/
public final TermVectorsResponse termvectors(TermVectorsRequest request, RequestOptions options) throws IOException {
return performRequestAndParseEntity(request, RequestConverters::termVectors, options, TermVectorsResponse::fromXContent,
emptySet());
}
/**
* Asynchronously calls the Term Vectors API
*
* See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-termvectors.html">Term Vectors API on
* elastic.co</a>
* @param request the request
* @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
* @param listener the listener to be notified upon request completion
*/
public final void termvectorsAsync(TermVectorsRequest request, RequestOptions options, ActionListener<TermVectorsResponse> listener) {
performRequestAsyncAndParseEntity(request, RequestConverters::termVectors, options, TermVectorsResponse::fromXContent, listener,
emptySet());
}
/**
* Executes a request using the Ranking Evaluation API.
* See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/search-rank-eval.html">Ranking Evaluation API

View File

@ -36,7 +36,9 @@ import org.elasticsearch.common.Strings;
import java.io.IOException;
public class SnapshotRequestConverters {
final class SnapshotRequestConverters {
private SnapshotRequestConverters() {}
static Request getRepositories(GetRepositoriesRequest getRepositoriesRequest) {
String[] repositories = getRepositoriesRequest.repositories() == null ? Strings.EMPTY_ARRAY : getRepositoriesRequest.repositories();

View File

@ -24,7 +24,9 @@ import org.apache.http.client.methods.HttpPost;
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksRequest;
public class TasksRequestConverters {
final class TasksRequestConverters {
private TasksRequestConverters() {}
static Request cancelTasks(CancelTasksRequest cancelTasksRequest) {
Request request = new Request(HttpPost.METHOD_NAME, "/_tasks/_cancel");

View File

@ -19,6 +19,8 @@
package org.elasticsearch.client;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.client.watcher.DeactivateWatchRequest;
import org.elasticsearch.client.watcher.DeactivateWatchResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.watcher.ActivateWatchRequest;
import org.elasticsearch.client.watcher.ActivateWatchResponse;
@ -125,6 +127,35 @@ public final class WatcherClient {
PutWatchResponse::fromXContent, listener, emptySet());
}
/**
* Deactivate an existing watch
* See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/watcher-api-deactivate-watch.html">
* the docs</a> for more.
* @param request the request
* @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
* @return the response
* @throws IOException in case there is a problem sending the request or parsing back the response
*/
public DeactivateWatchResponse deactivateWatch(DeactivateWatchRequest request, RequestOptions options) throws IOException {
return restHighLevelClient.performRequestAndParseEntity(request, WatcherRequestConverters::deactivateWatch, options,
DeactivateWatchResponse::fromXContent, emptySet());
}
/**
* Asynchronously deactivate an existing watch
* See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/watcher-api-deactivate-watch.html">
* the docs</a> for more.
*
* @param request the request
* @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
* @param listener the listener to be notified upon request completion
*/
public void deactivateWatchAsync(DeactivateWatchRequest request, RequestOptions options,
ActionListener<DeactivateWatchResponse> listener) {
restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::deactivateWatch, options,
DeactivateWatchResponse::fromXContent, listener, emptySet());
}
/**
* Deletes a watch from the cluster
* See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/watcher-api-delete-watch.html">

View File

@ -24,6 +24,7 @@ import org.apache.http.client.methods.HttpPost;
import org.apache.http.client.methods.HttpPut;
import org.apache.http.entity.ByteArrayEntity;
import org.apache.http.entity.ContentType;
import org.elasticsearch.client.watcher.DeactivateWatchRequest;
import org.elasticsearch.client.watcher.ActivateWatchRequest;
import org.elasticsearch.client.watcher.AckWatchRequest;
import org.elasticsearch.client.watcher.StartWatchServiceRequest;
@ -32,7 +33,9 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.protocol.xpack.watcher.DeleteWatchRequest;
import org.elasticsearch.protocol.xpack.watcher.PutWatchRequest;
public class WatcherRequestConverters {
final class WatcherRequestConverters {
private WatcherRequestConverters() {}
static Request startWatchService(StartWatchServiceRequest startWatchServiceRequest) {
String endpoint = new RequestConverters.EndpointBuilder()
@ -73,6 +76,17 @@ public class WatcherRequestConverters {
return request;
}
static Request deactivateWatch(DeactivateWatchRequest deactivateWatchRequest) {
String endpoint = new RequestConverters.EndpointBuilder()
.addPathPartAsIs("_xpack")
.addPathPartAsIs("watcher")
.addPathPartAsIs("watch")
.addPathPart(deactivateWatchRequest.getWatchId())
.addPathPartAsIs("_deactivate")
.build();
return new Request(HttpPut.METHOD_NAME, endpoint);
}
static Request deleteWatch(DeleteWatchRequest deleteWatchRequest) {
String endpoint = new RequestConverters.EndpointBuilder()
.addPathPartAsIs("_xpack")

View File

@ -27,7 +27,9 @@ import java.util.EnumSet;
import java.util.Locale;
import java.util.stream.Collectors;
public class XPackRequestConverters {
final class XPackRequestConverters {
private XPackRequestConverters() {}
static Request info(XPackInfoRequest infoRequest) {
Request request = new Request(HttpGet.METHOD_NAME, "/_xpack");

View File

@ -0,0 +1,228 @@
/*
* 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.client.core;
import org.elasticsearch.client.Validatable;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.io.InputStream;
import java.util.Map;
public class TermVectorsRequest implements ToXContentObject, Validatable {
private final String index;
private final String type;
private String id = null;
private String routing = null;
private String preference = null;
private boolean realtime = true;
private String[] fields = null;
private boolean requestPositions = true;
private boolean requestPayloads = true;
private boolean requestOffsets = true;
private boolean requestFieldStatistics = true;
private boolean requestTermStatistics = false;
private Map<String, String> perFieldAnalyzer = null;
private Map<String, Integer> filterSettings = null;
private XContentBuilder docBuilder = null;
/**
* Constructs TermVectorRequest for the given document
* @param index - index of the document
* @param type - type of the document
* @param docId - id of the document
*/
public TermVectorsRequest(String index, String type, String docId) {
this(index, type);
this.id = docId;
}
/**
* Constructs TermVectorRequest for an artificial document
* @param index - index of the document
* @param type - type of the document
*/
public TermVectorsRequest(String index, String type) {
this.index = index;
this.type = type;
}
/**
* Returns the index of the request
*/
public String getIndex() {
return index;
}
/**
* Returns the type of the request
*/
public String getType() {
return type;
}
/**
* Returns the id of the request
* can be NULL if there is no document ID
*/
public String getId() {
return id;
}
/**
* Sets the fields for which term vectors information should be retrieved
*/
public void setFields(String... fields) {
this.fields = fields;
}
public String[] getFields() {
return fields;
}
/**
* Sets whether to request term positions
*/
public void setPositions(boolean requestPositions) {
this.requestPositions = requestPositions;
}
/**
* Sets whether to request term payloads
*/
public void setPayloads(boolean requestPayloads) {
this.requestPayloads = requestPayloads;
}
/**
* Sets whether to request term offsets
*/
public void setOffsets(boolean requestOffsets) {
this.requestOffsets = requestOffsets;
}
/**
* Sets whether to request field statistics
*/
public void setFieldStatistics(boolean requestFieldStatistics) {
this.requestFieldStatistics = requestFieldStatistics;
}
/**
* Sets whether to request term statistics
*/
public void setTermStatistics(boolean requestTermStatistics) {
this.requestTermStatistics = requestTermStatistics;
}
/**
* Sets different analyzers than the one at the fields
*/
public void setPerFieldAnalyzer(Map<String, String> perFieldAnalyzer) {
this.perFieldAnalyzer = perFieldAnalyzer;
}
/**
* Sets an artifical document on what to request _termvectors
*/
public void setDoc(XContentBuilder docBuilder) {
this.docBuilder = docBuilder;
}
/**
* Sets conditions for terms filtering
*/
public void setFilterSettings(Map<String, Integer> filterSettings) {
this.filterSettings = filterSettings;
}
/**
* Sets a routing to route a request to a particular shard
*/
public void setRouting(String routing) {
this.routing = routing;
}
public String getRouting() {
return routing;
}
/**
* Set a preference of which shard copies to execute the request
*/
public void setPreference(String preference) {
this.preference = preference;
}
public String getPreference() {
return preference;
}
/**
* Sets if the request should be realtime or near-realtime
*/
public void setRealtime(boolean realtime) {
this.realtime = realtime;
}
/**
* Returns if the request is realtime(true) or near-realtime(false)
*/
public boolean getRealtime() {
return realtime;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
// set values only when different from defaults
if (requestPositions == false) builder.field("positions", false);
if (requestPayloads == false) builder.field("payloads", false);
if (requestOffsets == false) builder.field("offsets", false);
if (requestFieldStatistics == false) builder.field("field_statistics", false);
if (requestTermStatistics) builder.field("term_statistics", true);
if (perFieldAnalyzer != null) builder.field("per_field_analyzer", perFieldAnalyzer);
if (docBuilder != null) {
BytesReference doc = BytesReference.bytes(docBuilder);
try (InputStream stream = doc.streamInput()) {
builder.rawField("doc", stream, docBuilder.contentType());
}
}
if (filterSettings != null) {
builder.startObject("filter");
String[] filterSettingNames =
{"max_num_terms", "min_term_freq", "max_term_freq", "min_doc_freq", "max_doc_freq", "min_word_length", "max_word_length"};
for (String settingName : filterSettingNames) {
if (filterSettings.containsKey(settingName)) builder.field(settingName, filterSettings.get(settingName));
}
builder.endObject();
}
builder.endObject();
return builder;
}
}

View File

@ -0,0 +1,486 @@
/*
* 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.client.core;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
import java.util.Collections;
import java.util.List;
import java.util.Comparator;
import java.util.Objects;
public class TermVectorsResponse {
private final String index;
private final String type;
private final String id;
private final long docVersion;
private final boolean found;
private final long tookInMillis;
private final List<TermVector> termVectorList;
public TermVectorsResponse(
String index, String type, String id, long version, boolean found, long tookInMillis, List<TermVector> termVectorList) {
this.index = index;
this.type = type;
this.id = id;
this.docVersion = version;
this.found = found;
this.tookInMillis = tookInMillis;
this.termVectorList = termVectorList;
}
private static ConstructingObjectParser<TermVectorsResponse, Void> PARSER = new ConstructingObjectParser<>("term_vectors", true,
args -> {
// as the response comes from server, we are sure that args[6] will be a list of TermVector
@SuppressWarnings("unchecked") List<TermVector> termVectorList = (List<TermVector>) args[6];
if (termVectorList != null) {
Collections.sort(termVectorList, Comparator.comparing(TermVector::getFieldName));
}
return new TermVectorsResponse(
(String) args[0],
(String) args[1],
(String) args[2],
(long) args[3],
(boolean) args[4],
(long) args[5],
termVectorList
);
}
);
static {
PARSER.declareString(constructorArg(), new ParseField("_index"));
PARSER.declareString(constructorArg(), new ParseField("_type"));
PARSER.declareString(optionalConstructorArg(), new ParseField("_id"));
PARSER.declareLong(constructorArg(), new ParseField("_version"));
PARSER.declareBoolean(constructorArg(), new ParseField("found"));
PARSER.declareLong(constructorArg(), new ParseField("took"));
PARSER.declareNamedObjects(optionalConstructorArg(),
(p, c, fieldName) -> TermVector.fromXContent(p, fieldName), new ParseField("term_vectors"));
}
public static TermVectorsResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}
/**
* Returns the index for the response
*/
public String getIndex() {
return index;
}
/**
* Returns the type for the response
*/
public String getType() {
return type;
}
/**
* Returns the id of the request
* can be NULL if there is no document ID
*/
public String getId() {
return id;
}
/**
* Returns if the document is found
* always <code>true</code> for artificial documents
*/
public boolean getFound() {
return found;
}
/**
* Returns the document version
*/
public long getDocVersion() {
return docVersion;
}
/**
* Returns the time that a request took in milliseconds
*/
public long getTookInMillis() {
return tookInMillis;
}
/**
* Returns the list of term vectors
*/
public List<TermVector> getTermVectorsList(){
return termVectorList;
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (!(obj instanceof TermVectorsResponse)) return false;
TermVectorsResponse other = (TermVectorsResponse) obj;
return index.equals(other.index)
&& type.equals(other.type)
&& Objects.equals(id, other.id)
&& docVersion == other.docVersion
&& found == other.found
&& tookInMillis == tookInMillis
&& Objects.equals(termVectorList, other.termVectorList);
}
@Override
public int hashCode() {
return Objects.hash(index, type, id, docVersion, found, tookInMillis, termVectorList);
}
public static final class TermVector {
private static ConstructingObjectParser<TermVector, String> PARSER = new ConstructingObjectParser<>("term_vector", true,
(args, ctxFieldName) -> {
// as the response comes from server, we are sure that args[1] will be a list of Term
@SuppressWarnings("unchecked") List<Term> terms = (List<Term>) args[1];
if (terms != null) {
Collections.sort(terms, Comparator.comparing(Term::getTerm));
}
return new TermVector(ctxFieldName, (FieldStatistics) args[0], terms);
}
);
static {
PARSER.declareObject(optionalConstructorArg(),
(p,c) -> FieldStatistics.fromXContent(p), new ParseField("field_statistics"));
PARSER.declareNamedObjects(optionalConstructorArg(), (p, c, term) -> Term.fromXContent(p, term), new ParseField("terms"));
}
private final String fieldName;
@Nullable
private final FieldStatistics fieldStatistics;
@Nullable
private final List<Term> terms;
public TermVector(String fieldName, FieldStatistics fieldStatistics, List<Term> terms) {
this.fieldName = fieldName;
this.fieldStatistics = fieldStatistics;
this.terms = terms;
}
public static TermVector fromXContent(XContentParser parser, String fieldName) {
return PARSER.apply(parser, fieldName);
}
/**
* Returns the field name of the current term vector
*/
public String getFieldName() {
return fieldName;
}
/**
* Returns the list of terms for the current term vector
*/
public List<Term> getTerms() {
return terms;
}
/**
* Returns the field statistics for the current field
*/
public FieldStatistics getFieldStatistics() {
return fieldStatistics;
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (!(obj instanceof TermVector)) return false;
TermVector other = (TermVector) obj;
return fieldName.equals(other.fieldName)
&& Objects.equals(fieldStatistics, other.fieldStatistics)
&& Objects.equals(terms, other.terms);
}
@Override
public int hashCode() {
return Objects.hash(fieldName, fieldStatistics, terms);
}
// Class containing a general field statistics for the field
public static final class FieldStatistics {
private static ConstructingObjectParser<FieldStatistics, Void> PARSER = new ConstructingObjectParser<>(
"field_statistics", true,
args -> {
return new FieldStatistics((long) args[0], (int) args[1], (long) args[2]);
}
);
static {
PARSER.declareLong(constructorArg(), new ParseField("sum_doc_freq"));
PARSER.declareInt(constructorArg(), new ParseField("doc_count"));
PARSER.declareLong(constructorArg(), new ParseField("sum_ttf"));
}
private final long sumDocFreq;
private final int docCount;
private final long sumTotalTermFreq;
public FieldStatistics(long sumDocFreq, int docCount, long sumTotalTermFreq) {
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
this.sumTotalTermFreq = sumTotalTermFreq;
}
public static FieldStatistics fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}
/*
* Returns how many documents this field contains
*/
public int getDocCount() {
return docCount;
}
/**
* Returns the sum of document frequencies for all terms in this field
*/
public long getSumDocFreq() {
return sumDocFreq;
}
/**
* Returns the sum of total term frequencies of all terms in this field
*/
public long getSumTotalTermFreq() {
return sumTotalTermFreq;
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (!(obj instanceof FieldStatistics)) return false;
FieldStatistics other = (FieldStatistics) obj;
return docCount == other.docCount
&& sumDocFreq == other.sumDocFreq
&& sumTotalTermFreq == other.sumTotalTermFreq;
}
@Override
public int hashCode() {
return Objects.hash(docCount, sumDocFreq, sumTotalTermFreq);
}
}
public static final class Term {
private static ConstructingObjectParser<Term, String> PARSER = new ConstructingObjectParser<>("token", true,
(args, ctxTerm) -> {
// as the response comes from server, we are sure that args[4] will be a list of Token
@SuppressWarnings("unchecked") List<Token> tokens = (List<Token>) args[4];
if (tokens != null) {
Collections.sort(
tokens,
Comparator.comparing(Token::getPosition, Comparator.nullsFirst(Integer::compareTo))
.thenComparing(Token::getStartOffset, Comparator.nullsFirst(Integer::compareTo))
.thenComparing(Token::getEndOffset, Comparator.nullsFirst(Integer::compareTo))
);
}
return new Term(ctxTerm, (int) args[0], (Integer) args[1], (Long) args[2], (Float) args[3], tokens);
}
);
static {
PARSER.declareInt(constructorArg(), new ParseField("term_freq"));
PARSER.declareInt(optionalConstructorArg(), new ParseField("doc_freq"));
PARSER.declareLong(optionalConstructorArg(), new ParseField("ttf"));
PARSER.declareFloat(optionalConstructorArg(), new ParseField("score"));
PARSER.declareObjectArray(optionalConstructorArg(), (p,c) -> Token.fromXContent(p), new ParseField("tokens"));
}
private final String term;
private final int termFreq;
@Nullable
private final Integer docFreq;
@Nullable
private final Long totalTermFreq;
@Nullable
private final Float score;
@Nullable
private final List<Token> tokens;
public Term(String term, int termFreq, Integer docFreq, Long totalTermFreq, Float score, List<Token> tokens) {
this.term = term;
this.termFreq = termFreq;
this.docFreq = docFreq;
this.totalTermFreq = totalTermFreq;
this.score = score;
this.tokens = tokens;
}
public static Term fromXContent(XContentParser parser, String term) {
return PARSER.apply(parser, term);
}
/**
* Returns the string representation of the term
*/
public String getTerm() {
return term;
}
/**
* Returns term frequency - the number of times this term occurs in the current document
*/
public int getTermFreq() {
return termFreq;
}
/**
* Returns document frequency - the number of documents in the index that contain this term
*/
public Integer getDocFreq() {
return docFreq;
}
/**
* Returns total term frequency - the number of times this term occurs across all documents
*/
public Long getTotalTermFreq( ){
return totalTermFreq;
}
/**
* Returns tf-idf score, if the request used some form of terms filtering
*/
public Float getScore(){
return score;
}
/**
* Returns a list of tokens for the term
*/
public List<Token> getTokens() {
return tokens;
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (!(obj instanceof Term)) return false;
Term other = (Term) obj;
return term.equals(other.term)
&& termFreq == other.termFreq
&& Objects.equals(docFreq, other.docFreq)
&& Objects.equals(totalTermFreq, other.totalTermFreq)
&& Objects.equals(score, other.score)
&& Objects.equals(tokens, other.tokens);
}
@Override
public int hashCode() {
return Objects.hash(term, termFreq, docFreq, totalTermFreq, score, tokens);
}
}
public static final class Token {
private static ConstructingObjectParser<Token, Void> PARSER = new ConstructingObjectParser<>("token", true,
args -> {
return new Token((Integer) args[0], (Integer) args[1], (Integer) args[2], (String) args[3]);
});
static {
PARSER.declareInt(optionalConstructorArg(), new ParseField("start_offset"));
PARSER.declareInt(optionalConstructorArg(), new ParseField("end_offset"));
PARSER.declareInt(optionalConstructorArg(), new ParseField("position"));
PARSER.declareString(optionalConstructorArg(), new ParseField("payload"));
}
@Nullable
private final Integer startOffset;
@Nullable
private final Integer endOffset;
@Nullable
private final Integer position;
@Nullable
private final String payload;
public Token(Integer startOffset, Integer endOffset, Integer position, String payload) {
this.startOffset = startOffset;
this.endOffset = endOffset;
this.position = position;
this.payload = payload;
}
public static Token fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}
/**
* Returns the start offset of the token in the document's field
*/
public Integer getStartOffset() {
return startOffset;
}
/**
* Returns the end offset of the token in the document's field
*/
public Integer getEndOffset() {
return endOffset;
}
/**
* Returns the position of the token in the document's field
*/
public Integer getPosition() {
return position;
}
/**
* Returns the payload of the token or <code>null</code> if the payload doesn't exist
*/
public String getPayload() {
return payload;
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (!(obj instanceof Token)) return false;
Token other = (Token) obj;
return Objects.equals(startOffset, other.startOffset)
&& Objects.equals(endOffset,other.endOffset)
&& Objects.equals(position, other.position)
&& Objects.equals(payload, other.payload);
}
@Override
public int hashCode() {
return Objects.hash(startOffset, endOffset, position, payload);
}
}
}
}

View File

@ -42,11 +42,6 @@ import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optiona
/**
* This class holds the configuration details of a rollup job, such as the groupings, metrics, what
* index to rollup and where to roll them to.
*
* When the configuration is stored server side, if there is no {@link MetricConfig} for the fields referenced in the
* {@link HistogramGroupConfig} and {@link DateHistogramGroupConfig} in the passed {@link GroupConfig},
* then default metrics of {@code ["min", "max"]} are provided
*
*/
public class RollupJobConfig implements Validatable, ToXContentObject {

View File

@ -16,24 +16,28 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.client.watcher;
package org.elasticsearch.painless;
import org.elasticsearch.client.Validatable;
import org.elasticsearch.protocol.xpack.watcher.PutWatchRequest;
import org.elasticsearch.index.fielddata.ScriptDocValues;
import java.util.Objects;
import java.util.Map;
public class DeactivateWatchRequest implements Validatable {
private final String watchId;
/**
* Generic script interface that Painless implements for all Elasticsearch scripts.
*/
public abstract class GenericElasticsearchScript {
public DeactivateWatchRequest(String watchId) {
public GenericElasticsearchScript() {}
Objects.requireNonNull(watchId, "watch id is missing");
if (PutWatchRequest.isValidId(watchId) == false) {
throw new IllegalArgumentException("watch id contains whitespace");
}
public static final String[] PARAMETERS = new String[] {"params", "_score", "doc", "_value", "ctx"};
public abstract Object execute(
Map<String, Object> params, double _score, Map<String, ScriptDocValues<?>> doc, Object _value, Map<?, ?> ctx);
this.watchId = watchId;
}
public abstract boolean needs_score();
public abstract boolean needsCtx();
public String getWatchId() {
return watchId;
}
}

View File

@ -0,0 +1,65 @@
/*
* 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.client.watcher;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
import java.util.Objects;
public class DeactivateWatchResponse {
private WatchStatus status;
private static final ParseField STATUS_FIELD = new ParseField("status");
private static final ConstructingObjectParser<DeactivateWatchResponse, Void> PARSER
= new ConstructingObjectParser<>("x_pack_deactivate_watch_response", true,
(fields) -> new DeactivateWatchResponse((WatchStatus) fields[0]));
static {
PARSER.declareObject(ConstructingObjectParser.constructorArg(),
(parser, context) -> WatchStatus.parse(parser),
STATUS_FIELD);
}
public static DeactivateWatchResponse fromXContent(XContentParser parser) throws IOException {
return PARSER.parse(parser, null);
}
public DeactivateWatchResponse(WatchStatus status) {
this.status = status;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
DeactivateWatchResponse that = (DeactivateWatchResponse) o;
return Objects.equals(status, that.status);
}
@Override
public int hashCode() {
return Objects.hash(status);
}
public WatchStatus getStatus() {
return status;
}
}

View File

@ -44,12 +44,15 @@ import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.client.core.TermVectorsRequest;
import org.elasticsearch.client.core.TermVectorsResponse;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.get.GetResult;
@ -73,6 +76,7 @@ import org.joda.time.format.DateTimeFormat;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
@ -80,6 +84,7 @@ import java.util.concurrent.atomic.AtomicReference;
import static java.util.Collections.singletonMap;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.lessThan;
@ -196,7 +201,7 @@ public class CrudIT extends ESRestHighLevelClientTestCase {
ElasticsearchException exception = expectThrows(ElasticsearchException.class,
() -> execute(getRequest, highLevelClient()::get, highLevelClient()::getAsync));
assertEquals(RestStatus.NOT_FOUND, exception.status());
assertEquals("Elasticsearch exception [type=index_not_found_exception, reason=no such index]", exception.getMessage());
assertEquals("Elasticsearch exception [type=index_not_found_exception, reason=no such index [index]]", exception.getMessage());
assertEquals("index", exception.getMetadata("es.index").get(0));
}
IndexRequest index = new IndexRequest("index", "type", "id");
@ -283,7 +288,7 @@ public class CrudIT extends ESRestHighLevelClientTestCase {
assertEquals("id1", response.getResponses()[0].getFailure().getId());
assertEquals("type", response.getResponses()[0].getFailure().getType());
assertEquals("index", response.getResponses()[0].getFailure().getIndex());
assertEquals("Elasticsearch exception [type=index_not_found_exception, reason=no such index]",
assertEquals("Elasticsearch exception [type=index_not_found_exception, reason=no such index [index]]",
response.getResponses()[0].getFailure().getFailure().getMessage());
assertTrue(response.getResponses()[1].isFailed());
@ -291,7 +296,7 @@ public class CrudIT extends ESRestHighLevelClientTestCase {
assertEquals("id2", response.getResponses()[1].getId());
assertEquals("type", response.getResponses()[1].getType());
assertEquals("index", response.getResponses()[1].getIndex());
assertEquals("Elasticsearch exception [type=index_not_found_exception, reason=no such index]",
assertEquals("Elasticsearch exception [type=index_not_found_exception, reason=no such index [index]]",
response.getResponses()[1].getFailure().getFailure().getMessage());
}
BulkRequest bulk = new BulkRequest();
@ -1154,4 +1159,80 @@ public class CrudIT extends ESRestHighLevelClientTestCase {
assertEquals(routing, getResponse.getField("_routing").getValue());
}
}
// Not entirely sure if _termvectors belongs to CRUD, and in the absence of a better place, will have it here
public void testTermvectors() throws IOException {
final String sourceIndex = "index1";
{
// prepare : index docs
Settings settings = Settings.builder()
.put("number_of_shards", 1)
.put("number_of_replicas", 0)
.build();
String mappings = "\"_doc\":{\"properties\":{\"field\":{\"type\":\"text\"}}}";
createIndex(sourceIndex, settings, mappings);
assertEquals(
RestStatus.OK,
highLevelClient().bulk(
new BulkRequest()
.add(new IndexRequest(sourceIndex, "_doc", "1")
.source(Collections.singletonMap("field", "value1"), XContentType.JSON))
.add(new IndexRequest(sourceIndex, "_doc", "2")
.source(Collections.singletonMap("field", "value2"), XContentType.JSON))
.setRefreshPolicy(RefreshPolicy.IMMEDIATE),
RequestOptions.DEFAULT
).status()
);
}
{
// test _termvectors on real documents
TermVectorsRequest tvRequest = new TermVectorsRequest(sourceIndex, "_doc", "1");
tvRequest.setFields("field");
TermVectorsResponse tvResponse = execute(tvRequest, highLevelClient()::termvectors, highLevelClient()::termvectorsAsync);
TermVectorsResponse.TermVector.Token expectedToken = new TermVectorsResponse.TermVector.Token(0, 6, 0, null);
TermVectorsResponse.TermVector.Term expectedTerm = new TermVectorsResponse.TermVector.Term(
"value1", 1, null, null, null, Collections.singletonList(expectedToken));
TermVectorsResponse.TermVector.FieldStatistics expectedFieldStats =
new TermVectorsResponse.TermVector.FieldStatistics(2, 2, 2);
TermVectorsResponse.TermVector expectedTV =
new TermVectorsResponse.TermVector("field", expectedFieldStats, Collections.singletonList(expectedTerm));
List<TermVectorsResponse.TermVector> expectedTVlist = Collections.singletonList(expectedTV);
assertThat(tvResponse.getIndex(), equalTo(sourceIndex));
assertThat(Integer.valueOf(tvResponse.getId()), equalTo(1));
assertTrue(tvResponse.getFound());
assertEquals(expectedTVlist, tvResponse.getTermVectorsList());
}
{
// test _termvectors on artificial documents
TermVectorsRequest tvRequest = new TermVectorsRequest(sourceIndex, "_doc");
XContentBuilder docBuilder = XContentFactory.jsonBuilder();
docBuilder.startObject().field("field", "valuex").endObject();
tvRequest.setDoc(docBuilder);
TermVectorsResponse tvResponse = execute(tvRequest, highLevelClient()::termvectors, highLevelClient()::termvectorsAsync);
TermVectorsResponse.TermVector.Token expectedToken = new TermVectorsResponse.TermVector.Token(0, 6, 0, null);
TermVectorsResponse.TermVector.Term expectedTerm = new TermVectorsResponse.TermVector.Term(
"valuex", 1, null, null, null, Collections.singletonList(expectedToken));
TermVectorsResponse.TermVector.FieldStatistics expectedFieldStats =
new TermVectorsResponse.TermVector.FieldStatistics(2, 2, 2);
TermVectorsResponse.TermVector expectedTV =
new TermVectorsResponse.TermVector("field", expectedFieldStats, Collections.singletonList(expectedTerm));
List<TermVectorsResponse.TermVector> expectedTVlist = Collections.singletonList(expectedTV);
assertThat(tvResponse.getIndex(), equalTo(sourceIndex));
assertTrue(tvResponse.getFound());
assertEquals(expectedTVlist, tvResponse.getTermVectorsList());
}
}
// Not entirely sure if _termvectors belongs to CRUD, and in the absence of a better place, will have it here
public void testTermvectorsWithNonExistentIndex() {
TermVectorsRequest request = new TermVectorsRequest("non-existent", "non-existent", "non-existent");
ElasticsearchException exception = expectThrows(ElasticsearchException.class,
() -> execute(request, highLevelClient()::termvectors, highLevelClient()::termvectorsAsync));
assertEquals(RestStatus.NOT_FOUND, exception.status());
}
}

View File

@ -109,7 +109,7 @@ public class GetAliasesResponseTests extends AbstractXContentTestCase<GetAliases
" \"root_cause\": [" +
" {" +
" \"type\": \"index_not_found_exception\"," +
" \"reason\": \"no such index\"," +
" \"reason\": \"no such index [index]\"," +
" \"resource.type\": \"index_or_alias\"," +
" \"resource.id\": \"index\"," +
" \"index_uuid\": \"_na_\"," +
@ -117,7 +117,7 @@ public class GetAliasesResponseTests extends AbstractXContentTestCase<GetAliases
" }" +
" ]," +
" \"type\": \"index_not_found_exception\"," +
" \"reason\": \"no such index\"," +
" \"reason\": \"no such index [index]\"," +
" \"resource.type\": \"index_or_alias\"," +
" \"resource.id\": \"index\"," +
" \"index_uuid\": \"_na_\"," +
@ -131,7 +131,7 @@ public class GetAliasesResponseTests extends AbstractXContentTestCase<GetAliases
assertThat(getAliasesResponse.getError(), nullValue());
assertThat(getAliasesResponse.status(), equalTo(RestStatus.NOT_FOUND));
assertThat(getAliasesResponse.getException().getMessage(),
equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index]"));
equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index [index]]"));
}
}

View File

@ -573,7 +573,8 @@ public class IndicesClientIT extends ESRestHighLevelClientTestCase {
ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> execute(nonExistentIndexRequest,
highLevelClient().indices()::updateAliases, highLevelClient().indices()::updateAliasesAsync));
assertThat(exception.status(), equalTo(RestStatus.NOT_FOUND));
assertThat(exception.getMessage(), equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index]"));
assertThat(exception.getMessage(),
equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index [non_existent_index]]"));
assertThat(exception.getMetadata("es.index"), hasItem(nonExistentIndex));
createIndex(index, Settings.EMPTY);
@ -583,7 +584,8 @@ public class IndicesClientIT extends ESRestHighLevelClientTestCase {
exception = expectThrows(ElasticsearchStatusException.class,
() -> execute(mixedRequest, highLevelClient().indices()::updateAliases, highLevelClient().indices()::updateAliasesAsync));
assertThat(exception.status(), equalTo(RestStatus.NOT_FOUND));
assertThat(exception.getMessage(), equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index]"));
assertThat(exception.getMessage(),
equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index [non_existent_index]]"));
assertThat(exception.getMetadata("es.index"), hasItem(nonExistentIndex));
assertThat(exception.getMetadata("es.index"), not(hasItem(index)));
assertThat(aliasExists(index, alias), equalTo(false));
@ -595,7 +597,8 @@ public class IndicesClientIT extends ESRestHighLevelClientTestCase {
exception = expectThrows(ElasticsearchException.class, () -> execute(removeIndexRequest, highLevelClient().indices()::updateAliases,
highLevelClient().indices()::updateAliasesAsync));
assertThat(exception.status(), equalTo(RestStatus.NOT_FOUND));
assertThat(exception.getMessage(), equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index]"));
assertThat(exception.getMessage(),
equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index [non_existent_index]]"));
assertThat(exception.getMetadata("es.index"), hasItem(nonExistentIndex));
assertThat(exception.getMetadata("es.index"), not(hasItem(index)));
assertThat(aliasExists(index, alias), equalTo(false));
@ -1060,7 +1063,7 @@ public class IndicesClientIT extends ESRestHighLevelClientTestCase {
highLevelClient().indices()::getAliasAsync);
assertThat(getAliasesResponse.status(), equalTo(RestStatus.NOT_FOUND));
assertThat(getAliasesResponse.getException().getMessage(),
equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index]"));
equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index [index]]"));
}
{
GetAliasesRequest getAliasesRequest = new GetAliasesRequest(alias);
@ -1077,7 +1080,7 @@ public class IndicesClientIT extends ESRestHighLevelClientTestCase {
highLevelClient().indices()::getAliasAsync);
assertThat(getAliasesResponse.status(), equalTo(RestStatus.NOT_FOUND));
assertThat(getAliasesResponse.getException().getMessage(),
equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index]"));
equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index [non_existent_index]]"));
}
{
GetAliasesRequest getAliasesRequest = new GetAliasesRequest().indices(index, "non_existent_index").aliases(alias);
@ -1085,7 +1088,7 @@ public class IndicesClientIT extends ESRestHighLevelClientTestCase {
highLevelClient().indices()::getAliasAsync);
assertThat(getAliasesResponse.status(), equalTo(RestStatus.NOT_FOUND));
assertThat(getAliasesResponse.getException().getMessage(),
equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index]"));
equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index [non_existent_index]]"));
}
{
GetAliasesRequest getAliasesRequest = new GetAliasesRequest().indices("non_existent_index*");
@ -1199,7 +1202,8 @@ public class IndicesClientIT extends ESRestHighLevelClientTestCase {
ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> execute(indexUpdateSettingsRequest,
highLevelClient().indices()::putSettings, highLevelClient().indices()::putSettingsAsync));
assertEquals(RestStatus.NOT_FOUND, exception.status());
assertThat(exception.getMessage(), equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index]"));
assertThat(exception.getMessage(),
equalTo("Elasticsearch exception [type=index_not_found_exception, reason=no such index [index]]"));
createIndex(index, Settings.EMPTY);
exception = expectThrows(ElasticsearchException.class, () -> execute(indexUpdateSettingsRequest,

View File

@ -53,6 +53,7 @@ import org.elasticsearch.action.support.master.AcknowledgedRequest;
import org.elasticsearch.action.support.master.MasterNodeReadRequest;
import org.elasticsearch.action.support.master.MasterNodeRequest;
import org.elasticsearch.action.support.replication.ReplicationRequest;
import org.elasticsearch.client.core.TermVectorsRequest;
import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.client.RequestConverters.EndpointBuilder;
import org.elasticsearch.common.CheckedBiConsumer;
@ -1177,6 +1178,46 @@ public class RequestConvertersTests extends ESTestCase {
assertToXContentBody(explainRequest, request.getEntity());
}
public void testTermVectors() throws IOException {
String index = randomAlphaOfLengthBetween(3, 10);
String type = randomAlphaOfLengthBetween(3, 10);
String id = randomAlphaOfLengthBetween(3, 10);
TermVectorsRequest tvRequest = new TermVectorsRequest(index, type, id);
Map<String, String> expectedParams = new HashMap<>();
String[] fields;
if (randomBoolean()) {
String routing = randomAlphaOfLengthBetween(3, 10);
tvRequest.setRouting(routing);
expectedParams.put("routing", routing);
}
if (randomBoolean()) {
tvRequest.setRealtime(false);
expectedParams.put("realtime", "false");
}
boolean hasFields = randomBoolean();
if (hasFields) {
fields = generateRandomStringArray(10, 5, false, false);
tvRequest.setFields(fields);
}
Request request = RequestConverters.termVectors(tvRequest);
StringJoiner endpoint = new StringJoiner("/", "/", "");
endpoint.add(index).add(type).add(id).add("_termvectors");
assertEquals(HttpGet.METHOD_NAME, request.getMethod());
assertEquals(endpoint.toString(), request.getEndpoint());
if (hasFields) {
assertThat(request.getParameters(), hasKey("fields"));
String[] requestFields = Strings.splitStringByCommaToArray(request.getParameters().get("fields"));
assertArrayEquals(tvRequest.getFields(), requestFields);
}
for (Map.Entry<String, String> param : expectedParams.entrySet()) {
assertThat(request.getParameters(), hasEntry(param.getKey(), param.getValue()));
}
assertToXContentBody(tvRequest, request.getEntity());
}
public void testFieldCaps() {
// Create a random request.
String[] indices = randomIndicesNames(0, 5);

View File

@ -675,8 +675,7 @@ public class RestHighLevelClientTests extends ESTestCase {
"mtermvectors",
"render_search_template",
"scripts_painless_execute",
"tasks.get",
"termvectors"
"tasks.get"
};
//These API are not required for high-level client feature completeness
String[] notRequiredApi = new String[] {

View File

@ -46,7 +46,6 @@ import org.elasticsearch.client.rollup.job.config.GroupConfig;
import org.elasticsearch.client.rollup.job.config.MetricConfig;
import org.elasticsearch.client.rollup.job.config.RollupJobConfig;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
@ -169,9 +168,7 @@ public class RollupIT extends ESRestHighLevelClientTestCase {
public void testPutAndGetRollupJob() throws Exception {
// TODO expand this to also test with histogram and terms?
final GroupConfig groups = new GroupConfig(new DateHistogramGroupConfig("date", DateHistogramInterval.DAY));
final List<MetricConfig> metrics = Arrays.asList(
new MetricConfig("value", SUPPORTED_METRICS),
new MetricConfig("date", Arrays.asList(MaxAggregationBuilder.NAME)));
final List<MetricConfig> metrics = Collections.singletonList(new MetricConfig("value", SUPPORTED_METRICS));
final TimeValue timeout = TimeValue.timeValueSeconds(randomIntBetween(30, 600));
PutRollupJobRequest putRollupJobRequest =
@ -199,28 +196,21 @@ public class RollupIT extends ESRestHighLevelClientTestCase {
assertEquals(groups.getDateHistogram().getTimeZone(), source.get("date.date_histogram.time_zone"));
for (MetricConfig metric : metrics) {
if (metric.getField().equals("value")) {
for (String name : metric.getMetrics()) {
Number value = (Number) source.get(metric.getField() + "." + name + ".value");
if ("min".equals(name)) {
assertEquals(min, value.intValue());
} else if ("max".equals(name)) {
assertEquals(max, value.intValue());
} else if ("sum".equals(name)) {
assertEquals(sum, value.doubleValue(), 0.0d);
} else if ("avg".equals(name)) {
assertEquals(sum, value.doubleValue(), 0.0d);
Number avgCount = (Number) source.get(metric.getField() + "." + name + "._count");
assertEquals(numDocs, avgCount.intValue());
} else if ("value_count".equals(name)) {
assertEquals(numDocs, value.intValue());
}
for (String name : metric.getMetrics()) {
Number value = (Number) source.get(metric.getField() + "." + name + ".value");
if ("min".equals(name)) {
assertEquals(min, value.intValue());
} else if ("max".equals(name)) {
assertEquals(max, value.intValue());
} else if ("sum".equals(name)) {
assertEquals(sum, value.doubleValue(), 0.0d);
} else if ("avg".equals(name)) {
assertEquals(sum, value.doubleValue(), 0.0d);
Number avgCount = (Number) source.get(metric.getField() + "." + name + "._count");
assertEquals(numDocs, avgCount.intValue());
} else if ("value_count".equals(name)) {
assertEquals(numDocs, value.intValue());
}
} else {
Number value = (Number) source.get(metric.getField() + ".max.value");
assertEquals(
DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parser().parseDateTime("2018-01-01T00:59:50").getMillis(),
value.longValue());
}
}
});

View File

@ -1079,7 +1079,7 @@ public class SearchIT extends ESRestHighLevelClientTestCase {
assertThat(exception.status(), equalTo(RestStatus.NOT_FOUND));
assertThat(exception.getIndex().getName(), equalTo("non_existent_index"));
assertThat(exception.getDetailedMessage(),
containsString("Elasticsearch exception [type=index_not_found_exception, reason=no such index]"));
containsString("Elasticsearch exception [type=index_not_found_exception, reason=no such index [non_existent_index]]"));
}
{
ExplainRequest explainRequest = new ExplainRequest("index1", "doc", "999");

View File

@ -19,6 +19,10 @@
package org.elasticsearch.client;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.client.watcher.DeactivateWatchRequest;
import org.elasticsearch.client.watcher.DeactivateWatchResponse;
import org.elasticsearch.client.watcher.ActivateWatchRequest;
import org.elasticsearch.client.watcher.ActivateWatchResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.watcher.AckWatchRequest;
import org.elasticsearch.client.watcher.AckWatchResponse;
@ -73,6 +77,23 @@ public class WatcherIT extends ESRestHighLevelClientTestCase {
return highLevelClient().watcher().putWatch(putWatchRequest, RequestOptions.DEFAULT);
}
public void testDeactivateWatch() throws Exception {
// Deactivate a watch that exists
String watchId = randomAlphaOfLength(10);
createWatch(watchId);
DeactivateWatchResponse response = highLevelClient().watcher().deactivateWatch(
new DeactivateWatchRequest(watchId), RequestOptions.DEFAULT);
assertThat(response.getStatus().state().isActive(), is(false));
}
public void testDeactivateWatch404() throws Exception {
// Deactivate a watch that does not exist
String watchId = randomAlphaOfLength(10);
ElasticsearchStatusException exception = expectThrows(ElasticsearchStatusException.class,
() -> highLevelClient().watcher().deactivateWatch(new DeactivateWatchRequest(watchId), RequestOptions.DEFAULT));
assertEquals(RestStatus.NOT_FOUND, exception.status());
}
public void testDeleteWatch() throws Exception {
// delete watch that exists
{

View File

@ -22,6 +22,7 @@ package org.elasticsearch.client;
import org.apache.http.client.methods.HttpDelete;
import org.apache.http.client.methods.HttpPost;
import org.apache.http.client.methods.HttpPut;
import org.elasticsearch.client.watcher.DeactivateWatchRequest;
import org.elasticsearch.client.watcher.ActivateWatchRequest;
import org.elasticsearch.client.watcher.AckWatchRequest;
import org.elasticsearch.client.watcher.StartWatchServiceRequest;
@ -83,6 +84,15 @@ public class WatcherRequestConvertersTests extends ESTestCase {
assertThat(bos.toString("UTF-8"), is(body));
}
public void testDeactivateWatch() {
String watchId = randomAlphaOfLength(10);
DeactivateWatchRequest deactivateWatchRequest = new DeactivateWatchRequest(watchId);
Request request = WatcherRequestConverters.deactivateWatch(deactivateWatchRequest);
assertEquals(HttpPut.METHOD_NAME, request.getMethod());
assertEquals("/_xpack/watcher/watch/" + watchId + "/_deactivate", request.getEndpoint());
}
public void testDeleteWatch() {
DeleteWatchRequest deleteWatchRequest = new DeleteWatchRequest();
String watchId = randomAlphaOfLength(10);

View File

@ -0,0 +1,203 @@
/*
* 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.client.core;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.test.ESTestCase;
import java.util.ArrayList;
import java.util.List;
import java.io.IOException;
import java.util.Collections;
import java.util.Comparator;
import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester;
public class TermVectorsResponseTests extends ESTestCase {
public void testFromXContent() throws IOException {
xContentTester(
this::createParser,
this::createTestInstance,
this::toXContent,
TermVectorsResponse::fromXContent)
.supportsUnknownFields(true)
.randomFieldsExcludeFilter(field ->
field.endsWith("term_vectors") || field.endsWith("terms") || field.endsWith("tokens"))
.test();
}
private void toXContent(TermVectorsResponse response, XContentBuilder builder) throws IOException {
builder.startObject();
builder.field("_index", response.getIndex());
builder.field("_type", response.getType());
if (response.getId() != null) {
builder.field("_id", response.getId());
}
builder.field("_version", response.getDocVersion());
builder.field("found", response.getFound());
builder.field("took", response.getTookInMillis());
List<TermVectorsResponse.TermVector> termVectorList = response.getTermVectorsList();
if (termVectorList != null) {
Collections.sort(termVectorList, Comparator.comparing(TermVectorsResponse.TermVector::getFieldName));
builder.startObject("term_vectors");
for (TermVectorsResponse.TermVector tv : termVectorList) {
toXContent(tv, builder);
}
builder.endObject();
}
builder.endObject();
}
private void toXContent(TermVectorsResponse.TermVector tv, XContentBuilder builder) throws IOException {
builder.startObject(tv.getFieldName());
// build fields_statistics
if (tv.getFieldStatistics() != null) {
builder.startObject("field_statistics");
builder.field("sum_doc_freq", tv.getFieldStatistics().getSumDocFreq());
builder.field("doc_count", tv.getFieldStatistics().getDocCount());
builder.field("sum_ttf", tv.getFieldStatistics().getSumTotalTermFreq());
builder.endObject();
}
// build terms
List<TermVectorsResponse.TermVector.Term> terms = tv.getTerms();
if (terms != null) {
Collections.sort(terms, Comparator.comparing(TermVectorsResponse.TermVector.Term::getTerm));
builder.startObject("terms");
for (TermVectorsResponse.TermVector.Term term : terms) {
builder.startObject(term.getTerm());
// build term_statistics
if (term.getDocFreq() != null) builder.field("doc_freq", term.getDocFreq());
if (term.getTotalTermFreq() != null) builder.field("ttf", term.getTotalTermFreq());
builder.field("term_freq", term.getTermFreq());
// build tokens
List<TermVectorsResponse.TermVector.Token> tokens = term.getTokens();
if (tokens != null) {
Collections.sort(
tokens,
Comparator.comparing(TermVectorsResponse.TermVector.Token::getPosition, Comparator.nullsFirst(Integer::compareTo))
.thenComparing(TermVectorsResponse.TermVector.Token::getStartOffset, Comparator.nullsFirst(Integer::compareTo))
.thenComparing(TermVectorsResponse.TermVector.Token::getEndOffset, Comparator.nullsFirst(Integer::compareTo))
);
builder.startArray("tokens");
for (TermVectorsResponse.TermVector.Token token : tokens) {
builder.startObject();
if (token.getPosition() != null) builder.field("position", token.getPosition());
if (token.getStartOffset()!= null) builder.field("start_offset", token.getStartOffset());
if (token.getEndOffset() != null) builder.field("end_offset", token.getEndOffset());
if (token.getPayload() != null) builder.field("payload", token.getPayload());
builder.endObject();
}
builder.endArray();
}
if (term.getScore() != null) builder.field("score", term.getScore());
builder.endObject();
}
builder.endObject();
}
builder.endObject();
}
protected TermVectorsResponse createTestInstance() {
String index = randomAlphaOfLength(5);
String type = randomAlphaOfLength(5);
String id = String.valueOf(randomIntBetween(1,100));
long version = randomNonNegativeLong();
long tookInMillis = randomNonNegativeLong();
boolean found = randomBoolean();
List<TermVectorsResponse.TermVector> tvList = null;
if (found == true){
boolean hasFieldStatistics = randomBoolean();
boolean hasTermStatistics = randomBoolean();
boolean hasScores = randomBoolean();
boolean hasOffsets = randomBoolean();
boolean hasPositions = randomBoolean();
boolean hasPayloads = randomBoolean();
int fieldsCount = randomIntBetween(1, 3);
tvList = new ArrayList<>(fieldsCount);
for (int i = 0; i < fieldsCount; i++) {
tvList.add(randomTermVector(hasFieldStatistics, hasTermStatistics, hasScores, hasOffsets, hasPositions, hasPayloads));
}
}
TermVectorsResponse tvresponse = new TermVectorsResponse(index, type, id, version, found, tookInMillis, tvList);
return tvresponse;
}
private TermVectorsResponse.TermVector randomTermVector(boolean hasFieldStatistics, boolean hasTermStatistics, boolean hasScores,
boolean hasOffsets, boolean hasPositions, boolean hasPayloads) {
TermVectorsResponse.TermVector.FieldStatistics fs = null;
if (hasFieldStatistics) {
long sumDocFreq = randomNonNegativeLong();
int docCount = randomInt(1000);
long sumTotalTermFreq = randomNonNegativeLong();
fs = new TermVectorsResponse.TermVector.FieldStatistics(sumDocFreq, docCount, sumTotalTermFreq);
}
int termsCount = randomIntBetween(1, 5);
List<TermVectorsResponse.TermVector.Term> terms = new ArrayList<>(termsCount);
for (int i = 0; i < termsCount; i++) {
terms.add(randomTerm(hasTermStatistics, hasScores, hasOffsets, hasPositions, hasPayloads));
}
TermVectorsResponse.TermVector tv = new TermVectorsResponse.TermVector("field" + randomAlphaOfLength(2), fs, terms);
return tv;
}
private TermVectorsResponse.TermVector.Term randomTerm(boolean hasTermStatistics, boolean hasScores,
boolean hasOffsets, boolean hasPositions, boolean hasPayloads) {
String termTxt = "term" + randomAlphaOfLength(2);
int termFreq = randomInt(10000);
Integer docFreq = null;
Long totalTermFreq = null;
Float score = null;
List<TermVectorsResponse.TermVector.Token> tokens = null;
if (hasTermStatistics) {
docFreq = randomInt(1000);
totalTermFreq = randomNonNegativeLong();
}
if (hasScores) score = randomFloat();
if (hasOffsets || hasPositions || hasPayloads ){
int tokensCount = randomIntBetween(1, 5);
tokens = new ArrayList<>(tokensCount);
for (int i = 0; i < tokensCount; i++) {
Integer startOffset = null;
Integer endOffset = null;
Integer position = null;
String payload = null;
if (hasOffsets) {
startOffset = randomInt(1000);
endOffset = randomInt(2000);
}
if (hasPositions) position = randomInt(100);
if (hasPayloads) payload = "payload" + randomAlphaOfLength(2);
TermVectorsResponse.TermVector.Token token =
new TermVectorsResponse.TermVector.Token(startOffset, endOffset, position, payload);
tokens.add(token);
}
}
TermVectorsResponse.TermVector.Term term =
new TermVectorsResponse.TermVector.Term(termTxt, termFreq, docFreq, totalTermFreq, score, tokens);
return term;
}
}

View File

@ -25,6 +25,8 @@ import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.LatchedActionListener;
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
import org.elasticsearch.action.bulk.BackoffPolicy;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkProcessor;
@ -52,6 +54,8 @@ import org.elasticsearch.client.RequestOptions;
import org.elasticsearch.client.Response;
import org.elasticsearch.client.RestHighLevelClient;
import org.elasticsearch.client.RethrottleRequest;
import org.elasticsearch.client.core.TermVectorsRequest;
import org.elasticsearch.client.core.TermVectorsResponse;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.settings.Settings;
@ -1503,6 +1507,125 @@ public class CRUDDocumentationIT extends ESRestHighLevelClientTestCase {
}
}
// Not entirely sure if _termvectors belongs to CRUD, and in the absence of a better place, will have it here
public void testTermVectors() throws Exception {
RestHighLevelClient client = highLevelClient();
CreateIndexRequest authorsRequest = new CreateIndexRequest("authors").mapping("doc", "user", "type=keyword");
CreateIndexResponse authorsResponse = client.indices().create(authorsRequest, RequestOptions.DEFAULT);
assertTrue(authorsResponse.isAcknowledged());
client.index(new IndexRequest("index", "doc", "1").source("user", "kimchy"), RequestOptions.DEFAULT);
Response refreshResponse = client().performRequest(new Request("POST", "/authors/_refresh"));
assertEquals(200, refreshResponse.getStatusLine().getStatusCode());
{
// tag::term-vectors-request
TermVectorsRequest request = new TermVectorsRequest("authors", "doc", "1");
request.setFields("user");
// end::term-vectors-request
}
{
// tag::term-vectors-request-artificial
TermVectorsRequest request = new TermVectorsRequest("authors", "doc");
XContentBuilder docBuilder = XContentFactory.jsonBuilder();
docBuilder.startObject().field("user", "guest-user").endObject();
request.setDoc(docBuilder); // <1>
// end::term-vectors-request-artificial
// tag::term-vectors-request-optional-arguments
request.setFieldStatistics(false); // <1>
request.setTermStatistics(true); // <2>
request.setPositions(false); // <3>
request.setOffsets(false); // <4>
request.setPayloads(false); // <5>
Map<String, Integer> filterSettings = new HashMap<>();
filterSettings.put("max_num_terms", 3);
filterSettings.put("min_term_freq", 1);
filterSettings.put("max_term_freq", 10);
filterSettings.put("min_doc_freq", 1);
filterSettings.put("max_doc_freq", 100);
filterSettings.put("min_word_length", 1);
filterSettings.put("max_word_length", 10);
request.setFilterSettings(filterSettings); // <6>
Map<String, String> perFieldAnalyzer = new HashMap<>();
perFieldAnalyzer.put("user", "keyword");
request.setPerFieldAnalyzer(perFieldAnalyzer); // <7>
request.setRealtime(false); // <8>
request.setRouting("routing"); // <9>
// end::term-vectors-request-optional-arguments
}
TermVectorsRequest request = new TermVectorsRequest("authors", "doc", "1");
request.setFields("user");
// tag::term-vectors-execute
TermVectorsResponse response = client.termvectors(request, RequestOptions.DEFAULT);
// end::term-vectors-execute
// tag::term-vectors-response
String index = response.getIndex(); // <1>
String type = response.getType(); // <2>
String id = response.getId(); // <3>
boolean found = response.getFound(); // <4>
// end::term-vectors-response
// tag::term-vectors-term-vectors
if (response.getTermVectorsList() != null) {
List<TermVectorsResponse.TermVector> tvList = response.getTermVectorsList();
for (TermVectorsResponse.TermVector tv : tvList) {
String fieldname = tv.getFieldName(); // <1>
int docCount = tv.getFieldStatistics().getDocCount(); // <2>
long sumTotalTermFreq = tv.getFieldStatistics().getSumTotalTermFreq(); // <3>
long sumDocFreq = tv.getFieldStatistics().getSumDocFreq(); // <4>
if (tv.getTerms() != null) {
List<TermVectorsResponse.TermVector.Term> terms = tv.getTerms(); // <5>
for (TermVectorsResponse.TermVector.Term term : terms) {
String termStr = term.getTerm(); // <6>
int termFreq = term.getTermFreq(); // <7>
int docFreq = term.getDocFreq(); // <8>
long totalTermFreq = term.getTotalTermFreq(); // <9>
float score = term.getScore(); // <10>
if (term.getTokens() != null) {
List<TermVectorsResponse.TermVector.Token> tokens = term.getTokens(); // <11>
for (TermVectorsResponse.TermVector.Token token : tokens) {
int position = token.getPosition(); // <12>
int startOffset = token.getStartOffset(); // <13>
int endOffset = token.getEndOffset(); // <14>
String payload = token.getPayload(); // <15>
}
}
}
}
}
}
// end::term-vectors-term-vectors
// tag::term-vectors-execute-listener
ActionListener<TermVectorsResponse> listener = new ActionListener<TermVectorsResponse>() {
@Override
public void onResponse(TermVectorsResponse termVectorsResponse) {
// <1>
}
@Override
public void onFailure(Exception e) {
// <2>
}
};
// end::term-vectors-execute-listener
CountDownLatch latch = new CountDownLatch(1);
listener = new LatchedActionListener<>(listener, latch);
// tag::term-vectors-execute-async
client.termvectorsAsync(request, RequestOptions.DEFAULT, listener); // <1>
// end::term-vectors-execute-async
assertTrue(latch.await(30L, TimeUnit.SECONDS));
}
@SuppressWarnings("unused")
public void testMultiGet() throws Exception {
RestHighLevelClient client = highLevelClient();
@ -1592,7 +1715,7 @@ public class CRUDDocumentationIT extends ESRestHighLevelClientTestCase {
// TODO status is broken! fix in a followup
// assertEquals(RestStatus.NOT_FOUND, ee.status()); // <4>
assertThat(e.getMessage(),
containsString("reason=no such index")); // <5>
containsString("reason=no such index [missing_index]")); // <5>
// end::multi-get-indexnotfound
// tag::multi-get-execute-listener

View File

@ -32,6 +32,8 @@ import org.elasticsearch.client.watcher.AckWatchRequest;
import org.elasticsearch.client.watcher.AckWatchResponse;
import org.elasticsearch.client.watcher.ActionStatus;
import org.elasticsearch.client.watcher.ActionStatus.AckStatus;
import org.elasticsearch.client.watcher.DeactivateWatchRequest;
import org.elasticsearch.client.watcher.DeactivateWatchResponse;
import org.elasticsearch.client.watcher.StartWatchServiceRequest;
import org.elasticsearch.client.watcher.StopWatchServiceRequest;
import org.elasticsearch.client.watcher.WatchStatus;
@ -47,6 +49,8 @@ import org.elasticsearch.rest.RestStatus;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import static org.hamcrest.Matchers.is;
public class WatcherDocumentationIT extends ESRestHighLevelClientTestCase {
public void testStartStopWatchService() throws Exception {
@ -297,6 +301,57 @@ public class WatcherDocumentationIT extends ESRestHighLevelClientTestCase {
}
}
public void testDeactivateWatch() throws Exception {
RestHighLevelClient client = highLevelClient();
{
BytesReference watch = new BytesArray("{ \n" +
" \"trigger\": { \"schedule\": { \"interval\": \"10h\" } },\n" +
" \"input\": { \"simple\": { \"foo\" : \"bar\" } },\n" +
" \"actions\": { \"logme\": { \"logging\": { \"text\": \"{{ctx.payload}}\" } } }\n" +
"}");
PutWatchRequest putWatchRequest = new PutWatchRequest("my_watch_id", watch, XContentType.JSON);
client.watcher().putWatch(putWatchRequest, RequestOptions.DEFAULT);
}
{
//tag::deactivate-watch-execute
DeactivateWatchRequest request = new DeactivateWatchRequest("my_watch_id");
DeactivateWatchResponse response = client.watcher().deactivateWatch(request, RequestOptions.DEFAULT);
//end::deactivate-watch-execute
assertThat(response.getStatus().state().isActive(), is(false));
}
{
DeactivateWatchRequest request = new DeactivateWatchRequest("my_watch_id");
// tag::deactivate-watch-execute-listener
ActionListener<DeactivateWatchResponse> listener = new ActionListener<DeactivateWatchResponse>() {
@Override
public void onResponse(DeactivateWatchResponse response) {
// <1>
}
@Override
public void onFailure(Exception e) {
// <2>
}
};
// end::deactivate-watch-execute-listener
// For testing, replace the empty listener by a blocking listener.
final CountDownLatch latch = new CountDownLatch(1);
listener = new LatchedActionListener<>(listener, latch);
// tag::deactivate-watch-execute-async
client.watcher().deactivateWatchAsync(request, RequestOptions.DEFAULT, listener); // <1>
// end::deactivate-watch-execute-async
assertTrue(latch.await(30L, TimeUnit.SECONDS));
}
}
public void testActivateWatch() throws Exception {
RestHighLevelClient client = highLevelClient();

View File

@ -0,0 +1,41 @@
/*
* 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.client.watcher;
import org.elasticsearch.test.ESTestCase;
import static org.hamcrest.Matchers.is;
public class DeactivateWatchRequestTests extends ESTestCase {
public void testNullId() {
NullPointerException actual = expectThrows(NullPointerException.class, () -> new DeactivateWatchRequest(null));
assertNotNull(actual);
assertThat(actual.getMessage(), is("watch id is missing"));
}
public void testInvalidId() {
IllegalArgumentException actual = expectThrows(IllegalArgumentException.class,
() -> new DeactivateWatchRequest("Watch id has spaces"));
assertNotNull(actual);
assertThat(actual.getMessage(), is("watch id contains whitespace"));
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.client.watcher;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
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.test.ESTestCase;
import java.io.IOException;
public class DeactivateWatchResponseTests extends ESTestCase {
public void testBasicParsing() throws IOException {
XContentType contentType = randomFrom(XContentType.values());
int version = randomInt();
ExecutionState executionState = randomFrom(ExecutionState.values());
XContentBuilder builder = XContentFactory.contentBuilder(contentType).startObject()
.startObject("status")
.field("version", version)
.field("execution_state", executionState)
.endObject()
.endObject();
BytesReference bytes = BytesReference.bytes(builder);
DeactivateWatchResponse response = parse(contentType, bytes);
WatchStatus status = response.getStatus();
assertNotNull(status);
assertEquals(version, status.version());
assertEquals(executionState, status.getExecutionState());
}
private DeactivateWatchResponse parse(XContentType contentType, BytesReference bytes) throws IOException {
XContentParser parser = XContentFactory.xContent(contentType)
.createParser(NamedXContentRegistry.EMPTY, null, bytes.streamInput());
parser.nextToken();
return DeactivateWatchResponse.fromXContent(parser);
}
}

View File

@ -36,7 +36,7 @@ import org.apache.http.nio.entity.NStringEntity;
import org.apache.http.ssl.SSLContextBuilder;
import org.apache.http.ssl.SSLContexts;
import org.apache.http.util.EntityUtils;
import org.elasticsearch.client.HttpAsyncResponseConsumerFactory.HeapBufferedResponseConsumerFactory;
import org.elasticsearch.client.HttpAsyncResponseConsumerFactory;
import org.elasticsearch.client.Node;
import org.elasticsearch.client.NodeSelector;
import org.elasticsearch.client.Request;
@ -84,7 +84,8 @@ public class RestClientDocumentation {
RequestOptions.Builder builder = RequestOptions.DEFAULT.toBuilder();
builder.addHeader("Authorization", "Bearer " + TOKEN); // <1>
builder.setHttpAsyncResponseConsumerFactory( // <2>
new HeapBufferedResponseConsumerFactory(30 * 1024 * 1024 * 1024));
new HttpAsyncResponseConsumerFactory
.HeapBufferedResponseConsumerFactory(30 * 1024 * 1024 * 1024));
COMMON_OPTIONS = builder.build();
}
// end::rest-client-options-singleton

View File

@ -61,17 +61,7 @@ integTestCluster {
systemProperty 'es.scripting.update.ctx_in_params', 'false'
}
// remove when https://github.com/elastic/elasticsearch/issues/31305 is fixed
if (rootProject.ext.compilerJavaVersion.isJava11()) {
integTestRunner {
systemProperty 'tests.rest.blacklist', [
'plugins/ingest-attachment/line_164',
'plugins/ingest-attachment/line_117'
].join(',')
}
}
// Build the cluster with all plugins
// build the cluster with all plugins
project.rootProject.subprojects.findAll { it.parent.path == ':plugins' }.each { subproj ->
/* Skip repositories. We just aren't going to be able to test them so it
* doesn't make sense to waste time installing them. */

View File

@ -0,0 +1,101 @@
--
:api: term-vectors
:request: TermVectorsRequest
:response: TermVectorsResponse
--
[id="{upid}-{api}"]
=== Term Vectors API
Term Vectors API returns information and statistics on terms in the fields
of a particular document. The document could be stored in the index or
artificially provided by the user.
[id="{upid}-{api}-request"]
==== Term Vectors Request
A +{request}+ expects an `index`, a `type` and an `id` to specify
a certain document, and fields for which the information is retrieved.
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests-file}[{api}-request]
--------------------------------------------------
Term vectors can also be generated for artificial documents, that is for
documents not present in the index:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests-file}[{api}-request-artificial]
--------------------------------------------------
<1> An artificial document is provided as an `XContentBuilder` object,
the Elasticsearch built-in helper to generate JSON content.
===== Optional arguments
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests-file}[{api}-request-optional-arguments]
--------------------------------------------------
<1> Set `fieldStatistics` to `false` (default is `true`) to omit document count,
sum of document frequencies, sum of total term frequencies.
<2> Set `termStatistics` to `true` (default is `false`) to display
total term frequency and document frequency.
<3> Set `positions` to `false` (default is `true`) to omit the output of
positions.
<4> Set `offsets` to `false` (default is `true`) to omit the output of
offsets.
<5> Set `payloads` to `false` (default is `true`) to omit the output of
payloads.
<6> Set `filterSettings` to filter the terms that can be returned based
on their tf-idf scores.
<7> Set `perFieldAnalyzer` to specify a different analyzer than
the one that the field has.
<8> Set `realtime` to `false` (default is `true`) to retrieve term vectors
near realtime.
<9> Set a routing parameter
include::../execution.asciidoc[]
[id="{upid}-{api}-response"]
==== TermVectorsResponse
The `TermVectorsResponse` contains the following information:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests-file}[{api}-response]
--------------------------------------------------
<1> The index name of the document.
<2> The type name of the document.
<3> The id of the document.
<4> Indicates whether or not the document found.
===== Inspecting Term Vectors
If `TermVectorsResponse` contains non-null list of term vectors,
more information about each term vector can be obtained using the following:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests-file}[{api}-term-vectors]
--------------------------------------------------
<1> The name of the current field
<2> Fields statistics for the current field - document count
<3> Fields statistics for the current field - sum of total term frequencies
<4> Fields statistics for the current field - sum of document frequencies
<5> Terms for the current field
<6> The name of the term
<7> Term frequency of the term
<8> Document frequency of the term
<9> Total term frequency of the term
<10> Score of the term
<11> Tokens of the term
<12> Position of the token
<13> Start offset of the token
<14> End offset of the token
<15> Payload of the token

View File

@ -119,68 +119,6 @@ include-tagged::{doc-tests}/RollupDocumentationIT.java[x-pack-rollup-put-rollup-
<2> Adds the metrics to compute on the `temperature` field
<3> Adds the metrics to compute on the `voltage` field
By default, metrics `min`/`max` for the fields in `DateHistogramGroupConfig` and
`HistogramGroupConfig` are added to the configuration unless the user already provided
metrics for those fields.
So, for the following configuration:
[source,js]
--------------------------------------------------
"groups" : {
"date_histogram": {
"field": "timestamp",
"interval": "1h",
"delay": "7d",
"time_zone": "UTC"
},
"terms": {
"fields": ["hostname", "datacenter"]
},
"histogram": {
"fields": ["load", "net_in", "net_out"],
"interval": 5
},
},
"metrics": [
{
"field": "load",
"metrics": ["max"]
},
{
"field": "net_in",
"metrics": ["max"]
}
]
--------------------------------------------------
// NOTCONSOLE
The following will be the metrics in the configuration after
the defaults are added server side. Note the default metrics
provided for the fields `timestamp` and `net_out`
[source,js]
--------------------------------------------------
"metrics": [
{
"field": "load",
"metrics": ["max"]
},
{
"field": "net_in",
"metrics": ["max"]
},
{
"field": "timestamp",
"metrics": ["min", "max"]
},
{
"field": "net_out",
"metrics": ["min", "max"]
}
]
--------------------------------------------------
// NOTCONSOLE
[[java-rest-high-x-pack-rollup-put-rollup-job-execution]]
==== Execution

View File

@ -14,6 +14,7 @@ Single document APIs::
* <<{upid}-exists>>
* <<{upid}-delete>>
* <<{upid}-update>>
* <<{upid}-term-vectors>>
[[multi-doc]]
Multi-document APIs::
@ -29,6 +30,7 @@ include::document/get.asciidoc[]
include::document/exists.asciidoc[]
include::document/delete.asciidoc[]
include::document/update.asciidoc[]
include::document/term-vectors.asciidoc[]
include::document/bulk.asciidoc[]
include::document/multi-get.asciidoc[]
include::document/reindex.asciidoc[]
@ -344,6 +346,7 @@ The Java High Level REST Client supports the following Watcher APIs:
* <<{upid}-stop-watch-service>>
* <<java-rest-high-x-pack-watcher-put-watch>>
* <<java-rest-high-x-pack-watcher-delete-watch>>
* <<java-rest-high-watcher-deactivate-watch>>
* <<{upid}-ack-watch>>
* <<{upid}-activate-watch>>
@ -352,6 +355,7 @@ include::watcher/stop-watch-service.asciidoc[]
include::watcher/put-watch.asciidoc[]
include::watcher/delete-watch.asciidoc[]
include::watcher/ack-watch.asciidoc[]
include::watcher/deactivate-watch.asciidoc[]
include::watcher/activate-watch.asciidoc[]
== Graph APIs
@ -372,4 +376,4 @@ don't leak into the rest of the documentation.
:response!:
:doc-tests-file!:
:upid!:
--
--

View File

@ -0,0 +1,10 @@
--
:api: deactivate-watch
:request: deactivateWatchRequet
:response: deactivateWatchResponse
:doc-tests-file: {doc-tests}/WatcherDocumentationIT.java
--
[[java-rest-high-watcher-deactivate-watch]]
=== Deactivate Watch API
include::../execution.asciidoc[]

View File

@ -234,6 +234,13 @@ specific index module:
The length of time that a <<delete-versioning,deleted document's version number>> remains available for <<index-versioning,further versioned operations>>.
Defaults to `60s`.
`index.default_pipeline`::
The default <<ingest,ingest node>> pipeline for this index. Index requests will fail
if the default pipeline is set and the pipeline does not exist. The default may be
overridden using the `pipeline` parameter. The special pipeline name `_none` indicates
no ingest pipeline should be run.
[float]
=== Settings in other index modules

View File

@ -35,6 +35,9 @@ PUT my-index/_doc/my-id?pipeline=my_pipeline_id
// CONSOLE
// TEST[catch:bad_request]
An index may also declare a <<dynamic-index-settings,default pipeline>> that will be used in the
absence of the `pipeline` parameter.
See <<ingest-apis,Ingest APIs>> for more information about creating, adding, and deleting pipelines.
--

View File

@ -776,16 +776,16 @@ Accepts a single value or an array of values.
[options="header"]
|======
| Name | Required | Default | Description
| `field` | yes | - | The field to be appended to
| `value` | yes | - | The value to be appended
| `field` | yes | - | The field to be appended to. Supports <<accessing-template-fields,template snippets>>.
| `value` | yes | - | The value to be appended. Supports <<accessing-template-fields,template snippets>>.
|======
[source,js]
--------------------------------------------------
{
"append": {
"field": "field1",
"value": ["item2", "item3", "item4"]
"field": "tags",
"value": ["production", "{{app}}", "{{owner}}"]
}
}
--------------------------------------------------
@ -812,7 +812,7 @@ the field is not a supported format or resultant value exceeds 2^63.
--------------------------------------------------
{
"bytes": {
"field": "foo"
"field": "file.size"
}
}
--------------------------------------------------
@ -850,7 +850,7 @@ still be updated with the unconverted field value.
--------------------------------------------------
{
"convert": {
"field" : "foo",
"field" : "url.port",
"type": "integer"
}
}
@ -874,8 +874,8 @@ in the same order they were defined as part of the processor definition.
| `field` | yes | - | The field to get the date from.
| `target_field` | no | @timestamp | The field that will hold the parsed date.
| `formats` | yes | - | An array of the expected date formats. Can be a Joda pattern or one of the following formats: ISO8601, UNIX, UNIX_MS, or TAI64N.
| `timezone` | no | UTC | The timezone to use when parsing the date.
| `locale` | no | ENGLISH | The locale to use when parsing the date, relevant when parsing month names or week days.
| `timezone` | no | UTC | The timezone to use when parsing the date. Supports <<accessing-template-fields,template snippets>>.
| `locale` | no | ENGLISH | The locale to use when parsing the date, relevant when parsing month names or week days. Supports <<accessing-template-fields,template snippets>>.
|======
Here is an example that adds the parsed date to the `timestamp` field based on the `initial_date` field:
@ -913,8 +913,8 @@ the timezone and locale values.
"field" : "initial_date",
"target_field" : "timestamp",
"formats" : ["ISO8601"],
"timezone" : "{{ my_timezone }}",
"locale" : "{{ my_locale }}"
"timezone" : "{{my_timezone}}",
"locale" : "{{my_locale}}"
}
}
]
@ -1059,12 +1059,12 @@ understands this to mean `2016-04-01` as is explained in the <<date-math-index-n
|======
| Name | Required | Default | Description
| `field` | yes | - | The field to get the date or timestamp from.
| `index_name_prefix` | no | - | A prefix of the index name to be prepended before the printed date.
| `date_rounding` | yes | - | How to round the date when formatting the date into the index name. Valid values are: `y` (year), `M` (month), `w` (week), `d` (day), `h` (hour), `m` (minute) and `s` (second).
| `index_name_prefix` | no | - | A prefix of the index name to be prepended before the printed date. Supports <<accessing-template-fields,template snippets>>.
| `date_rounding` | yes | - | How to round the date when formatting the date into the index name. Valid values are: `y` (year), `M` (month), `w` (week), `d` (day), `h` (hour), `m` (minute) and `s` (second). Supports <<accessing-template-fields,template snippets>>.
| `date_formats` | no | yyyy-MM-dd'T'HH:mm:ss.SSSZ | An array of the expected date formats for parsing dates / timestamps in the document being preprocessed. Can be a Joda pattern or one of the following formats: ISO8601, UNIX, UNIX_MS, or TAI64N.
| `timezone` | no | UTC | The timezone to use when parsing the date and when date math index supports resolves expressions into concrete index names.
| `locale` | no | ENGLISH | The locale to use when parsing the date from the document being preprocessed, relevant when parsing month names or week days.
| `index_name_format` | no | yyyy-MM-dd | The format to be used when printing the parsed date into the index name. An valid Joda pattern is expected here.
| `index_name_format` | no | yyyy-MM-dd | The format to be used when printing the parsed date into the index name. An valid Joda pattern is expected here. Supports <<accessing-template-fields,template snippets>>.
|======
[[dissect-processor]]
@ -1260,6 +1260,21 @@ Reference key modifier example
* error = REFUSED
|======
[[drop-processor]]
=== Drop Processor
Drops the document without raising any errors. This is useful to prevent the document from
getting indexed based on some condition.
[source,js]
--------------------------------------------------
{
"drop": {
"if" : "ctx.network_name == 'Guest'"
}
}
--------------------------------------------------
// NOTCONSOLE
[[dot-expand-processor]]
=== Dot Expander Processor
@ -1390,14 +1405,15 @@ to the requester.
[options="header"]
|======
| Name | Required | Default | Description
| `message` | yes | - | The error message of the `FailException` thrown by the processor
| `message` | yes | - | The error message thrown by the processor. Supports <<accessing-template-fields,template snippets>>.
|======
[source,js]
--------------------------------------------------
{
"fail": {
"message": "an error message"
"if" : "ctx.tags.contains('production') != true",
"message": "The production tag is not present, found tags: {{tags}}"
}
}
--------------------------------------------------
@ -2093,6 +2109,120 @@ Converts a string to its lowercase equivalent.
--------------------------------------------------
// NOTCONSOLE
[[pipeline-processor]]
=== Pipeline Processor
Executes another pipeline.
[[pipeline-options]]
.Pipeline Options
[options="header"]
|======
| Name | Required | Default | Description
| `name` | yes | - | The name of the pipeline to execute
|======
[source,js]
--------------------------------------------------
{
"pipeline": {
"name": "inner-pipeline"
}
}
--------------------------------------------------
// NOTCONSOLE
An example of using this processor for nesting pipelines would be:
Define an inner pipeline:
[source,js]
--------------------------------------------------
PUT _ingest/pipeline/pipelineA
{
"description" : "inner pipeline",
"processors" : [
{
"set" : {
"field": "inner_pipeline_set",
"value": "inner"
}
}
]
}
--------------------------------------------------
// CONSOLE
Define another pipeline that uses the previously defined inner pipeline:
[source,js]
--------------------------------------------------
PUT _ingest/pipeline/pipelineB
{
"description" : "outer pipeline",
"processors" : [
{
"pipeline" : {
"name": "pipelineA"
}
},
{
"set" : {
"field": "outer_pipeline_set",
"value": "outer"
}
}
]
}
--------------------------------------------------
// CONSOLE
// TEST[continued]
Now indexing a document while applying the outer pipeline will see the inner pipeline executed
from the outer pipeline:
[source,js]
--------------------------------------------------
PUT /myindex/_doc/1?pipeline=pipelineB
{
"field": "value"
}
--------------------------------------------------
// CONSOLE
// TEST[continued]
Response from the index request:
[source,js]
--------------------------------------------------
{
"_index": "myindex",
"_type": "_doc",
"_id": "1",
"_version": 1,
"result": "created",
"_shards": {
"total": 2,
"successful": 1,
"failed": 0
},
"_seq_no": 0,
"_primary_term": 1,
}
--------------------------------------------------
// TESTRESPONSE
Indexed document:
[source,js]
--------------------------------------------------
{
"field": "value",
"inner_pipeline_set": "inner",
"outer_pipeline_set": "outer"
}
--------------------------------------------------
// NOTCONSOLE
[[remove-processor]]
=== Remove Processor
Removes existing fields. If one field doesn't exist, an exception will be thrown.
@ -2102,7 +2232,7 @@ Removes existing fields. If one field doesn't exist, an exception will be thrown
[options="header"]
|======
| Name | Required | Default | Description
| `field` | yes | - | Fields to be removed
| `field` | yes | - | Fields to be removed. Supports <<accessing-template-fields,template snippets>>.
| `ignore_missing` | no | `false` | If `true` and `field` does not exist or is `null`, the processor quietly exits without modifying the document
|======
@ -2112,7 +2242,7 @@ Here is an example to remove a single field:
--------------------------------------------------
{
"remove": {
"field": "foo"
"field": "user_agent"
}
}
--------------------------------------------------
@ -2124,7 +2254,7 @@ To remove multiple fields, you can use the following query:
--------------------------------------------------
{
"remove": {
"field": ["foo", "bar"]
"field": ["user_agent", "url"]
}
}
--------------------------------------------------
@ -2138,18 +2268,18 @@ Renames an existing field. If the field doesn't exist or the new name is already
.Rename Options
[options="header"]
|======
| Name | Required | Default | Description
| `field` | yes | - | The field to be renamed
| `target_field` | yes | - | The new name of the field
| `ignore_missing` | no | `false` | If `true` and `field` does not exist, the processor quietly exits without modifying the document
| Name | Required | Default | Description
| `field` | yes | - | The field to be renamed. Supports <<accessing-template-fields,template snippets>>.
| `target_field` | yes | - | The new name of the field. Supports <<accessing-template-fields,template snippets>>.
| `ignore_missing` | no | `false` | If `true` and `field` does not exist, the processor quietly exits without modifying the document
|======
[source,js]
--------------------------------------------------
{
"rename": {
"field": "foo",
"target_field": "foobar"
"field": "provider",
"target_field": "cloud.provider"
}
}
--------------------------------------------------
@ -2267,18 +2397,18 @@ its value will be replaced with the provided one.
.Set Options
[options="header"]
|======
| Name | Required | Default | Description
| `field` | yes | - | The field to insert, upsert, or update
| `value` | yes | - | The value to be set for the field
| `override`| no | true | If processor will update fields with pre-existing non-null-valued field. When set to `false`, such fields will not be touched.
| Name | Required | Default | Description
| `field` | yes | - | The field to insert, upsert, or update. Supports <<accessing-template-fields,template snippets>>.
| `value` | yes | - | The value to be set for the field. Supports <<accessing-template-fields,template snippets>>.
| `override` | no | true | If processor will update fields with pre-existing non-null-valued field. When set to `false`, such fields will not be touched.
|======
[source,js]
--------------------------------------------------
{
"set": {
"field": "field1",
"value": 582.1
"field": "host.os.name",
"value": "{{os}}"
}
}
--------------------------------------------------
@ -2331,7 +2461,7 @@ Throws an error when the field is not an array.
--------------------------------------------------
{
"sort": {
"field": "field_to_sort",
"field": "array_field_to_sort",
"order": "desc"
}
}

View File

@ -18,3 +18,10 @@ primary shards of the opened index to be allocated.
[float]
==== Shard preferences `_primary`, `_primary_first`, `_replica`, and `_replica_first` are removed
These shard preferences are removed in favour of the `_prefer_nodes` and `_only_nodes` preferences.
[float]
==== Cluster-wide shard soft limit
Clusters now have soft limits on the total number of open shards in the cluster
based on the number of nodes and the `cluster.max_shards_per_node` cluster
setting, to prevent accidental operations that would destabilize the cluster.
More information can be found in the <<misc-cluster,documentation for that setting>>.

View File

@ -22,6 +22,12 @@ appropriate request directly.
* All classes present in `org.elasticsearch.search.aggregations.metrics.*` packages
were moved to a single `org.elasticsearch.search.aggregations.metrics` package.
* All classes present in `org.elasticsearch.search.aggregations.pipeline.*` packages
were moved to a single `org.elasticsearch.search.aggregations.pipeline` package. In
addition, `org.elasticsearch.search.aggregations.pipeline.PipelineAggregationBuilders`
was moved to `org.elasticsearch.search.aggregations.PipelineAggregationBuilders`
[float]
==== `Retry.withBackoff` methods with `Settings` removed

View File

@ -22,6 +22,48 @@ user with access to the <<cluster-update-settings,cluster-update-settings>>
API can make the cluster read-write again.
[[cluster-shard-limit]]
==== Cluster Shard Limit
In a Elasticsearch 7.0 and later, there will be a soft limit on the number of
shards in a cluster, based on the number of nodes in the cluster. This is
intended to prevent operations which may unintentionally destabilize the
cluster. Prior to 7.0, actions which would result in the cluster going over the
limit will issue a deprecation warning.
NOTE: You can set the system property `es.enforce_max_shards_per_node` to `true`
to opt in to strict enforcement of the shard limit. If this system property is
set, actions which would result in the cluster going over the limit will result
in an error, rather than a deprecation warning. This property will be removed in
Elasticsearch 7.0, as strict enforcement of the limit will be the default and
only behavior.
If an operation, such as creating a new index, restoring a snapshot of an index,
or opening a closed index would lead to the number of shards in the cluster
going over this limit, the operation will issue a deprecation warning.
If the cluster is already over the limit, due to changes in node membership or
setting changes, all operations that create or open indices will issue warnings
until either the limit is increased as described below, or some indices are
<<indices-open-close,closed>> or <<indices-delete-index,deleted>> to bring the
number of shards below the limit.
Replicas count towards this limit, but closed indexes do not. An index with 5
primary shards and 2 replicas will be counted as 15 shards. Any closed index
is counted as 0, no matter how many shards and replicas it contains.
The limit defaults to 1,000 shards per node, and be dynamically adjusted using
the following property:
`cluster.max_shards_per_node`::
Controls the number of shards allowed in the cluster per node.
For example, a 3-node cluster with the default setting would allow 3,000 shards
total, across all open indexes. If the above setting is changed to 1,500, then
the cluster would allow 4,500 shards total.
[[user-defined-data]]
==== User Defined Cluster Metadata
@ -109,4 +151,4 @@ Enable or disable allocation for persistent tasks:
This setting does not affect the persistent tasks that are already being executed.
Only newly created persistent tasks, or tasks that must be reassigned (after a node
left the cluster, for example), are impacted by this setting.
--
--

View File

@ -30,6 +30,10 @@ Available settings include:
Control the resource limits on the shard recovery process.
<<search-settings,Search Settings>>::
Control global search settings.
include::indices/circuit_breaker.asciidoc[]
include::indices/fielddata.asciidoc[]
@ -42,3 +46,5 @@ include::indices/request_cache.asciidoc[]
include::indices/recovery.asciidoc[]
include::indices/search-settings.asciidoc[]

View File

@ -0,0 +1,16 @@
[[search-settings]]
=== Search Settings
The following _expert_ setting can be set to manage global search limits.
`indices.query.bool.max_clause_count`::
Defaults to `1024`.
This setting limits the number of clauses a Lucene BooleanQuery can have. The
default of 1024 is quite high and should normally be sufficient. This limit does
not only affect Elasticsearchs `bool` query, but many other queries are rewritten to Lucene's
BooleanQuery internally. The limit is in place to prevent searches from becoming to large
and taking up too much CPU and memory. In case you consider to increase this setting,
make sure you exhausted all other options to avoid having to do this. Higher values can lead
to performance degradations and memory issues, especially in clusters with a high load or
few resources.

View File

@ -88,13 +88,6 @@ Which will yield the following response:
"metrics" : [
"avg"
]
},
{
"field": "timestamp",
"metrics": [
"max",
"min"
]
}
],
"timeout" : "20s",
@ -215,13 +208,6 @@ Which will yield the following response:
"metrics" : [
"avg"
]
},
{
"field": "timestamp",
"metrics": [
"min",
"max"
]
}
],
"timeout" : "20s",
@ -271,13 +257,6 @@ Which will yield the following response:
"metrics" : [
"avg"
]
},
{
"field": "timestamp",
"metrics": [
"min",
"max"
]
}
],
"timeout" : "20s",

View File

@ -68,7 +68,7 @@ PUT _xpack/rollup/job/sensor
"groups" : {
"date_histogram": {
"field": "timestamp",
"interval": "60m",
"interval": "1h",
"delay": "7d"
},
"terms": {
@ -98,84 +98,4 @@ When the job is created, you receive the following results:
"acknowledged": true
}
----
// TESTRESPONSE
By default the metrics `min`/`max` are added
for the fields in the `date_histogram` and `histogram` configurations.
If this behavior is not desired, explicitly configure metrics
for those fields. This will override the defaults.
If the following is provided
[source,js]
--------------------------------------------------
PUT _xpack/rollup/job/sensor2
{
"index_pattern": "sensor-*",
"rollup_index": "sensor_rollup",
"cron": "*/30 * * * * ?",
"page_size" :1000,
"groups" : {
"date_histogram": {
"field": "timestamp",
"interval": "60m",
"delay": "7d"
},
"histogram": {
"fields": ["voltage", "temperature"],
"interval": 5
}
},
"metrics": [
{
"field": "temperature",
"metrics": ["min", "max", "sum"]
}
]
}
--------------------------------------------------
// NOTCONSOLE
// TEST[setup:sensor_index]
The actual config when created in the cluster will look as follows.
[source,js]
--------------------------------------------------
{
"index_pattern": "sensor-*",
"rollup_index": "sensor_rollup",
"cron": "*/30 * * * * ?",
"page_size" :1000,
"groups" : {
"date_histogram": {
"field": "timestamp",
"interval": "60m",
"delay": "7d"
},
"histogram": {
"fields": ["voltage", "temperature"],
"interval": 5
}
},
"metrics": [
{
"field": "temperature",
"metrics": ["min", "max", "sum"]
},
{
"field": "voltage", <1>
"metrics": ["min", "max"]
},
{
"field": "timestamp", <1>
"metrics": ["min", "max"]
}
]
}
--------------------------------------------------
// NOTCONSOLE
<1> Notice the new default metrics gathered for `voltage` and `timestamp`.
Since these fields were referenced in `groups.histogram` and
`groups.date_histogram` configurations
respectively and no metrics were requested for them,
they both got the default metrics of `["min", "max"]`.
// TESTRESPONSE

View File

@ -124,12 +124,6 @@ Which will yield the following response:
"time_zone" : "UTC",
"interval" : "1h",
"delay": "7d"
},
{
"agg": "max"
},
{
"agg": "min"
}
],
"voltage" : [

View File

@ -120,12 +120,6 @@ This will yield the following response:
"time_zone" : "UTC",
"interval" : "1h",
"delay": "7d"
},
{
"agg" : "max"
},
{
"agg" : "min"
}
],
"voltage" : [

View File

@ -47,7 +47,8 @@ public class CharArraysTests extends ESTestCase {
assertFalse(CharArrays.charsBeginsWith(randomAlphaOfLength(4), null));
assertFalse(CharArrays.charsBeginsWith(null, null));
assertFalse(CharArrays.charsBeginsWith(null, randomAlphaOfLength(4).toCharArray()));
assertFalse(CharArrays.charsBeginsWith(randomAlphaOfLength(2), randomAlphaOfLengthBetween(3, 8).toCharArray()));
final String undesiredPrefix = randomAlphaOfLength(2);
assertFalse(CharArrays.charsBeginsWith(undesiredPrefix, randomAlphaOfLengthNotBeginningWith(undesiredPrefix, 3, 8)));
final String prefix = randomAlphaOfLengthBetween(2, 4);
assertTrue(CharArrays.charsBeginsWith(prefix, prefix.toCharArray()));
@ -72,4 +73,12 @@ public class CharArraysTests extends ESTestCase {
assertFalse(CharArrays.constantTimeEquals(value, other));
assertFalse(CharArrays.constantTimeEquals(value.toCharArray(), other.toCharArray()));
}
private char[] randomAlphaOfLengthNotBeginningWith(String undesiredPrefix, int min, int max) {
char[] nonMatchingValue;
do {
nonMatchingValue = randomAlphaOfLengthBetween(min, max).toCharArray();
} while (new String(nonMatchingValue).startsWith(undesiredPrefix));
return nonMatchingValue;
}
}

View File

@ -1,6 +1,9 @@
# Integration tests for Matrix Aggs Plugin
#
"Matrix stats aggs loaded":
- skip:
reason: "contains is a newly added assertion"
features: contains
- do:
cluster.state: {}

View File

@ -1,4 +1,7 @@
"Module loaded":
- skip:
reason: "contains is a newly added assertion"
features: contains
- do:
cluster.state: {}

View File

@ -1,4 +1,7 @@
"Ingest common installed":
- skip:
reason: "contains is a newly added assertion"
features: contains
- do:
cluster.state: {}

View File

@ -26,27 +26,23 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.DoubleValues;
import org.apache.lucene.search.DoubleValuesSource;
import org.elasticsearch.script.GeneralScriptException;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.script.ScoreScript;
import java.io.IOException;
/**
* A bridge to evaluate an {@link Expression} against {@link Bindings} in the context
* of a {@link SearchScript}.
* of a {@link ScoreScript}.
*/
class ExpressionSearchScript implements SearchScript.LeafFactory {
class ExpressionScoreScript implements ScoreScript.LeafFactory {
final Expression exprScript;
final SimpleBindings bindings;
final DoubleValuesSource source;
final ReplaceableConstDoubleValueSource specialValue; // _value
final boolean needsScores;
private final Expression exprScript;
private final DoubleValuesSource source;
private final boolean needsScores;
ExpressionSearchScript(Expression e, SimpleBindings b, ReplaceableConstDoubleValueSource v, boolean needsScores) {
exprScript = e;
bindings = b;
source = exprScript.getDoubleValuesSource(bindings);
specialValue = v;
ExpressionScoreScript(Expression e, SimpleBindings b, boolean needsScores) {
this.exprScript = e;
this.source = exprScript.getDoubleValuesSource(b);
this.needsScores = needsScores;
}
@ -55,15 +51,14 @@ class ExpressionSearchScript implements SearchScript.LeafFactory {
return needsScores;
}
@Override
public SearchScript newInstance(final LeafReaderContext leaf) throws IOException {
return new SearchScript(null, null, null) {
public ScoreScript newInstance(final LeafReaderContext leaf) throws IOException {
return new ScoreScript(null, null, null) {
// Fake the scorer until setScorer is called.
DoubleValues values = source.getValues(leaf, new DoubleValues() {
@Override
public double doubleValue() throws IOException {
return getScore();
return get_score();
}
@Override
@ -73,10 +68,7 @@ class ExpressionSearchScript implements SearchScript.LeafFactory {
});
@Override
public Object run() { return Double.valueOf(runAsDouble()); }
@Override
public double runAsDouble() {
public double execute() {
try {
return values.doubleValue();
} catch (Exception exception) {
@ -92,24 +84,6 @@ class ExpressionSearchScript implements SearchScript.LeafFactory {
throw new IllegalStateException("Can't advance to doc using " + exprScript, e);
}
}
@Override
public void setNextAggregationValue(Object value) {
// _value isn't used in script if specialValue == null
if (specialValue != null) {
if (value instanceof Number) {
specialValue.setValue(((Number)value).doubleValue());
} else {
throw new GeneralScriptException("Cannot use expression with text variable using " + exprScript);
}
}
}
@Override
public void setNextVar(String name, Object value) {
// other per-document variables aren't supported yet, even if they are numbers
// but we shouldn't encourage this anyway.
}
};
}

View File

@ -23,10 +23,8 @@ import org.apache.lucene.expressions.Expression;
import org.apache.lucene.expressions.SimpleBindings;
import org.apache.lucene.expressions.js.JavascriptCompiler;
import org.apache.lucene.expressions.js.VariableContext;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
import org.apache.lucene.search.Scorable;
import org.apache.lucene.search.SortField;
import org.elasticsearch.SpecialPermission;
import org.elasticsearch.common.Nullable;
@ -48,11 +46,9 @@ import org.elasticsearch.script.ScoreScript;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptEngine;
import org.elasticsearch.script.ScriptException;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.script.TermsSetQueryScript;
import org.elasticsearch.search.lookup.SearchLookup;
import java.io.IOException;
import java.security.AccessControlContext;
import java.security.AccessController;
import java.security.PrivilegedAction;
@ -63,8 +59,9 @@ import java.util.List;
import java.util.Map;
/**
* Provides the infrastructure for Lucene expressions as a scripting language for Elasticsearch. Only
* {@link SearchScript}s are supported.
* Provides the infrastructure for Lucene expressions as a scripting language for Elasticsearch.
*
* Only contexts returning numeric types or {@link Object} are supported.
*/
public class ExpressionScriptEngine extends AbstractComponent implements ScriptEngine {
@ -111,10 +108,7 @@ public class ExpressionScriptEngine extends AbstractComponent implements ScriptE
}
}
});
if (context.instanceClazz.equals(SearchScript.class)) {
SearchScript.Factory factory = (p, lookup) -> newSearchScript(expr, lookup, p);
return context.factoryClazz.cast(factory);
} else if (context.instanceClazz.equals(BucketAggregationScript.class)) {
if (context.instanceClazz.equals(BucketAggregationScript.class)) {
return context.factoryClazz.cast(newBucketAggregationScriptFactory(expr));
} else if (context.instanceClazz.equals(BucketAggregationSelectorScript.class)) {
BucketAggregationScript.Factory factory = newBucketAggregationScriptFactory(expr);
@ -178,40 +172,6 @@ public class ExpressionScriptEngine extends AbstractComponent implements ScriptE
};
}
private SearchScript.LeafFactory newSearchScript(Expression expr, SearchLookup lookup, @Nullable Map<String, Object> vars) {
// NOTE: if we need to do anything complicated with bindings in the future, we can just extend Bindings,
// instead of complicating SimpleBindings (which should stay simple)
SimpleBindings bindings = new SimpleBindings();
ReplaceableConstDoubleValueSource specialValue = null;
boolean needsScores = false;
for (String variable : expr.variables) {
try {
if (variable.equals("_score")) {
bindings.add(new SortField("_score", SortField.Type.SCORE));
needsScores = true;
} else if (variable.equals("_value")) {
specialValue = new ReplaceableConstDoubleValueSource();
bindings.add("_value", specialValue);
// noop: _value is special for aggregations, and is handled in ExpressionScriptBindings
// TODO: if some uses it in a scoring expression, they will get a nasty failure when evaluating...need a
// way to know this is for aggregations and so _value is ok to have...
} else if (vars != null && vars.containsKey(variable)) {
bindFromParams(vars, bindings, variable);
} else {
// delegate valuesource creation based on field's type
// there are three types of "fields" to expressions, and each one has a different "api" of variables and methods.
final ValueSource valueSource = getDocValueSource(variable, lookup);
needsScores |= valueSource.getSortField(false).needsScores();
bindings.add(variable, valueSource.asDoubleValuesSource());
}
} catch (Exception e) {
// we defer "binding" of variables until here: give context for that variable
throw convertToScriptException("link error", expr.sourceText, variable, e);
}
}
return new ExpressionSearchScript(expr, bindings, specialValue, needsScores);
}
private NumberSortScript.LeafFactory newSortScript(Expression expr, SearchLookup lookup, @Nullable Map<String, Object> vars) {
// NOTE: if we need to do anything complicated with bindings in the future, we can just extend Bindings,
// instead of complicating SimpleBindings (which should stay simple)
@ -315,13 +275,13 @@ public class ExpressionScriptEngine extends AbstractComponent implements ScriptE
* See https://github.com/elastic/elasticsearch/issues/26429.
*/
private FilterScript.LeafFactory newFilterScript(Expression expr, SearchLookup lookup, @Nullable Map<String, Object> vars) {
SearchScript.LeafFactory searchLeafFactory = newSearchScript(expr, lookup, vars);
ScoreScript.LeafFactory searchLeafFactory = newScoreScript(expr, lookup, vars);
return ctx -> {
SearchScript script = searchLeafFactory.newInstance(ctx);
ScoreScript script = searchLeafFactory.newInstance(ctx);
return new FilterScript(vars, lookup, ctx) {
@Override
public boolean execute() {
return script.runAsDouble() != 0.0;
return script.execute() != 0.0;
}
@Override
public void setDocument(int docid) {
@ -332,39 +292,37 @@ public class ExpressionScriptEngine extends AbstractComponent implements ScriptE
}
private ScoreScript.LeafFactory newScoreScript(Expression expr, SearchLookup lookup, @Nullable Map<String, Object> vars) {
SearchScript.LeafFactory searchLeafFactory = newSearchScript(expr, lookup, vars);
return new ScoreScript.LeafFactory() {
@Override
public boolean needs_score() {
return searchLeafFactory.needs_score();
// NOTE: if we need to do anything complicated with bindings in the future, we can just extend Bindings,
// instead of complicating SimpleBindings (which should stay simple)
SimpleBindings bindings = new SimpleBindings();
ReplaceableConstDoubleValueSource specialValue = null;
boolean needsScores = false;
for (String variable : expr.variables) {
try {
if (variable.equals("_score")) {
bindings.add(new SortField("_score", SortField.Type.SCORE));
needsScores = true;
} else if (variable.equals("_value")) {
specialValue = new ReplaceableConstDoubleValueSource();
bindings.add("_value", specialValue);
// noop: _value is special for aggregations, and is handled in ExpressionScriptBindings
// TODO: if some uses it in a scoring expression, they will get a nasty failure when evaluating...need a
// way to know this is for aggregations and so _value is ok to have...
} else if (vars != null && vars.containsKey(variable)) {
bindFromParams(vars, bindings, variable);
} else {
// delegate valuesource creation based on field's type
// there are three types of "fields" to expressions, and each one has a different "api" of variables and methods.
final ValueSource valueSource = getDocValueSource(variable, lookup);
needsScores |= valueSource.getSortField(false).needsScores();
bindings.add(variable, valueSource.asDoubleValuesSource());
}
} catch (Exception e) {
// we defer "binding" of variables until here: give context for that variable
throw convertToScriptException("link error", expr.sourceText, variable, e);
}
@Override
public ScoreScript newInstance(LeafReaderContext ctx) throws IOException {
SearchScript script = searchLeafFactory.newInstance(ctx);
return new ScoreScript(vars, lookup, ctx) {
@Override
public double execute() {
return script.runAsDouble();
}
@Override
public void setDocument(int docid) {
script.setDocument(docid);
}
@Override
public void setScorer(Scorable scorer) {
script.setScorer(scorer);
}
@Override
public double get_score() {
return script.getScore();
}
};
}
};
}
return new ExpressionScoreScript(expr, bindings, needsScores);
}
/**

View File

@ -53,7 +53,7 @@ import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDI
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilders.bucketScript;
import static org.elasticsearch.search.aggregations.PipelineAggregatorBuilders.bucketScript;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo;

View File

@ -1,6 +1,9 @@
# Integration tests for Expression scripts
#
"Expression loaded":
- skip:
reason: "contains is a newly added assertion"
features: contains
- do:
cluster.state: {}

View File

@ -169,7 +169,7 @@ public class MultiSearchTemplateIT extends ESIntegTestCase {
MultiSearchTemplateResponse.Item response4 = response.getResponses()[3];
assertThat(response4.isFailure(), is(true));
assertThat(response4.getFailure(), instanceOf(IndexNotFoundException.class));
assertThat(response4.getFailure().getMessage(), equalTo("no such index"));
assertThat(response4.getFailure().getMessage(), equalTo("no such index [unknown]"));
MultiSearchTemplateResponse.Item response5 = response.getResponses()[4];
assertThat(response5.isFailure(), is(false));

View File

@ -1,6 +1,9 @@
# Integration tests for Mustache scripts
#
"Mustache loaded":
- skip:
reason: "contains is a newly added assertion"
features: contains
- do:
cluster.state: {}

View File

@ -66,13 +66,17 @@ public final class Whitelist {
/** The {@link List} of all the whitelisted Painless class bindings. */
public final List<WhitelistClassBinding> whitelistClassBindings;
/** The {@link List} of all the whitelisted Painless instance bindings. */
public final List<WhitelistInstanceBinding> whitelistInstanceBindings;
/** Standard constructor. All values must be not {@code null}. */
public Whitelist(ClassLoader classLoader, List<WhitelistClass> whitelistClasses,
List<WhitelistMethod> whitelistImportedMethods, List<WhitelistClassBinding> whitelistClassBindings) {
public Whitelist(ClassLoader classLoader, List<WhitelistClass> whitelistClasses, List<WhitelistMethod> whitelistImportedMethods,
List<WhitelistClassBinding> whitelistClassBindings, List<WhitelistInstanceBinding> whitelistInstanceBindings) {
this.classLoader = Objects.requireNonNull(classLoader);
this.whitelistClasses = Collections.unmodifiableList(Objects.requireNonNull(whitelistClasses));
this.whitelistImportedMethods = Collections.unmodifiableList(Objects.requireNonNull(whitelistImportedMethods));
this.whitelistClassBindings = Collections.unmodifiableList(Objects.requireNonNull(whitelistClassBindings));
this.whitelistInstanceBindings = Collections.unmodifiableList(Objects.requireNonNull(whitelistInstanceBindings));
}
}

View File

@ -42,9 +42,7 @@ public class WhitelistClassBinding {
/** The method name for this class binding. */
public final String methodName;
/**
* The canonical type name for the return type.
*/
/** The canonical type name for the return type. */
public final String returnCanonicalTypeName;
/**

View File

@ -0,0 +1,61 @@
/*
* 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.painless.spi;
import java.util.List;
import java.util.Objects;
/**
* An instance binding represents a method call that stores state. Each instance binding must provide
* exactly one public method name. The canonical type name parameters provided must match those of the
* method. The method for an instance binding will target the specified Java instance.
*/
public class WhitelistInstanceBinding {
/** Information about where this constructor was whitelisted from. */
public final String origin;
/** The Java instance this instance binding targets. */
public final Object targetInstance;
/** The method name for this class binding. */
public final String methodName;
/** The canonical type name for the return type. */
public final String returnCanonicalTypeName;
/**
* A {@link List} of {@link String}s that are the Painless type names for the parameters of the
* constructor which can be used to look up the Java constructor through reflection.
*/
public final List<String> canonicalTypeNameParameters;
/** Standard constructor. All values must be not {@code null}. */
public WhitelistInstanceBinding(String origin, Object targetInstance,
String methodName, String returnCanonicalTypeName, List<String> canonicalTypeNameParameters) {
this.origin = Objects.requireNonNull(origin);
this.targetInstance = Objects.requireNonNull(targetInstance);
this.methodName = Objects.requireNonNull(methodName);
this.returnCanonicalTypeName = Objects.requireNonNull(returnCanonicalTypeName);
this.canonicalTypeNameParameters = Objects.requireNonNull(canonicalTypeNameParameters);
}
}

View File

@ -29,6 +29,7 @@ import java.security.AccessController;
import java.security.PrivilegedAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
/** Loads and creates a {@link Whitelist} from one to many text files. */
@ -392,7 +393,7 @@ public final class WhitelistLoader {
ClassLoader loader = AccessController.doPrivileged((PrivilegedAction<ClassLoader>)resource::getClassLoader);
return new Whitelist(loader, whitelistClasses, whitelistStatics, whitelistClassBindings);
return new Whitelist(loader, whitelistClasses, whitelistStatics, whitelistClassBindings, Collections.emptyList());
}
private WhitelistLoader() {}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.painless;
import org.elasticsearch.bootstrap.BootstrapInfo;
import org.elasticsearch.painless.Locals.LocalMethod;
import org.elasticsearch.painless.antlr.Walker;
import org.elasticsearch.painless.lookup.PainlessLookup;
import org.elasticsearch.painless.node.SSource;
@ -222,8 +221,8 @@ final class Compiler {
ScriptClassInfo scriptClassInfo = new ScriptClassInfo(painlessLookup, scriptClass);
SSource root = Walker.buildPainlessTree(scriptClassInfo, reserved, name, source, settings, painlessLookup,
null);
Map<String, LocalMethod> localMethods = root.analyze(painlessLookup);
root.write();
root.analyze(painlessLookup);
Map<String, Object> statics = root.write();
try {
Class<? extends PainlessScript> clazz = loader.defineScript(CLASS_NAME, root.getBytes());
@ -231,7 +230,10 @@ final class Compiler {
clazz.getField("$SOURCE").set(null, source);
clazz.getField("$STATEMENTS").set(null, root.getStatements());
clazz.getField("$DEFINITION").set(null, painlessLookup);
clazz.getField("$LOCALS").set(null, localMethods);
for (Map.Entry<String, Object> statik : statics.entrySet()) {
clazz.getField(statik.getKey()).set(null, statik.getValue());
}
return clazz.getConstructors()[0];
} catch (Exception exception) { // Catch everything to let the user know this is something caused internally.

View File

@ -31,7 +31,8 @@ import java.util.Map;
public class Globals {
private final Map<String,SFunction> syntheticMethods = new HashMap<>();
private final Map<String,Constant> constantInitializers = new HashMap<>();
private final Map<String,Class<?>> bindings = new HashMap<>();
private final Map<String,Class<?>> classBindings = new HashMap<>();
private final Map<Object,String> instanceBindings = new HashMap<>();
private final BitSet statements;
/** Create a new Globals from the set of statement boundaries */
@ -56,14 +57,19 @@ public class Globals {
}
}
/** Adds a new binding to be written as a local variable */
public String addBinding(Class<?> type) {
String name = "$binding$" + bindings.size();
bindings.put(name, type);
/** Adds a new class binding to be written as a local variable */
public String addClassBinding(Class<?> type) {
String name = "$class_binding$" + classBindings.size();
classBindings.put(name, type);
return name;
}
/** Adds a new binding to be written as a local variable */
public String addInstanceBinding(Object instance) {
return instanceBindings.computeIfAbsent(instance, key -> "$instance_binding$" + instanceBindings.size());
}
/** Returns the current synthetic methods */
public Map<String,SFunction> getSyntheticMethods() {
return syntheticMethods;
@ -75,8 +81,13 @@ public class Globals {
}
/** Returns the current bindings */
public Map<String,Class<?>> getBindings() {
return bindings;
public Map<String,Class<?>> getClassBindings() {
return classBindings;
}
/** Returns the current bindings */
public Map<Object,String> getInstanceBindings() {
return instanceBindings;
}
/** Returns the set of statement boundaries */

View File

@ -40,7 +40,7 @@ import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestHandler;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptEngine;
import org.elasticsearch.search.aggregations.pipeline.movfn.MovingFunctionScript;
import org.elasticsearch.search.aggregations.pipeline.MovingFunctionScript;
import java.util.ArrayList;
import java.util.Arrays;

View File

@ -19,7 +19,6 @@
package org.elasticsearch.painless;
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.SpecialPermission;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
@ -29,7 +28,6 @@ import org.elasticsearch.painless.spi.Whitelist;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptEngine;
import org.elasticsearch.script.ScriptException;
import org.elasticsearch.script.SearchScript;
import org.objectweb.asm.ClassWriter;
import org.objectweb.asm.Opcodes;
import org.objectweb.asm.Type;
@ -37,7 +35,6 @@ import org.objectweb.asm.commons.GeneratorAdapter;
import java.lang.invoke.MethodType;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.security.AccessControlContext;
import java.security.AccessController;
@ -101,13 +98,8 @@ public final class PainlessScriptEngine extends AbstractComponent implements Scr
for (Map.Entry<ScriptContext<?>, List<Whitelist>> entry : contexts.entrySet()) {
ScriptContext<?> context = entry.getKey();
if (context.instanceClazz.equals(SearchScript.class)) {
contextsToCompilers.put(context, new Compiler(GenericElasticsearchScript.class, null, null,
PainlessLookupBuilder.buildFromWhitelists(entry.getValue())));
} else {
contextsToCompilers.put(context, new Compiler(context.instanceClazz, context.factoryClazz, context.statefulFactoryClazz,
PainlessLookupBuilder.buildFromWhitelists(entry.getValue())));
}
contextsToCompilers.put(context, new Compiler(context.instanceClazz, context.factoryClazz, context.statefulFactoryClazz,
PainlessLookupBuilder.buildFromWhitelists(entry.getValue())));
}
this.contextsToCompilers = Collections.unmodifiableMap(contextsToCompilers);
@ -126,54 +118,24 @@ public final class PainlessScriptEngine extends AbstractComponent implements Scr
public <T> T compile(String scriptName, String scriptSource, ScriptContext<T> context, Map<String, String> params) {
Compiler compiler = contextsToCompilers.get(context);
if (context.instanceClazz.equals(SearchScript.class)) {
Constructor<?> constructor = compile(compiler, scriptName, scriptSource, params);
boolean needsScore;
// Check we ourselves are not being called by unprivileged code.
SpecialPermission.check();
try {
GenericElasticsearchScript newInstance = (GenericElasticsearchScript)constructor.newInstance();
needsScore = newInstance.needs_score();
} catch (InstantiationException | IllegalAccessException | InvocationTargetException e) {
throw new IllegalArgumentException("internal error");
// Create our loader (which loads compiled code with no permissions).
final Loader loader = AccessController.doPrivileged(new PrivilegedAction<Loader>() {
@Override
public Loader run() {
return compiler.createLoader(getClass().getClassLoader());
}
});
SearchScript.Factory factory = (p, lookup) -> new SearchScript.LeafFactory() {
@Override
public SearchScript newInstance(final LeafReaderContext context) {
try {
// a new instance is required for the class bindings model to work correctly
GenericElasticsearchScript newInstance = (GenericElasticsearchScript)constructor.newInstance();
return new ScriptImpl(newInstance, p, lookup, context);
} catch (InstantiationException | IllegalAccessException | InvocationTargetException e) {
throw new IllegalArgumentException("internal error");
}
}
@Override
public boolean needs_score() {
return needsScore;
}
};
return context.factoryClazz.cast(factory);
MainMethodReserved reserved = new MainMethodReserved();
compile(contextsToCompilers.get(context), loader, reserved, scriptName, scriptSource, params);
if (context.statefulFactoryClazz != null) {
return generateFactory(loader, context, reserved, generateStatefulFactory(loader, context, reserved));
} else {
// Check we ourselves are not being called by unprivileged code.
SpecialPermission.check();
// Create our loader (which loads compiled code with no permissions).
final Loader loader = AccessController.doPrivileged(new PrivilegedAction<Loader>() {
@Override
public Loader run() {
return compiler.createLoader(getClass().getClassLoader());
}
});
MainMethodReserved reserved = new MainMethodReserved();
compile(contextsToCompilers.get(context), loader, reserved, scriptName, scriptSource, params);
if (context.statefulFactoryClazz != null) {
return generateFactory(loader, context, reserved, generateStatefulFactory(loader, context, reserved));
} else {
return generateFactory(loader, context, reserved, WriterConstants.CLASS_TYPE);
}
return generateFactory(loader, context, reserved, WriterConstants.CLASS_TYPE);
}
}

View File

@ -1,112 +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.painless;
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.lookup.LeafSearchLookup;
import org.elasticsearch.search.lookup.SearchLookup;
import java.util.HashMap;
import java.util.Map;
import java.util.function.DoubleSupplier;
import java.util.function.Function;
/**
* ScriptImpl can be used as a {@link SearchScript}
* to run a previously compiled Painless script.
*/
final class ScriptImpl extends SearchScript {
/**
* The Painless script that can be run.
*/
private final GenericElasticsearchScript script;
/**
* A map that can be used to access input parameters at run-time.
*/
private final Map<String, Object> variables;
/**
* Looks up the {@code _score} from {@link #scorer} if {@code _score} is used, otherwise returns {@code 0.0}.
*/
private final DoubleSupplier scoreLookup;
/**
* Looks up the {@code ctx} from the {@link #variables} if {@code ctx} is used, otherwise return {@code null}.
*/
private final Function<Map<String, Object>, Map<?, ?>> ctxLookup;
/**
* Current _value for aggregation
* @see #setNextAggregationValue(Object)
*/
private Object aggregationValue;
/**
* Creates a ScriptImpl for the a previously compiled Painless script.
* @param script The previously compiled Painless script.
* @param vars The initial variables to run the script with.
* @param lookup The lookup to allow search fields to be available if this is run as a search script.
*/
ScriptImpl(GenericElasticsearchScript script, Map<String, Object> vars, SearchLookup lookup, LeafReaderContext leafContext) {
super(null, lookup, leafContext);
this.script = script;
this.variables = new HashMap<>();
if (vars != null) {
variables.putAll(vars);
}
LeafSearchLookup leafLookup = getLeafLookup();
if (leafLookup != null) {
variables.putAll(leafLookup.asMap());
}
scoreLookup = script.needs_score() ? this::getScore : () -> 0.0;
ctxLookup = script.needsCtx() ? variables -> (Map<?, ?>) variables.get("ctx") : variables -> null;
}
@Override
public Map<String, Object> getParams() {
return variables;
}
@Override
public void setNextVar(final String name, final Object value) {
variables.put(name, value);
}
@Override
public void setNextAggregationValue(Object value) {
this.aggregationValue = value;
}
@Override
public Object run() {
return script.execute(variables, scoreLookup.getAsDouble(), getDoc(), aggregationValue, ctxLookup.apply(variables));
}
@Override
public double runAsDouble() {
return ((Number)run()).doubleValue();
}
}

View File

@ -60,7 +60,6 @@ public class PainlessClassBinding {
@Override
public int hashCode() {
return Objects.hash(javaConstructor, javaMethod, returnType, typeParameters);
}
}

View File

@ -0,0 +1,64 @@
/*
* 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.painless.lookup;
import java.lang.reflect.Method;
import java.util.List;
import java.util.Objects;
public class PainlessInstanceBinding {
public final Object targetInstance;
public final Method javaMethod;
public final Class<?> returnType;
public final List<Class<?>> typeParameters;
PainlessInstanceBinding(Object targetInstance, Method javaMethod, Class<?> returnType, List<Class<?>> typeParameters) {
this.targetInstance = targetInstance;
this.javaMethod = javaMethod;
this.returnType = returnType;
this.typeParameters = typeParameters;
}
@Override
public boolean equals(Object object) {
if (this == object) {
return true;
}
if (object == null || getClass() != object.getClass()) {
return false;
}
PainlessInstanceBinding that = (PainlessInstanceBinding)object;
return targetInstance == that.targetInstance &&
Objects.equals(javaMethod, that.javaMethod) &&
Objects.equals(returnType, that.returnType) &&
Objects.equals(typeParameters, that.typeParameters);
}
@Override
public int hashCode() {
return Objects.hash(targetInstance, javaMethod, returnType, typeParameters);
}
}

View File

@ -40,13 +40,15 @@ public final class PainlessLookup {
private final Map<String, PainlessMethod> painlessMethodKeysToImportedPainlessMethods;
private final Map<String, PainlessClassBinding> painlessMethodKeysToPainlessClassBindings;
private final Map<String, PainlessInstanceBinding> painlessMethodKeysToPainlessInstanceBindings;
PainlessLookup(
Map<String, Class<?>> javaClassNamesToClasses,
Map<String, Class<?>> canonicalClassNamesToClasses,
Map<Class<?>, PainlessClass> classesToPainlessClasses,
Map<String, PainlessMethod> painlessMethodKeysToImportedPainlessMethods,
Map<String, PainlessClassBinding> painlessMethodKeysToPainlessClassBindings) {
Map<String, PainlessClassBinding> painlessMethodKeysToPainlessClassBindings,
Map<String, PainlessInstanceBinding> painlessMethodKeysToPainlessInstanceBindings) {
Objects.requireNonNull(javaClassNamesToClasses);
Objects.requireNonNull(canonicalClassNamesToClasses);
@ -54,6 +56,7 @@ public final class PainlessLookup {
Objects.requireNonNull(painlessMethodKeysToImportedPainlessMethods);
Objects.requireNonNull(painlessMethodKeysToPainlessClassBindings);
Objects.requireNonNull(painlessMethodKeysToPainlessInstanceBindings);
this.javaClassNamesToClasses = javaClassNamesToClasses;
this.canonicalClassNamesToClasses = Collections.unmodifiableMap(canonicalClassNamesToClasses);
@ -61,6 +64,7 @@ public final class PainlessLookup {
this.painlessMethodKeysToImportedPainlessMethods = Collections.unmodifiableMap(painlessMethodKeysToImportedPainlessMethods);
this.painlessMethodKeysToPainlessClassBindings = Collections.unmodifiableMap(painlessMethodKeysToPainlessClassBindings);
this.painlessMethodKeysToPainlessInstanceBindings = Collections.unmodifiableMap(painlessMethodKeysToPainlessInstanceBindings);
}
public Class<?> javaClassNameToClass(String javaClassName) {
@ -200,6 +204,14 @@ public final class PainlessLookup {
return painlessMethodKeysToPainlessClassBindings.get(painlessMethodKey);
}
public PainlessInstanceBinding lookupPainlessInstanceBinding(String methodName, int arity) {
Objects.requireNonNull(methodName);
String painlessMethodKey = buildPainlessMethodKey(methodName, arity);
return painlessMethodKeysToPainlessInstanceBindings.get(painlessMethodKey);
}
public PainlessMethod lookupFunctionalInterfacePainlessMethod(Class<?> targetClass) {
PainlessClass targetPainlessClass = classesToPainlessClasses.get(targetClass);

View File

@ -24,6 +24,7 @@ import org.elasticsearch.painless.spi.WhitelistClass;
import org.elasticsearch.painless.spi.WhitelistClassBinding;
import org.elasticsearch.painless.spi.WhitelistConstructor;
import org.elasticsearch.painless.spi.WhitelistField;
import org.elasticsearch.painless.spi.WhitelistInstanceBinding;
import org.elasticsearch.painless.spi.WhitelistMethod;
import java.lang.invoke.MethodHandle;
@ -50,10 +51,11 @@ import static org.elasticsearch.painless.lookup.PainlessLookupUtility.typesToCan
public final class PainlessLookupBuilder {
private static final Map<PainlessConstructor , PainlessConstructor> painlessConstructorCache = new HashMap<>();
private static final Map<PainlessMethod , PainlessMethod> painlessMethodCache = new HashMap<>();
private static final Map<PainlessField , PainlessField> painlessFieldCache = new HashMap<>();
private static final Map<PainlessClassBinding, PainlessClassBinding> painlessClassBindingCache = new HashMap<>();
private static final Map<PainlessConstructor , PainlessConstructor> painlessConstructorCache = new HashMap<>();
private static final Map<PainlessMethod , PainlessMethod> painlessMethodCache = new HashMap<>();
private static final Map<PainlessField , PainlessField> painlessFieldCache = new HashMap<>();
private static final Map<PainlessClassBinding , PainlessClassBinding> painlessClassBindingCache = new HashMap<>();
private static final Map<PainlessInstanceBinding, PainlessInstanceBinding> painlessInstanceBindingCache = new HashMap<>();
private static final Pattern CLASS_NAME_PATTERN = Pattern.compile("^[_a-zA-Z][._a-zA-Z0-9]*$");
private static final Pattern METHOD_NAME_PATTERN = Pattern.compile("^[_a-zA-Z][_a-zA-Z0-9]*$");
@ -108,9 +110,15 @@ public final class PainlessLookupBuilder {
for (WhitelistClassBinding whitelistClassBinding : whitelist.whitelistClassBindings) {
origin = whitelistClassBinding.origin;
painlessLookupBuilder.addPainlessClassBinding(
whitelist.classLoader, whitelistClassBinding.targetJavaClassName,
whitelistClassBinding.methodName, whitelistClassBinding.returnCanonicalTypeName,
whitelistClassBinding.canonicalTypeNameParameters);
whitelist.classLoader, whitelistClassBinding.targetJavaClassName, whitelistClassBinding.methodName,
whitelistClassBinding.returnCanonicalTypeName, whitelistClassBinding.canonicalTypeNameParameters);
}
for (WhitelistInstanceBinding whitelistInstanceBinding : whitelist.whitelistInstanceBindings) {
origin = whitelistInstanceBinding.origin;
painlessLookupBuilder.addPainlessInstanceBinding(
whitelistInstanceBinding.targetInstance, whitelistInstanceBinding.methodName,
whitelistInstanceBinding.returnCanonicalTypeName, whitelistInstanceBinding.canonicalTypeNameParameters);
}
}
} catch (Exception exception) {
@ -134,6 +142,7 @@ public final class PainlessLookupBuilder {
private final Map<String, PainlessMethod> painlessMethodKeysToImportedPainlessMethods;
private final Map<String, PainlessClassBinding> painlessMethodKeysToPainlessClassBindings;
private final Map<String, PainlessInstanceBinding> painlessMethodKeysToPainlessInstanceBindings;
public PainlessLookupBuilder() {
javaClassNamesToClasses = new HashMap<>();
@ -142,6 +151,7 @@ public final class PainlessLookupBuilder {
painlessMethodKeysToImportedPainlessMethods = new HashMap<>();
painlessMethodKeysToPainlessClassBindings = new HashMap<>();
painlessMethodKeysToPainlessInstanceBindings = new HashMap<>();
}
private Class<?> canonicalTypeNameToType(String canonicalTypeName) {
@ -763,6 +773,10 @@ public final class PainlessLookupBuilder {
throw new IllegalArgumentException("imported method and class binding cannot have the same name [" + methodName + "]");
}
if (painlessMethodKeysToPainlessInstanceBindings.containsKey(painlessMethodKey)) {
throw new IllegalArgumentException("imported method and instance binding cannot have the same name [" + methodName + "]");
}
MethodHandle methodHandle;
try {
@ -783,7 +797,7 @@ public final class PainlessLookupBuilder {
painlessMethodKeysToImportedPainlessMethods.put(painlessMethodKey, newImportedPainlessMethod);
} else if (newImportedPainlessMethod.equals(existingImportedPainlessMethod) == false) {
throw new IllegalArgumentException("cannot add imported methods with the same name and arity " +
"but are not equivalent for methods " +
"but do not have equivalent methods " +
"[[" + targetCanonicalClassName + "], [" + methodName + "], " +
"[" + typeToCanonicalTypeName(returnType) + "], " +
typesToCanonicalTypeNames(typeParameters) + "] and " +
@ -942,6 +956,11 @@ public final class PainlessLookupBuilder {
}
}
if (isValidType(returnType) == false) {
throw new IllegalArgumentException("return type [" + typeToCanonicalTypeName(returnType) + "] not found for class binding " +
"[[" + targetCanonicalClassName + "], [" + methodName + "], " + typesToCanonicalTypeNames(typeParameters) + "]");
}
if (javaMethod.getReturnType() != typeToJavaType(returnType)) {
throw new IllegalArgumentException("return type [" + typeToCanonicalTypeName(javaMethod.getReturnType()) + "] " +
"does not match the specified returned type [" + typeToCanonicalTypeName(returnType) + "] " +
@ -955,6 +974,15 @@ public final class PainlessLookupBuilder {
throw new IllegalArgumentException("class binding and imported method cannot have the same name [" + methodName + "]");
}
if (painlessMethodKeysToPainlessInstanceBindings.containsKey(painlessMethodKey)) {
throw new IllegalArgumentException("class binding and instance binding cannot have the same name [" + methodName + "]");
}
if (Modifier.isStatic(javaMethod.getModifiers())) {
throw new IllegalArgumentException("class binding [[" + targetClass.getCanonicalName() + "], [" + methodName + "], " +
typesToCanonicalTypeNames(typeParameters) + "] cannot be static");
}
PainlessClassBinding existingPainlessClassBinding = painlessMethodKeysToPainlessClassBindings.get(painlessMethodKey);
PainlessClassBinding newPainlessClassBinding =
new PainlessClassBinding(javaConstructor, javaMethod, returnType, typeParameters);
@ -962,9 +990,9 @@ public final class PainlessLookupBuilder {
if (existingPainlessClassBinding == null) {
newPainlessClassBinding = painlessClassBindingCache.computeIfAbsent(newPainlessClassBinding, key -> key);
painlessMethodKeysToPainlessClassBindings.put(painlessMethodKey, newPainlessClassBinding);
} else if (newPainlessClassBinding.equals(existingPainlessClassBinding)) {
} else if (newPainlessClassBinding.equals(existingPainlessClassBinding) == false) {
throw new IllegalArgumentException("cannot add class bindings with the same name and arity " +
"but are not equivalent for methods " +
"but do not have equivalent methods " +
"[[" + targetCanonicalClassName + "], " +
"[" + methodName + "], " +
"[" + typeToCanonicalTypeName(returnType) + "], " +
@ -976,6 +1004,136 @@ public final class PainlessLookupBuilder {
}
}
public void addPainlessInstanceBinding(Object targetInstance,
String methodName, String returnCanonicalTypeName, List<String> canonicalTypeNameParameters) {
Objects.requireNonNull(targetInstance);
Objects.requireNonNull(methodName);
Objects.requireNonNull(returnCanonicalTypeName);
Objects.requireNonNull(canonicalTypeNameParameters);
Class<?> targetClass = targetInstance.getClass();
String targetCanonicalClassName = typeToCanonicalTypeName(targetClass);
List<Class<?>> typeParameters = new ArrayList<>(canonicalTypeNameParameters.size());
for (String canonicalTypeNameParameter : canonicalTypeNameParameters) {
Class<?> typeParameter = canonicalTypeNameToType(canonicalTypeNameParameter);
if (typeParameter == null) {
throw new IllegalArgumentException("type parameter [" + canonicalTypeNameParameter + "] not found for instance binding " +
"[[" + targetCanonicalClassName + "], [" + methodName + "], " + canonicalTypeNameParameters + "]");
}
typeParameters.add(typeParameter);
}
Class<?> returnType = canonicalTypeNameToType(returnCanonicalTypeName);
if (returnType == null) {
throw new IllegalArgumentException("return type [" + returnCanonicalTypeName + "] not found for class binding " +
"[[" + targetCanonicalClassName + "], [" + methodName + "], " + canonicalTypeNameParameters + "]");
}
addPainlessInstanceBinding(targetInstance, methodName, returnType, typeParameters);
}
public void addPainlessInstanceBinding(Object targetInstance, String methodName, Class<?> returnType, List<Class<?>> typeParameters) {
Objects.requireNonNull(targetInstance);
Objects.requireNonNull(methodName);
Objects.requireNonNull(returnType);
Objects.requireNonNull(typeParameters);
Class<?> targetClass = targetInstance.getClass();
if (targetClass == def.class) {
throw new IllegalArgumentException("cannot add instance binding as reserved class [" + DEF_CLASS_NAME + "]");
}
String targetCanonicalClassName = typeToCanonicalTypeName(targetClass);
Class<?> existingTargetClass = javaClassNamesToClasses.get(targetClass.getName());
if (existingTargetClass == null) {
javaClassNamesToClasses.put(targetClass.getName(), targetClass);
} else if (existingTargetClass != targetClass) {
throw new IllegalArgumentException("class [" + targetCanonicalClassName + "] " +
"cannot represent multiple java classes with the same name from different class loaders");
}
if (METHOD_NAME_PATTERN.matcher(methodName).matches() == false) {
throw new IllegalArgumentException(
"invalid method name [" + methodName + "] for instance binding [" + targetCanonicalClassName + "].");
}
int typeParametersSize = typeParameters.size();
List<Class<?>> javaTypeParameters = new ArrayList<>(typeParametersSize);
for (Class<?> typeParameter : typeParameters) {
if (isValidType(typeParameter) == false) {
throw new IllegalArgumentException("type parameter [" + typeToCanonicalTypeName(typeParameter) + "] " +
"not found for instance binding [[" + targetCanonicalClassName + "], [" + methodName + "], " +
typesToCanonicalTypeNames(typeParameters) + "]");
}
javaTypeParameters.add(typeToJavaType(typeParameter));
}
if (isValidType(returnType) == false) {
throw new IllegalArgumentException("return type [" + typeToCanonicalTypeName(returnType) + "] not found for imported method " +
"[[" + targetCanonicalClassName + "], [" + methodName + "], " + typesToCanonicalTypeNames(typeParameters) + "]");
}
Method javaMethod;
try {
javaMethod = targetClass.getMethod(methodName, javaTypeParameters.toArray(new Class<?>[typeParametersSize]));
} catch (NoSuchMethodException nsme) {
throw new IllegalArgumentException("instance binding reflection object [[" + targetCanonicalClassName + "], " +
"[" + methodName + "], " + typesToCanonicalTypeNames(typeParameters) + "] not found", nsme);
}
if (javaMethod.getReturnType() != typeToJavaType(returnType)) {
throw new IllegalArgumentException("return type [" + typeToCanonicalTypeName(javaMethod.getReturnType()) + "] " +
"does not match the specified returned type [" + typeToCanonicalTypeName(returnType) + "] " +
"for instance binding [[" + targetClass.getCanonicalName() + "], [" + methodName + "], " +
typesToCanonicalTypeNames(typeParameters) + "]");
}
if (Modifier.isStatic(javaMethod.getModifiers())) {
throw new IllegalArgumentException("instance binding [[" + targetClass.getCanonicalName() + "], [" + methodName + "], " +
typesToCanonicalTypeNames(typeParameters) + "] cannot be static");
}
String painlessMethodKey = buildPainlessMethodKey(methodName, typeParametersSize);
if (painlessMethodKeysToImportedPainlessMethods.containsKey(painlessMethodKey)) {
throw new IllegalArgumentException("instance binding and imported method cannot have the same name [" + methodName + "]");
}
if (painlessMethodKeysToPainlessClassBindings.containsKey(painlessMethodKey)) {
throw new IllegalArgumentException("instance binding and class binding cannot have the same name [" + methodName + "]");
}
PainlessInstanceBinding existingPainlessInstanceBinding = painlessMethodKeysToPainlessInstanceBindings.get(painlessMethodKey);
PainlessInstanceBinding newPainlessInstanceBinding =
new PainlessInstanceBinding(targetInstance, javaMethod, returnType, typeParameters);
if (existingPainlessInstanceBinding == null) {
newPainlessInstanceBinding = painlessInstanceBindingCache.computeIfAbsent(newPainlessInstanceBinding, key -> key);
painlessMethodKeysToPainlessInstanceBindings.put(painlessMethodKey, newPainlessInstanceBinding);
} else if (newPainlessInstanceBinding.equals(existingPainlessInstanceBinding) == false) {
throw new IllegalArgumentException("cannot add instances bindings with the same name and arity " +
"but do not have equivalent methods " +
"[[" + targetCanonicalClassName + "], " +
"[" + methodName + "], " +
"[" + typeToCanonicalTypeName(returnType) + "], " +
typesToCanonicalTypeNames(typeParameters) + "] and " +
"[[" + targetCanonicalClassName + "], " +
"[" + methodName + "], " +
"[" + typeToCanonicalTypeName(existingPainlessInstanceBinding.returnType) + "], " +
typesToCanonicalTypeNames(existingPainlessInstanceBinding.typeParameters) + "]");
}
}
public PainlessLookup build() {
copyPainlessClassMembers();
cacheRuntimeHandles();
@ -1003,8 +1161,13 @@ public final class PainlessLookupBuilder {
"must have the same classes as the keys of classes to painless classes");
}
return new PainlessLookup(javaClassNamesToClasses, canonicalClassNamesToClasses, classesToPainlessClasses,
painlessMethodKeysToImportedPainlessMethods, painlessMethodKeysToPainlessClassBindings);
return new PainlessLookup(
javaClassNamesToClasses,
canonicalClassNamesToClasses,
classesToPainlessClasses,
painlessMethodKeysToImportedPainlessMethods,
painlessMethodKeysToPainlessClassBindings,
painlessMethodKeysToPainlessInstanceBindings);
}
private void copyPainlessClassMembers() {

View File

@ -25,6 +25,7 @@ import org.elasticsearch.painless.Locals.LocalMethod;
import org.elasticsearch.painless.Location;
import org.elasticsearch.painless.MethodWriter;
import org.elasticsearch.painless.lookup.PainlessClassBinding;
import org.elasticsearch.painless.lookup.PainlessInstanceBinding;
import org.elasticsearch.painless.lookup.PainlessMethod;
import org.objectweb.asm.Label;
import org.objectweb.asm.Type;
@ -48,6 +49,7 @@ public final class ECallLocal extends AExpression {
private LocalMethod localMethod = null;
private PainlessMethod importedMethod = null;
private PainlessClassBinding classBinding = null;
private PainlessInstanceBinding instanceBinding = null;
public ECallLocal(Location location, String name, List<AExpression> arguments) {
super(location);
@ -74,8 +76,12 @@ public final class ECallLocal extends AExpression {
classBinding = locals.getPainlessLookup().lookupPainlessClassBinding(name, arguments.size());
if (classBinding == null) {
throw createError(
new IllegalArgumentException("Unknown call [" + name + "] with [" + arguments.size() + "] arguments."));
instanceBinding = locals.getPainlessLookup().lookupPainlessInstanceBinding(name, arguments.size());
if (instanceBinding == null) {
throw createError(
new IllegalArgumentException("Unknown call [" + name + "] with [" + arguments.size() + "] arguments."));
}
}
}
}
@ -91,6 +97,9 @@ public final class ECallLocal extends AExpression {
} else if (classBinding != null) {
typeParameters = new ArrayList<>(classBinding.typeParameters);
actual = classBinding.returnType;
} else if (instanceBinding != null) {
typeParameters = new ArrayList<>(instanceBinding.typeParameters);
actual = instanceBinding.returnType;
} else {
throw new IllegalStateException("Illegal tree structure.");
}
@ -125,7 +134,7 @@ public final class ECallLocal extends AExpression {
writer.invokeStatic(Type.getType(importedMethod.targetClass),
new Method(importedMethod.javaMethod.getName(), importedMethod.methodType.toMethodDescriptorString()));
} else if (classBinding != null) {
String name = globals.addBinding(classBinding.javaConstructor.getDeclaringClass());
String name = globals.addClassBinding(classBinding.javaConstructor.getDeclaringClass());
Type type = Type.getType(classBinding.javaConstructor.getDeclaringClass());
int javaConstructorParameterCount = classBinding.javaConstructor.getParameterCount();
@ -154,6 +163,18 @@ public final class ECallLocal extends AExpression {
}
writer.invokeVirtual(type, Method.getMethod(classBinding.javaMethod));
} else if (instanceBinding != null) {
String name = globals.addInstanceBinding(instanceBinding.targetInstance);
Type type = Type.getType(instanceBinding.targetInstance.getClass());
writer.loadThis();
writer.getStatic(CLASS_TYPE, name, type);
for (int argument = 0; argument < instanceBinding.javaMethod.getParameterCount(); ++argument) {
arguments.get(argument).write(writer, globals);
}
writer.invokeVirtual(type, Method.getMethod(instanceBinding.javaMethod));
} else {
throw new IllegalStateException("Illegal tree structure.");
}

View File

@ -164,7 +164,7 @@ public final class SSource extends AStatement {
throw new IllegalStateException("Illegal tree structure.");
}
public Map<String, LocalMethod> analyze(PainlessLookup painlessLookup) {
public void analyze(PainlessLookup painlessLookup) {
Map<String, LocalMethod> methods = new HashMap<>();
for (SFunction function : functions) {
@ -180,8 +180,6 @@ public final class SSource extends AStatement {
Locals locals = Locals.newProgramScope(painlessLookup, methods.values());
analyze(locals);
return locals.getMethods();
}
@Override
@ -228,7 +226,7 @@ public final class SSource extends AStatement {
}
}
public void write() {
public Map<String, Object> write() {
// Create the ClassWriter.
int classFrames = ClassWriter.COMPUTE_FRAMES | ClassWriter.COMPUTE_MAXS;
@ -359,13 +357,20 @@ public final class SSource extends AStatement {
clinit.endMethod();
}
// Write binding variables
for (Map.Entry<String, Class<?>> binding : globals.getBindings().entrySet()) {
String name = binding.getKey();
String descriptor = Type.getType(binding.getValue()).getDescriptor();
// Write class binding variables
for (Map.Entry<String, Class<?>> classBinding : globals.getClassBindings().entrySet()) {
String name = classBinding.getKey();
String descriptor = Type.getType(classBinding.getValue()).getDescriptor();
visitor.visitField(Opcodes.ACC_PRIVATE, name, descriptor, null, null).visitEnd();
}
// Write instance binding variables
for (Map.Entry<Object, String> instanceBinding : globals.getInstanceBindings().entrySet()) {
String name = instanceBinding.getValue();
String descriptor = Type.getType(instanceBinding.getKey().getClass()).getDescriptor();
visitor.visitField(Opcodes.ACC_PUBLIC | Opcodes.ACC_STATIC, name, descriptor, null, null).visitEnd();
}
// Write any needsVarName methods for used variables
for (org.objectweb.asm.commons.Method needsMethod : scriptClassInfo.getNeedsMethods()) {
String name = needsMethod.getName();
@ -382,6 +387,15 @@ public final class SSource extends AStatement {
visitor.visitEnd();
bytes = writer.toByteArray();
Map<String, Object> statics = new HashMap<>();
statics.put("$LOCALS", mainMethod.getMethods());
for (Map.Entry<Object, String> instanceBinding : globals.getInstanceBindings().entrySet()) {
statics.put(instanceBinding.getValue(), instanceBinding.getKey());
}
return statics;
}
@Override

View File

@ -19,7 +19,7 @@
# This file contains a whitelist for the Moving Function pipeline aggregator in core
class org.elasticsearch.search.aggregations.pipeline.movfn.MovingFunctions {
class org.elasticsearch.search.aggregations.pipeline.MovingFunctions {
double max(double[])
double min(double[])
double sum(double[])

View File

@ -20,14 +20,32 @@
package org.elasticsearch.painless;
import org.elasticsearch.painless.spi.Whitelist;
import org.elasticsearch.painless.spi.WhitelistInstanceBinding;
import org.elasticsearch.script.ScriptContext;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
public class BindingsTests extends ScriptTestCase {
public static class InstanceBindingTestClass {
private int value;
public InstanceBindingTestClass(int value) {
this.value = value;
}
public void setInstanceBindingValue(int value) {
this.value = value;
}
public int getInstanceBindingValue() {
return value;
}
}
public abstract static class BindingsTestScript {
public static final String[] PARAMETERS = { "test", "bound" };
public abstract int execute(int test, int bound);
@ -40,15 +58,29 @@ public class BindingsTests extends ScriptTestCase {
@Override
protected Map<ScriptContext<?>, List<Whitelist>> scriptContexts() {
Map<ScriptContext<?>, List<Whitelist>> contexts = super.scriptContexts();
contexts.put(BindingsTestScript.CONTEXT, Whitelist.BASE_WHITELISTS);
List<Whitelist> whitelists = new ArrayList<>(Whitelist.BASE_WHITELISTS);
InstanceBindingTestClass instanceBindingTestClass = new InstanceBindingTestClass(1);
WhitelistInstanceBinding getter = new WhitelistInstanceBinding("test", instanceBindingTestClass,
"setInstanceBindingValue", "void", Collections.singletonList("int"));
WhitelistInstanceBinding setter = new WhitelistInstanceBinding("test", instanceBindingTestClass,
"getInstanceBindingValue", "int", Collections.emptyList());
List<WhitelistInstanceBinding> instanceBindingsList = new ArrayList<>();
instanceBindingsList.add(getter);
instanceBindingsList.add(setter);
Whitelist instanceBindingsWhitelist = new Whitelist(instanceBindingTestClass.getClass().getClassLoader(),
Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), instanceBindingsList);
whitelists.add(instanceBindingsWhitelist);
contexts.put(BindingsTestScript.CONTEXT, whitelists);
return contexts;
}
public void testBasicBinding() {
public void testBasicClassBinding() {
assertEquals(15, exec("testAddWithState(4, 5, 6, 0.0)"));
}
public void testRepeatedBinding() {
public void testRepeatedClassBinding() {
String script = "testAddWithState(4, 5, test, 0.0)";
BindingsTestScript.Factory factory = scriptEngine.compile(null, script, BindingsTestScript.CONTEXT, Collections.emptyMap());
BindingsTestScript executableScript = factory.newInstance();
@ -58,7 +90,7 @@ public class BindingsTests extends ScriptTestCase {
assertEquals(16, executableScript.execute(7, 0));
}
public void testBoundBinding() {
public void testBoundClassBinding() {
String script = "testAddWithState(4, bound, test, 0.0)";
BindingsTestScript.Factory factory = scriptEngine.compile(null, script, BindingsTestScript.CONTEXT, Collections.emptyMap());
BindingsTestScript executableScript = factory.newInstance();
@ -66,4 +98,21 @@ public class BindingsTests extends ScriptTestCase {
assertEquals(10, executableScript.execute(5, 1));
assertEquals(9, executableScript.execute(4, 2));
}
public void testInstanceBinding() {
String script = "getInstanceBindingValue() + test + bound";
BindingsTestScript.Factory factory = scriptEngine.compile(null, script, BindingsTestScript.CONTEXT, Collections.emptyMap());
BindingsTestScript executableScript = factory.newInstance();
assertEquals(3, executableScript.execute(1, 1));
script = "setInstanceBindingValue(test + bound); getInstanceBindingValue()";
factory = scriptEngine.compile(null, script, BindingsTestScript.CONTEXT, Collections.emptyMap());
executableScript = factory.newInstance();
assertEquals(4, executableScript.execute(-2, 6));
script = "getInstanceBindingValue() + test + bound";
factory = scriptEngine.compile(null, script, BindingsTestScript.CONTEXT, Collections.emptyMap());
executableScript = factory.newInstance();
assertEquals(8, executableScript.execute(-2, 6));
}
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.painless;
import org.elasticsearch.painless.PainlessExecuteAction.PainlessTestScript;
import org.elasticsearch.painless.lookup.PainlessLookupBuilder;
import org.elasticsearch.painless.spi.Whitelist;
import org.objectweb.asm.util.Textifier;
@ -31,7 +32,7 @@ final class Debugger {
/** compiles source to bytecode, and returns debugging output */
static String toString(final String source) {
return toString(GenericElasticsearchScript.class, source, new CompilerSettings());
return toString(PainlessTestScript.class, source, new CompilerSettings());
}
/** compiles to bytecode, and returns debugging output */

View File

@ -91,7 +91,7 @@ public abstract class ScriptTestCase extends ESTestCase {
public Object exec(String script, Map<String, Object> vars, Map<String,String> compileParams, boolean picky) {
// test for ambiguity errors before running the actual script if picky is true
if (picky) {
ScriptClassInfo scriptClassInfo = new ScriptClassInfo(PAINLESS_LOOKUP, GenericElasticsearchScript.class);
ScriptClassInfo scriptClassInfo = new ScriptClassInfo(PAINLESS_LOOKUP, PainlessTestScript.class);
CompilerSettings pickySettings = new CompilerSettings();
pickySettings.setPicky(true);
pickySettings.setRegexesEnabled(CompilerSettings.REGEX_ENABLED.get(scriptEngineSettings()));

View File

@ -19,17 +19,26 @@
package org.elasticsearch.painless;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.memory.MemoryIndex;
import org.apache.lucene.search.Scorable;
import org.elasticsearch.painless.spi.Whitelist;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptedMetricAggContexts;
import org.elasticsearch.search.lookup.LeafSearchLookup;
import org.elasticsearch.search.lookup.SearchLookup;
import org.elasticsearch.search.lookup.SourceLookup;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class ScriptedMetricAggContextsTests extends ScriptTestCase {
@Override
protected Map<ScriptContext<?>, List<Whitelist>> scriptContexts() {
@ -57,7 +66,7 @@ public class ScriptedMetricAggContextsTests extends ScriptTestCase {
assertEquals(10, state.get("testField"));
}
public void testMapBasic() {
public void testMapBasic() throws IOException {
ScriptedMetricAggContexts.MapScript.Factory factory = scriptEngine.compile("test",
"state.testField = 2*_score", ScriptedMetricAggContexts.MapScript.CONTEXT, Collections.emptyMap());
@ -82,6 +91,32 @@ public class ScriptedMetricAggContextsTests extends ScriptTestCase {
assertEquals(1.0, state.get("testField"));
}
public void testMapSourceAccess() throws IOException {
ScriptedMetricAggContexts.MapScript.Factory factory = scriptEngine.compile("test",
"state.testField = params._source.three", ScriptedMetricAggContexts.MapScript.CONTEXT, Collections.emptyMap());
Map<String, Object> params = new HashMap<>();
Map<String, Object> state = new HashMap<>();
MemoryIndex index = new MemoryIndex();
// we don't need a real index, just need to construct a LeafReaderContext which cannot be mocked
LeafReaderContext leafReaderContext = index.createSearcher().getIndexReader().leaves().get(0);
SearchLookup lookup = mock(SearchLookup.class);
LeafSearchLookup leafLookup = mock(LeafSearchLookup.class);
when(lookup.getLeafSearchLookup(leafReaderContext)).thenReturn(leafLookup);
SourceLookup sourceLookup = mock(SourceLookup.class);
when(leafLookup.asMap()).thenReturn(Collections.singletonMap("_source", sourceLookup));
when(sourceLookup.get("three")).thenReturn(3);
ScriptedMetricAggContexts.MapScript.LeafFactory leafFactory = factory.newFactory(params, state, lookup);
ScriptedMetricAggContexts.MapScript script = leafFactory.newInstance(leafReaderContext);
script.execute();
assert(state.containsKey("testField"));
assertEquals(3, state.get("testField"));
}
public void testCombineBasic() {
ScriptedMetricAggContexts.CombineScript.Factory factory = scriptEngine.compile("test",
"state.testField = params.initialVal; return state.testField + params.inc", ScriptedMetricAggContexts.CombineScript.CONTEXT,

View File

@ -21,10 +21,10 @@ package org.elasticsearch.painless.node;
import org.elasticsearch.painless.CompilerSettings;
import org.elasticsearch.painless.FeatureTest;
import org.elasticsearch.painless.GenericElasticsearchScript;
import org.elasticsearch.painless.Locals.Variable;
import org.elasticsearch.painless.Location;
import org.elasticsearch.painless.Operation;
import org.elasticsearch.painless.PainlessExecuteAction.PainlessTestScript;
import org.elasticsearch.painless.ScriptClassInfo;
import org.elasticsearch.painless.antlr.Walker;
import org.elasticsearch.painless.lookup.PainlessCast;
@ -897,7 +897,7 @@ public class NodeToStringTests extends ESTestCase {
}
private SSource walk(String code) {
ScriptClassInfo scriptClassInfo = new ScriptClassInfo(painlessLookup, GenericElasticsearchScript.class);
ScriptClassInfo scriptClassInfo = new ScriptClassInfo(painlessLookup, PainlessTestScript.class);
CompilerSettings compilerSettings = new CompilerSettings();
compilerSettings.setRegexesEnabled(true);
try {

View File

@ -1,6 +1,9 @@
# Integration tests for Painless Plugin
#
"Painless plugin loaded":
- skip:
reason: "contains is a newly added assertion"
features: contains
- do:
cluster.state: {}

View File

@ -23,8 +23,8 @@
indices.refresh: {}
- do:
index: test
search:
index: test
body:
query:
script:
@ -45,8 +45,8 @@
- match: { hits.hits.1.fields.sNum1.0: 3.0 }
- do:
index: test
search:
index: test
body:
query:
script:
@ -70,8 +70,8 @@
- match: { hits.hits.1.fields.sNum1.0: 3.0 }
- do:
index: test
search:
index: test
body:
query:
script:
@ -96,8 +96,8 @@
- match: { hits.hits.2.fields.sNum1.0: 3.0 }
- do:
index: test
search:
index: test
body:
query:
script:
@ -127,8 +127,8 @@
indices.refresh: {}
- do:
index: test
search:
index: test
body:
query:
function_score:
@ -149,8 +149,8 @@
- match: { hits.hits.1._id: "1" }
- do:
index: test
search:
index: test
body:
query:
function_score:
@ -171,8 +171,8 @@
- match: { hits.hits.1._id: "2" }
- do:
index: test
search:
index: test
body:
query:
function_score:
@ -193,8 +193,8 @@
- match: { hits.hits.1._id: "1" }
- do:
index: test
search:
index: test
body:
query:
function_score:
@ -215,8 +215,8 @@
- match: { hits.hits.1._id: "1" }
- do:
index: test
search:
index: test
body:
query:
function_score:
@ -237,8 +237,8 @@
- match: { hits.hits.1._id: "1" }
- do:
index: test
search:
index: test
body:
query:
function_score:
@ -274,8 +274,8 @@
indices.refresh: {}
- do:
index: test
search:
index: test
body:
query:
function_score:
@ -325,8 +325,8 @@
- do:
index: test
search:
index: test
body:
query:
function_score:
@ -364,8 +364,8 @@
- do:
index: test
search:
index: test
body:
script_fields:
foobar:
@ -391,8 +391,8 @@
- do:
index: test
search:
index: test
body:
aggs:
value_agg:
@ -428,8 +428,8 @@
- do:
catch: bad_request
index: test
search:
index: test
body:
aggs:
genre:

View File

@ -120,7 +120,7 @@ public class ReindexFailureTests extends ReindexTestCase {
assertThat(e.getMessage(),
either(containsString("all shards failed"))
.or(containsString("No search context found"))
.or(containsString("no such index"))
.or(containsString("no such index [source]"))
);
return;
}

View File

@ -19,7 +19,7 @@ teardown:
transient:
action.auto_create_index: false
- do:
catch: /no such index and \[action.auto_create_index\] is \[false\]/
catch: /no such index \[dest\] and \[action.auto_create_index\] is \[false\]/
reindex:
body:
source:
@ -41,7 +41,7 @@ teardown:
id: 1
body: { "text": "test" }
- do:
catch: /no such index and \[action.auto_create_index\] \(\[test\]\) doesn't match/
catch: /no such index \[dest\] and \[action.auto_create_index\] \(\[test\]\) doesn't match/
reindex:
body:
source:
@ -63,7 +63,7 @@ teardown:
id: 1
body: { "text": "test" }
- do:
catch: /no such index and \[action.auto_create_index\] contains \[-dest\] which forbids automatic creation of the index/
catch: /no such index \[dest\] and \[action.auto_create_index\] contains \[-dest\] which forbids automatic creation of the index/
reindex:
body:
source:

View File

@ -103,6 +103,9 @@ teardown:
---
"Module repository-url is loaded":
- skip:
reason: "contains is a newly added assertion"
features: contains
- do:
cluster.state: {}

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