From 0cce58f12cfe863ec00bbba4e4b222dc9265e0e3 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 19 Jun 2024 08:11:29 -0400 Subject: [PATCH] Use intended version for elasticsearch tests (#31639) * Remove elasticsearch-test-5,6 which never exercised on intended version * And both versions are EOL * Fix dependency * fix spotbug --- ...PreCommit_Java_ElasticSearch_IO_Direct.yml | 1 - .../elasticsearch-tests-5/build.gradle | 47 --- .../src/test/contrib/create_elk_container.sh | 24 -- .../io/elasticsearch/ElasticsearchIOIT.java | 213 ------------- .../io/elasticsearch/ElasticsearchIOTest.java | 301 ------------------ .../src/test/resources/clientkeystore | Bin 1942 -> 0 bytes .../elasticsearch-tests-6/build.gradle | 46 --- .../src/test/contrib/create_elk_container.sh | 24 -- .../io/elasticsearch/ElasticsearchIOIT.java | 213 ------------- .../io/elasticsearch/ElasticsearchIOTest.java | 294 ----------------- .../src/test/resources/clientkeystore | Bin 1942 -> 0 bytes .../elasticsearch-tests-7/build.gradle | 3 +- .../elasticsearch-tests-8/build.gradle | 9 +- .../elasticsearch-tests-common/build.gradle | 32 +- .../ElasticsearchIOITCommon.java | 1 + .../ElasticsearchIOTestCommon.java | 8 +- .../ElasticsearchIOTestUtils.java | 1 + .../sdk/io/elasticsearch/package-info.java | 19 ++ settings.gradle.kts | 3 - 19 files changed, 51 insertions(+), 1188 deletions(-) delete mode 100644 sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/build.gradle delete mode 100755 sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/contrib/create_elk_container.sh delete mode 100644 sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java delete mode 100644 sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java delete mode 100644 sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/resources/clientkeystore delete mode 100644 sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/build.gradle delete mode 100755 sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/contrib/create_elk_container.sh delete mode 100644 sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java delete mode 100644 sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java delete mode 100644 sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/resources/clientkeystore rename sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/{test => main}/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOITCommon.java (99%) rename sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/{test => main}/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java (99%) rename sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/{test => main}/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java (99%) create mode 100644 sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/package-info.java diff --git a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml index 3cf5de2f356f..78ab882d4774 100644 --- a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml @@ -98,7 +98,6 @@ jobs: uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: | - :sdks:java:io:elasticsearch-tests:elasticsearch-tests-5:build \ :sdks:java:io:elasticsearch-tests:elasticsearch-tests-7:build \ :sdks:java:io:elasticsearch-tests:elasticsearch-tests-8:build \ :sdks:java:io:elasticsearch-tests:elasticsearch-tests-common:build \ diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/build.gradle b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/build.gradle deleted file mode 100644 index affc9db08828..000000000000 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/build.gradle +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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. - */ - -plugins { id 'org.apache.beam.module' } -applyJavaNature( - publish: false, - archivesBaseName: 'beam-sdks-java-io-elasticsearch-tests-5' -) -provideIntegrationTestingDependencies() -enableJavaPerformanceTesting() - -description = "Apache Beam :: SDKs :: Java :: IO :: Elasticsearch-Tests :: 5.x" -ext.summary = "Tests of ElasticsearchIO on Elasticsearch 5.x" - -def elastic_search_version = "5.6.3" - -dependencies { - testImplementation project(path: ":sdks:java:io:elasticsearch-tests:elasticsearch-tests-common", configuration: "testRuntimeMigration") - testImplementation library.java.testcontainers_elasticsearch - - testImplementation project(path: ":sdks:java:core", configuration: "shadow") - testImplementation project(":sdks:java:io:elasticsearch") - testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") - testImplementation library.java.slf4j_api - testImplementation library.java.hamcrest - testImplementation library.java.junit - testImplementation "org.elasticsearch.client:elasticsearch-rest-client:$elastic_search_version" - testRuntimeOnly library.java.log4j2_api - testRuntimeOnly library.java.log4j2_core - testRuntimeOnly library.java.slf4j_jdk14 - testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") -} diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/contrib/create_elk_container.sh b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/contrib/create_elk_container.sh deleted file mode 100755 index 715cb6d42a22..000000000000 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/contrib/create_elk_container.sh +++ /dev/null @@ -1,24 +0,0 @@ -#!/bin/sh -################################################################################ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF 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. -# -################################################################################ - -#Create an ELK (Elasticsearch Logstash Kibana) container for ES v5.4.3 and compatible Logstash and Kibana versions, -#bind then on host ports, allow shell access to container and mount current directory on /home/$USER inside the container - -docker create -p 5601:5601 -p 9200:9200 -p 5044:5044 -p 5000:5000 -p 9300:9300 -it -v $(pwd):/home/$USER/ --name elk-5.4.3 sebp/elk:543 diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java deleted file mode 100644 index 9503853071fb..000000000000 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java +++ /dev/null @@ -1,213 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.elasticsearch; - -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO.ConnectionConfiguration; - -import org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOITCommon.ElasticsearchPipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.TestPipeline; -import org.elasticsearch.client.RestClient; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * A test of {@link ElasticsearchIO} on an independent Elasticsearch v5.x instance. - * - *

This test requires a running instance of Elasticsearch, and the test dataset must exist in the - * database. See {@link ElasticsearchIOITCommon} for instructions to achieve this. - * - *

You can run this test by doing the following from the beam parent module directory with the - * correct server IP: - * - *

- *  ./gradlew integrationTest -p sdks/java/io/elasticsearch-tests/elasticsearch-tests-5
- *  -DintegrationTestPipelineOptions='[
- *  "--elasticsearchServer=1.2.3.4",
- *  "--elasticsearchHttpPort=9200"]'
- *  --tests org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOIT
- *  -DintegrationTestRunner=direct
- * 
- * - *

It is likely that you will need to configure thread_pool.bulk.queue_size: 250 (or - * higher) in the backend Elasticsearch server for this test to run. - */ -@RunWith(JUnit4.class) -public class ElasticsearchIOIT { - private static RestClient restClient; - private static ElasticsearchPipelineOptions options; - private static ConnectionConfiguration readConnectionConfiguration; - private static ConnectionConfiguration writeConnectionConfiguration; - private static ConnectionConfiguration updateConnectionConfiguration; - private static ElasticsearchIOTestCommon elasticsearchIOTestCommon; - - @Rule public TestPipeline pipeline = TestPipeline.create(); - - @BeforeClass - public static void beforeClass() throws Exception { - PipelineOptionsFactory.register(ElasticsearchPipelineOptions.class); - options = TestPipeline.testingPipelineOptions().as(ElasticsearchPipelineOptions.class); - readConnectionConfiguration = - ElasticsearchIOITCommon.getConnectionConfiguration( - options, ElasticsearchIOITCommon.IndexMode.READ); - writeConnectionConfiguration = - ElasticsearchIOITCommon.getConnectionConfiguration( - options, ElasticsearchIOITCommon.IndexMode.WRITE); - updateConnectionConfiguration = - ElasticsearchIOITCommon.getConnectionConfiguration( - options, ElasticsearchIOITCommon.IndexMode.WRITE_PARTIAL); - restClient = readConnectionConfiguration.createClient(); - elasticsearchIOTestCommon = - new ElasticsearchIOTestCommon(readConnectionConfiguration, restClient, true); - } - - @AfterClass - public static void afterClass() throws Exception { - ElasticsearchIOTestUtils.deleteIndex(writeConnectionConfiguration, restClient); - ElasticsearchIOTestUtils.deleteIndex(updateConnectionConfiguration, restClient); - restClient.close(); - } - - @Test - public void testSplitsVolume() throws Exception { - elasticsearchIOTestCommon.testSplit(10_000); - } - - @Test - public void testReadVolume() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testRead(); - } - - @Test - public void testWriteVolume() throws Exception { - // cannot share elasticsearchIOTestCommon because tests run in parallel. - ElasticsearchIOTestCommon elasticsearchIOTestCommonWrite = - new ElasticsearchIOTestCommon(writeConnectionConfiguration, restClient, true); - elasticsearchIOTestCommonWrite.setPipeline(pipeline); - elasticsearchIOTestCommonWrite.testWrite(); - } - - @Test - public void testWriteVolumeStateful() throws Exception { - // cannot share elasticsearchIOTestCommon because tests run in parallel. - ElasticsearchIOTestCommon elasticsearchIOTestCommonWrite = - new ElasticsearchIOTestCommon(writeConnectionConfiguration, restClient, true); - elasticsearchIOTestCommonWrite.setPipeline(pipeline); - elasticsearchIOTestCommonWrite.testWriteStateful(); - } - - @Test - public void testSizesVolume() throws Exception { - elasticsearchIOTestCommon.testSizes(); - } - - /** - * This test verifies volume loading of Elasticsearch using explicit document IDs and routed to an - * index named the same as the scientist, and type which is based on the modulo 2 of the scientist - * name. The goal of this IT is to help observe and verify that the overhead of adding the - * functions to parse the document and extract the ID is acceptable. - */ - @Test - public void testWriteWithFullAddressingVolume() throws Exception { - // cannot share elasticsearchIOTestCommon because tests run in parallel. - ElasticsearchIOTestCommon elasticsearchIOTestCommonWrite = - new ElasticsearchIOTestCommon(writeConnectionConfiguration, restClient, true); - elasticsearchIOTestCommonWrite.setPipeline(pipeline); - elasticsearchIOTestCommonWrite.testWriteWithFullAddressing(); - } - - @Test - public void testWriteWithAllowableErrors() throws Exception { - elasticsearchIOTestCommon.testWriteWithAllowedErrors(); - } - - @Test - public void testWriteWithRouting() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithRouting(); - } - - @Test - public void testWriteScriptedUpsert() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteScriptedUpsert(); - } - - @Test - public void testWriteWithDocVersion() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithDocVersion(); - } - - /** - * This test verifies volume partial updates of Elasticsearch. The test dataset index is cloned - * and then a new field is added to each document using a partial update. The test then asserts - * the updates were applied. - */ - @Test - public void testWritePartialUpdate() throws Exception { - ElasticsearchIOTestUtils.copyIndex( - restClient, - readConnectionConfiguration.getIndex(), - updateConnectionConfiguration.getIndex()); - // cannot share elasticsearchIOTestCommon because tests run in parallel. - ElasticsearchIOTestCommon elasticsearchIOTestCommonUpdate = - new ElasticsearchIOTestCommon(updateConnectionConfiguration, restClient, true); - elasticsearchIOTestCommonUpdate.setPipeline(pipeline); - elasticsearchIOTestCommonUpdate.testWritePartialUpdate(); - } - - /** - * This test verifies volume deletes of Elasticsearch. The test dataset index is cloned and then - * around half of the documents are deleted and the other half is partially updated using bulk - * delete request. The test then asserts the documents were deleted successfully. - */ - @Test - public void testWriteWithIsDeletedFnWithPartialUpdates() throws Exception { - ElasticsearchIOTestUtils.copyIndex( - restClient, - readConnectionConfiguration.getIndex(), - updateConnectionConfiguration.getIndex()); - ElasticsearchIOTestCommon elasticsearchIOTestCommonDeleteFn = - new ElasticsearchIOTestCommon(updateConnectionConfiguration, restClient, true); - elasticsearchIOTestCommonDeleteFn.setPipeline(pipeline); - elasticsearchIOTestCommonDeleteFn.testWriteWithIsDeletedFnWithPartialUpdates(); - } - - /** - * This test verifies volume deletes of Elasticsearch. The test dataset index is cloned and then - * around half of the documents are deleted using bulk delete request. The test then asserts the - * documents were deleted successfully. - */ - @Test - public void testWriteWithIsDeletedFnWithoutPartialUpdate() throws Exception { - ElasticsearchIOTestUtils.copyIndex( - restClient, - readConnectionConfiguration.getIndex(), - updateConnectionConfiguration.getIndex()); - ElasticsearchIOTestCommon elasticsearchIOTestCommonDeleteFn = - new ElasticsearchIOTestCommon(updateConnectionConfiguration, restClient, true); - elasticsearchIOTestCommonDeleteFn.setPipeline(pipeline); - elasticsearchIOTestCommonDeleteFn.testWriteWithIsDeletedFnWithoutPartialUpdate(); - } -} diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java deleted file mode 100644 index a68d08c38044..000000000000 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java +++ /dev/null @@ -1,301 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.elasticsearch; - -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO.ConnectionConfiguration; -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.getEsIndex; -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.createConnectionConfig; -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.createIndex; -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.createTestContainer; -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.deleteIndex; -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.setDefaultTemplate; - -import java.io.IOException; -import java.io.Serializable; -import java.net.URL; -import java.nio.file.Path; -import java.nio.file.Paths; -import org.apache.beam.sdk.testing.TestPipeline; -import org.elasticsearch.client.RestClient; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.testcontainers.elasticsearch.ElasticsearchContainer; - -/** Tests for {@link ElasticsearchIO} version 5. */ -public class ElasticsearchIOTest implements Serializable { - - private ElasticsearchIOTestCommon elasticsearchIOTestCommon; - private ConnectionConfiguration connectionConfiguration; - private static ElasticsearchContainer container; - private static RestClient client; - static final String IMAGE_TAG = "5.6.3"; - - @BeforeClass - public static void beforeClass() throws IOException { - // Create the elasticsearch container. - container = createTestContainer(IMAGE_TAG); - - // Start the container. This step might take some time... - container.start(); - client = ElasticsearchIOTestUtils.clientFromContainer(container); - setDefaultTemplate(client); - } - - @AfterClass - public static void afterClass() throws IOException { - client.close(); - container.stop(); - } - - @Before - public void setup() throws IOException { - if (connectionConfiguration == null) { - connectionConfiguration = createConnectionConfig(client); - elasticsearchIOTestCommon = - new ElasticsearchIOTestCommon(connectionConfiguration, client, false); - - deleteIndex(client, getEsIndex()); - } - } - - @Rule public TestPipeline pipeline = TestPipeline.create(); - - @Test - public void testSizes() throws Exception { - // need to create the index using the helper method (not create it at first insertion) - // for the indexSettings() to be run - createIndex(elasticsearchIOTestCommon.restClient, getEsIndex()); - elasticsearchIOTestCommon.testSizes(); - } - - @Test - public void testSizesWithAlias() throws Exception { - // need to create the index using the helper method (not create it at first insertion) - // for the indexSettings() to be run - createIndex(elasticsearchIOTestCommon.restClient, getEsIndex(), true); - elasticsearchIOTestCommon.testSizes(); - } - - @Test - public void testRead() throws Exception { - // need to create the index using the helper method (not create it at first insertion) - // for the indexSettings() to be run - createIndex(elasticsearchIOTestCommon.restClient, getEsIndex()); - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testRead(); - } - - @Test - public void testReadWithQueryString() throws Exception { - // need to create the index using the helper method (not create it at first insertion) - // for the indexSettings() to be run - createIndex(elasticsearchIOTestCommon.restClient, getEsIndex()); - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testReadWithQueryString(); - } - - @Test - public void testReadWithQueryValueProvider() throws Exception { - // need to create the index using the helper method (not create it at first insertion) - // for the indexSettings() to be run - createIndex(elasticsearchIOTestCommon.restClient, getEsIndex()); - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testReadWithQueryValueProvider(); - } - - @Test - public void testWrite() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWrite(); - } - - @Rule public ExpectedException expectedException = ExpectedException.none(); - - @Test - public void testWriteWithErrors() throws Exception { - elasticsearchIOTestCommon.setExpectedException(expectedException); - elasticsearchIOTestCommon.testWriteWithErrors(); - } - - @Test - public void testWriteWithErrorsReturned() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithErrorsReturned(); - } - - @Test - public void testWriteWithErrorsReturnedAllowedErrors() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithErrorsReturnedAllowedErrors(); - } - - @Test - public void testWriteWithMaxBatchSize() throws Exception { - elasticsearchIOTestCommon.testWriteWithMaxBatchSize(); - } - - @Test - public void testWriteWithMaxBatchSizeBytes() throws Exception { - elasticsearchIOTestCommon.testWriteWithMaxBatchSizeBytes(); - } - - @Test - public void testSplit() throws Exception { - // need to create the index using the helper method (not create it at first insertion) - // for the indexSettings() to be run - createIndex(elasticsearchIOTestCommon.restClient, getEsIndex()); - elasticsearchIOTestCommon.testSplit(2_000); - } - - @Test - public void testWriteWithIdFn() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithIdFn(); - } - - @Test - public void testWriteWithIndexFn() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithIndexFn(); - } - - @Test - public void testWriteWithTypeFn() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithTypeFn2x5x(); - } - - @Test - public void testWriteFullAddressing() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithFullAddressing(); - } - - @Test - public void testWritePartialUpdate() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWritePartialUpdate(); - } - - @Test - public void testWriteAppendOnly() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteAppendOnly(); - } - - @Test(expected = Exception.class) - public void testWriteAppendOnlyDeleteNotAllowed() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteAppendOnlyDeleteNotAllowed(); - } - - @Test - public void testWriteWithAllowableErrors() throws Exception { - elasticsearchIOTestCommon.testWriteWithAllowedErrors(); - } - - @Test - public void testWriteWithRouting() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithRouting(); - } - - @Test - public void testWriteScriptedUpsert() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteScriptedUpsert(); - } - - @Test - public void testWriteWithDocVersion() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithDocVersion(); - } - - @Test - public void testMaxParallelRequestsPerWindow() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testMaxParallelRequestsPerWindow(); - } - - @Test - public void testReadWithMetadata() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testReadWithMetadata(); - } - - @Test - public void testDefaultRetryPredicate() throws IOException { - elasticsearchIOTestCommon.testDefaultRetryPredicate(client); - } - - @Test - public void testWriteRetry() throws Throwable { - elasticsearchIOTestCommon.setExpectedException(expectedException); - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteRetry(); - } - - @Test - public void testWriteRetryValidRequest() throws Throwable { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteRetryValidRequest(); - } - - @Test - public void testWriteWithIsDeleteFn() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithIsDeletedFnWithPartialUpdates(); - elasticsearchIOTestCommon.testWriteWithIsDeletedFnWithoutPartialUpdate(); - } - - @Test - public void testDocToBulkAndBulkIO() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testDocToBulkAndBulkIO(); - } - - @Test - public void testDocumentCoder() throws Exception { - elasticsearchIOTestCommon.testDocumentCoder(); - } - - @Test - public void testPDone() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testPipelineDone(); - } - - @Test - public void testValidSSLAndUsernameConfiguration() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("clientkeystore"); - Path filePath = Paths.get(fileUrl.toURI()); - elasticsearchIOTestCommon.testValidSSLAndUsernameConfiguration( - filePath.toAbsolutePath().toString()); - } - - @Test - public void testWriteWindowPreservation() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWindowPreservation(); - } -} diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/resources/clientkeystore b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/resources/clientkeystore deleted file mode 100644 index a99abd7bc6be49e119087c4d8cc7fbe196434908..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1942 zcmcK4c{tSj9tZH5-z;bd4I{g#V~bfWgP|Gw(nLeXzRrnbX|gnUtivF(92F^aGqC_-7vl4Ve1NgVfNDOcC+-up-Q|MPsG_aCq4`^WcrKF{~_K3G3k2Z2DqVZh%4 z_X`eWghm`PQ3dhk#o<*Mh7A;9j>3;En;prc384V&CZONI&=uWGcDV3}MExATZ2RT$YnOurwy^o{o^hlf9J}T$z z*U6xKSoD%EFIQ{U9-fNdFd-Bmt$G?2xfu~&VkPdncgd_bNzI?+sqWlOa8dQaQbFw_ z5Q69QrASxYM;n2zr`Sdp#AIC!H-D5c`l+y4e0R%2H10B1GFCl3sY3@7>zY)upNb9f z6~5oiIl~*-8x?#OqZ-+EO#U8fE=?^4mhgtke-c((+46W!qSJweTB5l%MVQRRChE(c zzz4UFD>c_+g#1}gE`2# z1N)FZ{>Ppa#Zc+1l$^D`h3CD%**{pnF<6`l0f~HwXKjO?IS}8L}@LoSX#vU2Zue#VLAH zpig06nQuc~Z8L22mP?#u)do}EzB|Z>xGuD#<*4RlAiDFJrmyE5d1Fb_aJ4oJ5 zwZPfVZ~qoX*Ve%hK_F-Yh7G-U80Rtw00sc4=LK&jMg%V8ooWSUK%fu+2Ex#>2#gRM z;sk{w0Vf((1cUfD1d&i*hEK@XI1ed zB;T6r<^-X#gTP=5`mXJaQc{tgP??zXgPZpUK`NvU$CMUz#bW>L-DI;=RzcAR=tDwP zop$8PFCGh|Ms@TP@oS}@i<6T0_^j5vR?M6BMQayyfY2{tl0?Z>Qc$J^tru^elW2&$ z-7~)xmU}9cYLa0ns9(;tU*cVpQLl`f^DH-=wf*rcG}m~17^T9akEQhrt|Xk0n~?8U z83@%EVC*U{Qhv9xgIysKz2#nC1mr;-+xb2XoHeGc&dLJLmmtxr? zM!BW~{^)cN*?WDjU#Lmx%H1MdmFkwI@y}_$R&e=dT}nxgcFk^yr**RwtBuF5u24nG z0FjaOtNhrn=M^3Gt_F&C>#)aM?j zTGXF8>Y>sq9|-J!%tDu|u6n*y3DQ;7)fCWnB>7HouZh~J^h?ZEzuJpM+5lvpa8~qc zX^sRF%N>IHt{VJ64qn!01?{q3;%tt~+e=QeKj0GG>Y)=Rq`d6*YgdN2HZV$Sdikmq zwWcCpf7-&dRjdN(i9;#!lIPOPoo=~Bg}EhAn&RlUZ`$vZtyOz`&X5sfe`fTV3m@a8 zzSaKmFJhd4J1#j5oqV)h9hUT%eCOcx_41V&t(sjEjWYm;uvsY;kM|yYh=Ynft2}{G z#K<3(f&5V@3Mvu3ex;4Q&2*n$6PSY5p7kUTx|aN3mq1KF7$ZOqL4h~U$)dGK*6RFN zm&mEJxiiY$8w$c^c{B(Y R%G~BD?CHxl?mIB7{{This test requires a running instance of Elasticsearch, and the test dataset must exist in the - * database. See {@link ElasticsearchIOITCommon} for instructions to achieve this. - * - *

You can run this test by doing the following from the beam parent module directory with the - * correct server IP: - * - *

- *  ./gradlew integrationTest -p sdks/java/io/elasticsearch-tests/elasticsearch-tests-6
- *  -DintegrationTestPipelineOptions='[
- *  "--elasticsearchServer=1.2.3.4",
- *  "--elasticsearchHttpPort=9200"]'
- *  --tests org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOIT
- *  -DintegrationTestRunner=direct
- * 
- * - *

It is likely that you will need to configure thread_pool.bulk.queue_size: 250 (or - * higher) in the backend Elasticsearch server for this test to run. - */ -@RunWith(JUnit4.class) -public class ElasticsearchIOIT { - private static RestClient restClient; - private static ElasticsearchPipelineOptions options; - private static ConnectionConfiguration readConnectionConfiguration; - private static ConnectionConfiguration writeConnectionConfiguration; - private static ConnectionConfiguration updateConnectionConfiguration; - private static ElasticsearchIOTestCommon elasticsearchIOTestCommon; - - @Rule public TestPipeline pipeline = TestPipeline.create(); - - @BeforeClass - public static void beforeClass() throws Exception { - PipelineOptionsFactory.register(ElasticsearchPipelineOptions.class); - options = TestPipeline.testingPipelineOptions().as(ElasticsearchPipelineOptions.class); - readConnectionConfiguration = - ElasticsearchIOITCommon.getConnectionConfiguration( - options, ElasticsearchIOITCommon.IndexMode.READ); - writeConnectionConfiguration = - ElasticsearchIOITCommon.getConnectionConfiguration( - options, ElasticsearchIOITCommon.IndexMode.WRITE); - updateConnectionConfiguration = - ElasticsearchIOITCommon.getConnectionConfiguration( - options, ElasticsearchIOITCommon.IndexMode.WRITE_PARTIAL); - restClient = readConnectionConfiguration.createClient(); - elasticsearchIOTestCommon = - new ElasticsearchIOTestCommon(readConnectionConfiguration, restClient, true); - } - - @AfterClass - public static void afterClass() throws Exception { - ElasticsearchIOTestUtils.deleteIndex(writeConnectionConfiguration, restClient); - ElasticsearchIOTestUtils.deleteIndex(updateConnectionConfiguration, restClient); - restClient.close(); - } - - @Test - public void testSplitsVolume() throws Exception { - elasticsearchIOTestCommon.testSplit(10_000); - } - - @Test - public void testReadVolume() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testRead(); - } - - @Test - public void testWriteVolume() throws Exception { - // cannot share elasticsearchIOTestCommon because tests run in parallel. - ElasticsearchIOTestCommon elasticsearchIOTestCommonWrite = - new ElasticsearchIOTestCommon(writeConnectionConfiguration, restClient, true); - elasticsearchIOTestCommonWrite.setPipeline(pipeline); - elasticsearchIOTestCommonWrite.testWrite(); - } - - @Test - public void testWriteVolumeStateful() throws Exception { - // cannot share elasticsearchIOTestCommon because tests run in parallel. - ElasticsearchIOTestCommon elasticsearchIOTestCommonWrite = - new ElasticsearchIOTestCommon(writeConnectionConfiguration, restClient, true); - elasticsearchIOTestCommonWrite.setPipeline(pipeline); - elasticsearchIOTestCommonWrite.testWriteStateful(); - } - - @Test - public void testSizesVolume() throws Exception { - elasticsearchIOTestCommon.testSizes(); - } - - /** - * This test verifies volume loading of Elasticsearch using explicit document IDs and routed to an - * index named the same as the scientist, and type which is based on the modulo 2 of the scientist - * name. The goal of this IT is to help observe and verify that the overhead of adding the - * functions to parse the document and extract the ID is acceptable. - */ - @Test - public void testWriteWithFullAddressingVolume() throws Exception { - // cannot share elasticsearchIOTestCommon because tests run in parallel. - ElasticsearchIOTestCommon elasticsearchIOTestCommonWrite = - new ElasticsearchIOTestCommon(writeConnectionConfiguration, restClient, true); - elasticsearchIOTestCommonWrite.setPipeline(pipeline); - elasticsearchIOTestCommonWrite.testWriteWithFullAddressing(); - } - - @Test - public void testWriteWithAllowableErrors() throws Exception { - elasticsearchIOTestCommon.testWriteWithAllowedErrors(); - } - - @Test - public void testWriteWithRouting() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithRouting(); - } - - @Test - public void testWriteScriptedUpsert() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteScriptedUpsert(); - } - - @Test - public void testWriteWithDocVersion() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithDocVersion(); - } - - /** - * This test verifies volume partial updates of Elasticsearch. The test dataset index is cloned - * and then a new field is added to each document using a partial update. The test then asserts - * the updates were applied. - */ - @Test - public void testWritePartialUpdate() throws Exception { - ElasticsearchIOTestUtils.copyIndex( - restClient, - readConnectionConfiguration.getIndex(), - updateConnectionConfiguration.getIndex()); - // cannot share elasticsearchIOTestCommon because tests run in parallel. - ElasticsearchIOTestCommon elasticsearchIOTestCommonUpdate = - new ElasticsearchIOTestCommon(updateConnectionConfiguration, restClient, true); - elasticsearchIOTestCommonUpdate.setPipeline(pipeline); - elasticsearchIOTestCommonUpdate.testWritePartialUpdate(); - } - - /** - * This test verifies volume deletes of Elasticsearch. The test dataset index is cloned and then - * around half of the documents are deleted and the other half is partially updated using bulk - * delete request. The test then asserts the documents were deleted successfully. - */ - @Test - public void testWriteWithIsDeletedFnWithPartialUpdates() throws Exception { - ElasticsearchIOTestUtils.copyIndex( - restClient, - readConnectionConfiguration.getIndex(), - updateConnectionConfiguration.getIndex()); - ElasticsearchIOTestCommon elasticsearchIOTestCommonDeleteFn = - new ElasticsearchIOTestCommon(updateConnectionConfiguration, restClient, true); - elasticsearchIOTestCommonDeleteFn.setPipeline(pipeline); - elasticsearchIOTestCommonDeleteFn.testWriteWithIsDeletedFnWithPartialUpdates(); - } - - /** - * This test verifies volume deletes of Elasticsearch. The test dataset index is cloned and then - * around half of the documents are deleted using bulk delete request. The test then asserts the - * documents were deleted successfully. - */ - @Test - public void testWriteWithIsDeletedFnWithoutPartialUpdate() throws Exception { - ElasticsearchIOTestUtils.copyIndex( - restClient, - readConnectionConfiguration.getIndex(), - updateConnectionConfiguration.getIndex()); - ElasticsearchIOTestCommon elasticsearchIOTestCommonDeleteFn = - new ElasticsearchIOTestCommon(updateConnectionConfiguration, restClient, true); - elasticsearchIOTestCommonDeleteFn.setPipeline(pipeline); - elasticsearchIOTestCommonDeleteFn.testWriteWithIsDeletedFnWithoutPartialUpdate(); - } -} diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java deleted file mode 100644 index be98bfe16e81..000000000000 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java +++ /dev/null @@ -1,294 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.elasticsearch; - -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO.ConnectionConfiguration; -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.getEsIndex; -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.createConnectionConfig; -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.createIndex; -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.createTestContainer; -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.deleteIndex; -import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.setDefaultTemplate; - -import java.io.IOException; -import java.io.Serializable; -import java.net.URL; -import java.nio.file.Path; -import java.nio.file.Paths; -import org.apache.beam.sdk.testing.TestPipeline; -import org.elasticsearch.client.RestClient; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.testcontainers.elasticsearch.ElasticsearchContainer; - -/** Tests for {@link ElasticsearchIO} version 6. */ -public class ElasticsearchIOTest implements Serializable { - - private ElasticsearchIOTestCommon elasticsearchIOTestCommon; - private ConnectionConfiguration connectionConfiguration; - private static ElasticsearchContainer container; - private static RestClient client; - static final String IMAGE_TAG = "6.4.0"; - - @BeforeClass - public static void beforeClass() throws IOException { - // Create the elasticsearch container. - container = createTestContainer(IMAGE_TAG); - - // Start the container. This step might take some time... - container.start(); - client = ElasticsearchIOTestUtils.clientFromContainer(container); - setDefaultTemplate(client); - } - - @AfterClass - public static void afterClass() throws IOException { - client.close(); - container.stop(); - } - - @Before - public void setup() throws IOException { - if (connectionConfiguration == null) { - connectionConfiguration = createConnectionConfig(client); - elasticsearchIOTestCommon = - new ElasticsearchIOTestCommon(connectionConfiguration, client, false); - deleteIndex(client, getEsIndex()); - } - } - - @Rule public TestPipeline pipeline = TestPipeline.create(); - - @Test - public void testSizes() throws Exception { - // need to create the index using the helper method (not create it at first insertion) - // for the indexSettings() to be run - createIndex(elasticsearchIOTestCommon.restClient, getEsIndex()); - elasticsearchIOTestCommon.testSizes(); - } - - @Test - public void testSizesWithAlias() throws Exception { - // need to create the index using the helper method (not create it at first insertion) - // for the indexSettings() to be run - createIndex(elasticsearchIOTestCommon.restClient, getEsIndex(), true); - elasticsearchIOTestCommon.testSizes(); - } - - @Test - public void testRead() throws Exception { - // need to create the index using the helper method (not create it at first insertion) - // for the indexSettings() to be run - createIndex(elasticsearchIOTestCommon.restClient, getEsIndex()); - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testRead(); - } - - @Test - public void testReadWithQueryString() throws Exception { - // need to create the index using the helper method (not create it at first insertion) - // for the indexSettings() to be run - createIndex(elasticsearchIOTestCommon.restClient, getEsIndex()); - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testReadWithQueryString(); - } - - @Test - public void testReadWithQueryValueProvider() throws Exception { - // need to create the index using the helper method (not create it at first insertion) - // for the indexSettings() to be run - createIndex(elasticsearchIOTestCommon.restClient, getEsIndex()); - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testReadWithQueryValueProvider(); - } - - @Test - public void testWrite() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWrite(); - } - - @Rule public ExpectedException expectedException = ExpectedException.none(); - - @Test - public void testWriteWithErrors() throws Exception { - elasticsearchIOTestCommon.setExpectedException(expectedException); - elasticsearchIOTestCommon.testWriteWithErrors(); - } - - @Test - public void testWriteWithErrorsReturned() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithErrorsReturned(); - } - - @Test - public void testWriteWithErrorsReturnedAllowedErrors() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithErrorsReturnedAllowedErrors(); - } - - @Test - public void testWriteWithMaxBatchSize() throws Exception { - elasticsearchIOTestCommon.testWriteWithMaxBatchSize(); - } - - @Test - public void testWriteWithMaxBatchSizeBytes() throws Exception { - elasticsearchIOTestCommon.testWriteWithMaxBatchSizeBytes(); - } - - @Test - public void testSplit() throws Exception { - // need to create the index using the helper method (not create it at first insertion) - // for the indexSettings() to be run - createIndex(elasticsearchIOTestCommon.restClient, getEsIndex()); - elasticsearchIOTestCommon.testSplit(2_000); - } - - @Test - public void testWriteWithIdFn() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithIdFn(); - } - - @Test - public void testWriteWithIndexFn() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithIndexFn(); - } - - @Test - public void testWriteFullAddressing() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithFullAddressing(); - } - - @Test - public void testWritePartialUpdate() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWritePartialUpdate(); - } - - @Test - public void testWriteAppendOnly() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteAppendOnly(); - } - - @Test(expected = Exception.class) - public void testWriteAppendOnlyDeleteNotAllowed() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteAppendOnlyDeleteNotAllowed(); - } - - @Test - public void testWriteWithAllowableErrors() throws Exception { - elasticsearchIOTestCommon.testWriteWithAllowedErrors(); - } - - @Test - public void testWriteWithRouting() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithRouting(); - } - - @Test - public void testWriteScriptedUpsert() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteScriptedUpsert(); - } - - @Test - public void testWriteWithDocVersion() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithDocVersion(); - } - - @Test - public void testMaxParallelRequestsPerWindow() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testMaxParallelRequestsPerWindow(); - } - - @Test - public void testReadWithMetadata() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testReadWithMetadata(); - } - - @Test - public void testDefaultRetryPredicate() throws IOException { - elasticsearchIOTestCommon.testDefaultRetryPredicate(client); - } - - @Test - public void testWriteRetry() throws Throwable { - elasticsearchIOTestCommon.setExpectedException(expectedException); - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteRetry(); - } - - @Test - public void testWriteRetryValidRequest() throws Throwable { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteRetryValidRequest(); - } - - @Test - public void testWriteWithIsDeleteFn() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWithIsDeletedFnWithPartialUpdates(); - elasticsearchIOTestCommon.testWriteWithIsDeletedFnWithoutPartialUpdate(); - } - - @Test - public void testDocToBulkAndBulkIO() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testDocToBulkAndBulkIO(); - } - - @Test - public void testDocumentCoder() throws Exception { - elasticsearchIOTestCommon.testDocumentCoder(); - } - - @Test - public void testPDone() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testPipelineDone(); - } - - @Test - public void testValidSSLAndUsernameConfiguration() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("clientkeystore"); - Path filePath = Paths.get(fileUrl.toURI()); - elasticsearchIOTestCommon.testValidSSLAndUsernameConfiguration( - filePath.toAbsolutePath().toString()); - } - - @Test - public void testWriteWindowPreservation() throws Exception { - elasticsearchIOTestCommon.setPipeline(pipeline); - elasticsearchIOTestCommon.testWriteWindowPreservation(); - } -} diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/resources/clientkeystore b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/resources/clientkeystore deleted file mode 100644 index a99abd7bc6be49e119087c4d8cc7fbe196434908..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1942 zcmcK4c{tSj9tZH5-z;bd4I{g#V~bfWgP|Gw(nLeXzRrnbX|gnUtivF(92F^aGqC_-7vl4Ve1NgVfNDOcC+-up-Q|MPsG_aCq4`^WcrKF{~_K3G3k2Z2DqVZh%4 z_X`eWghm`PQ3dhk#o<*Mh7A;9j>3;En;prc384V&CZONI&=uWGcDV3}MExATZ2RT$YnOurwy^o{o^hlf9J}T$z z*U6xKSoD%EFIQ{U9-fNdFd-Bmt$G?2xfu~&VkPdncgd_bNzI?+sqWlOa8dQaQbFw_ z5Q69QrASxYM;n2zr`Sdp#AIC!H-D5c`l+y4e0R%2H10B1GFCl3sY3@7>zY)upNb9f z6~5oiIl~*-8x?#OqZ-+EO#U8fE=?^4mhgtke-c((+46W!qSJweTB5l%MVQRRChE(c zzz4UFD>c_+g#1}gE`2# z1N)FZ{>Ppa#Zc+1l$^D`h3CD%**{pnF<6`l0f~HwXKjO?IS}8L}@LoSX#vU2Zue#VLAH zpig06nQuc~Z8L22mP?#u)do}EzB|Z>xGuD#<*4RlAiDFJrmyE5d1Fb_aJ4oJ5 zwZPfVZ~qoX*Ve%hK_F-Yh7G-U80Rtw00sc4=LK&jMg%V8ooWSUK%fu+2Ex#>2#gRM z;sk{w0Vf((1cUfD1d&i*hEK@XI1ed zB;T6r<^-X#gTP=5`mXJaQc{tgP??zXgPZpUK`NvU$CMUz#bW>L-DI;=RzcAR=tDwP zop$8PFCGh|Ms@TP@oS}@i<6T0_^j5vR?M6BMQayyfY2{tl0?Z>Qc$J^tru^elW2&$ z-7~)xmU}9cYLa0ns9(;tU*cVpQLl`f^DH-=wf*rcG}m~17^T9akEQhrt|Xk0n~?8U z83@%EVC*U{Qhv9xgIysKz2#nC1mr;-+xb2XoHeGc&dLJLmmtxr? zM!BW~{^)cN*?WDjU#Lmx%H1MdmFkwI@y}_$R&e=dT}nxgcFk^yr**RwtBuF5u24nG z0FjaOtNhrn=M^3Gt_F&C>#)aM?j zTGXF8>Y>sq9|-J!%tDu|u6n*y3DQ;7)fCWnB>7HouZh~J^h?ZEzuJpM+5lvpa8~qc zX^sRF%N>IHt{VJ64qn!01?{q3;%tt~+e=QeKj0GG>Y)=Rq`d6*YgdN2HZV$Sdikmq zwWcCpf7-&dRjdN(i9;#!lIPOPoo=~Bg}EhAn&RlUZ`$vZtyOz`&X5sfe`fTV3m@a8 zzSaKmFJhd4J1#j5oqV)h9hUT%eCOcx_41V&t(sjEjWYm;uvsY;kM|yYh=Ynft2}{G z#K<3(f&5V@3Mvu3ex;4Q&2*n$6PSY5p7kUTx|aN3mq1KF7$ZOqL4h~U$)dGK*6RFN zm&mEJxiiY$8w$c^c{B(Y R%G~BD?CHxl?mIB7{{This is independent from the tests so that for read tests it can be run separately after data * store creation rather than every time (which can be more fragile.) */ +@SuppressWarnings("nullness") public class ElasticsearchIOITCommon { /** Pipeline options for elasticsearch tests. */ diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java similarity index 99% rename from sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java rename to sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java index 23675b08ccbc..750b0954f4c6 100644 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java +++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java @@ -56,7 +56,6 @@ import java.io.IOException; import java.io.PipedInputStream; import java.io.PipedOutputStream; -import java.io.Serializable; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; @@ -116,7 +115,8 @@ import org.slf4j.LoggerFactory; /** Common test class for {@link ElasticsearchIO}. */ -class ElasticsearchIOTestCommon implements Serializable { +@SuppressWarnings("nullness") +class ElasticsearchIOTestCommon { private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchIOTestCommon.class); @@ -1236,7 +1236,7 @@ void testWriteWithIsDeletedFnWithPartialUpdates() throws Exception { List data = new ArrayList<>(); for (int i = 0; i < numDocs; i++) { // Scientist names at odd index to be deleted. - data.add(String.format("{\"id\" : %s, \"is_deleted\": %b}", i, i % 2 == 1)); + data.add(String.format("{\"id\" : %s, \"is_deleted\": %b}", i, i % 2 != 0)); } pipeline @@ -1277,7 +1277,7 @@ void testWriteWithIsDeletedFnWithoutPartialUpdate() throws Exception { List data = new ArrayList<>(); for (int i = 0; i < numDocs; i++) { // Scientist names at odd index to be deleted. - if (i % 2 == 1) { + if (i % 2 != 0) { data.add(String.format("{\"id\" : %s, \"is_deleted\": %b}", i, true)); } } diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java similarity index 99% rename from sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java rename to sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java index 6bc73049cc27..7e3cd58fd202 100644 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java +++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java @@ -52,6 +52,7 @@ import org.testcontainers.utility.DockerImageName; /** Test utilities to use with {@link ElasticsearchIO}. */ +@SuppressWarnings("nullness") class ElasticsearchIOTestUtils { static final int ELASTICSEARCH_DEFAULT_PORT = 9200; static final String ELASTICSEARCH_PASSWORD = "superSecure"; diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/package-info.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/package-info.java new file mode 100644 index 000000000000..b5170130719c --- /dev/null +++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ +/** Common test utilities for Elasticsearch. */ +package org.apache.beam.sdk.io.elasticsearch; diff --git a/settings.gradle.kts b/settings.gradle.kts index bd224c34dc79..d239499933c0 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -219,9 +219,6 @@ include(":sdks:java:io:contextualtextio") include(":sdks:java:io:debezium") include(":sdks:java:io:debezium:expansion-service") include(":sdks:java:io:elasticsearch") -include(":sdks:java:io:elasticsearch-tests:elasticsearch-tests-2") -include(":sdks:java:io:elasticsearch-tests:elasticsearch-tests-5") -include(":sdks:java:io:elasticsearch-tests:elasticsearch-tests-6") include(":sdks:java:io:elasticsearch-tests:elasticsearch-tests-7") include(":sdks:java:io:elasticsearch-tests:elasticsearch-tests-8") include(":sdks:java:io:elasticsearch-tests:elasticsearch-tests-common")