diff --git a/common/build.gradle b/common/build.gradle index 5cf219fbae..109cad59cb 100644 --- a/common/build.gradle +++ b/common/build.gradle @@ -39,8 +39,8 @@ dependencies { api group: 'org.apache.commons', name: 'commons-lang3', version: '3.12.0' api group: 'com.squareup.okhttp3', name: 'okhttp', version: '4.9.3' implementation 'com.github.babbel:okhttp-aws-signer:1.0.2' - api group: 'com.amazonaws', name: 'aws-java-sdk-core', version: '1.12.1' - api group: 'com.amazonaws', name: 'aws-java-sdk-sts', version: '1.12.1' + api group: 'com.amazonaws', name: 'aws-java-sdk-core', version: '1.12.545' + api group: 'com.amazonaws', name: 'aws-java-sdk-sts', version: '1.12.545' implementation "com.github.seancfoley:ipaddress:5.4.0" testImplementation group: 'junit', name: 'junit', version: '4.13.2' diff --git a/common/src/main/java/org/opensearch/sql/common/setting/Settings.java b/common/src/main/java/org/opensearch/sql/common/setting/Settings.java index be780e8d80..8daf0e9bf6 100644 --- a/common/src/main/java/org/opensearch/sql/common/setting/Settings.java +++ b/common/src/main/java/org/opensearch/sql/common/setting/Settings.java @@ -35,7 +35,7 @@ public enum Key { METRICS_ROLLING_WINDOW("plugins.query.metrics.rolling_window"), METRICS_ROLLING_INTERVAL("plugins.query.metrics.rolling_interval"), - + SPARK_EXECUTION_ENGINE_CONFIG("plugins.query.executionengine.spark.config"), CLUSTER_NAME("cluster.name"); @Getter private final String keyValue; diff --git a/core/src/main/java/org/opensearch/sql/datasource/DataSourceService.java b/core/src/main/java/org/opensearch/sql/datasource/DataSourceService.java index 3d6ddc864e..6dace50f99 100644 --- a/core/src/main/java/org/opensearch/sql/datasource/DataSourceService.java +++ b/core/src/main/java/org/opensearch/sql/datasource/DataSourceService.java @@ -39,6 +39,15 @@ public interface DataSourceService { */ DataSourceMetadata getDataSourceMetadata(String name); + /** + * Returns dataSourceMetadata object with specific name. The returned objects contain all the + * metadata information without any filtering. + * + * @param name name of the {@link DataSource}. + * @return set of {@link DataSourceMetadata}. + */ + DataSourceMetadata getRawDataSourceMetadata(String name); + /** * Register {@link DataSource} defined by {@link DataSourceMetadata}. * diff --git a/core/src/test/java/org/opensearch/sql/analysis/AnalyzerTestBase.java b/core/src/test/java/org/opensearch/sql/analysis/AnalyzerTestBase.java index f09bc5d380..a16d57673e 100644 --- a/core/src/test/java/org/opensearch/sql/analysis/AnalyzerTestBase.java +++ b/core/src/test/java/org/opensearch/sql/analysis/AnalyzerTestBase.java @@ -208,6 +208,11 @@ public DataSourceMetadata getDataSourceMetadata(String name) { return null; } + @Override + public DataSourceMetadata getRawDataSourceMetadata(String name) { + return null; + } + @Override public void createDataSource(DataSourceMetadata metadata) { throw new UnsupportedOperationException("unsupported operation"); diff --git a/datasources/src/main/java/org/opensearch/sql/datasources/service/DataSourceServiceImpl.java b/datasources/src/main/java/org/opensearch/sql/datasources/service/DataSourceServiceImpl.java index 2ac480bbf2..d6c1907f84 100644 --- a/datasources/src/main/java/org/opensearch/sql/datasources/service/DataSourceServiceImpl.java +++ b/datasources/src/main/java/org/opensearch/sql/datasources/service/DataSourceServiceImpl.java @@ -64,29 +64,17 @@ public Set getDataSourceMetadata(boolean isDefaultDataSource } @Override - public DataSourceMetadata getDataSourceMetadata(String datasourceName) { - Optional dataSourceMetadataOptional = - getDataSourceMetadataFromName(datasourceName); - if (dataSourceMetadataOptional.isEmpty()) { - throw new IllegalArgumentException( - "DataSource with name: " + datasourceName + " doesn't exist."); - } - removeAuthInfo(dataSourceMetadataOptional.get()); - return dataSourceMetadataOptional.get(); + public DataSourceMetadata getDataSourceMetadata(String dataSourceName) { + DataSourceMetadata dataSourceMetadata = getRawDataSourceMetadata(dataSourceName); + removeAuthInfo(dataSourceMetadata); + return dataSourceMetadata; } @Override public DataSource getDataSource(String dataSourceName) { - Optional dataSourceMetadataOptional = - getDataSourceMetadataFromName(dataSourceName); - if (dataSourceMetadataOptional.isEmpty()) { - throw new DataSourceNotFoundException( - String.format("DataSource with name %s doesn't exist.", dataSourceName)); - } else { - DataSourceMetadata dataSourceMetadata = dataSourceMetadataOptional.get(); - this.dataSourceUserAuthorizationHelper.authorizeDataSource(dataSourceMetadata); - return dataSourceLoaderCache.getOrLoadDataSource(dataSourceMetadata); - } + DataSourceMetadata dataSourceMetadata = getRawDataSourceMetadata(dataSourceName); + this.dataSourceUserAuthorizationHelper.authorizeDataSource(dataSourceMetadata); + return dataSourceLoaderCache.getOrLoadDataSource(dataSourceMetadata); } @Override @@ -146,11 +134,20 @@ private void validateDataSourceMetaData(DataSourceMetadata metadata) { + " Properties are required parameters."); } - private Optional getDataSourceMetadataFromName(String dataSourceName) { + @Override + public DataSourceMetadata getRawDataSourceMetadata(String dataSourceName) { if (dataSourceName.equals(DEFAULT_DATASOURCE_NAME)) { - return Optional.of(DataSourceMetadata.defaultOpenSearchDataSourceMetadata()); + return DataSourceMetadata.defaultOpenSearchDataSourceMetadata(); + } else { - return this.dataSourceMetadataStorage.getDataSourceMetadata(dataSourceName); + Optional dataSourceMetadataOptional = + this.dataSourceMetadataStorage.getDataSourceMetadata(dataSourceName); + if (dataSourceMetadataOptional.isEmpty()) { + throw new DataSourceNotFoundException( + String.format("DataSource with name %s doesn't exist.", dataSourceName)); + } else { + return dataSourceMetadataOptional.get(); + } } } diff --git a/datasources/src/test/java/org/opensearch/sql/datasources/service/DataSourceServiceImplTest.java b/datasources/src/test/java/org/opensearch/sql/datasources/service/DataSourceServiceImplTest.java index 56d3586c6e..eb28495541 100644 --- a/datasources/src/test/java/org/opensearch/sql/datasources/service/DataSourceServiceImplTest.java +++ b/datasources/src/test/java/org/opensearch/sql/datasources/service/DataSourceServiceImplTest.java @@ -359,11 +359,11 @@ void testRemovalOfAuthorizationInfo() { @Test void testGetDataSourceMetadataForNonExistingDataSource() { when(dataSourceMetadataStorage.getDataSourceMetadata("testDS")).thenReturn(Optional.empty()); - IllegalArgumentException exception = + DataSourceNotFoundException exception = assertThrows( - IllegalArgumentException.class, + DataSourceNotFoundException.class, () -> dataSourceService.getDataSourceMetadata("testDS")); - assertEquals("DataSource with name: testDS doesn't exist.", exception.getMessage()); + assertEquals("DataSource with name testDS doesn't exist.", exception.getMessage()); } @Test @@ -385,4 +385,28 @@ void testGetDataSourceMetadataForSpecificDataSourceName() { assertFalse(dataSourceMetadata.getProperties().containsKey("prometheus.auth.password")); verify(dataSourceMetadataStorage, times(1)).getDataSourceMetadata("testDS"); } + + @Test + void testGetRawDataSourceMetadata() { + HashMap properties = new HashMap<>(); + properties.put("prometheus.uri", "https://localhost:9090"); + properties.put("prometheus.auth.type", "basicauth"); + properties.put("prometheus.auth.username", "username"); + properties.put("prometheus.auth.password", "password"); + DataSourceMetadata dataSourceMetadata = + new DataSourceMetadata( + "testDS", + DataSourceType.PROMETHEUS, + Collections.singletonList("prometheus_access"), + properties); + when(dataSourceMetadataStorage.getDataSourceMetadata("testDS")) + .thenReturn(Optional.of(dataSourceMetadata)); + + DataSourceMetadata dataSourceMetadata1 = dataSourceService.getRawDataSourceMetadata("testDS"); + assertEquals("testDS", dataSourceMetadata1.getName()); + assertEquals(DataSourceType.PROMETHEUS, dataSourceMetadata1.getConnector()); + assertTrue(dataSourceMetadata1.getProperties().containsKey("prometheus.auth.type")); + assertTrue(dataSourceMetadata1.getProperties().containsKey("prometheus.auth.username")); + assertTrue(dataSourceMetadata1.getProperties().containsKey("prometheus.auth.password")); + } } diff --git a/docs/user/interfaces/asyncqueryinterface.rst b/docs/user/interfaces/asyncqueryinterface.rst new file mode 100644 index 0000000000..98990b795b --- /dev/null +++ b/docs/user/interfaces/asyncqueryinterface.rst @@ -0,0 +1,108 @@ +.. highlight:: sh + +======================= +Async Query Interface Endpoints +======================= + +.. rubric:: Table of contents + +.. contents:: + :local: + :depth: 1 + + +Introduction +============ + +For supporting `S3Glue <../ppl/admin/connector/s3glue_connector.rst>`_ and Cloudwatch datasources connectors, we have introduced a new execution engine on top of Spark. +All the queries to be executed on spark execution engine can only be submitted via Async Query APIs. Below sections will list all the new APIs introduced. + + +Configuration required for Async Query APIs +====================================== +Currently, we only support AWS emr serverless as SPARK execution engine. The details of execution engine should be configured under +``plugins.query.executionengine.spark.config`` cluster setting. The value should be a stringified json comprising of ``applicationId``, ``executionRoleARN``,``region``. +Sample Setting Value :: + + plugins.query.executionengine.spark.config: '{"applicationId":"xxxxx", "executionRoleARN":"arn:aws:iam::***********:role/emr-job-execution-role","region":"eu-west-1"}' + + +If this setting is not configured during bootstrap, Async Query APIs will be disabled and it requires a cluster restart to enable them back again. +We make use of default aws credentials chain to make calls to the emr serverless application and also make sure the default credentials +have pass role permissions for emr-job-execution-role mentioned in the engine configuration. + + + +Async Query Creation API +====================================== +If security plugin is enabled, this API can only be invoked by users with permission ``cluster:admin/opensearch/ql/async_query/create``. + +HTTP URI: _plugins/_query/_async_query +HTTP VERB: POST + + + +Sample Request:: + + curl --location 'http://localhost:9200/_plugins/_async_query' \ + --header 'Content-Type: application/json' \ + --data '{ + "kind" : "sql", + "query" : "select * from my_glue.default.http_logs limit 10" + }' + +Sample Response:: + + { + "queryId": "00fd796ut1a7eg0q" + } + +Async Query Result API +====================================== +If security plugin is enabled, this API can only be invoked by users with permission ``cluster:admin/opensearch/ql/async_query/result``. +Async Query Creation and Result Query permissions are orthogonal, so any user with result api permissions and queryId can query the corresponding query results irrespective of the user who created the async query. + + +HTTP URI: _plugins/_query/_async_query/{queryId} +HTTP VERB: GET + + +Sample Request BODY:: + + curl --location --request GET 'http://localhost:9200/_plugins/_async_query/00fd796ut1a7eg0q' \ + --header 'Content-Type: application/json' \ + --data '{ + "query" : "select * from default.http_logs limit 1" + }' + +Sample Response if the Query is in Progress :: + + {"status":"RUNNING"} + +Sample Response If the Query is successful :: + + { + "schema": [ + { + "name": "indexed_col_name", + "type": "string" + }, + { + "name": "data_type", + "type": "string" + }, + { + "name": "skip_type", + "type": "string" + } + ], + "datarows": [ + [ + "status", + "int", + "VALUE_SET" + ] + ], + "total": 1, + "size": 1 + } diff --git a/integ-test/build.gradle b/integ-test/build.gradle index 0404900450..dc92f9ebb3 100644 --- a/integ-test/build.gradle +++ b/integ-test/build.gradle @@ -162,6 +162,7 @@ configurations.all { resolutionStrategy.force "org.jetbrains.kotlin:kotlin-stdlib-jdk7:1.5.31" resolutionStrategy.force "joda-time:joda-time:2.10.12" resolutionStrategy.force "org.slf4j:slf4j-api:1.7.36" + resolutionStrategy.force "com.amazonaws:aws-java-sdk-core:1.12.545" } configurations { diff --git a/integ-test/src/test/java/org/opensearch/sql/datasource/DataSourceAPIsIT.java b/integ-test/src/test/java/org/opensearch/sql/datasource/DataSourceAPIsIT.java index e1d071d522..2e7c064062 100644 --- a/integ-test/src/test/java/org/opensearch/sql/datasource/DataSourceAPIsIT.java +++ b/integ-test/src/test/java/org/opensearch/sql/datasource/DataSourceAPIsIT.java @@ -174,12 +174,12 @@ public void deleteDataSourceTest() { Assert.assertThrows( ResponseException.class, () -> client().performRequest(prometheusGetRequest)); Assert.assertEquals( - 400, prometheusGetResponseException.getResponse().getStatusLine().getStatusCode()); + 404, prometheusGetResponseException.getResponse().getStatusLine().getStatusCode()); String prometheusGetResponseString = getResponseBody(prometheusGetResponseException.getResponse()); JsonObject errorMessage = new Gson().fromJson(prometheusGetResponseString, JsonObject.class); Assert.assertEquals( - "DataSource with name: delete_prometheus doesn't exist.", + "DataSource with name delete_prometheus doesn't exist.", errorMessage.get("error").getAsJsonObject().get("details").getAsString()); } diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/setting/OpenSearchSettings.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/setting/OpenSearchSettings.java index 48ceacaf10..76bda07607 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/setting/OpenSearchSettings.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/setting/OpenSearchSettings.java @@ -129,6 +129,12 @@ public class OpenSearchSettings extends Settings { Setting.Property.NodeScope, Setting.Property.Dynamic); + public static final Setting SPARK_EXECUTION_ENGINE_CONFIG = + Setting.simpleString( + Key.SPARK_EXECUTION_ENGINE_CONFIG.getKeyValue(), + Setting.Property.NodeScope, + Setting.Property.Dynamic); + /** Construct OpenSearchSetting. The OpenSearchSetting must be singleton. */ @SuppressWarnings("unchecked") public OpenSearchSettings(ClusterSettings clusterSettings) { @@ -193,6 +199,12 @@ public OpenSearchSettings(ClusterSettings clusterSettings) { Key.DATASOURCES_URI_HOSTS_DENY_LIST, DATASOURCE_URI_HOSTS_DENY_LIST, new Updater(Key.DATASOURCES_URI_HOSTS_DENY_LIST)); + register( + settingBuilder, + clusterSettings, + Key.SPARK_EXECUTION_ENGINE_CONFIG, + SPARK_EXECUTION_ENGINE_CONFIG, + new Updater(Key.SPARK_EXECUTION_ENGINE_CONFIG)); registerNonDynamicSettings( settingBuilder, clusterSettings, Key.CLUSTER_NAME, ClusterName.CLUSTER_NAME_SETTING); defaultSettings = settingBuilder.build(); @@ -257,6 +269,7 @@ public static List> pluginSettings() { .add(METRICS_ROLLING_WINDOW_SETTING) .add(METRICS_ROLLING_INTERVAL_SETTING) .add(DATASOURCE_URI_HOSTS_DENY_LIST) + .add(SPARK_EXECUTION_ENGINE_CONFIG) .build(); } diff --git a/plugin/build.gradle b/plugin/build.gradle index 9e2011059d..53d2e21f10 100644 --- a/plugin/build.gradle +++ b/plugin/build.gradle @@ -152,8 +152,8 @@ dependencies { testImplementation group: 'net.bytebuddy', name: 'byte-buddy-agent', version: '1.12.13' testImplementation group: 'org.hamcrest', name: 'hamcrest-library', version: '2.1' - testImplementation group: 'org.mockito', name: 'mockito-core', version: '5.4.0' - testImplementation group: 'org.mockito', name: 'mockito-junit-jupiter', version: '5.4.0' + testImplementation group: 'org.mockito', name: 'mockito-core', version: '5.5.0' + testImplementation group: 'org.mockito', name: 'mockito-junit-jupiter', version: '5.5.0' testImplementation 'org.junit.jupiter:junit-jupiter:5.6.2' } diff --git a/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java b/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java index 80e1a6b1a3..ed10b1e3e6 100644 --- a/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java +++ b/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java @@ -5,10 +5,16 @@ package org.opensearch.sql.plugin; +import static org.opensearch.sql.common.setting.Settings.Key.SPARK_EXECUTION_ENGINE_CONFIG; import static org.opensearch.sql.datasource.model.DataSourceMetadata.defaultOpenSearchDataSourceMetadata; +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; +import com.amazonaws.services.emrserverless.AWSEMRServerless; +import com.amazonaws.services.emrserverless.AWSEMRServerlessClientBuilder; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import java.security.AccessController; +import java.security.PrivilegedAction; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -83,16 +89,23 @@ import org.opensearch.sql.plugin.transport.TransportPPLQueryAction; import org.opensearch.sql.plugin.transport.TransportPPLQueryResponse; import org.opensearch.sql.prometheus.storage.PrometheusStorageFactory; -import org.opensearch.sql.spark.rest.RestJobManagementAction; +import org.opensearch.sql.spark.asyncquery.AsyncQueryExecutorService; +import org.opensearch.sql.spark.asyncquery.AsyncQueryExecutorServiceImpl; +import org.opensearch.sql.spark.asyncquery.AsyncQueryJobMetadataStorageService; +import org.opensearch.sql.spark.asyncquery.OpensearchAsyncQueryJobMetadataStorageService; +import org.opensearch.sql.spark.client.EmrServerlessClientImpl; +import org.opensearch.sql.spark.client.SparkJobClient; +import org.opensearch.sql.spark.config.SparkExecutionEngineConfig; +import org.opensearch.sql.spark.dispatcher.SparkQueryDispatcher; +import org.opensearch.sql.spark.response.JobExecutionResponseReader; +import org.opensearch.sql.spark.rest.RestAsyncQueryManagementAction; import org.opensearch.sql.spark.storage.SparkStorageFactory; -import org.opensearch.sql.spark.transport.TransportCreateJobRequestAction; -import org.opensearch.sql.spark.transport.TransportDeleteJobRequestAction; -import org.opensearch.sql.spark.transport.TransportGetJobRequestAction; -import org.opensearch.sql.spark.transport.TransportGetQueryResultRequestAction; -import org.opensearch.sql.spark.transport.model.CreateJobActionResponse; -import org.opensearch.sql.spark.transport.model.DeleteJobActionResponse; -import org.opensearch.sql.spark.transport.model.GetJobActionResponse; -import org.opensearch.sql.spark.transport.model.GetJobQueryResultActionResponse; +import org.opensearch.sql.spark.transport.TransportCancelAsyncQueryRequestAction; +import org.opensearch.sql.spark.transport.TransportCreateAsyncQueryRequestAction; +import org.opensearch.sql.spark.transport.TransportGetAsyncQueryResultAction; +import org.opensearch.sql.spark.transport.model.CancelAsyncQueryActionResponse; +import org.opensearch.sql.spark.transport.model.CreateAsyncQueryActionResponse; +import org.opensearch.sql.spark.transport.model.GetAsyncQueryResultActionResponse; import org.opensearch.sql.storage.DataSourceFactory; import org.opensearch.threadpool.ExecutorBuilder; import org.opensearch.threadpool.FixedExecutorBuilder; @@ -110,6 +123,7 @@ public class SQLPlugin extends Plugin implements ActionPlugin, ScriptPlugin { private NodeClient client; private DataSourceServiceImpl dataSourceService; + private AsyncQueryExecutorService asyncQueryExecutorService; private Injector injector; public String name() { @@ -142,7 +156,7 @@ public List getRestHandlers( new RestPPLStatsAction(settings, restController), new RestQuerySettingsAction(settings, restController), new RestDataSourceQueryAction(), - new RestJobManagementAction()); + new RestAsyncQueryManagementAction()); } /** Register action and handler so that transportClient can find proxy for action. */ @@ -168,18 +182,17 @@ public List getRestHandlers( TransportDeleteDataSourceAction.NAME, DeleteDataSourceActionResponse::new), TransportDeleteDataSourceAction.class), new ActionHandler<>( - new ActionType<>(TransportCreateJobRequestAction.NAME, CreateJobActionResponse::new), - TransportCreateJobRequestAction.class), - new ActionHandler<>( - new ActionType<>(TransportGetJobRequestAction.NAME, GetJobActionResponse::new), - TransportGetJobRequestAction.class), + new ActionType<>( + TransportCreateAsyncQueryRequestAction.NAME, CreateAsyncQueryActionResponse::new), + TransportCreateAsyncQueryRequestAction.class), new ActionHandler<>( new ActionType<>( - TransportGetQueryResultRequestAction.NAME, GetJobQueryResultActionResponse::new), - TransportGetQueryResultRequestAction.class), + TransportGetAsyncQueryResultAction.NAME, GetAsyncQueryResultActionResponse::new), + TransportGetAsyncQueryResultAction.class), new ActionHandler<>( - new ActionType<>(TransportDeleteJobRequestAction.NAME, DeleteJobActionResponse::new), - TransportDeleteJobRequestAction.class)); + new ActionType<>( + TransportCancelAsyncQueryRequestAction.NAME, CancelAsyncQueryActionResponse::new), + TransportCancelAsyncQueryRequestAction.class)); } @Override @@ -202,6 +215,16 @@ public Collection createComponents( dataSourceService.createDataSource(defaultOpenSearchDataSourceMetadata()); LocalClusterState.state().setClusterService(clusterService); LocalClusterState.state().setPluginSettings((OpenSearchSettings) pluginSettings); + if (StringUtils.isEmpty(this.pluginSettings.getSettingValue(SPARK_EXECUTION_ENGINE_CONFIG))) { + LOGGER.warn( + String.format( + "Async Query APIs are disabled as %s is not configured in cluster settings. " + + "Please configure and restart the domain to enable Async Query APIs", + SPARK_EXECUTION_ENGINE_CONFIG.getKeyValue())); + this.asyncQueryExecutorService = new AsyncQueryExecutorServiceImpl(); + } else { + this.asyncQueryExecutorService = createAsyncQueryExecutorService(); + } ModulesBuilder modules = new ModulesBuilder(); modules.add(new OpenSearchPluginModule()); @@ -213,7 +236,7 @@ public Collection createComponents( }); injector = modules.createInjector(); - return ImmutableList.of(dataSourceService); + return ImmutableList.of(dataSourceService, asyncQueryExecutorService); } @Override @@ -270,4 +293,34 @@ private DataSourceServiceImpl createDataSourceService() { dataSourceMetadataStorage, dataSourceUserAuthorizationHelper); } + + private AsyncQueryExecutorService createAsyncQueryExecutorService() { + AsyncQueryJobMetadataStorageService asyncQueryJobMetadataStorageService = + new OpensearchAsyncQueryJobMetadataStorageService(client, clusterService); + SparkJobClient sparkJobClient = createEMRServerlessClient(); + JobExecutionResponseReader jobExecutionResponseReader = new JobExecutionResponseReader(client); + SparkQueryDispatcher sparkQueryDispatcher = + new SparkQueryDispatcher( + sparkJobClient, this.dataSourceService, jobExecutionResponseReader); + return new AsyncQueryExecutorServiceImpl( + asyncQueryJobMetadataStorageService, sparkQueryDispatcher, pluginSettings); + } + + private SparkJobClient createEMRServerlessClient() { + String sparkExecutionEngineConfigString = + this.pluginSettings.getSettingValue(SPARK_EXECUTION_ENGINE_CONFIG); + return AccessController.doPrivileged( + (PrivilegedAction) + () -> { + SparkExecutionEngineConfig sparkExecutionEngineConfig = + SparkExecutionEngineConfig.toSparkExecutionEngineConfig( + sparkExecutionEngineConfigString); + AWSEMRServerless awsemrServerless = + AWSEMRServerlessClientBuilder.standard() + .withRegion(sparkExecutionEngineConfig.getRegion()) + .withCredentials(new DefaultAWSCredentialsProviderChain()) + .build(); + return new EmrServerlessClientImpl(awsemrServerless); + }); + } } diff --git a/plugin/src/main/plugin-metadata/plugin-security.policy b/plugin/src/main/plugin-metadata/plugin-security.policy index aec517aa84..fcf70c01f9 100644 --- a/plugin/src/main/plugin-metadata/plugin-security.policy +++ b/plugin/src/main/plugin-metadata/plugin-security.policy @@ -15,4 +15,13 @@ grant { // ml-commons client permission java.lang.RuntimePermission "setContextClassLoader"; + + // aws credentials + permission java.io.FilePermission "${user.home}${/}.aws${/}*", "read"; + + // Permissions for aws emr servless sdk + permission javax.management.MBeanServerPermission "createMBeanServer"; + permission javax.management.MBeanServerPermission "findMBeanServer"; + permission javax.management.MBeanPermission "com.amazonaws.metrics.*", "*"; + permission javax.management.MBeanTrustPermission "register"; }; diff --git a/spark/build.gradle b/spark/build.gradle index b93e3327ce..fb9a1e0e4b 100644 --- a/spark/build.gradle +++ b/spark/build.gradle @@ -15,11 +15,14 @@ repositories { dependencies { api project(':core') + implementation project(':protocol') implementation project(':datasources') implementation group: 'org.opensearch', name: 'opensearch', version: "${opensearch_version}" implementation group: 'org.json', name: 'json', version: '20230227' - implementation group: 'com.amazonaws', name: 'aws-java-sdk-emr', version: '1.12.1' + api group: 'com.amazonaws', name: 'aws-java-sdk-emr', version: '1.12.545' + api group: 'com.amazonaws', name: 'aws-java-sdk-emrserverless', version: '1.12.545' + implementation group: 'commons-io', name: 'commons-io', version: '2.8.0' testImplementation('org.junit.jupiter:junit-jupiter:5.6.2') testImplementation group: 'org.mockito', name: 'mockito-core', version: '5.2.0' @@ -56,7 +59,9 @@ jacocoTestCoverageVerification { excludes = [ 'org.opensearch.sql.spark.data.constants.*', 'org.opensearch.sql.spark.rest.*', - 'org.opensearch.sql.spark.transport.model.*' + 'org.opensearch.sql.spark.transport.model.*', + 'org.opensearch.sql.spark.asyncquery.model.*', + 'org.opensearch.sql.spark.asyncquery.exceptions.*' ] limit { counter = 'LINE' diff --git a/spark/src/main/java/org/opensearch/sql/spark/asyncquery/AsyncQueryExecutorService.java b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/AsyncQueryExecutorService.java new file mode 100644 index 0000000000..df13daa2a2 --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/AsyncQueryExecutorService.java @@ -0,0 +1,32 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.asyncquery; + +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryExecutionResponse; +import org.opensearch.sql.spark.rest.model.CreateAsyncQueryRequest; +import org.opensearch.sql.spark.rest.model.CreateAsyncQueryResponse; + +/** + * AsyncQueryExecutorService exposes functionality to create, get results and cancel an async query. + */ +public interface AsyncQueryExecutorService { + + /** + * Creates async query job based on the request and returns queryId in the response. + * + * @param createAsyncQueryRequest createAsyncQueryRequest. + * @return {@link CreateAsyncQueryResponse} + */ + CreateAsyncQueryResponse createAsyncQuery(CreateAsyncQueryRequest createAsyncQueryRequest); + + /** + * Returns async query response for a given queryId. + * + * @param queryId queryId. + * @return {@link AsyncQueryExecutionResponse} + */ + AsyncQueryExecutionResponse getAsyncQueryResults(String queryId); +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/asyncquery/AsyncQueryExecutorServiceImpl.java b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/AsyncQueryExecutorServiceImpl.java new file mode 100644 index 0000000000..e5ed65920e --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/AsyncQueryExecutorServiceImpl.java @@ -0,0 +1,107 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.asyncquery; + +import static org.opensearch.sql.common.setting.Settings.Key.SPARK_EXECUTION_ENGINE_CONFIG; + +import com.amazonaws.services.emrserverless.model.JobRunState; +import java.security.AccessController; +import java.security.PrivilegedAction; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import lombok.AllArgsConstructor; +import org.json.JSONObject; +import org.opensearch.sql.common.setting.Settings; +import org.opensearch.sql.data.model.ExprValue; +import org.opensearch.sql.spark.asyncquery.exceptions.AsyncQueryNotFoundException; +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryExecutionResponse; +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryJobMetadata; +import org.opensearch.sql.spark.config.SparkExecutionEngineConfig; +import org.opensearch.sql.spark.dispatcher.SparkQueryDispatcher; +import org.opensearch.sql.spark.functions.response.DefaultSparkSqlFunctionResponseHandle; +import org.opensearch.sql.spark.rest.model.CreateAsyncQueryRequest; +import org.opensearch.sql.spark.rest.model.CreateAsyncQueryResponse; + +/** AsyncQueryExecutorService implementation of {@link AsyncQueryExecutorService}. */ +@AllArgsConstructor +public class AsyncQueryExecutorServiceImpl implements AsyncQueryExecutorService { + private AsyncQueryJobMetadataStorageService asyncQueryJobMetadataStorageService; + private SparkQueryDispatcher sparkQueryDispatcher; + private Settings settings; + private Boolean isSparkJobExecutionEnabled; + + public AsyncQueryExecutorServiceImpl() { + this.isSparkJobExecutionEnabled = Boolean.FALSE; + } + + public AsyncQueryExecutorServiceImpl( + AsyncQueryJobMetadataStorageService asyncQueryJobMetadataStorageService, + SparkQueryDispatcher sparkQueryDispatcher, + Settings settings) { + this.isSparkJobExecutionEnabled = Boolean.TRUE; + this.asyncQueryJobMetadataStorageService = asyncQueryJobMetadataStorageService; + this.sparkQueryDispatcher = sparkQueryDispatcher; + this.settings = settings; + } + + @Override + public CreateAsyncQueryResponse createAsyncQuery( + CreateAsyncQueryRequest createAsyncQueryRequest) { + validateSparkExecutionEngineSettings(); + String sparkExecutionEngineConfigString = + settings.getSettingValue(SPARK_EXECUTION_ENGINE_CONFIG); + SparkExecutionEngineConfig sparkExecutionEngineConfig = + AccessController.doPrivileged( + (PrivilegedAction) + () -> + SparkExecutionEngineConfig.toSparkExecutionEngineConfig( + sparkExecutionEngineConfigString)); + String jobId = + sparkQueryDispatcher.dispatch( + sparkExecutionEngineConfig.getApplicationId(), + createAsyncQueryRequest.getQuery(), + sparkExecutionEngineConfig.getExecutionRoleARN()); + asyncQueryJobMetadataStorageService.storeJobMetadata( + new AsyncQueryJobMetadata(jobId, sparkExecutionEngineConfig.getApplicationId())); + return new CreateAsyncQueryResponse(jobId); + } + + @Override + public AsyncQueryExecutionResponse getAsyncQueryResults(String queryId) { + validateSparkExecutionEngineSettings(); + Optional jobMetadata = + asyncQueryJobMetadataStorageService.getJobMetadata(queryId); + if (jobMetadata.isPresent()) { + JSONObject jsonObject = + sparkQueryDispatcher.getQueryResponse( + jobMetadata.get().getApplicationId(), jobMetadata.get().getJobId()); + if (JobRunState.SUCCESS.toString().equals(jsonObject.getString("status"))) { + DefaultSparkSqlFunctionResponseHandle sparkSqlFunctionResponseHandle = + new DefaultSparkSqlFunctionResponseHandle(jsonObject); + List result = new ArrayList<>(); + while (sparkSqlFunctionResponseHandle.hasNext()) { + result.add(sparkSqlFunctionResponseHandle.next()); + } + return new AsyncQueryExecutionResponse( + JobRunState.SUCCESS.toString(), sparkSqlFunctionResponseHandle.schema(), result); + } else { + return new AsyncQueryExecutionResponse(jsonObject.getString("status"), null, null); + } + } + throw new AsyncQueryNotFoundException(String.format("QueryId: %s not found", queryId)); + } + + private void validateSparkExecutionEngineSettings() { + if (!isSparkJobExecutionEnabled) { + throw new IllegalArgumentException( + String.format( + "Async Query APIs are disabled as %s is not configured in cluster settings. Please" + + " configure the setting and restart the domain to enable Async Query APIs", + SPARK_EXECUTION_ENGINE_CONFIG.getKeyValue())); + } + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/asyncquery/AsyncQueryJobMetadataStorageService.java b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/AsyncQueryJobMetadataStorageService.java new file mode 100644 index 0000000000..4ce34458cd --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/AsyncQueryJobMetadataStorageService.java @@ -0,0 +1,18 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.asyncquery; + +import java.util.Optional; +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryJobMetadata; + +public interface AsyncQueryJobMetadataStorageService { + + void storeJobMetadata(AsyncQueryJobMetadata asyncQueryJobMetadata); + + Optional getJobMetadata(String jobId); +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/asyncquery/OpensearchAsyncQueryJobMetadataStorageService.java b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/OpensearchAsyncQueryJobMetadataStorageService.java new file mode 100644 index 0000000000..cee38d10f8 --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/OpensearchAsyncQueryJobMetadataStorageService.java @@ -0,0 +1,171 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.asyncquery; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import org.apache.commons.io.IOUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.DocWriteRequest; +import org.opensearch.action.DocWriteResponse; +import org.opensearch.action.admin.indices.create.CreateIndexRequest; +import org.opensearch.action.admin.indices.create.CreateIndexResponse; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.index.IndexResponse; +import org.opensearch.action.search.SearchRequest; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.action.support.WriteRequest; +import org.opensearch.client.Client; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.action.ActionFuture; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.QueryBuilders; +import org.opensearch.search.SearchHit; +import org.opensearch.search.builder.SearchSourceBuilder; +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryJobMetadata; + +/** Opensearch implementation of {@link AsyncQueryJobMetadataStorageService} */ +public class OpensearchAsyncQueryJobMetadataStorageService + implements AsyncQueryJobMetadataStorageService { + + public static final String JOB_METADATA_INDEX = ".ql-job-metadata"; + private static final String JOB_METADATA_INDEX_MAPPING_FILE_NAME = + "job-metadata-index-mapping.yml"; + private static final String JOB_METADATA_INDEX_SETTINGS_FILE_NAME = + "job-metadata-index-settings.yml"; + private static final Logger LOG = LogManager.getLogger(); + private final Client client; + private final ClusterService clusterService; + + /** + * This class implements JobMetadataStorageService interface using OpenSearch as underlying + * storage. + * + * @param client opensearch NodeClient. + * @param clusterService ClusterService. + */ + public OpensearchAsyncQueryJobMetadataStorageService( + Client client, ClusterService clusterService) { + this.client = client; + this.clusterService = clusterService; + } + + @Override + public void storeJobMetadata(AsyncQueryJobMetadata asyncQueryJobMetadata) { + if (!this.clusterService.state().routingTable().hasIndex(JOB_METADATA_INDEX)) { + createJobMetadataIndex(); + } + IndexRequest indexRequest = new IndexRequest(JOB_METADATA_INDEX); + indexRequest.id(asyncQueryJobMetadata.getJobId()); + indexRequest.opType(DocWriteRequest.OpType.CREATE); + indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + ActionFuture indexResponseActionFuture; + IndexResponse indexResponse; + try (ThreadContext.StoredContext storedContext = + client.threadPool().getThreadContext().stashContext()) { + indexRequest.source(AsyncQueryJobMetadata.convertToXContent(asyncQueryJobMetadata)); + indexResponseActionFuture = client.index(indexRequest); + indexResponse = indexResponseActionFuture.actionGet(); + } catch (Exception e) { + throw new RuntimeException(e); + } + + if (indexResponse.getResult().equals(DocWriteResponse.Result.CREATED)) { + LOG.debug("JobMetadata : {} successfully created", asyncQueryJobMetadata.getJobId()); + } else { + throw new RuntimeException( + "Saving job metadata information failed with result : " + + indexResponse.getResult().getLowercase()); + } + } + + @Override + public Optional getJobMetadata(String jobId) { + if (!this.clusterService.state().routingTable().hasIndex(JOB_METADATA_INDEX)) { + createJobMetadataIndex(); + return Optional.empty(); + } + return searchInJobMetadataIndex(QueryBuilders.termQuery("jobId", jobId)).stream().findFirst(); + } + + private void createJobMetadataIndex() { + try { + InputStream mappingFileStream = + OpensearchAsyncQueryJobMetadataStorageService.class + .getClassLoader() + .getResourceAsStream(JOB_METADATA_INDEX_MAPPING_FILE_NAME); + InputStream settingsFileStream = + OpensearchAsyncQueryJobMetadataStorageService.class + .getClassLoader() + .getResourceAsStream(JOB_METADATA_INDEX_SETTINGS_FILE_NAME); + CreateIndexRequest createIndexRequest = new CreateIndexRequest(JOB_METADATA_INDEX); + createIndexRequest + .mapping(IOUtils.toString(mappingFileStream, StandardCharsets.UTF_8), XContentType.YAML) + .settings( + IOUtils.toString(settingsFileStream, StandardCharsets.UTF_8), XContentType.YAML); + ActionFuture createIndexResponseActionFuture; + try (ThreadContext.StoredContext ignored = + client.threadPool().getThreadContext().stashContext()) { + createIndexResponseActionFuture = client.admin().indices().create(createIndexRequest); + } + CreateIndexResponse createIndexResponse = createIndexResponseActionFuture.actionGet(); + if (createIndexResponse.isAcknowledged()) { + LOG.info("Index: {} creation Acknowledged", JOB_METADATA_INDEX); + } else { + throw new RuntimeException("Index creation is not acknowledged."); + } + } catch (Throwable e) { + throw new RuntimeException( + "Internal server error while creating" + + JOB_METADATA_INDEX + + " index:: " + + e.getMessage()); + } + } + + private List searchInJobMetadataIndex(QueryBuilder query) { + SearchRequest searchRequest = new SearchRequest(); + searchRequest.indices(JOB_METADATA_INDEX); + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); + searchSourceBuilder.query(query); + searchSourceBuilder.size(1); + searchRequest.source(searchSourceBuilder); + // https://github.com/opensearch-project/sql/issues/1801. + searchRequest.preference("_primary_first"); + ActionFuture searchResponseActionFuture; + try (ThreadContext.StoredContext ignored = + client.threadPool().getThreadContext().stashContext()) { + searchResponseActionFuture = client.search(searchRequest); + } + SearchResponse searchResponse = searchResponseActionFuture.actionGet(); + if (searchResponse.status().getStatus() != 200) { + throw new RuntimeException( + "Fetching job metadata information failed with status : " + searchResponse.status()); + } else { + List list = new ArrayList<>(); + for (SearchHit searchHit : searchResponse.getHits().getHits()) { + String sourceAsString = searchHit.getSourceAsString(); + AsyncQueryJobMetadata asyncQueryJobMetadata; + try { + asyncQueryJobMetadata = AsyncQueryJobMetadata.toJobMetadata(sourceAsString); + } catch (IOException e) { + throw new RuntimeException(e); + } + list.add(asyncQueryJobMetadata); + } + return list; + } + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/asyncquery/exceptions/AsyncQueryNotFoundException.java b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/exceptions/AsyncQueryNotFoundException.java new file mode 100644 index 0000000000..80a0c34b70 --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/exceptions/AsyncQueryNotFoundException.java @@ -0,0 +1,15 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.asyncquery.exceptions; + +/** AsyncQueryNotFoundException. */ +public class AsyncQueryNotFoundException extends RuntimeException { + public AsyncQueryNotFoundException(String message) { + super(message); + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/asyncquery/model/AsyncQueryExecutionResponse.java b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/model/AsyncQueryExecutionResponse.java new file mode 100644 index 0000000000..84dcc490ba --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/model/AsyncQueryExecutionResponse.java @@ -0,0 +1,21 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.asyncquery.model; + +import java.util.List; +import lombok.Data; +import org.opensearch.sql.data.model.ExprValue; +import org.opensearch.sql.executor.ExecutionEngine; + +/** AsyncQueryExecutionResponse to store the response form spark job execution. */ +@Data +public class AsyncQueryExecutionResponse { + private final String status; + private final ExecutionEngine.Schema schema; + private final List results; +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/asyncquery/model/AsyncQueryJobMetadata.java b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/model/AsyncQueryJobMetadata.java new file mode 100644 index 0000000000..60ec53987e --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/model/AsyncQueryJobMetadata.java @@ -0,0 +1,100 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.asyncquery.model; + +import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; + +import com.google.gson.Gson; +import java.io.IOException; +import lombok.AllArgsConstructor; +import lombok.Data; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.xcontent.DeprecationHandler; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; + +/** This class models all the metadata required for a job. */ +@Data +@AllArgsConstructor +public class AsyncQueryJobMetadata { + private String jobId; + private String applicationId; + + @Override + public String toString() { + return new Gson().toJson(this); + } + + /** + * Converts JobMetadata to XContentBuilder. + * + * @param metadata metadata. + * @return XContentBuilder {@link XContentBuilder} + * @throws Exception Exception. + */ + public static XContentBuilder convertToXContent(AsyncQueryJobMetadata metadata) throws Exception { + XContentBuilder builder = XContentFactory.jsonBuilder(); + builder.startObject(); + builder.field("jobId", metadata.getJobId()); + builder.field("applicationId", metadata.getApplicationId()); + builder.endObject(); + return builder; + } + + /** + * Converts json string to DataSourceMetadata. + * + * @param json jsonstring. + * @return jobmetadata {@link AsyncQueryJobMetadata} + * @throws java.io.IOException IOException. + */ + public static AsyncQueryJobMetadata toJobMetadata(String json) throws IOException { + try (XContentParser parser = + XContentType.JSON + .xContent() + .createParser( + NamedXContentRegistry.EMPTY, + DeprecationHandler.THROW_UNSUPPORTED_OPERATION, + json)) { + return toJobMetadata(parser); + } + } + + /** + * Convert xcontent parser to JobMetadata. + * + * @param parser parser. + * @return JobMetadata {@link AsyncQueryJobMetadata} + * @throws IOException IOException. + */ + public static AsyncQueryJobMetadata toJobMetadata(XContentParser parser) throws IOException { + String jobId = null; + String applicationId = null; + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + String fieldName = parser.currentName(); + parser.nextToken(); + switch (fieldName) { + case "jobId": + jobId = parser.textOrNull(); + break; + case "applicationId": + applicationId = parser.textOrNull(); + break; + default: + throw new IllegalArgumentException("Unknown field: " + fieldName); + } + } + if (jobId == null || applicationId == null) { + throw new IllegalArgumentException("jobId and applicationId are required fields."); + } + return new AsyncQueryJobMetadata(jobId, applicationId); + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/asyncquery/model/AsyncQueryResult.java b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/model/AsyncQueryResult.java new file mode 100644 index 0000000000..6d6bce8fbc --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/model/AsyncQueryResult.java @@ -0,0 +1,29 @@ +package org.opensearch.sql.spark.asyncquery.model; + +import java.util.Collection; +import lombok.Getter; +import org.opensearch.sql.data.model.ExprValue; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.pagination.Cursor; +import org.opensearch.sql.protocol.response.QueryResult; + +/** AsyncQueryResult for async query APIs. */ +public class AsyncQueryResult extends QueryResult { + + @Getter private final String status; + + public AsyncQueryResult( + String status, + ExecutionEngine.Schema schema, + Collection exprValues, + Cursor cursor) { + super(schema, exprValues, cursor); + this.status = status; + } + + public AsyncQueryResult( + String status, ExecutionEngine.Schema schema, Collection exprValues) { + super(schema, exprValues); + this.status = status; + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/asyncquery/model/S3GlueSparkSubmitParameters.java b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/model/S3GlueSparkSubmitParameters.java new file mode 100644 index 0000000000..fadb8a67a9 --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/asyncquery/model/S3GlueSparkSubmitParameters.java @@ -0,0 +1,97 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.asyncquery.model; + +import static org.opensearch.sql.spark.data.constants.SparkConstants.AWS_SNAPSHOT_REPOSITORY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.DEFAULT_CLASS_NAME; +import static org.opensearch.sql.spark.data.constants.SparkConstants.DEFAULT_GLUE_CATALOG_CREDENTIALS_PROVIDER_FACTORY_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.DEFAULT_S3_AWS_CREDENTIALS_PROVIDER_VALUE; +import static org.opensearch.sql.spark.data.constants.SparkConstants.EMR_ASSUME_ROLE_CREDENTIALS_PROVIDER; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_CATALOG_JAR; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_CREDENTIALS_PROVIDER_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_DEFAULT_AUTH; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_DEFAULT_HOST; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_DEFAULT_PORT; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_DEFAULT_REGION; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_DEFAULT_SCHEME; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_INDEX_STORE_AUTH_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_INDEX_STORE_AWSREGION_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_INDEX_STORE_HOST_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_INDEX_STORE_PORT_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_INDEX_STORE_SCHEME_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_SQL_EXTENSION; +import static org.opensearch.sql.spark.data.constants.SparkConstants.GLUE_CATALOG_HIVE_JAR; +import static org.opensearch.sql.spark.data.constants.SparkConstants.GLUE_HIVE_CATALOG_FACTORY_CLASS; +import static org.opensearch.sql.spark.data.constants.SparkConstants.HADOOP_CATALOG_CREDENTIALS_PROVIDER_FACTORY_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.HIVE_METASTORE_CLASS_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.JAVA_HOME_LOCATION; +import static org.opensearch.sql.spark.data.constants.SparkConstants.S3_AWS_CREDENTIALS_PROVIDER_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_DRIVER_ENV_JAVA_HOME_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_EXECUTOR_ENV_JAVA_HOME_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_JARS_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_JAR_PACKAGES_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_JAR_REPOSITORIES_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_SQL_EXTENSIONS_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_STANDALONE_PACKAGE; + +import java.util.LinkedHashMap; +import java.util.Map; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class S3GlueSparkSubmitParameters { + + private String className; + private Map config; + public static final String SPACE = " "; + public static final String EQUALS = "="; + + public S3GlueSparkSubmitParameters() { + this.className = DEFAULT_CLASS_NAME; + this.config = new LinkedHashMap<>(); + this.config.put(S3_AWS_CREDENTIALS_PROVIDER_KEY, DEFAULT_S3_AWS_CREDENTIALS_PROVIDER_VALUE); + this.config.put( + HADOOP_CATALOG_CREDENTIALS_PROVIDER_FACTORY_KEY, + DEFAULT_GLUE_CATALOG_CREDENTIALS_PROVIDER_FACTORY_KEY); + this.config.put(SPARK_JARS_KEY, GLUE_CATALOG_HIVE_JAR + "," + FLINT_CATALOG_JAR); + this.config.put(SPARK_JAR_PACKAGES_KEY, SPARK_STANDALONE_PACKAGE); + this.config.put(SPARK_JAR_REPOSITORIES_KEY, AWS_SNAPSHOT_REPOSITORY); + this.config.put(SPARK_DRIVER_ENV_JAVA_HOME_KEY, JAVA_HOME_LOCATION); + this.config.put(SPARK_EXECUTOR_ENV_JAVA_HOME_KEY, JAVA_HOME_LOCATION); + this.config.put(FLINT_INDEX_STORE_HOST_KEY, FLINT_DEFAULT_HOST); + this.config.put(FLINT_INDEX_STORE_PORT_KEY, FLINT_DEFAULT_PORT); + this.config.put(FLINT_INDEX_STORE_SCHEME_KEY, FLINT_DEFAULT_SCHEME); + this.config.put(FLINT_INDEX_STORE_AUTH_KEY, FLINT_DEFAULT_AUTH); + this.config.put(FLINT_INDEX_STORE_AWSREGION_KEY, FLINT_DEFAULT_REGION); + this.config.put(FLINT_CREDENTIALS_PROVIDER_KEY, EMR_ASSUME_ROLE_CREDENTIALS_PROVIDER); + this.config.put(SPARK_SQL_EXTENSIONS_KEY, FLINT_SQL_EXTENSION); + this.config.put(HIVE_METASTORE_CLASS_KEY, GLUE_HIVE_CATALOG_FACTORY_CLASS); + } + + public void addParameter(String key, String value) { + this.config.put(key, value); + } + + @Override + public String toString() { + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append(" --class "); + stringBuilder.append(this.className); + stringBuilder.append(SPACE); + for (String key : config.keySet()) { + stringBuilder.append(" --conf "); + stringBuilder.append(key); + stringBuilder.append(EQUALS); + stringBuilder.append(config.get(key)); + stringBuilder.append(SPACE); + } + return stringBuilder.toString(); + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/client/EmrClientImpl.java b/spark/src/main/java/org/opensearch/sql/spark/client/EmrClientImpl.java index 1a3304994b..4e66cd9a00 100644 --- a/spark/src/main/java/org/opensearch/sql/spark/client/EmrClientImpl.java +++ b/spark/src/main/java/org/opensearch/sql/spark/client/EmrClientImpl.java @@ -5,7 +5,7 @@ package org.opensearch.sql.spark.client; -import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_INDEX_NAME; +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_RESPONSE_BUFFER_INDEX_NAME; import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_SQL_APPLICATION_JAR; import com.amazonaws.services.elasticmapreduce.AmazonElasticMapReduce; @@ -74,7 +74,7 @@ void runEmrApplication(String query) { flint.getFlintIntegrationJar(), sparkApplicationJar, query, - SPARK_INDEX_NAME, + SPARK_RESPONSE_BUFFER_INDEX_NAME, flint.getFlintHost(), flint.getFlintPort(), flint.getFlintScheme(), diff --git a/spark/src/main/java/org/opensearch/sql/spark/client/EmrServerlessClientImpl.java b/spark/src/main/java/org/opensearch/sql/spark/client/EmrServerlessClientImpl.java new file mode 100644 index 0000000000..b554c4cd23 --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/client/EmrServerlessClientImpl.java @@ -0,0 +1,68 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.client; + +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_RESPONSE_BUFFER_INDEX_NAME; +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_SQL_APPLICATION_JAR; + +import com.amazonaws.services.emrserverless.AWSEMRServerless; +import com.amazonaws.services.emrserverless.model.GetJobRunRequest; +import com.amazonaws.services.emrserverless.model.GetJobRunResult; +import com.amazonaws.services.emrserverless.model.JobDriver; +import com.amazonaws.services.emrserverless.model.SparkSubmit; +import com.amazonaws.services.emrserverless.model.StartJobRunRequest; +import com.amazonaws.services.emrserverless.model.StartJobRunResult; +import java.security.AccessController; +import java.security.PrivilegedAction; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +public class EmrServerlessClientImpl implements SparkJobClient { + + private final AWSEMRServerless emrServerless; + private static final Logger logger = LogManager.getLogger(EmrServerlessClientImpl.class); + + public EmrServerlessClientImpl(AWSEMRServerless emrServerless) { + this.emrServerless = emrServerless; + } + + @Override + public String startJobRun( + String query, + String jobName, + String applicationId, + String executionRoleArn, + String sparkSubmitParams) { + StartJobRunRequest request = + new StartJobRunRequest() + .withName(jobName) + .withApplicationId(applicationId) + .withExecutionRoleArn(executionRoleArn) + .withJobDriver( + new JobDriver() + .withSparkSubmit( + new SparkSubmit() + .withEntryPoint(SPARK_SQL_APPLICATION_JAR) + .withEntryPointArguments(query, SPARK_RESPONSE_BUFFER_INDEX_NAME) + .withSparkSubmitParameters(sparkSubmitParams))); + StartJobRunResult startJobRunResult = + AccessController.doPrivileged( + (PrivilegedAction) () -> emrServerless.startJobRun(request)); + logger.info("Job Run ID: " + startJobRunResult.getJobRunId()); + return startJobRunResult.getJobRunId(); + } + + @Override + public GetJobRunResult getJobRunResult(String applicationId, String jobId) { + GetJobRunRequest request = + new GetJobRunRequest().withApplicationId(applicationId).withJobRunId(jobId); + GetJobRunResult getJobRunResult = + AccessController.doPrivileged( + (PrivilegedAction) () -> emrServerless.getJobRun(request)); + logger.info("Job Run state: " + getJobRunResult.getJobRun().getState()); + return getJobRunResult; + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/client/SparkJobClient.java b/spark/src/main/java/org/opensearch/sql/spark/client/SparkJobClient.java new file mode 100644 index 0000000000..ff9f4acedd --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/client/SparkJobClient.java @@ -0,0 +1,22 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.client; + +import com.amazonaws.services.emrserverless.model.GetJobRunResult; + +public interface SparkJobClient { + + String startJobRun( + String query, + String jobName, + String applicationId, + String executionRoleArn, + String sparkSubmitParams); + + GetJobRunResult getJobRunResult(String applicationId, String jobId); +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/config/SparkExecutionEngineConfig.java b/spark/src/main/java/org/opensearch/sql/spark/config/SparkExecutionEngineConfig.java new file mode 100644 index 0000000000..4f928c4f1f --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/config/SparkExecutionEngineConfig.java @@ -0,0 +1,22 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.config; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.google.gson.Gson; +import lombok.Data; + +@Data +@JsonIgnoreProperties(ignoreUnknown = true) +public class SparkExecutionEngineConfig { + private String applicationId; + private String region; + private String executionRoleARN; + + public static SparkExecutionEngineConfig toSparkExecutionEngineConfig(String jsonString) { + return new Gson().fromJson(jsonString, SparkExecutionEngineConfig.class); + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/data/constants/SparkConstants.java b/spark/src/main/java/org/opensearch/sql/spark/data/constants/SparkConstants.java index 65d5a01ba2..21db8b9478 100644 --- a/spark/src/main/java/org/opensearch/sql/spark/data/constants/SparkConstants.java +++ b/spark/src/main/java/org/opensearch/sql/spark/data/constants/SparkConstants.java @@ -8,13 +8,64 @@ public class SparkConstants { public static final String EMR = "emr"; public static final String STEP_ID_FIELD = "stepId.keyword"; - public static final String SPARK_SQL_APPLICATION_JAR = "s3://spark-datasource/sql-job.jar"; - public static final String SPARK_INDEX_NAME = ".query_execution_result"; + // TODO should be replaced with mvn jar. + public static final String SPARK_SQL_APPLICATION_JAR = + "s3://flint-data-dp-eu-west-1-beta/code/flint/sql-job.jar"; + public static final String SPARK_RESPONSE_BUFFER_INDEX_NAME = ".query_execution_result"; + // TODO should be replaced with mvn jar. public static final String FLINT_INTEGRATION_JAR = "s3://spark-datasource/flint-spark-integration-assembly-0.1.0-SNAPSHOT.jar"; + // TODO should be replaced with mvn jar. + public static final String GLUE_CATALOG_HIVE_JAR = + "s3://flint-data-dp-eu-west-1-beta/code/flint/AWSGlueDataCatalogHiveMetaStoreAuth-1.0.jar"; + // TODO should be replaced with mvn jar. + public static final String FLINT_CATALOG_JAR = + "s3://flint-data-dp-eu-west-1-beta/code/flint/flint-catalog.jar"; public static final String FLINT_DEFAULT_HOST = "localhost"; public static final String FLINT_DEFAULT_PORT = "9200"; public static final String FLINT_DEFAULT_SCHEME = "http"; public static final String FLINT_DEFAULT_AUTH = "-1"; public static final String FLINT_DEFAULT_REGION = "us-west-2"; + public static final String DEFAULT_CLASS_NAME = "org.opensearch.sql.FlintJob"; + public static final String S3_AWS_CREDENTIALS_PROVIDER_KEY = + "spark.hadoop.fs.s3.customAWSCredentialsProvider"; + public static final String DRIVER_ENV_ASSUME_ROLE_ARN_KEY = + "spark.emr-serverless.driverEnv.ASSUME_ROLE_CREDENTIALS_ROLE_ARN"; + public static final String EXECUTOR_ENV_ASSUME_ROLE_ARN_KEY = + "spark.executorEnv.ASSUME_ROLE_CREDENTIALS_ROLE_ARN"; + public static final String HADOOP_CATALOG_CREDENTIALS_PROVIDER_FACTORY_KEY = + "spark.hadoop.aws.catalog.credentials.provider.factory.class"; + public static final String HIVE_METASTORE_GLUE_ARN_KEY = "spark.hive.metastore.glue.role.arn"; + public static final String SPARK_JARS_KEY = "spark.jars"; + public static final String SPARK_JAR_PACKAGES_KEY = "spark.jars.packages"; + public static final String SPARK_JAR_REPOSITORIES_KEY = "spark.jars.repositories"; + public static final String SPARK_DRIVER_ENV_JAVA_HOME_KEY = + "spark.emr-serverless.driverEnv.JAVA_HOME"; + public static final String SPARK_EXECUTOR_ENV_JAVA_HOME_KEY = "spark.executorEnv.JAVA_HOME"; + public static final String FLINT_INDEX_STORE_HOST_KEY = "spark.datasource.flint.host"; + public static final String FLINT_INDEX_STORE_PORT_KEY = "spark.datasource.flint.port"; + public static final String FLINT_INDEX_STORE_SCHEME_KEY = "spark.datasource.flint.scheme"; + public static final String FLINT_INDEX_STORE_AUTH_KEY = "spark.datasource.flint.auth"; + public static final String FLINT_INDEX_STORE_AWSREGION_KEY = "spark.datasource.flint.region"; + public static final String FLINT_CREDENTIALS_PROVIDER_KEY = + "spark.datasource.flint.customAWSCredentialsProvider"; + public static final String SPARK_SQL_EXTENSIONS_KEY = "spark.sql.extensions"; + public static final String HIVE_METASTORE_CLASS_KEY = + "spark.hadoop.hive.metastore.client.factory.class"; + public static final String DEFAULT_S3_AWS_CREDENTIALS_PROVIDER_VALUE = + "com.amazonaws.emr.AssumeRoleAWSCredentialsProvider"; + public static final String DEFAULT_GLUE_CATALOG_CREDENTIALS_PROVIDER_FACTORY_KEY = + "com.amazonaws.glue.catalog.metastore.STSAssumeRoleSessionCredentialsProviderFactory"; + public static final String SPARK_STANDALONE_PACKAGE = + "org.opensearch:opensearch-spark-standalone_2.12:0.1.0-SNAPSHOT"; + public static final String AWS_SNAPSHOT_REPOSITORY = + "https://aws.oss.sonatype.org/content/repositories/snapshots"; + public static final String GLUE_HIVE_CATALOG_FACTORY_CLASS = + "com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory"; + public static final String FLINT_DELEGATE_CATALOG = "org.opensearch.sql.FlintDelegateCatalog"; + public static final String FLINT_SQL_EXTENSION = + "org.opensearch.flint.spark.FlintSparkExtensions"; + public static final String EMR_ASSUME_ROLE_CREDENTIALS_PROVIDER = + "com.amazonaws.emr.AssumeRoleAWSCredentialsProvider"; + public static final String JAVA_HOME_LOCATION = "/usr/lib/jvm/java-17-amazon-corretto.x86_64/"; } diff --git a/spark/src/main/java/org/opensearch/sql/spark/dispatcher/SparkQueryDispatcher.java b/spark/src/main/java/org/opensearch/sql/spark/dispatcher/SparkQueryDispatcher.java new file mode 100644 index 0000000000..f632ceaf6a --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/dispatcher/SparkQueryDispatcher.java @@ -0,0 +1,103 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.dispatcher; + +import static org.opensearch.sql.spark.data.constants.SparkConstants.DRIVER_ENV_ASSUME_ROLE_ARN_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.EXECUTOR_ENV_ASSUME_ROLE_ARN_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_DELEGATE_CATALOG; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_INDEX_STORE_AUTH_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_INDEX_STORE_AWSREGION_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_INDEX_STORE_HOST_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_INDEX_STORE_PORT_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.FLINT_INDEX_STORE_SCHEME_KEY; +import static org.opensearch.sql.spark.data.constants.SparkConstants.HIVE_METASTORE_GLUE_ARN_KEY; + +import com.amazonaws.services.emrserverless.model.GetJobRunResult; +import com.amazonaws.services.emrserverless.model.JobRunState; +import java.net.URI; +import java.net.URISyntaxException; +import lombok.AllArgsConstructor; +import org.json.JSONObject; +import org.opensearch.sql.datasource.DataSourceService; +import org.opensearch.sql.datasource.model.DataSourceMetadata; +import org.opensearch.sql.spark.asyncquery.model.S3GlueSparkSubmitParameters; +import org.opensearch.sql.spark.client.SparkJobClient; +import org.opensearch.sql.spark.response.JobExecutionResponseReader; + +/** This class takes care of understanding query and dispatching job query to emr serverless. */ +@AllArgsConstructor +public class SparkQueryDispatcher { + + private SparkJobClient sparkJobClient; + + private DataSourceService dataSourceService; + + private JobExecutionResponseReader jobExecutionResponseReader; + + public String dispatch(String applicationId, String query, String executionRoleARN) { + String datasourceName = getDataSourceName(); + try { + return sparkJobClient.startJobRun( + query, + "flint-opensearch-query", + applicationId, + executionRoleARN, + constructSparkParameters(datasourceName)); + } catch (URISyntaxException e) { + throw new IllegalArgumentException( + String.format( + "Bad URI in indexstore configuration of the : %s datasoure.", datasourceName)); + } + } + + // TODO : Fetch from Result Index and then make call to EMR Serverless. + public JSONObject getQueryResponse(String applicationId, String queryId) { + GetJobRunResult getJobRunResult = sparkJobClient.getJobRunResult(applicationId, queryId); + JSONObject result = new JSONObject(); + if (getJobRunResult.getJobRun().getState().equals(JobRunState.SUCCESS.toString())) { + result = jobExecutionResponseReader.getResultFromOpensearchIndex(queryId); + } + result.put("status", getJobRunResult.getJobRun().getState()); + return result; + } + + // TODO: Analyze given query + // Extract datasourceName + // Apply Authorizaiton. + private String getDataSourceName() { + return "my_glue"; + } + + // TODO: Analyze given query and get the role arn based on datasource type. + private String getDataSourceRoleARN(DataSourceMetadata dataSourceMetadata) { + return dataSourceMetadata.getProperties().get("glue.auth.role_arn"); + } + + private String constructSparkParameters(String datasourceName) throws URISyntaxException { + DataSourceMetadata dataSourceMetadata = + dataSourceService.getRawDataSourceMetadata(datasourceName); + S3GlueSparkSubmitParameters s3GlueSparkSubmitParameters = new S3GlueSparkSubmitParameters(); + s3GlueSparkSubmitParameters.addParameter( + DRIVER_ENV_ASSUME_ROLE_ARN_KEY, getDataSourceRoleARN(dataSourceMetadata)); + s3GlueSparkSubmitParameters.addParameter( + EXECUTOR_ENV_ASSUME_ROLE_ARN_KEY, getDataSourceRoleARN(dataSourceMetadata)); + s3GlueSparkSubmitParameters.addParameter( + HIVE_METASTORE_GLUE_ARN_KEY, getDataSourceRoleARN(dataSourceMetadata)); + String opensearchuri = dataSourceMetadata.getProperties().get("glue.indexstore.opensearch.uri"); + URI uri = new URI(opensearchuri); + String auth = dataSourceMetadata.getProperties().get("glue.indexstore.opensearch.auth"); + String region = dataSourceMetadata.getProperties().get("glue.indexstore.opensearch.region"); + s3GlueSparkSubmitParameters.addParameter(FLINT_INDEX_STORE_HOST_KEY, uri.getHost()); + s3GlueSparkSubmitParameters.addParameter( + FLINT_INDEX_STORE_PORT_KEY, String.valueOf(uri.getPort())); + s3GlueSparkSubmitParameters.addParameter(FLINT_INDEX_STORE_SCHEME_KEY, uri.getScheme()); + s3GlueSparkSubmitParameters.addParameter(FLINT_INDEX_STORE_AUTH_KEY, auth); + s3GlueSparkSubmitParameters.addParameter(FLINT_INDEX_STORE_AWSREGION_KEY, region); + s3GlueSparkSubmitParameters.addParameter( + "spark.sql.catalog." + datasourceName, FLINT_DELEGATE_CATALOG); + return s3GlueSparkSubmitParameters.toString(); + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/functions/response/DefaultSparkSqlFunctionResponseHandle.java b/spark/src/main/java/org/opensearch/sql/spark/functions/response/DefaultSparkSqlFunctionResponseHandle.java index 823ad2da29..9d0cd59cf8 100644 --- a/spark/src/main/java/org/opensearch/sql/spark/functions/response/DefaultSparkSqlFunctionResponseHandle.java +++ b/spark/src/main/java/org/opensearch/sql/spark/functions/response/DefaultSparkSqlFunctionResponseHandle.java @@ -15,7 +15,6 @@ import org.json.JSONObject; import org.opensearch.sql.data.model.ExprBooleanValue; import org.opensearch.sql.data.model.ExprByteValue; -import org.opensearch.sql.data.model.ExprDateValue; import org.opensearch.sql.data.model.ExprDoubleValue; import org.opensearch.sql.data.model.ExprFloatValue; import org.opensearch.sql.data.model.ExprIntegerValue; @@ -81,7 +80,8 @@ private static LinkedHashMap extractRow( } else if (type == ExprCoreType.FLOAT) { linkedHashMap.put(column.getName(), new ExprFloatValue(row.getFloat(column.getName()))); } else if (type == ExprCoreType.DATE) { - linkedHashMap.put(column.getName(), new ExprDateValue(row.getString(column.getName()))); + // TODO :: correct this to ExprTimestampValue + linkedHashMap.put(column.getName(), new ExprStringValue(row.getString(column.getName()))); } else if (type == ExprCoreType.TIMESTAMP) { linkedHashMap.put( column.getName(), new ExprTimestampValue(row.getString(column.getName()))); diff --git a/spark/src/main/java/org/opensearch/sql/spark/response/JobExecutionResponseReader.java b/spark/src/main/java/org/opensearch/sql/spark/response/JobExecutionResponseReader.java new file mode 100644 index 0000000000..8abb7cd11f --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/response/JobExecutionResponseReader.java @@ -0,0 +1,67 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.response; + +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_RESPONSE_BUFFER_INDEX_NAME; +import static org.opensearch.sql.spark.data.constants.SparkConstants.STEP_ID_FIELD; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.json.JSONObject; +import org.opensearch.action.search.SearchRequest; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.client.Client; +import org.opensearch.common.action.ActionFuture; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.QueryBuilders; +import org.opensearch.search.SearchHit; +import org.opensearch.search.builder.SearchSourceBuilder; + +public class JobExecutionResponseReader { + private final Client client; + private static final Logger LOG = LogManager.getLogger(); + + /** + * JobExecutionResponseReader for spark query. + * + * @param client Opensearch client + */ + public JobExecutionResponseReader(Client client) { + this.client = client; + } + + public JSONObject getResultFromOpensearchIndex(String jobId) { + return searchInSparkIndex(QueryBuilders.termQuery(STEP_ID_FIELD, jobId)); + } + + private JSONObject searchInSparkIndex(QueryBuilder query) { + SearchRequest searchRequest = new SearchRequest(); + searchRequest.indices(SPARK_RESPONSE_BUFFER_INDEX_NAME); + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); + searchSourceBuilder.query(query); + searchRequest.source(searchSourceBuilder); + ActionFuture searchResponseActionFuture; + try { + searchResponseActionFuture = client.search(searchRequest); + } catch (Exception e) { + throw new RuntimeException(e); + } + SearchResponse searchResponse = searchResponseActionFuture.actionGet(); + if (searchResponse.status().getStatus() != 200) { + throw new RuntimeException( + "Fetching result from " + + SPARK_RESPONSE_BUFFER_INDEX_NAME + + " index failed with status : " + + searchResponse.status()); + } else { + JSONObject data = new JSONObject(); + for (SearchHit searchHit : searchResponse.getHits().getHits()) { + data.put("data", searchHit.getSourceAsMap()); + } + return data; + } + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/response/SparkResponse.java b/spark/src/main/java/org/opensearch/sql/spark/response/SparkResponse.java index 3edb541384..496caba2c9 100644 --- a/spark/src/main/java/org/opensearch/sql/spark/response/SparkResponse.java +++ b/spark/src/main/java/org/opensearch/sql/spark/response/SparkResponse.java @@ -5,7 +5,7 @@ package org.opensearch.sql.spark.response; -import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_INDEX_NAME; +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_RESPONSE_BUFFER_INDEX_NAME; import com.google.common.annotations.VisibleForTesting; import lombok.Data; @@ -51,7 +51,7 @@ public JSONObject getResultFromOpensearchIndex() { private JSONObject searchInSparkIndex(QueryBuilder query) { SearchRequest searchRequest = new SearchRequest(); - searchRequest.indices(SPARK_INDEX_NAME); + searchRequest.indices(SPARK_RESPONSE_BUFFER_INDEX_NAME); SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); searchSourceBuilder.query(query); searchRequest.source(searchSourceBuilder); @@ -65,7 +65,7 @@ private JSONObject searchInSparkIndex(QueryBuilder query) { if (searchResponse.status().getStatus() != 200) { throw new RuntimeException( "Fetching result from " - + SPARK_INDEX_NAME + + SPARK_RESPONSE_BUFFER_INDEX_NAME + " index failed with status : " + searchResponse.status()); } else { @@ -80,7 +80,7 @@ private JSONObject searchInSparkIndex(QueryBuilder query) { @VisibleForTesting void deleteInSparkIndex(String id) { - DeleteRequest deleteRequest = new DeleteRequest(SPARK_INDEX_NAME); + DeleteRequest deleteRequest = new DeleteRequest(SPARK_RESPONSE_BUFFER_INDEX_NAME); deleteRequest.id(id); ActionFuture deleteResponseActionFuture; try { diff --git a/spark/src/main/java/org/opensearch/sql/spark/rest/RestJobManagementAction.java b/spark/src/main/java/org/opensearch/sql/spark/rest/RestAsyncQueryManagementAction.java similarity index 50% rename from spark/src/main/java/org/opensearch/sql/spark/rest/RestJobManagementAction.java rename to spark/src/main/java/org/opensearch/sql/spark/rest/RestAsyncQueryManagementAction.java index 669cbb6aca..56484688dc 100644 --- a/spark/src/main/java/org/opensearch/sql/spark/rest/RestJobManagementAction.java +++ b/spark/src/main/java/org/opensearch/sql/spark/rest/RestAsyncQueryManagementAction.java @@ -27,30 +27,27 @@ import org.opensearch.rest.RestRequest; import org.opensearch.sql.datasources.exceptions.ErrorMessage; import org.opensearch.sql.datasources.utils.Scheduler; -import org.opensearch.sql.spark.rest.model.CreateJobRequest; -import org.opensearch.sql.spark.transport.TransportCreateJobRequestAction; -import org.opensearch.sql.spark.transport.TransportDeleteJobRequestAction; -import org.opensearch.sql.spark.transport.TransportGetJobRequestAction; -import org.opensearch.sql.spark.transport.TransportGetQueryResultRequestAction; -import org.opensearch.sql.spark.transport.model.CreateJobActionRequest; -import org.opensearch.sql.spark.transport.model.CreateJobActionResponse; -import org.opensearch.sql.spark.transport.model.DeleteJobActionRequest; -import org.opensearch.sql.spark.transport.model.DeleteJobActionResponse; -import org.opensearch.sql.spark.transport.model.GetJobActionRequest; -import org.opensearch.sql.spark.transport.model.GetJobActionResponse; -import org.opensearch.sql.spark.transport.model.GetJobQueryResultActionRequest; -import org.opensearch.sql.spark.transport.model.GetJobQueryResultActionResponse; +import org.opensearch.sql.spark.rest.model.CreateAsyncQueryRequest; +import org.opensearch.sql.spark.transport.TransportCancelAsyncQueryRequestAction; +import org.opensearch.sql.spark.transport.TransportCreateAsyncQueryRequestAction; +import org.opensearch.sql.spark.transport.TransportGetAsyncQueryResultAction; +import org.opensearch.sql.spark.transport.model.CancelAsyncQueryActionRequest; +import org.opensearch.sql.spark.transport.model.CancelAsyncQueryActionResponse; +import org.opensearch.sql.spark.transport.model.CreateAsyncQueryActionRequest; +import org.opensearch.sql.spark.transport.model.CreateAsyncQueryActionResponse; +import org.opensearch.sql.spark.transport.model.GetAsyncQueryResultActionRequest; +import org.opensearch.sql.spark.transport.model.GetAsyncQueryResultActionResponse; -public class RestJobManagementAction extends BaseRestHandler { +public class RestAsyncQueryManagementAction extends BaseRestHandler { - public static final String JOB_ACTIONS = "job_actions"; - public static final String BASE_JOB_ACTION_URL = "/_plugins/_query/_jobs"; + public static final String ASYNC_QUERY_ACTIONS = "async_query_actions"; + public static final String BASE_ASYNC_QUERY_ACTION_URL = "/_plugins/_async_query"; - private static final Logger LOG = LogManager.getLogger(RestJobManagementAction.class); + private static final Logger LOG = LogManager.getLogger(RestAsyncQueryManagementAction.class); @Override public String getName() { - return JOB_ACTIONS; + return ASYNC_QUERY_ACTIONS; } @Override @@ -59,47 +56,38 @@ public List routes() { /* * - * Create a new job with spark execution engine. + * Create a new async query using spark execution engine. * Request URL: POST * Request body: - * Ref [org.opensearch.sql.spark.transport.model.SubmitJobActionRequest] + * Ref [org.opensearch.sql.spark.transport.model.CreateAsyncQueryActionRequest] * Response body: - * Ref [org.opensearch.sql.spark.transport.model.SubmitJobActionResponse] + * Ref [org.opensearch.sql.spark.transport.model.CreateAsyncQueryActionResponse] */ - new Route(POST, BASE_JOB_ACTION_URL), + new Route(POST, BASE_ASYNC_QUERY_ACTION_URL), /* * - * GET jobs with in spark execution engine. + * GET Async Query result with in spark execution engine. * Request URL: GET * Request body: - * Ref [org.opensearch.sql.spark.transport.model.SubmitJobActionRequest] + * Ref [org.opensearch.sql.spark.transport.model.GetAsyncQueryResultActionRequest] * Response body: - * Ref [org.opensearch.sql.spark.transport.model.SubmitJobActionResponse] + * Ref [org.opensearch.sql.spark.transport.model.GetAsyncQueryResultActionResponse] */ - new Route(GET, String.format(Locale.ROOT, "%s/{%s}", BASE_JOB_ACTION_URL, "jobId")), - new Route(GET, BASE_JOB_ACTION_URL), + new Route( + GET, String.format(Locale.ROOT, "%s/{%s}", BASE_ASYNC_QUERY_ACTION_URL, "queryId")), /* * * Cancel a job within spark execution engine. * Request URL: DELETE * Request body: - * Ref [org.opensearch.sql.spark.transport.model.SubmitJobActionRequest] + * Ref [org.opensearch.sql.spark.transport.model.CancelAsyncQueryActionRequest] * Response body: - * Ref [org.opensearch.sql.spark.transport.model.SubmitJobActionResponse] + * Ref [org.opensearch.sql.spark.transport.model.CancelAsyncQueryActionResponse] */ - new Route(DELETE, String.format(Locale.ROOT, "%s/{%s}", BASE_JOB_ACTION_URL, "jobId")), - - /* - * GET query result from job {{jobId}} execution. - * Request URL: GET - * Request body: - * Ref [org.opensearch.sql.spark.transport.model.GetJobQueryResultActionRequest] - * Response body: - * Ref [org.opensearch.sql.spark.transport.model.GetJobQueryResultActionResponse] - */ - new Route(GET, String.format(Locale.ROOT, "%s/{%s}/result", BASE_JOB_ACTION_URL, "jobId"))); + new Route( + DELETE, String.format(Locale.ROOT, "%s/{%s}", BASE_ASYNC_QUERY_ACTION_URL, "queryId"))); } @Override @@ -109,7 +97,7 @@ protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient case POST: return executePostRequest(restRequest, nodeClient); case GET: - return executeGetRequest(restRequest, nodeClient); + return executeGetAsyncQueryResultRequest(restRequest, nodeClient); case DELETE: return executeDeleteRequest(restRequest, nodeClient); default: @@ -122,23 +110,24 @@ protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient private RestChannelConsumer executePostRequest(RestRequest restRequest, NodeClient nodeClient) throws IOException { - CreateJobRequest submitJobRequest = - CreateJobRequest.fromXContentParser(restRequest.contentParser()); + CreateAsyncQueryRequest submitJobRequest = + CreateAsyncQueryRequest.fromXContentParser(restRequest.contentParser()); return restChannel -> Scheduler.schedule( nodeClient, () -> nodeClient.execute( - TransportCreateJobRequestAction.ACTION_TYPE, - new CreateJobActionRequest(submitJobRequest), + TransportCreateAsyncQueryRequestAction.ACTION_TYPE, + new CreateAsyncQueryActionRequest(submitJobRequest), new ActionListener<>() { @Override - public void onResponse(CreateJobActionResponse createJobActionResponse) { + public void onResponse( + CreateAsyncQueryActionResponse createAsyncQueryActionResponse) { restChannel.sendResponse( new BytesRestResponse( RestStatus.CREATED, "application/json; charset=UTF-8", - submitJobRequest.getQuery())); + createAsyncQueryActionResponse.getResult())); } @Override @@ -148,60 +137,25 @@ public void onFailure(Exception e) { })); } - private RestChannelConsumer executeGetRequest(RestRequest restRequest, NodeClient nodeClient) { - Boolean isResultRequest = restRequest.rawPath().contains("result"); - if (isResultRequest) { - return executeGetJobQueryResultRequest(nodeClient, restRequest); - } else { - return executeGetJobRequest(nodeClient, restRequest); - } - } - - private RestChannelConsumer executeGetJobQueryResultRequest( - NodeClient nodeClient, RestRequest restRequest) { - String jobId = restRequest.param("jobId"); + private RestChannelConsumer executeGetAsyncQueryResultRequest( + RestRequest restRequest, NodeClient nodeClient) { + String queryId = restRequest.param("queryId"); return restChannel -> Scheduler.schedule( nodeClient, () -> nodeClient.execute( - TransportGetQueryResultRequestAction.ACTION_TYPE, - new GetJobQueryResultActionRequest(jobId), + TransportGetAsyncQueryResultAction.ACTION_TYPE, + new GetAsyncQueryResultActionRequest(queryId), new ActionListener<>() { @Override public void onResponse( - GetJobQueryResultActionResponse getJobQueryResultActionResponse) { - restChannel.sendResponse( - new BytesRestResponse( - RestStatus.OK, - "application/json; charset=UTF-8", - getJobQueryResultActionResponse.getResult())); - } - - @Override - public void onFailure(Exception e) { - handleException(e, restChannel); - } - })); - } - - private RestChannelConsumer executeGetJobRequest(NodeClient nodeClient, RestRequest restRequest) { - String jobId = restRequest.param("jobId"); - return restChannel -> - Scheduler.schedule( - nodeClient, - () -> - nodeClient.execute( - TransportGetJobRequestAction.ACTION_TYPE, - new GetJobActionRequest(jobId), - new ActionListener<>() { - @Override - public void onResponse(GetJobActionResponse getJobActionResponse) { + GetAsyncQueryResultActionResponse getAsyncQueryResultActionResponse) { restChannel.sendResponse( new BytesRestResponse( RestStatus.OK, "application/json; charset=UTF-8", - getJobActionResponse.getResult())); + getAsyncQueryResultActionResponse.getResult())); } @Override @@ -226,22 +180,23 @@ private void handleException(Exception e, RestChannel restChannel) { } private RestChannelConsumer executeDeleteRequest(RestRequest restRequest, NodeClient nodeClient) { - String jobId = restRequest.param("jobId"); + String queryId = restRequest.param("queryId"); return restChannel -> Scheduler.schedule( nodeClient, () -> nodeClient.execute( - TransportDeleteJobRequestAction.ACTION_TYPE, - new DeleteJobActionRequest(jobId), + TransportCancelAsyncQueryRequestAction.ACTION_TYPE, + new CancelAsyncQueryActionRequest(queryId), new ActionListener<>() { @Override - public void onResponse(DeleteJobActionResponse deleteJobActionResponse) { + public void onResponse( + CancelAsyncQueryActionResponse cancelAsyncQueryActionResponse) { restChannel.sendResponse( new BytesRestResponse( RestStatus.OK, "application/json; charset=UTF-8", - deleteJobActionResponse.getResult())); + cancelAsyncQueryActionResponse.getResult())); } @Override diff --git a/spark/src/main/java/org/opensearch/sql/spark/rest/model/CreateJobRequest.java b/spark/src/main/java/org/opensearch/sql/spark/rest/model/CreateAsyncQueryRequest.java similarity index 71% rename from spark/src/main/java/org/opensearch/sql/spark/rest/model/CreateJobRequest.java rename to spark/src/main/java/org/opensearch/sql/spark/rest/model/CreateAsyncQueryRequest.java index ef29e857c8..1e46ae48d2 100644 --- a/spark/src/main/java/org/opensearch/sql/spark/rest/model/CreateJobRequest.java +++ b/spark/src/main/java/org/opensearch/sql/spark/rest/model/CreateAsyncQueryRequest.java @@ -14,22 +14,27 @@ @Data @AllArgsConstructor -public class CreateJobRequest { +public class CreateAsyncQueryRequest { private String query; + private String lang; - public static CreateJobRequest fromXContentParser(XContentParser parser) throws IOException { + public static CreateAsyncQueryRequest fromXContentParser(XContentParser parser) + throws IOException { String query = null; + String lang = null; ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser); while (parser.nextToken() != XContentParser.Token.END_OBJECT) { String fieldName = parser.currentName(); parser.nextToken(); if (fieldName.equals("query")) { query = parser.textOrNull(); + } else if (fieldName.equals("kind")) { + lang = parser.textOrNull(); } else { throw new IllegalArgumentException("Unknown field: " + fieldName); } } - return new CreateJobRequest(query); + return new CreateAsyncQueryRequest(query, lang); } } diff --git a/spark/src/main/java/org/opensearch/sql/spark/rest/model/CreateAsyncQueryResponse.java b/spark/src/main/java/org/opensearch/sql/spark/rest/model/CreateAsyncQueryResponse.java new file mode 100644 index 0000000000..8cfe57c2a6 --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/rest/model/CreateAsyncQueryResponse.java @@ -0,0 +1,15 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.rest.model; + +import lombok.AllArgsConstructor; +import lombok.Data; + +@Data +@AllArgsConstructor +public class CreateAsyncQueryResponse { + private String queryId; +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/TransportCancelAsyncQueryRequestAction.java b/spark/src/main/java/org/opensearch/sql/spark/transport/TransportCancelAsyncQueryRequestAction.java new file mode 100644 index 0000000000..990dbccd0b --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/transport/TransportCancelAsyncQueryRequestAction.java @@ -0,0 +1,41 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.transport; + +import org.opensearch.action.ActionType; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.HandledTransportAction; +import org.opensearch.common.inject.Inject; +import org.opensearch.core.action.ActionListener; +import org.opensearch.sql.spark.transport.model.CancelAsyncQueryActionRequest; +import org.opensearch.sql.spark.transport.model.CancelAsyncQueryActionResponse; +import org.opensearch.tasks.Task; +import org.opensearch.transport.TransportService; + +public class TransportCancelAsyncQueryRequestAction + extends HandledTransportAction { + + public static final String NAME = "cluster:admin/opensearch/ql/async_query/delete"; + public static final ActionType ACTION_TYPE = + new ActionType<>(NAME, CancelAsyncQueryActionResponse::new); + + @Inject + public TransportCancelAsyncQueryRequestAction( + TransportService transportService, ActionFilters actionFilters) { + super(NAME, transportService, actionFilters, CancelAsyncQueryActionRequest::new); + } + + @Override + protected void doExecute( + Task task, + CancelAsyncQueryActionRequest request, + ActionListener listener) { + String responseContent = "deleted_job"; + listener.onResponse(new CancelAsyncQueryActionResponse(responseContent)); + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/TransportCreateAsyncQueryRequestAction.java b/spark/src/main/java/org/opensearch/sql/spark/transport/TransportCreateAsyncQueryRequestAction.java new file mode 100644 index 0000000000..991eafdad9 --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/transport/TransportCreateAsyncQueryRequestAction.java @@ -0,0 +1,64 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.transport; + +import org.opensearch.action.ActionType; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.HandledTransportAction; +import org.opensearch.common.inject.Inject; +import org.opensearch.core.action.ActionListener; +import org.opensearch.sql.protocol.response.format.JsonResponseFormatter; +import org.opensearch.sql.spark.asyncquery.AsyncQueryExecutorService; +import org.opensearch.sql.spark.asyncquery.AsyncQueryExecutorServiceImpl; +import org.opensearch.sql.spark.rest.model.CreateAsyncQueryRequest; +import org.opensearch.sql.spark.rest.model.CreateAsyncQueryResponse; +import org.opensearch.sql.spark.transport.model.CreateAsyncQueryActionRequest; +import org.opensearch.sql.spark.transport.model.CreateAsyncQueryActionResponse; +import org.opensearch.tasks.Task; +import org.opensearch.transport.TransportService; + +public class TransportCreateAsyncQueryRequestAction + extends HandledTransportAction { + + private final AsyncQueryExecutorService asyncQueryExecutorService; + + public static final String NAME = "cluster:admin/opensearch/ql/async_query/create"; + public static final ActionType ACTION_TYPE = + new ActionType<>(NAME, CreateAsyncQueryActionResponse::new); + + @Inject + public TransportCreateAsyncQueryRequestAction( + TransportService transportService, + ActionFilters actionFilters, + AsyncQueryExecutorServiceImpl jobManagementService) { + super(NAME, transportService, actionFilters, CreateAsyncQueryActionRequest::new); + this.asyncQueryExecutorService = jobManagementService; + } + + @Override + protected void doExecute( + Task task, + CreateAsyncQueryActionRequest request, + ActionListener listener) { + try { + CreateAsyncQueryRequest createAsyncQueryRequest = request.getCreateAsyncQueryRequest(); + CreateAsyncQueryResponse createAsyncQueryResponse = + asyncQueryExecutorService.createAsyncQuery(createAsyncQueryRequest); + String responseContent = + new JsonResponseFormatter(JsonResponseFormatter.Style.PRETTY) { + @Override + protected Object buildJsonObject(CreateAsyncQueryResponse response) { + return response; + } + }.format(createAsyncQueryResponse); + listener.onResponse(new CreateAsyncQueryActionResponse(responseContent)); + } catch (Exception e) { + listener.onFailure(e); + } + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/TransportCreateJobRequestAction.java b/spark/src/main/java/org/opensearch/sql/spark/transport/TransportCreateJobRequestAction.java deleted file mode 100644 index 53ae9fad90..0000000000 --- a/spark/src/main/java/org/opensearch/sql/spark/transport/TransportCreateJobRequestAction.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * - * * Copyright OpenSearch Contributors - * * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.opensearch.sql.spark.transport; - -import org.opensearch.action.ActionType; -import org.opensearch.action.support.ActionFilters; -import org.opensearch.action.support.HandledTransportAction; -import org.opensearch.common.inject.Inject; -import org.opensearch.core.action.ActionListener; -import org.opensearch.sql.spark.transport.model.CreateJobActionRequest; -import org.opensearch.sql.spark.transport.model.CreateJobActionResponse; -import org.opensearch.tasks.Task; -import org.opensearch.transport.TransportService; - -public class TransportCreateJobRequestAction - extends HandledTransportAction { - - public static final String NAME = "cluster:admin/opensearch/ql/jobs/create"; - public static final ActionType ACTION_TYPE = - new ActionType<>(NAME, CreateJobActionResponse::new); - - @Inject - public TransportCreateJobRequestAction( - TransportService transportService, ActionFilters actionFilters) { - super(NAME, transportService, actionFilters, CreateJobActionRequest::new); - } - - @Override - protected void doExecute( - Task task, CreateJobActionRequest request, ActionListener listener) { - String responseContent = "submitted_job"; - listener.onResponse(new CreateJobActionResponse(responseContent)); - } -} diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/TransportDeleteJobRequestAction.java b/spark/src/main/java/org/opensearch/sql/spark/transport/TransportDeleteJobRequestAction.java deleted file mode 100644 index dcccb76272..0000000000 --- a/spark/src/main/java/org/opensearch/sql/spark/transport/TransportDeleteJobRequestAction.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * - * * Copyright OpenSearch Contributors - * * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.opensearch.sql.spark.transport; - -import org.opensearch.action.ActionType; -import org.opensearch.action.support.ActionFilters; -import org.opensearch.action.support.HandledTransportAction; -import org.opensearch.common.inject.Inject; -import org.opensearch.core.action.ActionListener; -import org.opensearch.sql.spark.transport.model.DeleteJobActionRequest; -import org.opensearch.sql.spark.transport.model.DeleteJobActionResponse; -import org.opensearch.tasks.Task; -import org.opensearch.transport.TransportService; - -public class TransportDeleteJobRequestAction - extends HandledTransportAction { - - public static final String NAME = "cluster:admin/opensearch/ql/jobs/delete"; - public static final ActionType ACTION_TYPE = - new ActionType<>(NAME, DeleteJobActionResponse::new); - - @Inject - public TransportDeleteJobRequestAction( - TransportService transportService, ActionFilters actionFilters) { - super(NAME, transportService, actionFilters, DeleteJobActionRequest::new); - } - - @Override - protected void doExecute( - Task task, DeleteJobActionRequest request, ActionListener listener) { - String responseContent = "deleted_job"; - listener.onResponse(new DeleteJobActionResponse(responseContent)); - } -} diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/TransportGetAsyncQueryResultAction.java b/spark/src/main/java/org/opensearch/sql/spark/transport/TransportGetAsyncQueryResultAction.java new file mode 100644 index 0000000000..c23706b184 --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/transport/TransportGetAsyncQueryResultAction.java @@ -0,0 +1,70 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.transport; + +import org.opensearch.action.ActionType; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.HandledTransportAction; +import org.opensearch.common.inject.Inject; +import org.opensearch.core.action.ActionListener; +import org.opensearch.sql.executor.pagination.Cursor; +import org.opensearch.sql.protocol.response.format.JsonResponseFormatter; +import org.opensearch.sql.protocol.response.format.ResponseFormatter; +import org.opensearch.sql.spark.asyncquery.AsyncQueryExecutorService; +import org.opensearch.sql.spark.asyncquery.AsyncQueryExecutorServiceImpl; +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryExecutionResponse; +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryResult; +import org.opensearch.sql.spark.transport.format.AsyncQueryResultResponseFormatter; +import org.opensearch.sql.spark.transport.model.GetAsyncQueryResultActionRequest; +import org.opensearch.sql.spark.transport.model.GetAsyncQueryResultActionResponse; +import org.opensearch.tasks.Task; +import org.opensearch.transport.TransportService; + +public class TransportGetAsyncQueryResultAction + extends HandledTransportAction< + GetAsyncQueryResultActionRequest, GetAsyncQueryResultActionResponse> { + + private final AsyncQueryExecutorService asyncQueryExecutorService; + + public static final String NAME = "cluster:admin/opensearch/ql/async_query/result"; + public static final ActionType ACTION_TYPE = + new ActionType<>(NAME, GetAsyncQueryResultActionResponse::new); + + @Inject + public TransportGetAsyncQueryResultAction( + TransportService transportService, + ActionFilters actionFilters, + AsyncQueryExecutorServiceImpl jobManagementService) { + super(NAME, transportService, actionFilters, GetAsyncQueryResultActionRequest::new); + this.asyncQueryExecutorService = jobManagementService; + } + + @Override + protected void doExecute( + Task task, + GetAsyncQueryResultActionRequest request, + ActionListener listener) { + try { + String jobId = request.getQueryId(); + AsyncQueryExecutionResponse asyncQueryExecutionResponse = + asyncQueryExecutorService.getAsyncQueryResults(jobId); + ResponseFormatter formatter = + new AsyncQueryResultResponseFormatter(JsonResponseFormatter.Style.PRETTY); + String responseContent = + formatter.format( + new AsyncQueryResult( + asyncQueryExecutionResponse.getStatus(), + asyncQueryExecutionResponse.getSchema(), + asyncQueryExecutionResponse.getResults(), + Cursor.None)); + listener.onResponse(new GetAsyncQueryResultActionResponse(responseContent)); + } catch (Exception e) { + listener.onFailure(e); + } + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/TransportGetJobRequestAction.java b/spark/src/main/java/org/opensearch/sql/spark/transport/TransportGetJobRequestAction.java deleted file mode 100644 index 96e002bd81..0000000000 --- a/spark/src/main/java/org/opensearch/sql/spark/transport/TransportGetJobRequestAction.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * - * * Copyright OpenSearch Contributors - * * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.opensearch.sql.spark.transport; - -import org.opensearch.action.ActionType; -import org.opensearch.action.support.ActionFilters; -import org.opensearch.action.support.HandledTransportAction; -import org.opensearch.common.inject.Inject; -import org.opensearch.core.action.ActionListener; -import org.opensearch.sql.spark.transport.model.GetJobActionRequest; -import org.opensearch.sql.spark.transport.model.GetJobActionResponse; -import org.opensearch.tasks.Task; -import org.opensearch.transport.TransportService; - -public class TransportGetJobRequestAction - extends HandledTransportAction { - - public static final String NAME = "cluster:admin/opensearch/ql/jobs/read"; - public static final ActionType ACTION_TYPE = - new ActionType<>(NAME, GetJobActionResponse::new); - - @Inject - public TransportGetJobRequestAction( - TransportService transportService, ActionFilters actionFilters) { - super(NAME, transportService, actionFilters, GetJobActionRequest::new); - } - - @Override - protected void doExecute( - Task task, GetJobActionRequest request, ActionListener listener) { - String responseContent; - if (request.getJobId() == null) { - responseContent = handleGetAllJobs(); - } else { - responseContent = handleGetJob(request.getJobId()); - } - listener.onResponse(new GetJobActionResponse(responseContent)); - } - - private String handleGetAllJobs() { - return "All Jobs Information."; - } - - private String handleGetJob(String jobId) { - return String.format("Job %s details.", jobId); - } -} diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/TransportGetQueryResultRequestAction.java b/spark/src/main/java/org/opensearch/sql/spark/transport/TransportGetQueryResultRequestAction.java deleted file mode 100644 index 6aba1b48b6..0000000000 --- a/spark/src/main/java/org/opensearch/sql/spark/transport/TransportGetQueryResultRequestAction.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * - * * Copyright OpenSearch Contributors - * * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.opensearch.sql.spark.transport; - -import org.opensearch.action.ActionType; -import org.opensearch.action.support.ActionFilters; -import org.opensearch.action.support.HandledTransportAction; -import org.opensearch.common.inject.Inject; -import org.opensearch.core.action.ActionListener; -import org.opensearch.sql.spark.transport.model.GetJobQueryResultActionRequest; -import org.opensearch.sql.spark.transport.model.GetJobQueryResultActionResponse; -import org.opensearch.tasks.Task; -import org.opensearch.transport.TransportService; - -public class TransportGetQueryResultRequestAction - extends HandledTransportAction< - GetJobQueryResultActionRequest, GetJobQueryResultActionResponse> { - - public static final String NAME = "cluster:admin/opensearch/ql/jobs/result"; - public static final ActionType ACTION_TYPE = - new ActionType<>(NAME, GetJobQueryResultActionResponse::new); - - @Inject - public TransportGetQueryResultRequestAction( - TransportService transportService, ActionFilters actionFilters) { - super(NAME, transportService, actionFilters, GetJobQueryResultActionRequest::new); - } - - @Override - protected void doExecute( - Task task, - GetJobQueryResultActionRequest request, - ActionListener listener) { - String responseContent = "job result"; - listener.onResponse(new GetJobQueryResultActionResponse(responseContent)); - } -} diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/format/AsyncQueryResultResponseFormatter.java b/spark/src/main/java/org/opensearch/sql/spark/transport/format/AsyncQueryResultResponseFormatter.java new file mode 100644 index 0000000000..c9eb5bbf59 --- /dev/null +++ b/spark/src/main/java/org/opensearch/sql/spark/transport/format/AsyncQueryResultResponseFormatter.java @@ -0,0 +1,90 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.transport.format; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import java.util.List; +import java.util.stream.Collectors; +import lombok.Builder; +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import org.opensearch.sql.protocol.response.QueryResult; +import org.opensearch.sql.protocol.response.format.JsonResponseFormatter; +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryResult; + +/** + * JSON response format with schema header and data rows. For example, + * + *
+ *  {
+ *      "schema": [
+ *          {
+ *              "name": "name",
+ *              "type": "string"
+ *          }
+ *      ],
+ *      "datarows": [
+ *          ["John"],
+ *          ["Smith"]
+ *      ],
+ *      "total": 2,
+ *      "size": 2
+ *  }
+ * 
+ */ +public class AsyncQueryResultResponseFormatter extends JsonResponseFormatter { + + public AsyncQueryResultResponseFormatter(Style style) { + super(style); + } + + @Override + public Object buildJsonObject(AsyncQueryResult response) { + JsonResponse.JsonResponseBuilder json = JsonResponse.builder(); + if (response.getStatus().equalsIgnoreCase("success")) { + json.total(response.size()).size(response.size()); + json.schema( + response.columnNameTypes().entrySet().stream() + .map((entry) -> new Column(entry.getKey(), entry.getValue())) + .collect(Collectors.toList())); + json.datarows(fetchDataRows(response)); + } + json.status(response.getStatus()); + return json.build(); + } + + private Object[][] fetchDataRows(QueryResult response) { + Object[][] rows = new Object[response.size()][]; + int i = 0; + for (Object[] values : response) { + rows[i++] = values; + } + return rows; + } + + /** org.json requires these inner data classes be public (and static) */ + @Builder + @Getter + @JsonIgnoreProperties(ignoreUnknown = true) + public static class JsonResponse { + + private final String status; + + private final List schema; + + private final Object[][] datarows; + + private Integer total; + private Integer size; + } + + @RequiredArgsConstructor + @Getter + public static class Column { + private final String name; + private final String type; + } +} diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/model/DeleteJobActionRequest.java b/spark/src/main/java/org/opensearch/sql/spark/transport/model/CancelAsyncQueryActionRequest.java similarity index 77% rename from spark/src/main/java/org/opensearch/sql/spark/transport/model/DeleteJobActionRequest.java rename to spark/src/main/java/org/opensearch/sql/spark/transport/model/CancelAsyncQueryActionRequest.java index eaf379047a..e12f184efe 100644 --- a/spark/src/main/java/org/opensearch/sql/spark/transport/model/DeleteJobActionRequest.java +++ b/spark/src/main/java/org/opensearch/sql/spark/transport/model/CancelAsyncQueryActionRequest.java @@ -14,12 +14,12 @@ import org.opensearch.core.common.io.stream.StreamInput; @AllArgsConstructor -public class DeleteJobActionRequest extends ActionRequest { +public class CancelAsyncQueryActionRequest extends ActionRequest { - private String jobId; + private String queryId; /** Constructor of SubmitJobActionRequest from StreamInput. */ - public DeleteJobActionRequest(StreamInput in) throws IOException { + public CancelAsyncQueryActionRequest(StreamInput in) throws IOException { super(in); } diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/model/CreateJobActionResponse.java b/spark/src/main/java/org/opensearch/sql/spark/transport/model/CancelAsyncQueryActionResponse.java similarity index 81% rename from spark/src/main/java/org/opensearch/sql/spark/transport/model/CreateJobActionResponse.java rename to spark/src/main/java/org/opensearch/sql/spark/transport/model/CancelAsyncQueryActionResponse.java index ce76d4a20d..af97140b49 100644 --- a/spark/src/main/java/org/opensearch/sql/spark/transport/model/CreateJobActionResponse.java +++ b/spark/src/main/java/org/opensearch/sql/spark/transport/model/CancelAsyncQueryActionResponse.java @@ -15,11 +15,11 @@ import org.opensearch.core.common.io.stream.StreamOutput; @RequiredArgsConstructor -public class CreateJobActionResponse extends ActionResponse { +public class CancelAsyncQueryActionResponse extends ActionResponse { @Getter private final String result; - public CreateJobActionResponse(StreamInput in) throws IOException { + public CancelAsyncQueryActionResponse(StreamInput in) throws IOException { super(in); result = in.readString(); } diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/model/CreateJobActionRequest.java b/spark/src/main/java/org/opensearch/sql/spark/transport/model/CreateAsyncQueryActionRequest.java similarity index 55% rename from spark/src/main/java/org/opensearch/sql/spark/transport/model/CreateJobActionRequest.java rename to spark/src/main/java/org/opensearch/sql/spark/transport/model/CreateAsyncQueryActionRequest.java index cbdcb617af..bcb329b2dc 100644 --- a/spark/src/main/java/org/opensearch/sql/spark/transport/model/CreateJobActionRequest.java +++ b/spark/src/main/java/org/opensearch/sql/spark/transport/model/CreateAsyncQueryActionRequest.java @@ -12,19 +12,19 @@ import org.opensearch.action.ActionRequest; import org.opensearch.action.ActionRequestValidationException; import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.sql.spark.rest.model.CreateJobRequest; +import org.opensearch.sql.spark.rest.model.CreateAsyncQueryRequest; -public class CreateJobActionRequest extends ActionRequest { +public class CreateAsyncQueryActionRequest extends ActionRequest { - @Getter private CreateJobRequest createJobRequest; + @Getter private CreateAsyncQueryRequest createAsyncQueryRequest; /** Constructor of CreateJobActionRequest from StreamInput. */ - public CreateJobActionRequest(StreamInput in) throws IOException { + public CreateAsyncQueryActionRequest(StreamInput in) throws IOException { super(in); } - public CreateJobActionRequest(CreateJobRequest createJobRequest) { - this.createJobRequest = createJobRequest; + public CreateAsyncQueryActionRequest(CreateAsyncQueryRequest createAsyncQueryRequest) { + this.createAsyncQueryRequest = createAsyncQueryRequest; } @Override diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetJobActionResponse.java b/spark/src/main/java/org/opensearch/sql/spark/transport/model/CreateAsyncQueryActionResponse.java similarity index 81% rename from spark/src/main/java/org/opensearch/sql/spark/transport/model/GetJobActionResponse.java rename to spark/src/main/java/org/opensearch/sql/spark/transport/model/CreateAsyncQueryActionResponse.java index f904afdb4e..de5acc2537 100644 --- a/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetJobActionResponse.java +++ b/spark/src/main/java/org/opensearch/sql/spark/transport/model/CreateAsyncQueryActionResponse.java @@ -15,11 +15,11 @@ import org.opensearch.core.common.io.stream.StreamOutput; @RequiredArgsConstructor -public class GetJobActionResponse extends ActionResponse { +public class CreateAsyncQueryActionResponse extends ActionResponse { @Getter private final String result; - public GetJobActionResponse(StreamInput in) throws IOException { + public CreateAsyncQueryActionResponse(StreamInput in) throws IOException { super(in); result = in.readString(); } diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/model/DeleteJobActionResponse.java b/spark/src/main/java/org/opensearch/sql/spark/transport/model/DeleteJobActionResponse.java deleted file mode 100644 index 38be57c21d..0000000000 --- a/spark/src/main/java/org/opensearch/sql/spark/transport/model/DeleteJobActionResponse.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * - * * Copyright OpenSearch Contributors - * * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.opensearch.sql.spark.transport.model; - -import java.io.IOException; -import lombok.Getter; -import lombok.RequiredArgsConstructor; -import org.opensearch.core.action.ActionResponse; -import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.core.common.io.stream.StreamOutput; - -@RequiredArgsConstructor -public class DeleteJobActionResponse extends ActionResponse { - - @Getter private final String result; - - public DeleteJobActionResponse(StreamInput in) throws IOException { - super(in); - result = in.readString(); - } - - @Override - public void writeTo(StreamOutput streamOutput) throws IOException { - streamOutput.writeString(result); - } -} diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetJobQueryResultActionRequest.java b/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetAsyncQueryResultActionRequest.java similarity index 76% rename from spark/src/main/java/org/opensearch/sql/spark/transport/model/GetJobQueryResultActionRequest.java rename to spark/src/main/java/org/opensearch/sql/spark/transport/model/GetAsyncQueryResultActionRequest.java index 1de7bae2c7..06faa75a26 100644 --- a/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetJobQueryResultActionRequest.java +++ b/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetAsyncQueryResultActionRequest.java @@ -15,12 +15,12 @@ import org.opensearch.core.common.io.stream.StreamInput; @AllArgsConstructor -public class GetJobQueryResultActionRequest extends ActionRequest { +public class GetAsyncQueryResultActionRequest extends ActionRequest { - @Getter private String jobId; + @Getter private String queryId; /** Constructor of GetJobQueryResultActionRequest from StreamInput. */ - public GetJobQueryResultActionRequest(StreamInput in) throws IOException { + public GetAsyncQueryResultActionRequest(StreamInput in) throws IOException { super(in); } diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetJobQueryResultActionResponse.java b/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetAsyncQueryResultActionResponse.java similarity index 80% rename from spark/src/main/java/org/opensearch/sql/spark/transport/model/GetJobQueryResultActionResponse.java rename to spark/src/main/java/org/opensearch/sql/spark/transport/model/GetAsyncQueryResultActionResponse.java index a7a8002c67..bb77bb131a 100644 --- a/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetJobQueryResultActionResponse.java +++ b/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetAsyncQueryResultActionResponse.java @@ -15,11 +15,11 @@ import org.opensearch.core.common.io.stream.StreamOutput; @RequiredArgsConstructor -public class GetJobQueryResultActionResponse extends ActionResponse { +public class GetAsyncQueryResultActionResponse extends ActionResponse { @Getter private final String result; - public GetJobQueryResultActionResponse(StreamInput in) throws IOException { + public GetAsyncQueryResultActionResponse(StreamInput in) throws IOException { super(in); result = in.readString(); } diff --git a/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetJobActionRequest.java b/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetJobActionRequest.java deleted file mode 100644 index f8969cde15..0000000000 --- a/spark/src/main/java/org/opensearch/sql/spark/transport/model/GetJobActionRequest.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * - * * Copyright OpenSearch Contributors - * * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.opensearch.sql.spark.transport.model; - -import java.io.IOException; -import lombok.AllArgsConstructor; -import lombok.Getter; -import lombok.NoArgsConstructor; -import org.opensearch.action.ActionRequest; -import org.opensearch.action.ActionRequestValidationException; -import org.opensearch.core.common.io.stream.StreamInput; - -@NoArgsConstructor -@AllArgsConstructor -public class GetJobActionRequest extends ActionRequest { - - @Getter private String jobId; - - /** Constructor of GetJobActionRequest from StreamInput. */ - public GetJobActionRequest(StreamInput in) throws IOException { - super(in); - } - - @Override - public ActionRequestValidationException validate() { - return null; - } -} diff --git a/spark/src/main/resources/job-metadata-index-mapping.yml b/spark/src/main/resources/job-metadata-index-mapping.yml new file mode 100644 index 0000000000..ec2c83a4df --- /dev/null +++ b/spark/src/main/resources/job-metadata-index-mapping.yml @@ -0,0 +1,20 @@ +--- +## +# Copyright OpenSearch Contributors +# SPDX-License-Identifier: Apache-2.0 +## + +# Schema file for the .ql-job-metadata index +# Also "dynamic" is set to "false" so that other fields can be added. +dynamic: false +properties: + jobId: + type: text + fields: + keyword: + type: keyword + applicationId: + type: text + fields: + keyword: + type: keyword \ No newline at end of file diff --git a/spark/src/main/resources/job-metadata-index-settings.yml b/spark/src/main/resources/job-metadata-index-settings.yml new file mode 100644 index 0000000000..be93f4645c --- /dev/null +++ b/spark/src/main/resources/job-metadata-index-settings.yml @@ -0,0 +1,11 @@ +--- +## +# Copyright OpenSearch Contributors +# SPDX-License-Identifier: Apache-2.0 +## + +# Settings file for the .ql-job-metadata index +index: + number_of_shards: "1" + auto_expand_replicas: "0-2" + number_of_replicas: "0" \ No newline at end of file diff --git a/spark/src/test/java/org/opensearch/sql/spark/asyncquery/AsyncQueryExecutorServiceImplTest.java b/spark/src/test/java/org/opensearch/sql/spark/asyncquery/AsyncQueryExecutorServiceImplTest.java new file mode 100644 index 0000000000..cf04278892 --- /dev/null +++ b/spark/src/test/java/org/opensearch/sql/spark/asyncquery/AsyncQueryExecutorServiceImplTest.java @@ -0,0 +1,145 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.asyncquery; + +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.when; +import static org.opensearch.sql.spark.constants.TestConstants.EMRS_APPLICATION_ID; +import static org.opensearch.sql.spark.constants.TestConstants.EMR_JOB_ID; +import static org.opensearch.sql.spark.utils.TestUtils.getJson; + +import com.amazonaws.services.emrserverless.model.JobRunState; +import java.io.IOException; +import java.util.HashMap; +import java.util.Optional; +import org.json.JSONObject; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.sql.common.setting.Settings; +import org.opensearch.sql.spark.asyncquery.exceptions.AsyncQueryNotFoundException; +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryExecutionResponse; +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryJobMetadata; +import org.opensearch.sql.spark.dispatcher.SparkQueryDispatcher; +import org.opensearch.sql.spark.rest.model.CreateAsyncQueryRequest; +import org.opensearch.sql.spark.rest.model.CreateAsyncQueryResponse; + +@ExtendWith(MockitoExtension.class) +public class AsyncQueryExecutorServiceImplTest { + + @Mock private SparkQueryDispatcher sparkQueryDispatcher; + @Mock private AsyncQueryJobMetadataStorageService asyncQueryJobMetadataStorageService; + @Mock private Settings settings; + + @Test + void testCreateAsyncQuery() { + AsyncQueryExecutorServiceImpl jobExecutorService = + new AsyncQueryExecutorServiceImpl( + asyncQueryJobMetadataStorageService, sparkQueryDispatcher, settings); + CreateAsyncQueryRequest createAsyncQueryRequest = + new CreateAsyncQueryRequest("select * from my_glue.default.http_logs", "sql"); + when(settings.getSettingValue(Settings.Key.SPARK_EXECUTION_ENGINE_CONFIG)) + .thenReturn( + "{\"applicationId\":\"00fd775baqpu4g0p\",\"executionRoleARN\":\"arn:aws:iam::270824043731:role/emr-job-execution-role\",\"region\":\"eu-west-1\"}"); + when(sparkQueryDispatcher.dispatch( + "00fd775baqpu4g0p", + "select * from my_glue.default.http_logs", + "arn:aws:iam::270824043731:role/emr-job-execution-role")) + .thenReturn(EMR_JOB_ID); + CreateAsyncQueryResponse createAsyncQueryResponse = + jobExecutorService.createAsyncQuery(createAsyncQueryRequest); + verify(asyncQueryJobMetadataStorageService, times(1)) + .storeJobMetadata(new AsyncQueryJobMetadata(EMR_JOB_ID, "00fd775baqpu4g0p")); + verify(settings, times(1)).getSettingValue(Settings.Key.SPARK_EXECUTION_ENGINE_CONFIG); + verify(sparkQueryDispatcher, times(1)) + .dispatch( + "00fd775baqpu4g0p", + "select * from my_glue.default.http_logs", + "arn:aws:iam::270824043731:role/emr-job-execution-role"); + Assertions.assertEquals(EMR_JOB_ID, createAsyncQueryResponse.getQueryId()); + } + + @Test + void testGetAsyncQueryResultsWithJobNotFoundException() { + AsyncQueryExecutorServiceImpl jobExecutorService = + new AsyncQueryExecutorServiceImpl( + asyncQueryJobMetadataStorageService, sparkQueryDispatcher, settings); + when(asyncQueryJobMetadataStorageService.getJobMetadata(EMR_JOB_ID)) + .thenReturn(Optional.empty()); + AsyncQueryNotFoundException asyncQueryNotFoundException = + Assertions.assertThrows( + AsyncQueryNotFoundException.class, + () -> jobExecutorService.getAsyncQueryResults(EMR_JOB_ID)); + Assertions.assertEquals( + "QueryId: " + EMR_JOB_ID + " not found", asyncQueryNotFoundException.getMessage()); + verifyNoInteractions(sparkQueryDispatcher); + verifyNoInteractions(settings); + } + + @Test + void testGetAsyncQueryResultsWithInProgressJob() { + AsyncQueryExecutorServiceImpl jobExecutorService = + new AsyncQueryExecutorServiceImpl( + asyncQueryJobMetadataStorageService, sparkQueryDispatcher, settings); + when(asyncQueryJobMetadataStorageService.getJobMetadata(EMR_JOB_ID)) + .thenReturn(Optional.of(new AsyncQueryJobMetadata(EMR_JOB_ID, EMRS_APPLICATION_ID))); + JSONObject jobResult = new JSONObject(); + jobResult.put("status", JobRunState.PENDING.toString()); + when(sparkQueryDispatcher.getQueryResponse(EMRS_APPLICATION_ID, EMR_JOB_ID)) + .thenReturn(jobResult); + AsyncQueryExecutionResponse asyncQueryExecutionResponse = + jobExecutorService.getAsyncQueryResults(EMR_JOB_ID); + + Assertions.assertNull(asyncQueryExecutionResponse.getResults()); + Assertions.assertNull(asyncQueryExecutionResponse.getSchema()); + Assertions.assertEquals("PENDING", asyncQueryExecutionResponse.getStatus()); + verifyNoInteractions(settings); + } + + @Test + void testGetAsyncQueryResultsWithSuccessJob() throws IOException { + when(asyncQueryJobMetadataStorageService.getJobMetadata(EMR_JOB_ID)) + .thenReturn(Optional.of(new AsyncQueryJobMetadata(EMR_JOB_ID, EMRS_APPLICATION_ID))); + JSONObject jobResult = new JSONObject(getJson("select_query_response.json")); + jobResult.put("status", JobRunState.SUCCESS.toString()); + when(sparkQueryDispatcher.getQueryResponse(EMRS_APPLICATION_ID, EMR_JOB_ID)) + .thenReturn(jobResult); + + AsyncQueryExecutorServiceImpl jobExecutorService = + new AsyncQueryExecutorServiceImpl( + asyncQueryJobMetadataStorageService, sparkQueryDispatcher, settings); + AsyncQueryExecutionResponse asyncQueryExecutionResponse = + jobExecutorService.getAsyncQueryResults(EMR_JOB_ID); + + Assertions.assertEquals("SUCCESS", asyncQueryExecutionResponse.getStatus()); + Assertions.assertEquals(1, asyncQueryExecutionResponse.getSchema().getColumns().size()); + Assertions.assertEquals( + "1", asyncQueryExecutionResponse.getSchema().getColumns().get(0).getName()); + Assertions.assertEquals( + 1, + ((HashMap) asyncQueryExecutionResponse.getResults().get(0).value()) + .get("1")); + verifyNoInteractions(settings); + } + + @Test + void testGetAsyncQueryResultsWithDisabledExecutionEngine() { + AsyncQueryExecutorService asyncQueryExecutorService = new AsyncQueryExecutorServiceImpl(); + IllegalArgumentException illegalArgumentException = + Assertions.assertThrows( + IllegalArgumentException.class, + () -> asyncQueryExecutorService.getAsyncQueryResults(EMR_JOB_ID)); + Assertions.assertEquals( + "Async Query APIs are disabled as plugins.query.executionengine.spark.config is not" + + " configured in cluster settings. Please configure the setting and restart the domain" + + " to enable Async Query APIs", + illegalArgumentException.getMessage()); + } +} diff --git a/spark/src/test/java/org/opensearch/sql/spark/asyncquery/OpensearchAsyncQueryAsyncQueryJobMetadataStorageServiceTest.java b/spark/src/test/java/org/opensearch/sql/spark/asyncquery/OpensearchAsyncQueryAsyncQueryJobMetadataStorageServiceTest.java new file mode 100644 index 0000000000..fe9da12ef0 --- /dev/null +++ b/spark/src/test/java/org/opensearch/sql/spark/asyncquery/OpensearchAsyncQueryAsyncQueryJobMetadataStorageServiceTest.java @@ -0,0 +1,246 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.asyncquery; + +import static org.opensearch.sql.spark.asyncquery.OpensearchAsyncQueryJobMetadataStorageService.JOB_METADATA_INDEX; +import static org.opensearch.sql.spark.constants.TestConstants.EMRS_APPLICATION_ID; +import static org.opensearch.sql.spark.constants.TestConstants.EMR_JOB_ID; + +import java.util.Optional; +import org.apache.lucene.search.TotalHits; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Answers; +import org.mockito.ArgumentMatchers; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.action.DocWriteResponse; +import org.opensearch.action.admin.indices.create.CreateIndexResponse; +import org.opensearch.action.index.IndexResponse; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.client.Client; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.action.ActionFuture; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.search.SearchHit; +import org.opensearch.search.SearchHits; +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryJobMetadata; + +@ExtendWith(MockitoExtension.class) +public class OpensearchAsyncQueryAsyncQueryJobMetadataStorageServiceTest { + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private Client client; + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private ClusterService clusterService; + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private SearchResponse searchResponse; + + @Mock private ActionFuture searchResponseActionFuture; + @Mock private ActionFuture createIndexResponseActionFuture; + @Mock private ActionFuture indexResponseActionFuture; + @Mock private IndexResponse indexResponse; + @Mock private SearchHit searchHit; + + @InjectMocks + private OpensearchAsyncQueryJobMetadataStorageService opensearchJobMetadataStorageService; + + @Test + public void testStoreJobMetadata() { + + Mockito.when(clusterService.state().routingTable().hasIndex(JOB_METADATA_INDEX)) + .thenReturn(Boolean.FALSE); + Mockito.when(client.admin().indices().create(ArgumentMatchers.any())) + .thenReturn(createIndexResponseActionFuture); + Mockito.when(createIndexResponseActionFuture.actionGet()) + .thenReturn(new CreateIndexResponse(true, true, JOB_METADATA_INDEX)); + Mockito.when(client.index(ArgumentMatchers.any())).thenReturn(indexResponseActionFuture); + Mockito.when(indexResponseActionFuture.actionGet()).thenReturn(indexResponse); + Mockito.when(indexResponse.getResult()).thenReturn(DocWriteResponse.Result.CREATED); + AsyncQueryJobMetadata asyncQueryJobMetadata = + new AsyncQueryJobMetadata(EMR_JOB_ID, EMRS_APPLICATION_ID); + + this.opensearchJobMetadataStorageService.storeJobMetadata(asyncQueryJobMetadata); + + Mockito.verify(client.admin().indices(), Mockito.times(1)).create(ArgumentMatchers.any()); + Mockito.verify(client, Mockito.times(1)).index(ArgumentMatchers.any()); + Mockito.verify(client.threadPool().getThreadContext(), Mockito.times(2)).stashContext(); + } + + @Test + public void testStoreJobMetadataWithOutCreatingIndex() { + Mockito.when(clusterService.state().routingTable().hasIndex(JOB_METADATA_INDEX)) + .thenReturn(Boolean.TRUE); + Mockito.when(client.index(ArgumentMatchers.any())).thenReturn(indexResponseActionFuture); + Mockito.when(indexResponseActionFuture.actionGet()).thenReturn(indexResponse); + Mockito.when(indexResponse.getResult()).thenReturn(DocWriteResponse.Result.CREATED); + AsyncQueryJobMetadata asyncQueryJobMetadata = + new AsyncQueryJobMetadata(EMR_JOB_ID, EMRS_APPLICATION_ID); + + this.opensearchJobMetadataStorageService.storeJobMetadata(asyncQueryJobMetadata); + + Mockito.verify(client.admin().indices(), Mockito.times(0)).create(ArgumentMatchers.any()); + Mockito.verify(client, Mockito.times(1)).index(ArgumentMatchers.any()); + Mockito.verify(client.threadPool().getThreadContext(), Mockito.times(1)).stashContext(); + } + + @Test + public void testStoreJobMetadataWithException() { + + Mockito.when(clusterService.state().routingTable().hasIndex(JOB_METADATA_INDEX)) + .thenReturn(Boolean.FALSE); + Mockito.when(client.admin().indices().create(ArgumentMatchers.any())) + .thenReturn(createIndexResponseActionFuture); + Mockito.when(createIndexResponseActionFuture.actionGet()) + .thenReturn(new CreateIndexResponse(true, true, JOB_METADATA_INDEX)); + Mockito.when(client.index(ArgumentMatchers.any())) + .thenThrow(new RuntimeException("error while indexing")); + + AsyncQueryJobMetadata asyncQueryJobMetadata = + new AsyncQueryJobMetadata(EMR_JOB_ID, EMRS_APPLICATION_ID); + RuntimeException runtimeException = + Assertions.assertThrows( + RuntimeException.class, + () -> this.opensearchJobMetadataStorageService.storeJobMetadata(asyncQueryJobMetadata)); + Assertions.assertEquals( + "java.lang.RuntimeException: error while indexing", runtimeException.getMessage()); + + Mockito.verify(client.admin().indices(), Mockito.times(1)).create(ArgumentMatchers.any()); + Mockito.verify(client, Mockito.times(1)).index(ArgumentMatchers.any()); + Mockito.verify(client.threadPool().getThreadContext(), Mockito.times(2)).stashContext(); + } + + @Test + public void testStoreJobMetadataWithIndexCreationFailed() { + + Mockito.when(clusterService.state().routingTable().hasIndex(JOB_METADATA_INDEX)) + .thenReturn(Boolean.FALSE); + Mockito.when(client.admin().indices().create(ArgumentMatchers.any())) + .thenReturn(createIndexResponseActionFuture); + Mockito.when(createIndexResponseActionFuture.actionGet()) + .thenReturn(new CreateIndexResponse(false, false, JOB_METADATA_INDEX)); + + AsyncQueryJobMetadata asyncQueryJobMetadata = + new AsyncQueryJobMetadata(EMR_JOB_ID, EMRS_APPLICATION_ID); + RuntimeException runtimeException = + Assertions.assertThrows( + RuntimeException.class, + () -> this.opensearchJobMetadataStorageService.storeJobMetadata(asyncQueryJobMetadata)); + Assertions.assertEquals( + "Internal server error while creating.ql-job-metadata index:: " + + "Index creation is not acknowledged.", + runtimeException.getMessage()); + + Mockito.verify(client.admin().indices(), Mockito.times(1)).create(ArgumentMatchers.any()); + Mockito.verify(client.threadPool().getThreadContext(), Mockito.times(1)).stashContext(); + } + + @Test + public void testStoreJobMetadataFailedWithNotFoundResponse() { + + Mockito.when(clusterService.state().routingTable().hasIndex(JOB_METADATA_INDEX)) + .thenReturn(Boolean.FALSE); + Mockito.when(client.admin().indices().create(ArgumentMatchers.any())) + .thenReturn(createIndexResponseActionFuture); + Mockito.when(createIndexResponseActionFuture.actionGet()) + .thenReturn(new CreateIndexResponse(true, true, JOB_METADATA_INDEX)); + Mockito.when(client.index(ArgumentMatchers.any())).thenReturn(indexResponseActionFuture); + Mockito.when(indexResponseActionFuture.actionGet()).thenReturn(indexResponse); + Mockito.when(indexResponse.getResult()).thenReturn(DocWriteResponse.Result.NOT_FOUND); + + AsyncQueryJobMetadata asyncQueryJobMetadata = + new AsyncQueryJobMetadata(EMR_JOB_ID, EMRS_APPLICATION_ID); + RuntimeException runtimeException = + Assertions.assertThrows( + RuntimeException.class, + () -> this.opensearchJobMetadataStorageService.storeJobMetadata(asyncQueryJobMetadata)); + Assertions.assertEquals( + "Saving job metadata information failed with result : not_found", + runtimeException.getMessage()); + + Mockito.verify(client.admin().indices(), Mockito.times(1)).create(ArgumentMatchers.any()); + Mockito.verify(client, Mockito.times(1)).index(ArgumentMatchers.any()); + Mockito.verify(client.threadPool().getThreadContext(), Mockito.times(2)).stashContext(); + } + + @Test + public void testGetJobMetadata() { + Mockito.when(clusterService.state().routingTable().hasIndex(JOB_METADATA_INDEX)) + .thenReturn(true); + Mockito.when(client.search(ArgumentMatchers.any())).thenReturn(searchResponseActionFuture); + Mockito.when(searchResponseActionFuture.actionGet()).thenReturn(searchResponse); + Mockito.when(searchResponse.status()).thenReturn(RestStatus.OK); + Mockito.when(searchResponse.getHits()) + .thenReturn( + new SearchHits( + new SearchHit[] {searchHit}, new TotalHits(21, TotalHits.Relation.EQUAL_TO), 1.0F)); + AsyncQueryJobMetadata asyncQueryJobMetadata = + new AsyncQueryJobMetadata(EMR_JOB_ID, EMRS_APPLICATION_ID); + Mockito.when(searchHit.getSourceAsString()).thenReturn(asyncQueryJobMetadata.toString()); + + Optional jobMetadataOptional = + opensearchJobMetadataStorageService.getJobMetadata(EMR_JOB_ID); + Assertions.assertTrue(jobMetadataOptional.isPresent()); + Assertions.assertEquals(EMR_JOB_ID, jobMetadataOptional.get().getJobId()); + Assertions.assertEquals(EMRS_APPLICATION_ID, jobMetadataOptional.get().getApplicationId()); + } + + @Test + public void testGetJobMetadataWith404SearchResponse() { + Mockito.when(clusterService.state().routingTable().hasIndex(JOB_METADATA_INDEX)) + .thenReturn(true); + Mockito.when(client.search(ArgumentMatchers.any())).thenReturn(searchResponseActionFuture); + Mockito.when(searchResponseActionFuture.actionGet()).thenReturn(searchResponse); + Mockito.when(searchResponse.status()).thenReturn(RestStatus.NOT_FOUND); + + RuntimeException runtimeException = + Assertions.assertThrows( + RuntimeException.class, + () -> opensearchJobMetadataStorageService.getJobMetadata(EMR_JOB_ID)); + Assertions.assertEquals( + "Fetching job metadata information failed with status : NOT_FOUND", + runtimeException.getMessage()); + } + + @Test + public void testGetJobMetadataWithParsingFailed() { + Mockito.when(clusterService.state().routingTable().hasIndex(JOB_METADATA_INDEX)) + .thenReturn(true); + Mockito.when(client.search(ArgumentMatchers.any())).thenReturn(searchResponseActionFuture); + Mockito.when(searchResponseActionFuture.actionGet()).thenReturn(searchResponse); + Mockito.when(searchResponse.status()).thenReturn(RestStatus.OK); + Mockito.when(searchResponse.getHits()) + .thenReturn( + new SearchHits( + new SearchHit[] {searchHit}, new TotalHits(21, TotalHits.Relation.EQUAL_TO), 1.0F)); + Mockito.when(searchHit.getSourceAsString()).thenReturn("..tesJOBs"); + + Assertions.assertThrows( + RuntimeException.class, + () -> opensearchJobMetadataStorageService.getJobMetadata(EMR_JOB_ID)); + } + + @Test + public void testGetJobMetadataWithNoIndex() { + Mockito.when(clusterService.state().routingTable().hasIndex(JOB_METADATA_INDEX)) + .thenReturn(Boolean.FALSE); + Mockito.when(client.admin().indices().create(ArgumentMatchers.any())) + .thenReturn(createIndexResponseActionFuture); + Mockito.when(createIndexResponseActionFuture.actionGet()) + .thenReturn(new CreateIndexResponse(true, true, JOB_METADATA_INDEX)); + Mockito.when(client.index(ArgumentMatchers.any())).thenReturn(indexResponseActionFuture); + + Optional jobMetadata = + opensearchJobMetadataStorageService.getJobMetadata(EMR_JOB_ID); + + Assertions.assertFalse(jobMetadata.isPresent()); + } +} diff --git a/spark/src/test/java/org/opensearch/sql/spark/client/EmrServerlessClientImplTest.java b/spark/src/test/java/org/opensearch/sql/spark/client/EmrServerlessClientImplTest.java new file mode 100644 index 0000000000..36f10cd08b --- /dev/null +++ b/spark/src/test/java/org/opensearch/sql/spark/client/EmrServerlessClientImplTest.java @@ -0,0 +1,48 @@ +/* Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.client; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.when; +import static org.opensearch.sql.spark.constants.TestConstants.EMRS_APPLICATION_ID; +import static org.opensearch.sql.spark.constants.TestConstants.EMRS_EXECUTION_ROLE; +import static org.opensearch.sql.spark.constants.TestConstants.EMRS_JOB_NAME; +import static org.opensearch.sql.spark.constants.TestConstants.QUERY; +import static org.opensearch.sql.spark.constants.TestConstants.SPARK_SUBMIT_PARAMETERS; + +import com.amazonaws.services.emrserverless.AWSEMRServerless; +import com.amazonaws.services.emrserverless.model.GetJobRunResult; +import com.amazonaws.services.emrserverless.model.JobRun; +import com.amazonaws.services.emrserverless.model.StartJobRunResult; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +@ExtendWith(MockitoExtension.class) +public class EmrServerlessClientImplTest { + @Mock private AWSEMRServerless emrServerless; + + @Test + void testStartJobRun() { + StartJobRunResult response = new StartJobRunResult(); + when(emrServerless.startJobRun(any())).thenReturn(response); + + EmrServerlessClientImpl emrServerlessClient = new EmrServerlessClientImpl(emrServerless); + emrServerlessClient.startJobRun( + QUERY, EMRS_JOB_NAME, EMRS_APPLICATION_ID, EMRS_EXECUTION_ROLE, SPARK_SUBMIT_PARAMETERS); + } + + @Test + void testGetJobRunState() { + JobRun jobRun = new JobRun(); + jobRun.setState("Running"); + GetJobRunResult response = new GetJobRunResult(); + response.setJobRun(jobRun); + when(emrServerless.getJobRun(any())).thenReturn(response); + EmrServerlessClientImpl emrServerlessClient = new EmrServerlessClientImpl(emrServerless); + emrServerlessClient.getJobRunResult(EMRS_APPLICATION_ID, "123"); + } +} diff --git a/spark/src/test/java/org/opensearch/sql/spark/constants/TestConstants.java b/spark/src/test/java/org/opensearch/sql/spark/constants/TestConstants.java index 2b1020568a..e455e6a049 100644 --- a/spark/src/test/java/org/opensearch/sql/spark/constants/TestConstants.java +++ b/spark/src/test/java/org/opensearch/sql/spark/constants/TestConstants.java @@ -7,5 +7,12 @@ public class TestConstants { public static final String QUERY = "select 1"; + public static final String TEST_DATASOURCE_NAME = "test_datasource_name"; public static final String EMR_CLUSTER_ID = "j-123456789"; + public static final String EMR_JOB_ID = "job-123xxx"; + public static final String EMRS_APPLICATION_ID = "app-xxxxx"; + public static final String EMRS_EXECUTION_ROLE = "execution_role"; + public static final String EMRS_DATASOURCE_ROLE = "datasource_role"; + public static final String EMRS_JOB_NAME = "job_name"; + public static final String SPARK_SUBMIT_PARAMETERS = "--conf org.flint.sql.SQLJob"; } diff --git a/spark/src/test/java/org/opensearch/sql/spark/dispatcher/SparkQueryDispatcherTest.java b/spark/src/test/java/org/opensearch/sql/spark/dispatcher/SparkQueryDispatcherTest.java new file mode 100644 index 0000000000..800bd59b72 --- /dev/null +++ b/spark/src/test/java/org/opensearch/sql/spark/dispatcher/SparkQueryDispatcherTest.java @@ -0,0 +1,174 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.dispatcher; + +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.when; +import static org.opensearch.sql.spark.constants.TestConstants.EMRS_APPLICATION_ID; +import static org.opensearch.sql.spark.constants.TestConstants.EMRS_EXECUTION_ROLE; +import static org.opensearch.sql.spark.constants.TestConstants.EMR_JOB_ID; +import static org.opensearch.sql.spark.constants.TestConstants.QUERY; + +import com.amazonaws.services.emrserverless.model.GetJobRunResult; +import com.amazonaws.services.emrserverless.model.JobRun; +import com.amazonaws.services.emrserverless.model.JobRunState; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import org.json.JSONObject; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.sql.datasource.DataSourceService; +import org.opensearch.sql.datasource.model.DataSourceMetadata; +import org.opensearch.sql.datasource.model.DataSourceType; +import org.opensearch.sql.spark.client.SparkJobClient; +import org.opensearch.sql.spark.response.JobExecutionResponseReader; + +@ExtendWith(MockitoExtension.class) +public class SparkQueryDispatcherTest { + + @Mock private SparkJobClient sparkJobClient; + @Mock private DataSourceService dataSourceService; + @Mock private JobExecutionResponseReader jobExecutionResponseReader; + + @Test + void testDispatch() { + SparkQueryDispatcher sparkQueryDispatcher = + new SparkQueryDispatcher(sparkJobClient, dataSourceService, jobExecutionResponseReader); + when(sparkJobClient.startJobRun( + QUERY, + "flint-opensearch-query", + EMRS_APPLICATION_ID, + EMRS_EXECUTION_ROLE, + constructExpectedSparkSubmitParameterString())) + .thenReturn(EMR_JOB_ID); + when(dataSourceService.getRawDataSourceMetadata("my_glue")) + .thenReturn(constructMyGlueDataSourceMetadata()); + String jobId = sparkQueryDispatcher.dispatch(EMRS_APPLICATION_ID, QUERY, EMRS_EXECUTION_ROLE); + verify(sparkJobClient, times(1)) + .startJobRun( + QUERY, + "flint-opensearch-query", + EMRS_APPLICATION_ID, + EMRS_EXECUTION_ROLE, + constructExpectedSparkSubmitParameterString()); + Assertions.assertEquals(EMR_JOB_ID, jobId); + } + + @Test + void testDispatchWithWrongURI() { + SparkQueryDispatcher sparkQueryDispatcher = + new SparkQueryDispatcher(sparkJobClient, dataSourceService, jobExecutionResponseReader); + when(dataSourceService.getRawDataSourceMetadata("my_glue")) + .thenReturn(constructMyGlueDataSourceMetadataWithBadURISyntax()); + IllegalArgumentException illegalArgumentException = + Assertions.assertThrows( + IllegalArgumentException.class, + () -> sparkQueryDispatcher.dispatch(EMRS_APPLICATION_ID, QUERY, EMRS_EXECUTION_ROLE)); + Assertions.assertEquals( + "Bad URI in indexstore configuration of the : my_glue datasoure.", + illegalArgumentException.getMessage()); + } + + private DataSourceMetadata constructMyGlueDataSourceMetadata() { + DataSourceMetadata dataSourceMetadata = new DataSourceMetadata(); + dataSourceMetadata.setName("my_glue"); + dataSourceMetadata.setConnector(DataSourceType.S3GLUE); + Map properties = new HashMap<>(); + properties.put("glue.auth.type", "iam_role"); + properties.put( + "glue.auth.role_arn", "arn:aws:iam::924196221507:role/FlintOpensearchServiceRole"); + properties.put( + "glue.indexstore.opensearch.uri", + "https://search-flint-dp-benchmark-cf5crj5mj2kfzvgwdeynkxnefy.eu-west-1.es.amazonaws.com"); + properties.put("glue.indexstore.opensearch.auth", "sigv4"); + properties.put("glue.indexstore.opensearch.region", "eu-west-1"); + dataSourceMetadata.setProperties(properties); + return dataSourceMetadata; + } + + private DataSourceMetadata constructMyGlueDataSourceMetadataWithBadURISyntax() { + DataSourceMetadata dataSourceMetadata = new DataSourceMetadata(); + dataSourceMetadata.setName("my_glue"); + dataSourceMetadata.setConnector(DataSourceType.S3GLUE); + Map properties = new HashMap<>(); + properties.put("glue.auth.type", "iam_role"); + properties.put( + "glue.auth.role_arn", "arn:aws:iam::924196221507:role/FlintOpensearchServiceRole"); + properties.put("glue.indexstore.opensearch.uri", "http://localhost:9090? param"); + properties.put("glue.indexstore.opensearch.auth", "sigv4"); + properties.put("glue.indexstore.opensearch.region", "eu-west-1"); + dataSourceMetadata.setProperties(properties); + return dataSourceMetadata; + } + + @Test + void testGetQueryResponse() { + SparkQueryDispatcher sparkQueryDispatcher = + new SparkQueryDispatcher(sparkJobClient, dataSourceService, jobExecutionResponseReader); + when(sparkJobClient.getJobRunResult(EMRS_APPLICATION_ID, EMR_JOB_ID)) + .thenReturn(new GetJobRunResult().withJobRun(new JobRun().withState(JobRunState.PENDING))); + JSONObject result = sparkQueryDispatcher.getQueryResponse(EMRS_APPLICATION_ID, EMR_JOB_ID); + Assertions.assertEquals("PENDING", result.get("status")); + verifyNoInteractions(jobExecutionResponseReader); + } + + @Test + void testGetQueryResponseWithSuccess() { + SparkQueryDispatcher sparkQueryDispatcher = + new SparkQueryDispatcher(sparkJobClient, dataSourceService, jobExecutionResponseReader); + when(sparkJobClient.getJobRunResult(EMRS_APPLICATION_ID, EMR_JOB_ID)) + .thenReturn(new GetJobRunResult().withJobRun(new JobRun().withState(JobRunState.SUCCESS))); + JSONObject queryResult = new JSONObject(); + queryResult.put("data", "result"); + when(jobExecutionResponseReader.getResultFromOpensearchIndex(EMR_JOB_ID)) + .thenReturn(queryResult); + JSONObject result = sparkQueryDispatcher.getQueryResponse(EMRS_APPLICATION_ID, EMR_JOB_ID); + verify(sparkJobClient, times(1)).getJobRunResult(EMRS_APPLICATION_ID, EMR_JOB_ID); + verify(jobExecutionResponseReader, times(1)).getResultFromOpensearchIndex(EMR_JOB_ID); + Assertions.assertEquals(new HashSet<>(Arrays.asList("data", "status")), result.keySet()); + Assertions.assertEquals("result", result.get("data")); + Assertions.assertEquals("SUCCESS", result.get("status")); + } + + String constructExpectedSparkSubmitParameterString() { + return " --class org.opensearch.sql.FlintJob --conf" + + " spark.hadoop.fs.s3.customAWSCredentialsProvider=com.amazonaws.emr.AssumeRoleAWSCredentialsProvider" + + " --conf" + + " spark.hadoop.aws.catalog.credentials.provider.factory.class=com.amazonaws.glue.catalog.metastore.STSAssumeRoleSessionCredentialsProviderFactory" + + " --conf" + + " spark.jars=s3://flint-data-dp-eu-west-1-beta/code/flint/AWSGlueDataCatalogHiveMetaStoreAuth-1.0.jar,s3://flint-data-dp-eu-west-1-beta/code/flint/flint-catalog.jar" + + " --conf" + + " spark.jars.packages=org.opensearch:opensearch-spark-standalone_2.12:0.1.0-SNAPSHOT" + + " --conf" + + " spark.jars.repositories=https://aws.oss.sonatype.org/content/repositories/snapshots" + + " --conf" + + " spark.emr-serverless.driverEnv.JAVA_HOME=/usr/lib/jvm/java-17-amazon-corretto.x86_64/" + + " --conf spark.executorEnv.JAVA_HOME=/usr/lib/jvm/java-17-amazon-corretto.x86_64/" + + " --conf" + + " spark.datasource.flint.host=search-flint-dp-benchmark-cf5crj5mj2kfzvgwdeynkxnefy.eu-west-1.es.amazonaws.com" + + " --conf spark.datasource.flint.port=-1 --conf" + + " spark.datasource.flint.scheme=https --conf spark.datasource.flint.auth=sigv4 " + + " --conf spark.datasource.flint.region=eu-west-1 --conf" + + " spark.datasource.flint.customAWSCredentialsProvider=com.amazonaws.emr.AssumeRoleAWSCredentialsProvider" + + " --conf spark.sql.extensions=org.opensearch.flint.spark.FlintSparkExtensions " + + " --conf" + + " spark.hadoop.hive.metastore.client.factory.class=com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory" + + " --conf" + + " spark.emr-serverless.driverEnv.ASSUME_ROLE_CREDENTIALS_ROLE_ARN=arn:aws:iam::924196221507:role/FlintOpensearchServiceRole" + + " --conf" + + " spark.executorEnv.ASSUME_ROLE_CREDENTIALS_ROLE_ARN=arn:aws:iam::924196221507:role/FlintOpensearchServiceRole" + + " --conf" + + " spark.hive.metastore.glue.role.arn=arn:aws:iam::924196221507:role/FlintOpensearchServiceRole" + + " --conf spark.sql.catalog.my_glue=org.opensearch.sql.FlintDelegateCatalog "; + } +} diff --git a/spark/src/test/java/org/opensearch/sql/spark/response/AsyncQueryExecutionResponseReaderTest.java b/spark/src/test/java/org/opensearch/sql/spark/response/AsyncQueryExecutionResponseReaderTest.java new file mode 100644 index 0000000000..17305fb905 --- /dev/null +++ b/spark/src/test/java/org/opensearch/sql/spark/response/AsyncQueryExecutionResponseReaderTest.java @@ -0,0 +1,78 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.response; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.when; +import static org.opensearch.sql.spark.constants.TestConstants.EMR_JOB_ID; +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_RESPONSE_BUFFER_INDEX_NAME; + +import java.util.Map; +import org.apache.lucene.search.TotalHits; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.client.Client; +import org.opensearch.common.action.ActionFuture; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.search.SearchHit; +import org.opensearch.search.SearchHits; + +@ExtendWith(MockitoExtension.class) +public class AsyncQueryExecutionResponseReaderTest { + @Mock private Client client; + @Mock private SearchResponse searchResponse; + @Mock private SearchHit searchHit; + @Mock private ActionFuture searchResponseActionFuture; + + @Test + public void testGetResultFromOpensearchIndex() { + when(client.search(any())).thenReturn(searchResponseActionFuture); + when(searchResponseActionFuture.actionGet()).thenReturn(searchResponse); + when(searchResponse.status()).thenReturn(RestStatus.OK); + when(searchResponse.getHits()) + .thenReturn( + new SearchHits( + new SearchHit[] {searchHit}, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F)); + Mockito.when(searchHit.getSourceAsMap()).thenReturn(Map.of("stepId", EMR_JOB_ID)); + JobExecutionResponseReader jobExecutionResponseReader = new JobExecutionResponseReader(client); + assertFalse(jobExecutionResponseReader.getResultFromOpensearchIndex(EMR_JOB_ID).isEmpty()); + } + + @Test + public void testInvalidSearchResponse() { + when(client.search(any())).thenReturn(searchResponseActionFuture); + when(searchResponseActionFuture.actionGet()).thenReturn(searchResponse); + when(searchResponse.status()).thenReturn(RestStatus.NO_CONTENT); + + JobExecutionResponseReader jobExecutionResponseReader = new JobExecutionResponseReader(client); + RuntimeException exception = + assertThrows( + RuntimeException.class, + () -> jobExecutionResponseReader.getResultFromOpensearchIndex(EMR_JOB_ID)); + Assertions.assertEquals( + "Fetching result from " + + SPARK_RESPONSE_BUFFER_INDEX_NAME + + " index failed with status : " + + RestStatus.NO_CONTENT, + exception.getMessage()); + } + + @Test + public void testSearchFailure() { + when(client.search(any())).thenThrow(RuntimeException.class); + JobExecutionResponseReader jobExecutionResponseReader = new JobExecutionResponseReader(client); + assertThrows( + RuntimeException.class, + () -> jobExecutionResponseReader.getResultFromOpensearchIndex(EMR_JOB_ID)); + } +} diff --git a/spark/src/test/java/org/opensearch/sql/spark/response/SparkResponseTest.java b/spark/src/test/java/org/opensearch/sql/spark/response/SparkResponseTest.java index 211561ac72..e234454021 100644 --- a/spark/src/test/java/org/opensearch/sql/spark/response/SparkResponseTest.java +++ b/spark/src/test/java/org/opensearch/sql/spark/response/SparkResponseTest.java @@ -10,7 +10,7 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.when; import static org.opensearch.sql.spark.constants.TestConstants.EMR_CLUSTER_ID; -import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_INDEX_NAME; +import static org.opensearch.sql.spark.data.constants.SparkConstants.SPARK_RESPONSE_BUFFER_INDEX_NAME; import java.util.Map; import org.apache.lucene.search.TotalHits; @@ -69,7 +69,7 @@ public void testInvalidSearchResponse() { assertThrows(RuntimeException.class, () -> sparkResponse.getResultFromOpensearchIndex()); Assertions.assertEquals( "Fetching result from " - + SPARK_INDEX_NAME + + SPARK_RESPONSE_BUFFER_INDEX_NAME + " index failed with status : " + RestStatus.NO_CONTENT, exception.getMessage()); diff --git a/spark/src/test/java/org/opensearch/sql/spark/transport/TransportDeleteJobRequestActionTest.java b/spark/src/test/java/org/opensearch/sql/spark/transport/TransportCancelAsyncQueryRequestActionTest.java similarity index 57% rename from spark/src/test/java/org/opensearch/sql/spark/transport/TransportDeleteJobRequestActionTest.java rename to spark/src/test/java/org/opensearch/sql/spark/transport/TransportCancelAsyncQueryRequestActionTest.java index 828b264343..c560c882c0 100644 --- a/spark/src/test/java/org/opensearch/sql/spark/transport/TransportDeleteJobRequestActionTest.java +++ b/spark/src/test/java/org/opensearch/sql/spark/transport/TransportCancelAsyncQueryRequestActionTest.java @@ -19,35 +19,37 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.action.support.ActionFilters; import org.opensearch.core.action.ActionListener; -import org.opensearch.sql.spark.transport.model.DeleteJobActionRequest; -import org.opensearch.sql.spark.transport.model.DeleteJobActionResponse; +import org.opensearch.sql.spark.transport.model.CancelAsyncQueryActionRequest; +import org.opensearch.sql.spark.transport.model.CancelAsyncQueryActionResponse; import org.opensearch.tasks.Task; import org.opensearch.transport.TransportService; @ExtendWith(MockitoExtension.class) -public class TransportDeleteJobRequestActionTest { +public class TransportCancelAsyncQueryRequestActionTest { @Mock private TransportService transportService; - @Mock private TransportDeleteJobRequestAction action; + @Mock private TransportCancelAsyncQueryRequestAction action; @Mock private Task task; - @Mock private ActionListener actionListener; + @Mock private ActionListener actionListener; - @Captor private ArgumentCaptor deleteJobActionResponseArgumentCaptor; + @Captor + private ArgumentCaptor deleteJobActionResponseArgumentCaptor; @BeforeEach public void setUp() { action = - new TransportDeleteJobRequestAction(transportService, new ActionFilters(new HashSet<>())); + new TransportCancelAsyncQueryRequestAction( + transportService, new ActionFilters(new HashSet<>())); } @Test public void testDoExecute() { - DeleteJobActionRequest request = new DeleteJobActionRequest("jobId"); + CancelAsyncQueryActionRequest request = new CancelAsyncQueryActionRequest("jobId"); action.doExecute(task, request, actionListener); Mockito.verify(actionListener).onResponse(deleteJobActionResponseArgumentCaptor.capture()); - DeleteJobActionResponse deleteJobActionResponse = + CancelAsyncQueryActionResponse cancelAsyncQueryActionResponse = deleteJobActionResponseArgumentCaptor.getValue(); - Assertions.assertEquals("deleted_job", deleteJobActionResponse.getResult()); + Assertions.assertEquals("deleted_job", cancelAsyncQueryActionResponse.getResult()); } } diff --git a/spark/src/test/java/org/opensearch/sql/spark/transport/TransportCreateAsyncQueryRequestActionTest.java b/spark/src/test/java/org/opensearch/sql/spark/transport/TransportCreateAsyncQueryRequestActionTest.java new file mode 100644 index 0000000000..6596a9e820 --- /dev/null +++ b/spark/src/test/java/org/opensearch/sql/spark/transport/TransportCreateAsyncQueryRequestActionTest.java @@ -0,0 +1,88 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.transport; + +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.util.HashSet; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.core.action.ActionListener; +import org.opensearch.sql.spark.asyncquery.AsyncQueryExecutorServiceImpl; +import org.opensearch.sql.spark.rest.model.CreateAsyncQueryRequest; +import org.opensearch.sql.spark.rest.model.CreateAsyncQueryResponse; +import org.opensearch.sql.spark.transport.model.CreateAsyncQueryActionRequest; +import org.opensearch.sql.spark.transport.model.CreateAsyncQueryActionResponse; +import org.opensearch.tasks.Task; +import org.opensearch.transport.TransportService; + +@ExtendWith(MockitoExtension.class) +public class TransportCreateAsyncQueryRequestActionTest { + + @Mock private TransportService transportService; + @Mock private TransportCreateAsyncQueryRequestAction action; + @Mock private AsyncQueryExecutorServiceImpl jobExecutorService; + @Mock private Task task; + @Mock private ActionListener actionListener; + + @Captor + private ArgumentCaptor createJobActionResponseArgumentCaptor; + + @Captor private ArgumentCaptor exceptionArgumentCaptor; + + @BeforeEach + public void setUp() { + action = + new TransportCreateAsyncQueryRequestAction( + transportService, new ActionFilters(new HashSet<>()), jobExecutorService); + } + + @Test + public void testDoExecute() { + CreateAsyncQueryRequest createAsyncQueryRequest = + new CreateAsyncQueryRequest("source = my_glue.default.alb_logs", "sql"); + CreateAsyncQueryActionRequest request = + new CreateAsyncQueryActionRequest(createAsyncQueryRequest); + when(jobExecutorService.createAsyncQuery(createAsyncQueryRequest)) + .thenReturn(new CreateAsyncQueryResponse("123")); + action.doExecute(task, request, actionListener); + Mockito.verify(actionListener).onResponse(createJobActionResponseArgumentCaptor.capture()); + CreateAsyncQueryActionResponse createAsyncQueryActionResponse = + createJobActionResponseArgumentCaptor.getValue(); + Assertions.assertEquals( + "{\n" + " \"queryId\": \"123\"\n" + "}", createAsyncQueryActionResponse.getResult()); + } + + @Test + public void testDoExecuteWithException() { + CreateAsyncQueryRequest createAsyncQueryRequest = + new CreateAsyncQueryRequest("source = my_glue.default.alb_logs", "sql"); + CreateAsyncQueryActionRequest request = + new CreateAsyncQueryActionRequest(createAsyncQueryRequest); + doThrow(new RuntimeException("Error")) + .when(jobExecutorService) + .createAsyncQuery(createAsyncQueryRequest); + action.doExecute(task, request, actionListener); + verify(jobExecutorService, times(1)).createAsyncQuery(createAsyncQueryRequest); + Mockito.verify(actionListener).onFailure(exceptionArgumentCaptor.capture()); + Exception exception = exceptionArgumentCaptor.getValue(); + Assertions.assertTrue(exception instanceof RuntimeException); + Assertions.assertEquals("Error", exception.getMessage()); + } +} diff --git a/spark/src/test/java/org/opensearch/sql/spark/transport/TransportCreateJobRequestActionTest.java b/spark/src/test/java/org/opensearch/sql/spark/transport/TransportCreateJobRequestActionTest.java deleted file mode 100644 index 4357899368..0000000000 --- a/spark/src/test/java/org/opensearch/sql/spark/transport/TransportCreateJobRequestActionTest.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * - * * Copyright OpenSearch Contributors - * * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.opensearch.sql.spark.transport; - -import java.util.HashSet; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Captor; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.action.support.ActionFilters; -import org.opensearch.core.action.ActionListener; -import org.opensearch.sql.spark.rest.model.CreateJobRequest; -import org.opensearch.sql.spark.transport.model.CreateJobActionRequest; -import org.opensearch.sql.spark.transport.model.CreateJobActionResponse; -import org.opensearch.tasks.Task; -import org.opensearch.transport.TransportService; - -@ExtendWith(MockitoExtension.class) -public class TransportCreateJobRequestActionTest { - - @Mock private TransportService transportService; - @Mock private TransportCreateJobRequestAction action; - @Mock private Task task; - @Mock private ActionListener actionListener; - - @Captor private ArgumentCaptor createJobActionResponseArgumentCaptor; - - @BeforeEach - public void setUp() { - action = - new TransportCreateJobRequestAction(transportService, new ActionFilters(new HashSet<>())); - } - - @Test - public void testDoExecute() { - CreateJobRequest createJobRequest = new CreateJobRequest("source = my_glue.default.alb_logs"); - CreateJobActionRequest request = new CreateJobActionRequest(createJobRequest); - - action.doExecute(task, request, actionListener); - Mockito.verify(actionListener).onResponse(createJobActionResponseArgumentCaptor.capture()); - CreateJobActionResponse createJobActionResponse = - createJobActionResponseArgumentCaptor.getValue(); - Assertions.assertEquals("submitted_job", createJobActionResponse.getResult()); - } -} diff --git a/spark/src/test/java/org/opensearch/sql/spark/transport/TransportGetAsyncQueryResultActionTest.java b/spark/src/test/java/org/opensearch/sql/spark/transport/TransportGetAsyncQueryResultActionTest.java new file mode 100644 index 0000000000..9e4cd75165 --- /dev/null +++ b/spark/src/test/java/org/opensearch/sql/spark/transport/TransportGetAsyncQueryResultActionTest.java @@ -0,0 +1,139 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.transport; + +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.opensearch.sql.data.model.ExprValueUtils.tupleValue; +import static org.opensearch.sql.data.type.ExprCoreType.INTEGER; +import static org.opensearch.sql.data.type.ExprCoreType.STRING; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.util.Arrays; +import java.util.HashSet; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.core.action.ActionListener; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.spark.asyncquery.AsyncQueryExecutorServiceImpl; +import org.opensearch.sql.spark.asyncquery.exceptions.AsyncQueryNotFoundException; +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryExecutionResponse; +import org.opensearch.sql.spark.transport.model.GetAsyncQueryResultActionRequest; +import org.opensearch.sql.spark.transport.model.GetAsyncQueryResultActionResponse; +import org.opensearch.tasks.Task; +import org.opensearch.transport.TransportService; + +@ExtendWith(MockitoExtension.class) +public class TransportGetAsyncQueryResultActionTest { + + @Mock private TransportService transportService; + @Mock private TransportGetAsyncQueryResultAction action; + @Mock private Task task; + @Mock private ActionListener actionListener; + @Mock private AsyncQueryExecutorServiceImpl jobExecutorService; + + @Captor + private ArgumentCaptor createJobActionResponseArgumentCaptor; + + @Captor private ArgumentCaptor exceptionArgumentCaptor; + + @BeforeEach + public void setUp() { + action = + new TransportGetAsyncQueryResultAction( + transportService, new ActionFilters(new HashSet<>()), jobExecutorService); + } + + @Test + public void testDoExecute() { + GetAsyncQueryResultActionRequest request = new GetAsyncQueryResultActionRequest("jobId"); + AsyncQueryExecutionResponse asyncQueryExecutionResponse = + new AsyncQueryExecutionResponse("IN_PROGRESS", null, null); + when(jobExecutorService.getAsyncQueryResults("jobId")).thenReturn(asyncQueryExecutionResponse); + action.doExecute(task, request, actionListener); + verify(actionListener).onResponse(createJobActionResponseArgumentCaptor.capture()); + GetAsyncQueryResultActionResponse getAsyncQueryResultActionResponse = + createJobActionResponseArgumentCaptor.getValue(); + Assertions.assertEquals( + "{\n" + " \"status\": \"IN_PROGRESS\"\n" + "}", + getAsyncQueryResultActionResponse.getResult()); + } + + @Test + public void testDoExecuteWithSuccessResponse() { + GetAsyncQueryResultActionRequest request = new GetAsyncQueryResultActionRequest("jobId"); + ExecutionEngine.Schema schema = + new ExecutionEngine.Schema( + ImmutableList.of( + new ExecutionEngine.Schema.Column("name", "name", STRING), + new ExecutionEngine.Schema.Column("age", "age", INTEGER))); + AsyncQueryExecutionResponse asyncQueryExecutionResponse = + new AsyncQueryExecutionResponse( + "SUCCESS", + schema, + Arrays.asList( + tupleValue(ImmutableMap.of("name", "John", "age", 20)), + tupleValue(ImmutableMap.of("name", "Smith", "age", 30)))); + when(jobExecutorService.getAsyncQueryResults("jobId")).thenReturn(asyncQueryExecutionResponse); + action.doExecute(task, request, actionListener); + verify(actionListener).onResponse(createJobActionResponseArgumentCaptor.capture()); + GetAsyncQueryResultActionResponse getAsyncQueryResultActionResponse = + createJobActionResponseArgumentCaptor.getValue(); + Assertions.assertEquals( + "{\n" + + " \"status\": \"SUCCESS\",\n" + + " \"schema\": [\n" + + " {\n" + + " \"name\": \"name\",\n" + + " \"type\": \"string\"\n" + + " },\n" + + " {\n" + + " \"name\": \"age\",\n" + + " \"type\": \"integer\"\n" + + " }\n" + + " ],\n" + + " \"datarows\": [\n" + + " [\n" + + " \"John\",\n" + + " 20\n" + + " ],\n" + + " [\n" + + " \"Smith\",\n" + + " 30\n" + + " ]\n" + + " ],\n" + + " \"total\": 2,\n" + + " \"size\": 2\n" + + "}", + getAsyncQueryResultActionResponse.getResult()); + } + + @Test + public void testDoExecuteWithException() { + GetAsyncQueryResultActionRequest request = new GetAsyncQueryResultActionRequest("123"); + doThrow(new AsyncQueryNotFoundException("JobId 123 not found")) + .when(jobExecutorService) + .getAsyncQueryResults("123"); + action.doExecute(task, request, actionListener); + verify(jobExecutorService, times(1)).getAsyncQueryResults("123"); + verify(actionListener).onFailure(exceptionArgumentCaptor.capture()); + Exception exception = exceptionArgumentCaptor.getValue(); + Assertions.assertTrue(exception instanceof RuntimeException); + Assertions.assertEquals("JobId 123 not found", exception.getMessage()); + } +} diff --git a/spark/src/test/java/org/opensearch/sql/spark/transport/TransportGetJobRequestActionTest.java b/spark/src/test/java/org/opensearch/sql/spark/transport/TransportGetJobRequestActionTest.java deleted file mode 100644 index 06d1ee8baf..0000000000 --- a/spark/src/test/java/org/opensearch/sql/spark/transport/TransportGetJobRequestActionTest.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * - * * Copyright OpenSearch Contributors - * * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.opensearch.sql.spark.transport; - -import java.util.HashSet; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Captor; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.action.support.ActionFilters; -import org.opensearch.core.action.ActionListener; -import org.opensearch.sql.spark.transport.model.GetJobActionRequest; -import org.opensearch.sql.spark.transport.model.GetJobActionResponse; -import org.opensearch.tasks.Task; -import org.opensearch.transport.TransportService; - -@ExtendWith(MockitoExtension.class) -public class TransportGetJobRequestActionTest { - - @Mock private TransportService transportService; - @Mock private TransportGetJobRequestAction action; - @Mock private Task task; - @Mock private ActionListener actionListener; - - @Captor private ArgumentCaptor getJobActionResponseArgumentCaptor; - - @BeforeEach - public void setUp() { - action = new TransportGetJobRequestAction(transportService, new ActionFilters(new HashSet<>())); - } - - @Test - public void testDoExecuteWithSingleJob() { - GetJobActionRequest request = new GetJobActionRequest("abcd"); - - action.doExecute(task, request, actionListener); - Mockito.verify(actionListener).onResponse(getJobActionResponseArgumentCaptor.capture()); - GetJobActionResponse getJobActionResponse = getJobActionResponseArgumentCaptor.getValue(); - Assertions.assertEquals("Job abcd details.", getJobActionResponse.getResult()); - } - - @Test - public void testDoExecuteWithAllJobs() { - GetJobActionRequest request = new GetJobActionRequest(); - action.doExecute(task, request, actionListener); - Mockito.verify(actionListener).onResponse(getJobActionResponseArgumentCaptor.capture()); - GetJobActionResponse getJobActionResponse = getJobActionResponseArgumentCaptor.getValue(); - Assertions.assertEquals("All Jobs Information.", getJobActionResponse.getResult()); - } -} diff --git a/spark/src/test/java/org/opensearch/sql/spark/transport/TransportGetQueryResultRequestActionTest.java b/spark/src/test/java/org/opensearch/sql/spark/transport/TransportGetQueryResultRequestActionTest.java deleted file mode 100644 index f22adead49..0000000000 --- a/spark/src/test/java/org/opensearch/sql/spark/transport/TransportGetQueryResultRequestActionTest.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * - * * Copyright OpenSearch Contributors - * * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.opensearch.sql.spark.transport; - -import java.util.HashSet; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Captor; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.action.support.ActionFilters; -import org.opensearch.core.action.ActionListener; -import org.opensearch.sql.spark.transport.model.GetJobQueryResultActionRequest; -import org.opensearch.sql.spark.transport.model.GetJobQueryResultActionResponse; -import org.opensearch.tasks.Task; -import org.opensearch.transport.TransportService; - -@ExtendWith(MockitoExtension.class) -public class TransportGetQueryResultRequestActionTest { - - @Mock private TransportService transportService; - @Mock private TransportGetQueryResultRequestAction action; - @Mock private Task task; - @Mock private ActionListener actionListener; - - @Captor - private ArgumentCaptor createJobActionResponseArgumentCaptor; - - @BeforeEach - public void setUp() { - action = - new TransportGetQueryResultRequestAction( - transportService, new ActionFilters(new HashSet<>())); - } - - @Test - public void testDoExecuteForSingleJob() { - GetJobQueryResultActionRequest request = new GetJobQueryResultActionRequest("jobId"); - action.doExecute(task, request, actionListener); - Mockito.verify(actionListener).onResponse(createJobActionResponseArgumentCaptor.capture()); - GetJobQueryResultActionResponse getJobQueryResultActionResponse = - createJobActionResponseArgumentCaptor.getValue(); - Assertions.assertEquals("job result", getJobQueryResultActionResponse.getResult()); - } -} diff --git a/spark/src/test/java/org/opensearch/sql/spark/transport/format/AsyncQueryResultResponseFormatterTest.java b/spark/src/test/java/org/opensearch/sql/spark/transport/format/AsyncQueryResultResponseFormatterTest.java new file mode 100644 index 0000000000..5ba5627665 --- /dev/null +++ b/spark/src/test/java/org/opensearch/sql/spark/transport/format/AsyncQueryResultResponseFormatterTest.java @@ -0,0 +1,40 @@ +package org.opensearch.sql.spark.transport.format; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.opensearch.sql.data.model.ExprValueUtils.tupleValue; +import static org.opensearch.sql.data.type.ExprCoreType.INTEGER; +import static org.opensearch.sql.data.type.ExprCoreType.STRING; +import static org.opensearch.sql.protocol.response.format.JsonResponseFormatter.Style.COMPACT; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.util.Arrays; +import org.junit.jupiter.api.Test; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.spark.asyncquery.model.AsyncQueryResult; + +public class AsyncQueryResultResponseFormatterTest { + + private final ExecutionEngine.Schema schema = + new ExecutionEngine.Schema( + ImmutableList.of( + new ExecutionEngine.Schema.Column("firstname", null, STRING), + new ExecutionEngine.Schema.Column("age", null, INTEGER))); + + @Test + void formatAsyncQueryResponse() { + AsyncQueryResult response = + new AsyncQueryResult( + "success", + schema, + Arrays.asList( + tupleValue(ImmutableMap.of("firstname", "John", "age", 20)), + tupleValue(ImmutableMap.of("firstname", "Smith", "age", 30)))); + AsyncQueryResultResponseFormatter formatter = new AsyncQueryResultResponseFormatter(COMPACT); + assertEquals( + "{\"status\":\"success\",\"schema\":[{\"name\":\"firstname\",\"type\":\"string\"}," + + "{\"name\":\"age\",\"type\":\"integer\"}],\"datarows\":" + + "[[\"John\",20],[\"Smith\",30]],\"total\":2,\"size\":2}", + formatter.format(response)); + } +}