From 006bf1e976ec0cd0919d248f494fe5ca09d28854 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Thu, 27 Jun 2024 15:51:46 -0700 Subject: [PATCH 01/18] [Feature] Flint query scheduler part1 - integrate job scheduler plugin Signed-off-by: Louis Chu --- .../src/main/antlr/SqlBaseParser.g4 | 8 + async-query/build.gradle | 2 + .../OpenSearchAsyncQueryScheduler.java | 237 ++++++++++++++++++ .../scheduler/SampleExtensionRestHandler.java | 133 ++++++++++ .../AsyncQuerySchedulerException.java | 22 ++ .../job/OpenSearchRefreshIndexJob.java | 124 +++++++++ .../OpenSearchRefreshIndexJobRequest.java | 108 ++++++++ .../async-query-scheduler-index-mapping.yml | 41 +++ .../async-query-scheduler-index-settings.yml | 11 + build.gradle | 2 +- common/build.gradle | 4 +- core/build.gradle | 2 +- integ-test/build.gradle | 2 +- legacy/build.gradle | 2 +- plugin/build.gradle | 11 +- .../org/opensearch/sql/plugin/SQLPlugin.java | 32 ++- ...rch.jobscheduler.spi.JobSchedulerExtension | 6 + ppl/build.gradle | 2 +- protocol/build.gradle | 2 +- sql/build.gradle | 2 +- 20 files changed, 740 insertions(+), 13 deletions(-) create mode 100644 async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java create mode 100644 async-query/src/main/java/org/opensearch/sql/spark/scheduler/SampleExtensionRestHandler.java create mode 100644 async-query/src/main/java/org/opensearch/sql/spark/scheduler/exceptions/AsyncQuerySchedulerException.java create mode 100644 async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java create mode 100644 async-query/src/main/java/org/opensearch/sql/spark/scheduler/model/OpenSearchRefreshIndexJobRequest.java create mode 100644 async-query/src/main/resources/async-query-scheduler-index-mapping.yml create mode 100644 async-query/src/main/resources/async-query-scheduler-index-settings.yml create mode 100644 plugin/src/main/resources/META-INF/services/org.opensearch.jobscheduler.spi.JobSchedulerExtension diff --git a/async-query-core/src/main/antlr/SqlBaseParser.g4 b/async-query-core/src/main/antlr/SqlBaseParser.g4 index a50051715e..2d748642f0 100644 --- a/async-query-core/src/main/antlr/SqlBaseParser.g4 +++ b/async-query-core/src/main/antlr/SqlBaseParser.g4 @@ -83,6 +83,14 @@ endLabel : multipartIdentifier ; +beginLabel + : multipartIdentifier COLON + ; + +endLabel + : multipartIdentifier + ; + singleExpression : namedExpression EOF ; diff --git a/async-query/build.gradle b/async-query/build.gradle index 5a4a0d729d..f4907b91c0 100644 --- a/async-query/build.gradle +++ b/async-query/build.gradle @@ -16,6 +16,8 @@ repositories { dependencies { + compileOnly "org.opensearch:opensearch-job-scheduler-spi:${opensearch_build}" + api project(':core') api project(':async-query-core') implementation project(':protocol') diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java new file mode 100644 index 0000000000..8d0abb5aeb --- /dev/null +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java @@ -0,0 +1,237 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.scheduler; + +import static org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.time.Instant; +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.delete.DeleteRequest; +import org.opensearch.action.delete.DeleteResponse; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.index.IndexResponse; +import org.opensearch.action.support.WriteRequest; +import org.opensearch.action.update.UpdateRequest; +import org.opensearch.action.update.UpdateResponse; +import org.opensearch.client.Client; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.action.ActionFuture; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.core.xcontent.XContentParserUtils; +import org.opensearch.index.engine.VersionConflictEngineException; +import org.opensearch.jobscheduler.spi.ScheduledJobParser; +import org.opensearch.jobscheduler.spi.ScheduledJobRunner; +import org.opensearch.jobscheduler.spi.schedule.ScheduleParser; +import org.opensearch.sql.spark.scheduler.exceptions.AsyncQuerySchedulerException; +import org.opensearch.sql.spark.scheduler.job.OpenSearchRefreshIndexJob; +import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; +import org.opensearch.threadpool.ThreadPool; + +public class OpenSearchAsyncQueryScheduler { + public static final String SCHEDULER_INDEX_NAME = ".async-query-scheduler"; + public static final String SCHEDULER_PLUGIN_JOB_TYPE = "async-query-scheduler"; + private static final String SCHEDULER_INDEX_MAPPING_FILE_NAME = + "async-query-scheduler-index-mapping.yml"; + private static final String SCHEDULER_INDEX_SETTINGS_FILE_NAME = + "async-query-scheduler-index-settings.yml"; + private static final Logger LOG = LogManager.getLogger(); + private Client client; + private ClusterService clusterService; + + public void loadJobResource(Client client, ClusterService clusterService, ThreadPool threadPool) { + this.client = client; + this.clusterService = clusterService; + OpenSearchRefreshIndexJob openSearchRefreshIndexJob = + OpenSearchRefreshIndexJob.getJobRunnerInstance(); + openSearchRefreshIndexJob.setClusterService(clusterService); + openSearchRefreshIndexJob.setThreadPool(threadPool); + openSearchRefreshIndexJob.setClient(client); + } + + public void scheduleJob(OpenSearchRefreshIndexJobRequest request) { + if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { + createAsyncQuerySchedulerIndex(); + } + IndexRequest indexRequest = new IndexRequest(SCHEDULER_INDEX_NAME); + indexRequest.id(request.getName()); + indexRequest.opType(DocWriteRequest.OpType.CREATE); + indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + ActionFuture indexResponseActionFuture; + IndexResponse indexResponse; + try { + indexRequest.source(request.toXContent(JsonXContent.contentBuilder(), EMPTY_PARAMS)); + indexResponseActionFuture = client.index(indexRequest); + indexResponse = indexResponseActionFuture.actionGet(); + } catch (VersionConflictEngineException exception) { + throw new IllegalArgumentException("A job already exists with name: " + request.getName()); + } catch (Exception e) { + throw new AsyncQuerySchedulerException(e); + } + + if (indexResponse.getResult().equals(DocWriteResponse.Result.CREATED)) { + LOG.debug("Job : {} successfully created", request.getName()); + } else { + throw new AsyncQuerySchedulerException( + "Schedule job failed with result : " + indexResponse.getResult().getLowercase()); + } + } + + public void unscheduleJob(String jobId) throws IOException { + if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { + throw new IllegalArgumentException("Job index does not exist."); + } + OpenSearchRefreshIndexJobRequest request = + OpenSearchRefreshIndexJobRequest.builder() + .jobName(jobId) + .enabled(false) + .lastUpdateTime(Instant.now()) + .build(); + updateJob(request); + } + + public void updateJob(OpenSearchRefreshIndexJobRequest request) throws IOException { + if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { + throw new IllegalArgumentException("Job index does not exist."); + } + UpdateRequest updateRequest = new UpdateRequest(SCHEDULER_INDEX_NAME, request.getName()); + updateRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + updateRequest.doc(request.toXContent(JsonXContent.contentBuilder(), EMPTY_PARAMS)); + + ActionFuture updateResponseActionFuture = client.update(updateRequest); + UpdateResponse updateResponse = updateResponseActionFuture.actionGet(); + + if (updateResponse.getResult().equals(DocWriteResponse.Result.UPDATED) + || updateResponse.getResult().equals(DocWriteResponse.Result.NOOP)) { + LOG.debug("Job : {} successfully updated", request.getName()); + } else { + throw new AsyncQuerySchedulerException( + "Update job failed with result : " + updateResponse.getResult().getLowercase()); + } + } + + public void removeJob(String jobId) { + if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { + throw new IllegalArgumentException("Job index does not exist."); + } + DeleteRequest deleteRequest = new DeleteRequest(SCHEDULER_INDEX_NAME, jobId); + deleteRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + ActionFuture deleteResponseActionFuture = client.delete(deleteRequest); + DeleteResponse deleteResponse = deleteResponseActionFuture.actionGet(); + + if (deleteResponse.getResult().equals(DocWriteResponse.Result.DELETED)) { + LOG.debug("Job : {} successfully deleted", jobId); + } else if (deleteResponse.getResult().equals(DocWriteResponse.Result.NOT_FOUND)) { + throw new AsyncQuerySchedulerException("Job : " + jobId + " doesn't exist"); + } else { + throw new AsyncQuerySchedulerException( + "Remove job failed with result : " + deleteResponse.getResult().getLowercase()); + } + } + + private void createAsyncQuerySchedulerIndex() { + try { + InputStream mappingFileStream = + OpenSearchAsyncQueryScheduler.class + .getClassLoader() + .getResourceAsStream(SCHEDULER_INDEX_MAPPING_FILE_NAME); + InputStream settingsFileStream = + OpenSearchAsyncQueryScheduler.class + .getClassLoader() + .getResourceAsStream(SCHEDULER_INDEX_SETTINGS_FILE_NAME); + CreateIndexRequest createIndexRequest = new CreateIndexRequest(SCHEDULER_INDEX_NAME); + createIndexRequest.mapping( + IOUtils.toString(mappingFileStream, StandardCharsets.UTF_8), XContentType.YAML); + createIndexRequest.settings( + IOUtils.toString(settingsFileStream, StandardCharsets.UTF_8), XContentType.YAML); + ActionFuture createIndexResponseActionFuture = + client.admin().indices().create(createIndexRequest); + CreateIndexResponse createIndexResponse = createIndexResponseActionFuture.actionGet(); + + if (createIndexResponse.isAcknowledged()) { + LOG.debug("Index: {} creation Acknowledged", SCHEDULER_INDEX_NAME); + } else { + throw new AsyncQuerySchedulerException("Index creation is not acknowledged."); + } + } catch (Throwable e) { + LOG.error("Error creating index: {}", SCHEDULER_INDEX_NAME, e); + throw new AsyncQuerySchedulerException( + "Internal server error while creating " + + SCHEDULER_INDEX_NAME + + " index: " + + e.getMessage(), + e); + } + } + + public static ScheduledJobRunner getJobRunner() { + return OpenSearchRefreshIndexJob.getJobRunnerInstance(); + } + + public static ScheduledJobParser getJobParser() { + return (parser, id, jobDocVersion) -> { + OpenSearchRefreshIndexJobRequest.OpenSearchRefreshIndexJobRequestBuilder builder = + OpenSearchRefreshIndexJobRequest.builder(); + XContentParserUtils.ensureExpectedToken( + XContentParser.Token.START_OBJECT, parser.nextToken(), parser); + + while (!parser.nextToken().equals(XContentParser.Token.END_OBJECT)) { + String fieldName = parser.currentName(); + parser.nextToken(); + switch (fieldName) { + case OpenSearchRefreshIndexJobRequest.JOB_NAME_FIELD: + builder.jobName(parser.text()); + break; + case OpenSearchRefreshIndexJobRequest.JOB_TYPE_FIELD: + builder.jobType(parser.text()); + break; + case OpenSearchRefreshIndexJobRequest.ENABLED_FIELD: + builder.enabled(parser.booleanValue()); + break; + case OpenSearchRefreshIndexJobRequest.ENABLED_TIME_FIELD: + builder.enabledTime(parseInstantValue(parser)); + break; + case OpenSearchRefreshIndexJobRequest.LAST_UPDATE_TIME_FIELD: + builder.lastUpdateTime(parseInstantValue(parser)); + break; + case OpenSearchRefreshIndexJobRequest.SCHEDULE_FIELD: + builder.schedule(ScheduleParser.parse(parser)); + break; + case OpenSearchRefreshIndexJobRequest.LOCK_DURATION_SECONDS: + builder.lockDurationSeconds(parser.longValue()); + break; + case OpenSearchRefreshIndexJobRequest.JITTER: + builder.jitter(parser.doubleValue()); + break; + default: + XContentParserUtils.throwUnknownToken(parser.currentToken(), parser.getTokenLocation()); + } + } + return builder.build(); + }; + } + + private static Instant parseInstantValue(XContentParser parser) throws IOException { + if (XContentParser.Token.VALUE_NULL.equals(parser.currentToken())) { + return null; + } + if (parser.currentToken().isValue()) { + return Instant.ofEpochMilli(parser.longValue()); + } + XContentParserUtils.throwUnknownToken(parser.currentToken(), parser.getTokenLocation()); + return null; + } +} diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/SampleExtensionRestHandler.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/SampleExtensionRestHandler.java new file mode 100644 index 0000000000..4dcee97499 --- /dev/null +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/SampleExtensionRestHandler.java @@ -0,0 +1,133 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.scheduler; + +import static org.opensearch.core.rest.RestStatus.BAD_REQUEST; +import static org.opensearch.core.rest.RestStatus.OK; + +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.client.node.NodeClient; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.jobscheduler.spi.schedule.IntervalSchedule; +import org.opensearch.rest.BaseRestHandler; +import org.opensearch.rest.BytesRestResponse; +import org.opensearch.rest.RestRequest; +import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; + +/** A sample rest handler that supports schedule and deschedule job operation */ +public class SampleExtensionRestHandler extends BaseRestHandler { + private static final Logger LOG = LogManager.getLogger(SampleExtensionRestHandler.class); + + public static final String WATCH_INDEX_URI = "/_plugins/scheduler_sample/watch"; + private final OpenSearchAsyncQueryScheduler scheduler; + + public SampleExtensionRestHandler(OpenSearchAsyncQueryScheduler scheduler) { + this.scheduler = scheduler; + } + + @Override + public String getName() { + return "Sample JobScheduler extension handler"; + } + + @Override + public List routes() { + return Collections.unmodifiableList( + Arrays.asList( + new Route(RestRequest.Method.POST, WATCH_INDEX_URI), + new Route(RestRequest.Method.DELETE, WATCH_INDEX_URI))); + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) { + switch (request.method()) { + case POST: + return handlePostRequest(request); + case DELETE: + return handleDeleteRequest(request); + default: + return restChannel -> { + restChannel.sendResponse( + new BytesRestResponse( + RestStatus.METHOD_NOT_ALLOWED, request.method() + " is not allowed.")); + }; + } + } + + private RestChannelConsumer handlePostRequest(RestRequest request) { + String jobName = request.param("job_name"); + String interval = request.param("interval"); + + if (jobName == null || interval == null) { + return restChannel -> + restChannel.sendResponse( + new BytesRestResponse( + BAD_REQUEST, "Missing required parameters: id, index, job_name, interval")); + } + + return restChannel -> { + try { + Instant now = Instant.now(); + OpenSearchRefreshIndexJobRequest jobRequest = + OpenSearchRefreshIndexJobRequest.builder() + .jobName(jobName) + .schedule( + new IntervalSchedule( + now, + Integer.parseInt(interval), + ChronoUnit + .MINUTES)) // Assuming ScheduleParser can parse the interval directly + .enabled(true) + .enabledTime(now) + .lastUpdateTime(now) + .lockDurationSeconds(1L) + .build(); + scheduler.scheduleJob(jobRequest); + XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject().field("message", "Scheduled job with name " + jobName).endObject(); + restChannel.sendResponse(new BytesRestResponse(OK, builder)); + } catch (Exception e) { + LOG.error("Failed to schedule job", e); + restChannel.sendResponse( + new BytesRestResponse(RestStatus.INTERNAL_SERVER_ERROR, e.getMessage())); + } + }; + } + + private RestChannelConsumer handleDeleteRequest(RestRequest request) { + String jobName = request.param("job_name"); + if (jobName == null) { + return restChannel -> + restChannel.sendResponse( + new BytesRestResponse(BAD_REQUEST, "Must specify jobName parameter")); + } + + return restChannel -> { + try { + scheduler.removeJob(jobName); + XContentBuilder builder = JsonXContent.contentBuilder(); + builder + .startObject() + .field("message", "Remove scheduled job with name " + jobName) + .endObject(); + restChannel.sendResponse(new BytesRestResponse(OK, builder)); + } catch (Exception e) { + LOG.error("Failed to schedule job", e); + restChannel.sendResponse( + new BytesRestResponse(RestStatus.INTERNAL_SERVER_ERROR, e.getMessage())); + } + }; + } +} diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/exceptions/AsyncQuerySchedulerException.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/exceptions/AsyncQuerySchedulerException.java new file mode 100644 index 0000000000..2b4f15f4fc --- /dev/null +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/exceptions/AsyncQuerySchedulerException.java @@ -0,0 +1,22 @@ +/* + * + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.sql.spark.scheduler.exceptions; + +public class AsyncQuerySchedulerException extends RuntimeException { + public AsyncQuerySchedulerException(String message) { + super(message); + } + + public AsyncQuerySchedulerException(Throwable cause) { + super(cause); + } + + public AsyncQuerySchedulerException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java new file mode 100644 index 0000000000..787eca9e0c --- /dev/null +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java @@ -0,0 +1,124 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.scheduler.job; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.client.Client; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.core.action.ActionListener; +import org.opensearch.jobscheduler.spi.JobExecutionContext; +import org.opensearch.jobscheduler.spi.ScheduledJobParameter; +import org.opensearch.jobscheduler.spi.ScheduledJobRunner; +import org.opensearch.jobscheduler.spi.utils.LockService; +import org.opensearch.plugins.Plugin; +import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; +import org.opensearch.threadpool.ThreadPool; + +/** + * The job runner class for scheduling refresh index query. + * + *

The job runner should be a singleton class if it uses OpenSearch client or other objects + * passed from OpenSearch. Because when registering the job runner to JobScheduler plugin, + * OpenSearch has not invoked plugins' createComponents() method. That is saying the plugin is not + * completely initialized, and the OpenSearch {@link org.opensearch.client.Client}, {@link + * ClusterService} and other objects are not available to plugin and this job runner. + * + *

So we have to move this job runner initialization to {@link Plugin} createComponents() method, + * and using singleton job runner to ensure we register a usable job runner instance to JobScheduler + * plugin. + */ +public class OpenSearchRefreshIndexJob implements ScheduledJobRunner { + + private static final Logger log = LogManager.getLogger(OpenSearchRefreshIndexJob.class); + + private static OpenSearchRefreshIndexJob INSTANCE; + + public static OpenSearchRefreshIndexJob getJobRunnerInstance() { + if (INSTANCE != null) { + return INSTANCE; + } + synchronized (OpenSearchRefreshIndexJob.class) { + if (INSTANCE == null) { + INSTANCE = new OpenSearchRefreshIndexJob(); + } + return INSTANCE; + } + } + + private ClusterService clusterService; + private ThreadPool threadPool; + private Client client; + + private OpenSearchRefreshIndexJob() { + // Singleton class, use getJobRunnerInstance method instead of constructor + } + + public void setClusterService(ClusterService clusterService) { + this.clusterService = clusterService; + } + + public void setThreadPool(ThreadPool threadPool) { + this.threadPool = threadPool; + } + + public void setClient(Client client) { + this.client = client; + } + + @Override + public void runJob(ScheduledJobParameter jobParameter, JobExecutionContext context) { + if (!(jobParameter instanceof OpenSearchRefreshIndexJobRequest)) { + throw new IllegalStateException( + "Job parameter is not instance of OpenSearchRefreshIndexJobRequest, type: " + + jobParameter.getClass().getCanonicalName()); + } + + if (this.clusterService == null) { + throw new IllegalStateException("ClusterService is not initialized."); + } + + if (this.threadPool == null) { + throw new IllegalStateException("ThreadPool is not initialized."); + } + + final LockService lockService = context.getLockService(); + + Runnable runnable = + () -> { + if (jobParameter.getLockDurationSeconds() != null) { + lockService.acquireLock( + jobParameter, + context, + ActionListener.wrap( + lock -> { + if (lock == null) { + return; + } + + // TODO: add logic to refresh index + log.info("Running job: " + jobParameter.getName()); + + lockService.release( + lock, + ActionListener.wrap( + released -> { + log.info("Released lock for job {}", jobParameter.getName()); + }, + exception -> { + throw new IllegalStateException( + "Failed to release lock.", exception); + })); + }, + exception -> { + throw new IllegalStateException("Failed to acquire lock.", exception); + })); + } + }; + + threadPool.generic().submit(runnable); + } +} diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/model/OpenSearchRefreshIndexJobRequest.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/model/OpenSearchRefreshIndexJobRequest.java new file mode 100644 index 0000000000..7eaa4e2d29 --- /dev/null +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/model/OpenSearchRefreshIndexJobRequest.java @@ -0,0 +1,108 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.scheduler.model; + +import java.io.IOException; +import java.time.Instant; +import lombok.Builder; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.jobscheduler.spi.ScheduledJobParameter; +import org.opensearch.jobscheduler.spi.schedule.Schedule; + +/** Represents a job request to refresh index. */ +@Builder +public class OpenSearchRefreshIndexJobRequest implements ScheduledJobParameter { + // Constant fields for JSON serialization + public static final String JOB_NAME_FIELD = "jobName"; + public static final String JOB_TYPE_FIELD = "jobType"; + public static final String LAST_UPDATE_TIME_FIELD = "lastUpdateTime"; + public static final String LAST_UPDATE_TIME_FIELD_READABLE = "last_update_time_field"; + public static final String SCHEDULE_FIELD = "schedule"; + public static final String ENABLED_TIME_FIELD = "enabledTime"; + public static final String ENABLED_TIME_FIELD_READABLE = "enabled_time_field"; + public static final String LOCK_DURATION_SECONDS = "lockDurationSeconds"; + public static final String JITTER = "jitter"; + public static final String ENABLED_FIELD = "enabled"; + + // name is doc id + private final String jobName; + private final String jobType; + private final Schedule schedule; + private final boolean enabled; + private final Instant lastUpdateTime; + private final Instant enabledTime; + private final Long lockDurationSeconds; + private final Double jitter; + + @Override + public String getName() { + return jobName; + } + + public String getJobType() { + return jobType; + } + + @Override + public Schedule getSchedule() { + return schedule; + } + + @Override + public boolean isEnabled() { + return enabled; + } + + @Override + public Instant getLastUpdateTime() { + return lastUpdateTime; + } + + @Override + public Instant getEnabledTime() { + return enabledTime; + } + + @Override + public Long getLockDurationSeconds() { + return lockDurationSeconds; + } + + @Override + public Double getJitter() { + return jitter; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) + throws IOException { + builder.startObject(); + builder.field(JOB_NAME_FIELD, getName()).field(ENABLED_FIELD, isEnabled()); + if (getSchedule() != null) { + builder.field(SCHEDULE_FIELD, getSchedule()); + } + if (getJobType() != null) { + builder.field(JOB_TYPE_FIELD, getJobType()); + } + if (getEnabledTime() != null) { + builder.timeField( + ENABLED_TIME_FIELD, ENABLED_TIME_FIELD_READABLE, getEnabledTime().toEpochMilli()); + } + builder.timeField( + LAST_UPDATE_TIME_FIELD, + LAST_UPDATE_TIME_FIELD_READABLE, + getLastUpdateTime().toEpochMilli()); + if (this.lockDurationSeconds != null) { + builder.field(LOCK_DURATION_SECONDS, this.lockDurationSeconds); + } + if (this.jitter != null) { + builder.field(JITTER, this.jitter); + } + builder.endObject(); + return builder; + } +} diff --git a/async-query/src/main/resources/async-query-scheduler-index-mapping.yml b/async-query/src/main/resources/async-query-scheduler-index-mapping.yml new file mode 100644 index 0000000000..36bd1b873e --- /dev/null +++ b/async-query/src/main/resources/async-query-scheduler-index-mapping.yml @@ -0,0 +1,41 @@ +--- +## +# Copyright OpenSearch Contributors +# SPDX-License-Identifier: Apache-2.0 +## + +# Schema file for the .async-query-scheduler index +# Also "dynamic" is set to "false" so that other fields cannot be added. +dynamic: false +properties: + name: + type: keyword + jobType: + type: keyword + lastUpdateTime: + type: date + format: epoch_millis + enabledTime: + type: date + format: epoch_millis + schedule: + properties: + initialDelay: + type: long + interval: + properties: + start_time: + type: date + format: "strict_date_time||epoch_millis" + period: + type: integer + unit: + type: keyword + enabled: + type: boolean + lockDurationSeconds: + type: long + null_value: -1 + jitter: + type: double + null_value: 0.0 \ No newline at end of file diff --git a/async-query/src/main/resources/async-query-scheduler-index-settings.yml b/async-query/src/main/resources/async-query-scheduler-index-settings.yml new file mode 100644 index 0000000000..386f1f4f34 --- /dev/null +++ b/async-query/src/main/resources/async-query-scheduler-index-settings.yml @@ -0,0 +1,11 @@ +--- +## +# Copyright OpenSearch Contributors +# SPDX-License-Identifier: Apache-2.0 +## + +# Settings file for the .async-query-scheduler index +index: + number_of_shards: "1" + auto_expand_replicas: "0-2" + number_of_replicas: "0" \ No newline at end of file diff --git a/build.gradle b/build.gradle index b3e09d7b50..d088883022 100644 --- a/build.gradle +++ b/build.gradle @@ -192,7 +192,7 @@ configurations.all { exclude group: "commons-logging", module: "commons-logging" // enforce 1.1.3, https://www.whitesourcesoftware.com/vulnerability-database/WS-2019-0379 resolutionStrategy.force 'commons-codec:commons-codec:1.13' - resolutionStrategy.force 'com.google.guava:guava:32.0.1-jre' + resolutionStrategy.force 'com.google.guava:guava:32.1.3-jre' } // updateVersion: Task to auto increment to the next development iteration diff --git a/common/build.gradle b/common/build.gradle index b4ee98a5b7..786629b027 100644 --- a/common/build.gradle +++ b/common/build.gradle @@ -34,7 +34,7 @@ repositories { dependencies { api "org.antlr:antlr4-runtime:4.7.1" - api group: 'com.google.guava', name: 'guava', version: '32.0.1-jre' + api group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' api group: 'org.apache.logging.log4j', name: 'log4j-core', version:"${versions.log4j}" api group: 'org.apache.commons', name: 'commons-lang3', version: '3.12.0' api group: 'org.apache.commons', name: 'commons-text', version: '1.10.0' @@ -46,7 +46,7 @@ dependencies { testImplementation group: 'junit', name: 'junit', version: '4.13.2' testImplementation group: 'org.assertj', name: 'assertj-core', version: '3.9.1' - testImplementation group: 'com.google.guava', name: 'guava', version: '32.0.1-jre' + testImplementation group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' testImplementation group: 'org.hamcrest', name: 'hamcrest-library', version: '2.1' testImplementation('org.junit.jupiter:junit-jupiter:5.9.3') testImplementation group: 'org.mockito', name: 'mockito-core', version: '5.7.0' diff --git a/core/build.gradle b/core/build.gradle index 655e7d92c2..9a2f08f148 100644 --- a/core/build.gradle +++ b/core/build.gradle @@ -46,7 +46,7 @@ pitest { } dependencies { - api group: 'com.google.guava', name: 'guava', version: '32.0.1-jre' + api group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' api group: 'org.apache.commons', name: 'commons-lang3', version: '3.12.0' api group: 'org.apache.commons', name: 'commons-text', version: '1.10.0' api group: 'com.facebook.presto', name: 'presto-matching', version: '0.240' diff --git a/integ-test/build.gradle b/integ-test/build.gradle index 93153cf737..22b6d24005 100644 --- a/integ-test/build.gradle +++ b/integ-test/build.gradle @@ -153,7 +153,7 @@ configurations.all { resolutionStrategy.force "commons-logging:commons-logging:1.2" // enforce 1.1.3, https://www.whitesourcesoftware.com/vulnerability-database/WS-2019-0379 resolutionStrategy.force 'commons-codec:commons-codec:1.13' - resolutionStrategy.force 'com.google.guava:guava:32.0.1-jre' + resolutionStrategy.force 'com.google.guava:guava:32.1.3-jre' resolutionStrategy.force "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" resolutionStrategy.force "com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:${versions.jackson}" resolutionStrategy.force "com.fasterxml.jackson.dataformat:jackson-dataformat-smile:${versions.jackson}" diff --git a/legacy/build.gradle b/legacy/build.gradle index 0467db183d..303387bdbe 100644 --- a/legacy/build.gradle +++ b/legacy/build.gradle @@ -107,7 +107,7 @@ dependencies { because 'https://www.whitesourcesoftware.com/vulnerability-database/WS-2019-0379' } } - implementation group: 'com.google.guava', name: 'guava', version: '32.0.1-jre' + implementation group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' implementation group: 'org.json', name: 'json', version:'20231013' implementation group: 'org.apache.commons', name: 'commons-lang3', version: '3.12.0' implementation group: 'org.apache.commons', name: 'commons-text', version: '1.10.0' diff --git a/plugin/build.gradle b/plugin/build.gradle index 710d81ed0a..7c7c882a22 100644 --- a/plugin/build.gradle +++ b/plugin/build.gradle @@ -48,6 +48,7 @@ opensearchplugin { name 'opensearch-sql' description 'OpenSearch SQL' classname 'org.opensearch.sql.plugin.SQLPlugin' + extendedPlugins = ['opensearch-job-scheduler'] licenseFile rootProject.file("LICENSE.txt") noticeFile rootProject.file("NOTICE") } @@ -98,7 +99,8 @@ configurations.all { resolutionStrategy.force "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" // enforce 1.1.3, https://www.whitesourcesoftware.com/vulnerability-database/WS-2019-0379 resolutionStrategy.force 'commons-codec:commons-codec:1.13' - resolutionStrategy.force 'com.google.guava:guava:32.0.1-jre' + resolutionStrategy.force 'com.google.guava:guava:32.1.3-jre' + resolutionStrategy.force 'com.google.guava:failureaccess:1.0.2' resolutionStrategy.force "com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:${versions.jackson}" resolutionStrategy.force "com.fasterxml.jackson.dataformat:jackson-dataformat-smile:${versions.jackson}" resolutionStrategy.force "com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:${versions.jackson}" @@ -139,6 +141,10 @@ spotless { } dependencies { + compileOnly "org.opensearch:opensearch-job-scheduler-spi:${opensearch_build}" + compileOnly 'com.google.guava:guava:32.1.3-jre' + compileOnly 'com.google.guava:failureaccess:1.0.2' + api "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" api "com.fasterxml.jackson.core:jackson-databind:${versions.jackson_databind}" api "com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}" @@ -204,11 +210,10 @@ dependencyLicenses.enabled = false // enable testingConventions check will cause errors like: "Classes ending with [Tests] must subclass [LuceneTestCase]" testingConventions.enabled = false -// TODO: need to verify the thirdPartyAudi +// TODO: need to verify the thirdPartyAudit // currently it complains missing classes like ibatis, mysql etc, should not be a problem thirdPartyAudit.enabled = false - apply plugin: 'com.netflix.nebula.ospackage' validateNebulaPom.enabled = false 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 b86ab9218a..39f3c64e12 100644 --- a/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java +++ b/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java @@ -42,6 +42,9 @@ import org.opensearch.env.Environment; import org.opensearch.env.NodeEnvironment; import org.opensearch.indices.SystemIndexDescriptor; +import org.opensearch.jobscheduler.spi.JobSchedulerExtension; +import org.opensearch.jobscheduler.spi.ScheduledJobParser; +import org.opensearch.jobscheduler.spi.ScheduledJobRunner; import org.opensearch.plugins.ActionPlugin; import org.opensearch.plugins.Plugin; import org.opensearch.plugins.ScriptPlugin; @@ -91,6 +94,8 @@ import org.opensearch.sql.spark.flint.FlintIndexMetadataServiceImpl; import org.opensearch.sql.spark.flint.operation.FlintIndexOpFactory; import org.opensearch.sql.spark.rest.RestAsyncQueryManagementAction; +import org.opensearch.sql.spark.scheduler.OpenSearchAsyncQueryScheduler; +import org.opensearch.sql.spark.scheduler.SampleExtensionRestHandler; import org.opensearch.sql.spark.storage.SparkStorageFactory; import org.opensearch.sql.spark.transport.TransportCancelAsyncQueryRequestAction; import org.opensearch.sql.spark.transport.TransportCreateAsyncQueryRequestAction; @@ -105,7 +110,8 @@ import org.opensearch.threadpool.ThreadPool; import org.opensearch.watcher.ResourceWatcherService; -public class SQLPlugin extends Plugin implements ActionPlugin, ScriptPlugin, SystemIndexPlugin { +public class SQLPlugin extends Plugin + implements ActionPlugin, ScriptPlugin, SystemIndexPlugin, JobSchedulerExtension { private static final Logger LOGGER = LogManager.getLogger(SQLPlugin.class); @@ -116,6 +122,7 @@ public class SQLPlugin extends Plugin implements ActionPlugin, ScriptPlugin, Sys private NodeClient client; private DataSourceServiceImpl dataSourceService; + private OpenSearchAsyncQueryScheduler asyncQueryScheduler; private Injector injector; public String name() { @@ -141,6 +148,7 @@ public List getRestHandlers( Metrics.getInstance().registerDefaultMetrics(); return Arrays.asList( + new SampleExtensionRestHandler(this.asyncQueryScheduler), new RestPPLQueryAction(), new RestSqlAction(settings, injector), new RestSqlStatsAction(settings, restController), @@ -208,6 +216,8 @@ public Collection createComponents( this.client = (NodeClient) client; this.dataSourceService = createDataSourceService(); dataSourceService.createDataSource(defaultOpenSearchDataSourceMetadata()); + this.asyncQueryScheduler = new OpenSearchAsyncQueryScheduler(); + this.asyncQueryScheduler.loadJobResource(client, clusterService, threadPool); LocalClusterState.state().setClusterService(clusterService); LocalClusterState.state().setPluginSettings((OpenSearchSettings) pluginSettings); LocalClusterState.state().setClient(client); @@ -243,6 +253,26 @@ public Collection createComponents( pluginSettings); } + @Override + public String getJobType() { + return OpenSearchAsyncQueryScheduler.SCHEDULER_PLUGIN_JOB_TYPE; + } + + @Override + public String getJobIndex() { + return OpenSearchAsyncQueryScheduler.SCHEDULER_INDEX_NAME; + } + + @Override + public ScheduledJobRunner getJobRunner() { + return OpenSearchAsyncQueryScheduler.getJobRunner(); + } + + @Override + public ScheduledJobParser getJobParser() { + return OpenSearchAsyncQueryScheduler.getJobParser(); + } + @Override public List> getExecutorBuilders(Settings settings) { return singletonList( diff --git a/plugin/src/main/resources/META-INF/services/org.opensearch.jobscheduler.spi.JobSchedulerExtension b/plugin/src/main/resources/META-INF/services/org.opensearch.jobscheduler.spi.JobSchedulerExtension new file mode 100644 index 0000000000..5337857c15 --- /dev/null +++ b/plugin/src/main/resources/META-INF/services/org.opensearch.jobscheduler.spi.JobSchedulerExtension @@ -0,0 +1,6 @@ +# +# Copyright OpenSearch Contributors +# SPDX-License-Identifier: Apache-2.0 +# + +org.opensearch.sql.plugin.SQLPlugin \ No newline at end of file diff --git a/ppl/build.gradle b/ppl/build.gradle index d58882d5e8..39bed7a359 100644 --- a/ppl/build.gradle +++ b/ppl/build.gradle @@ -48,7 +48,7 @@ dependencies { runtimeOnly group: 'org.reflections', name: 'reflections', version: '0.9.12' implementation "org.antlr:antlr4-runtime:4.7.1" - implementation group: 'com.google.guava', name: 'guava', version: '32.0.1-jre' + implementation group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' api group: 'org.json', name: 'json', version: '20231013' implementation group: 'org.apache.logging.log4j', name: 'log4j-core', version:"${versions.log4j}" api project(':common') diff --git a/protocol/build.gradle b/protocol/build.gradle index 5bbff68e51..d706b480c3 100644 --- a/protocol/build.gradle +++ b/protocol/build.gradle @@ -30,7 +30,7 @@ plugins { } dependencies { - implementation group: 'com.google.guava', name: 'guava', version: '32.0.1-jre' + implementation group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' implementation group: 'com.fasterxml.jackson.core', name: 'jackson-core', version: "${versions.jackson}" implementation group: 'com.fasterxml.jackson.core', name: 'jackson-databind', version: "${versions.jackson_databind}" implementation group: 'com.fasterxml.jackson.dataformat', name: 'jackson-dataformat-cbor', version: "${versions.jackson}" diff --git a/sql/build.gradle b/sql/build.gradle index 81872e6035..52ae0ac621 100644 --- a/sql/build.gradle +++ b/sql/build.gradle @@ -46,7 +46,7 @@ dependencies { antlr "org.antlr:antlr4:4.7.1" implementation "org.antlr:antlr4-runtime:4.7.1" - implementation group: 'com.google.guava', name: 'guava', version: '32.0.1-jre' + implementation group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' implementation group: 'org.json', name: 'json', version:'20231013' implementation project(':common') implementation project(':core') From aa7d13a857a954ba0bad36f883482b3daf6ff862 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Wed, 17 Jul 2024 09:10:28 -0700 Subject: [PATCH 02/18] Add comments Signed-off-by: Louis Chu --- .../spark/scheduler/OpenSearchAsyncQueryScheduler.java | 9 +++++++++ .../exceptions/AsyncQuerySchedulerException.java | 1 + 2 files changed, 10 insertions(+) diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java index 8d0abb5aeb..992d9979d7 100644 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java @@ -41,6 +41,7 @@ import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; import org.opensearch.threadpool.ThreadPool; +/** Scheduler class for managing asynchronous query jobs. */ public class OpenSearchAsyncQueryScheduler { public static final String SCHEDULER_INDEX_NAME = ".async-query-scheduler"; public static final String SCHEDULER_PLUGIN_JOB_TYPE = "async-query-scheduler"; @@ -52,6 +53,7 @@ public class OpenSearchAsyncQueryScheduler { private Client client; private ClusterService clusterService; + /** Loads job resources, setting up required services and job runner instance. */ public void loadJobResource(Client client, ClusterService clusterService, ThreadPool threadPool) { this.client = client; this.clusterService = clusterService; @@ -62,6 +64,7 @@ public void loadJobResource(Client client, ClusterService clusterService, Thread openSearchRefreshIndexJob.setClient(client); } + /** Schedules a new job by indexing it into the job index. */ public void scheduleJob(OpenSearchRefreshIndexJobRequest request) { if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { createAsyncQuerySchedulerIndex(); @@ -90,6 +93,7 @@ public void scheduleJob(OpenSearchRefreshIndexJobRequest request) { } } + /** Unschedules a job by marking it as disabled and updating its last update time. */ public void unscheduleJob(String jobId) throws IOException { if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { throw new IllegalArgumentException("Job index does not exist."); @@ -103,6 +107,7 @@ public void unscheduleJob(String jobId) throws IOException { updateJob(request); } + /** Updates an existing job with new parameters. */ public void updateJob(OpenSearchRefreshIndexJobRequest request) throws IOException { if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { throw new IllegalArgumentException("Job index does not exist."); @@ -123,6 +128,7 @@ public void updateJob(OpenSearchRefreshIndexJobRequest request) throws IOExcepti } } + /** Removes a job by deleting its document from the index. */ public void removeJob(String jobId) { if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { throw new IllegalArgumentException("Job index does not exist."); @@ -142,6 +148,7 @@ public void removeJob(String jobId) { } } + /** Creates the async query scheduler index with specified mappings and settings. */ private void createAsyncQuerySchedulerIndex() { try { InputStream mappingFileStream = @@ -177,10 +184,12 @@ private void createAsyncQuerySchedulerIndex() { } } + /** Returns the job runner instance for the scheduler. */ public static ScheduledJobRunner getJobRunner() { return OpenSearchRefreshIndexJob.getJobRunnerInstance(); } + /** Returns the job parser instance for the scheduler. */ public static ScheduledJobParser getJobParser() { return (parser, id, jobDocVersion) -> { OpenSearchRefreshIndexJobRequest.OpenSearchRefreshIndexJobRequestBuilder builder = diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/exceptions/AsyncQuerySchedulerException.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/exceptions/AsyncQuerySchedulerException.java index 2b4f15f4fc..c5cafa03bc 100644 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/exceptions/AsyncQuerySchedulerException.java +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/exceptions/AsyncQuerySchedulerException.java @@ -7,6 +7,7 @@ package org.opensearch.sql.spark.scheduler.exceptions; +/** Exception class for handling errors related to the asynchronous query scheduler. */ public class AsyncQuerySchedulerException extends RuntimeException { public AsyncQuerySchedulerException(String message) { super(message); From c7e8d6450e0bb5bf437f07ea5fc674dfe7e23a98 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Wed, 17 Jul 2024 09:52:43 -0700 Subject: [PATCH 03/18] Add unit test Signed-off-by: Louis Chu --- async-query/build.gradle | 2 +- .../OpenSearchAsyncQueryScheduler.java | 30 ++- .../AsyncQuerySchedulerException.java | 23 -- .../OpenSearchAsyncQuerySchedulerTest.java | 238 ++++++++++++++++++ .../job/OpenSearchRefreshIndexJobTest.java | 117 +++++++++ .../OpenSearchRefreshIndexJobRequestTest.java | 81 ++++++ 6 files changed, 455 insertions(+), 36 deletions(-) delete mode 100644 async-query/src/main/java/org/opensearch/sql/spark/scheduler/exceptions/AsyncQuerySchedulerException.java create mode 100644 async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java create mode 100644 async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java create mode 100644 async-query/src/test/java/org/opensearch/sql/spark/scheduler/model/OpenSearchRefreshIndexJobRequestTest.java diff --git a/async-query/build.gradle b/async-query/build.gradle index f4907b91c0..c885e2efb3 100644 --- a/async-query/build.gradle +++ b/async-query/build.gradle @@ -16,7 +16,7 @@ repositories { dependencies { - compileOnly "org.opensearch:opensearch-job-scheduler-spi:${opensearch_build}" + implementation "org.opensearch:opensearch-job-scheduler-spi:${opensearch_build}" api project(':core') api project(':async-query-core') diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java index 992d9979d7..573b409128 100644 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java @@ -32,11 +32,11 @@ import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.core.xcontent.XContentParserUtils; +import org.opensearch.index.engine.DocumentMissingException; import org.opensearch.index.engine.VersionConflictEngineException; import org.opensearch.jobscheduler.spi.ScheduledJobParser; import org.opensearch.jobscheduler.spi.ScheduledJobRunner; import org.opensearch.jobscheduler.spi.schedule.ScheduleParser; -import org.opensearch.sql.spark.scheduler.exceptions.AsyncQuerySchedulerException; import org.opensearch.sql.spark.scheduler.job.OpenSearchRefreshIndexJob; import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; import org.opensearch.threadpool.ThreadPool; @@ -82,13 +82,13 @@ public void scheduleJob(OpenSearchRefreshIndexJobRequest request) { } catch (VersionConflictEngineException exception) { throw new IllegalArgumentException("A job already exists with name: " + request.getName()); } catch (Exception e) { - throw new AsyncQuerySchedulerException(e); + throw new RuntimeException(e); } if (indexResponse.getResult().equals(DocWriteResponse.Result.CREATED)) { LOG.debug("Job : {} successfully created", request.getName()); } else { - throw new AsyncQuerySchedulerException( + throw new RuntimeException( "Schedule job failed with result : " + indexResponse.getResult().getLowercase()); } } @@ -115,15 +115,21 @@ public void updateJob(OpenSearchRefreshIndexJobRequest request) throws IOExcepti UpdateRequest updateRequest = new UpdateRequest(SCHEDULER_INDEX_NAME, request.getName()); updateRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); updateRequest.doc(request.toXContent(JsonXContent.contentBuilder(), EMPTY_PARAMS)); - - ActionFuture updateResponseActionFuture = client.update(updateRequest); - UpdateResponse updateResponse = updateResponseActionFuture.actionGet(); + UpdateResponse updateResponse; + try { + ActionFuture updateResponseActionFuture = client.update(updateRequest); + updateResponse = updateResponseActionFuture.actionGet(); + } catch (DocumentMissingException exception) { + throw new IllegalArgumentException("Job with name: " + request.getName() + " doesn't exist"); + } catch (Exception e) { + throw new RuntimeException(e); + } if (updateResponse.getResult().equals(DocWriteResponse.Result.UPDATED) || updateResponse.getResult().equals(DocWriteResponse.Result.NOOP)) { LOG.debug("Job : {} successfully updated", request.getName()); } else { - throw new AsyncQuerySchedulerException( + throw new RuntimeException( "Update job failed with result : " + updateResponse.getResult().getLowercase()); } } @@ -141,15 +147,15 @@ public void removeJob(String jobId) { if (deleteResponse.getResult().equals(DocWriteResponse.Result.DELETED)) { LOG.debug("Job : {} successfully deleted", jobId); } else if (deleteResponse.getResult().equals(DocWriteResponse.Result.NOT_FOUND)) { - throw new AsyncQuerySchedulerException("Job : " + jobId + " doesn't exist"); + throw new IllegalArgumentException("Job : " + jobId + " doesn't exist"); } else { - throw new AsyncQuerySchedulerException( + throw new RuntimeException( "Remove job failed with result : " + deleteResponse.getResult().getLowercase()); } } /** Creates the async query scheduler index with specified mappings and settings. */ - private void createAsyncQuerySchedulerIndex() { + void createAsyncQuerySchedulerIndex() { try { InputStream mappingFileStream = OpenSearchAsyncQueryScheduler.class @@ -171,11 +177,11 @@ private void createAsyncQuerySchedulerIndex() { if (createIndexResponse.isAcknowledged()) { LOG.debug("Index: {} creation Acknowledged", SCHEDULER_INDEX_NAME); } else { - throw new AsyncQuerySchedulerException("Index creation is not acknowledged."); + throw new RuntimeException("Index creation is not acknowledged."); } } catch (Throwable e) { LOG.error("Error creating index: {}", SCHEDULER_INDEX_NAME, e); - throw new AsyncQuerySchedulerException( + throw new RuntimeException( "Internal server error while creating " + SCHEDULER_INDEX_NAME + " index: " diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/exceptions/AsyncQuerySchedulerException.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/exceptions/AsyncQuerySchedulerException.java deleted file mode 100644 index c5cafa03bc..0000000000 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/exceptions/AsyncQuerySchedulerException.java +++ /dev/null @@ -1,23 +0,0 @@ -/* - * - * * Copyright OpenSearch Contributors - * * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.opensearch.sql.spark.scheduler.exceptions; - -/** Exception class for handling errors related to the asynchronous query scheduler. */ -public class AsyncQuerySchedulerException extends RuntimeException { - public AsyncQuerySchedulerException(String message) { - super(message); - } - - public AsyncQuerySchedulerException(Throwable cause) { - super(cause); - } - - public AsyncQuerySchedulerException(String message, Throwable cause) { - super(message, cause); - } -} diff --git a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java new file mode 100644 index 0000000000..c2765d6de9 --- /dev/null +++ b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java @@ -0,0 +1,238 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.scheduler; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.opensearch.sql.spark.scheduler.OpenSearchAsyncQueryScheduler.SCHEDULER_INDEX_NAME; + +import java.io.IOException; +import java.time.Instant; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Answers; +import org.mockito.ArgumentCaptor; +import org.mockito.ArgumentMatchers; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; +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.delete.DeleteRequest; +import org.opensearch.action.delete.DeleteResponse; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.index.IndexResponse; +import org.opensearch.action.support.WriteRequest; +import org.opensearch.action.update.UpdateRequest; +import org.opensearch.action.update.UpdateResponse; +import org.opensearch.client.Client; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.action.ActionFuture; +import org.opensearch.index.engine.VersionConflictEngineException; +import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; +import org.opensearch.threadpool.ThreadPool; + +public class OpenSearchAsyncQuerySchedulerTest { + + private static final String TEST_SCHEDULER_INDEX_NAME = "testQS"; + + @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 ThreadPool threadPool; + + @Mock private ActionFuture indexResponseActionFuture; + + @Mock private ActionFuture updateResponseActionFuture; + + @Mock private ActionFuture deleteResponseActionFuture; + + @Mock private ActionFuture createIndexResponseActionFuture; + + @Mock private IndexResponse indexResponse; + + @Mock private UpdateResponse updateResponse; + + private OpenSearchAsyncQueryScheduler scheduler; + + @BeforeEach + public void setup() { + MockitoAnnotations.openMocks(this); + scheduler = new OpenSearchAsyncQueryScheduler(); + scheduler.loadJobResource(client, clusterService, threadPool); + } + + @Test + public void testScheduleJob() { + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) + .thenReturn(Boolean.FALSE); + when(client.admin().indices().create(any(CreateIndexRequest.class))) + .thenReturn(createIndexResponseActionFuture); + when(createIndexResponseActionFuture.actionGet()) + .thenReturn(new CreateIndexResponse(true, true, TEST_SCHEDULER_INDEX_NAME)); + when(client.index(any(IndexRequest.class))).thenReturn(indexResponseActionFuture); + when(indexResponseActionFuture.actionGet()).thenReturn(indexResponse); + when(indexResponse.getResult()).thenReturn(DocWriteResponse.Result.CREATED); + + OpenSearchRefreshIndexJobRequest request = + OpenSearchRefreshIndexJobRequest.builder() + .jobName("testJob") + .lastUpdateTime(Instant.now()) + .build(); + + scheduler.scheduleJob(request); + + // Verify index created + verify(client.admin().indices(), Mockito.times(1)).create(ArgumentMatchers.any()); + + // Verify doc indexed + ArgumentCaptor captor = ArgumentCaptor.forClass(IndexRequest.class); + verify(client, Mockito.times(1)).index(captor.capture()); + IndexRequest capturedRequest = captor.getValue(); + assertEquals(request.getName(), capturedRequest.id()); + assertEquals(WriteRequest.RefreshPolicy.IMMEDIATE, capturedRequest.getRefreshPolicy()); + } + + @Test + public void testScheduleJobWithExistingJob() { + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) + .thenReturn(Boolean.TRUE); + + OpenSearchRefreshIndexJobRequest request = + OpenSearchRefreshIndexJobRequest.builder() + .jobName("testJob") + .lastUpdateTime(Instant.now()) + .build(); + + when(client.index(any(IndexRequest.class))).thenThrow(VersionConflictEngineException.class); + + IllegalArgumentException exception = + assertThrows( + IllegalArgumentException.class, + () -> { + scheduler.scheduleJob(request); + }); + + verify(client, Mockito.times(1)).index(ArgumentCaptor.forClass(IndexRequest.class).capture()); + assertEquals("A job already exists with name: testJob", exception.getMessage()); + } + + @Test + public void testUnscheduleJob() throws IOException { + String jobId = "testJob"; + + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); + + when(updateResponseActionFuture.actionGet()).thenReturn(updateResponse); + when(updateResponse.getResult()).thenReturn(DocWriteResponse.Result.UPDATED); + + when(client.update(any(UpdateRequest.class))).thenReturn(updateResponseActionFuture); + + scheduler.unscheduleJob(jobId); + + ArgumentCaptor captor = ArgumentCaptor.forClass(UpdateRequest.class); + verify(client).update(captor.capture()); + + UpdateRequest capturedRequest = captor.getValue(); + assertEquals(jobId, capturedRequest.id()); + assertEquals(WriteRequest.RefreshPolicy.IMMEDIATE, capturedRequest.getRefreshPolicy()); + } + + @Test + public void testUpdateJob() throws IOException { + OpenSearchRefreshIndexJobRequest request = + OpenSearchRefreshIndexJobRequest.builder() + .jobName("testJob") + .lastUpdateTime(Instant.now()) + .build(); + + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); + + when(updateResponseActionFuture.actionGet()).thenReturn(updateResponse); + when(updateResponse.getResult()).thenReturn(DocWriteResponse.Result.UPDATED); + + when(client.update(any(UpdateRequest.class))).thenReturn(updateResponseActionFuture); + + scheduler.updateJob(request); + + ArgumentCaptor captor = ArgumentCaptor.forClass(UpdateRequest.class); + verify(client).update(captor.capture()); + + UpdateRequest capturedRequest = captor.getValue(); + assertEquals(request.getName(), capturedRequest.id()); + assertEquals(WriteRequest.RefreshPolicy.IMMEDIATE, capturedRequest.getRefreshPolicy()); + } + + @Test + public void testRemoveJob() { + String jobId = "testJob"; + + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); + + DeleteResponse deleteResponse = mock(DeleteResponse.class); + when(deleteResponseActionFuture.actionGet()).thenReturn(deleteResponse); + when(deleteResponse.getResult()).thenReturn(DocWriteResponse.Result.DELETED); + + when(client.delete(any(DeleteRequest.class))).thenReturn(deleteResponseActionFuture); + + scheduler.removeJob(jobId); + + ArgumentCaptor captor = ArgumentCaptor.forClass(DeleteRequest.class); + verify(client).delete(captor.capture()); + + DeleteRequest capturedRequest = captor.getValue(); + assertEquals(jobId, capturedRequest.id()); + assertEquals(WriteRequest.RefreshPolicy.IMMEDIATE, capturedRequest.getRefreshPolicy()); + } + + @Test + public void testCreateAsyncQuerySchedulerIndex() { + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(false); + + CreateIndexResponse createIndexResponse = mock(CreateIndexResponse.class); + when(createIndexResponseActionFuture.actionGet()).thenReturn(createIndexResponse); + when(createIndexResponse.isAcknowledged()).thenReturn(true); + + when(client.admin().indices().create(any(CreateIndexRequest.class))) + .thenReturn(createIndexResponseActionFuture); + + scheduler.createAsyncQuerySchedulerIndex(); + + ArgumentCaptor captor = ArgumentCaptor.forClass(CreateIndexRequest.class); + verify(client.admin().indices()).create(captor.capture()); + + CreateIndexRequest capturedRequest = captor.getValue(); + assertEquals(SCHEDULER_INDEX_NAME, capturedRequest.index()); + } + + @Test + public void testCreateAsyncQuerySchedulerIndexFailure() throws IOException { + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(false); + + when(client.admin().indices().create(any(CreateIndexRequest.class))) + .thenThrow(new RuntimeException("Error creating index")); + + RuntimeException exception = + assertThrows( + RuntimeException.class, + () -> { + scheduler.createAsyncQuerySchedulerIndex(); + }); + + assertEquals( + "Internal server error while creating .async-query-scheduler index: Error creating index", + exception.getMessage()); + } +} diff --git a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java new file mode 100644 index 0000000000..c19d696e0e --- /dev/null +++ b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java @@ -0,0 +1,117 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.scheduler.job; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.time.Instant; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Answers; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.opensearch.client.Client; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.core.action.ActionListener; +import org.opensearch.jobscheduler.spi.JobExecutionContext; +import org.opensearch.jobscheduler.spi.ScheduledJobParameter; +import org.opensearch.jobscheduler.spi.utils.LockService; +import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; +import org.opensearch.threadpool.ThreadPool; + +public class OpenSearchRefreshIndexJobTest { + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private ClusterService clusterService; + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private ThreadPool threadPool; + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private Client client; + + @Mock private JobExecutionContext context; + + @Mock private LockService lockService; + + private OpenSearchRefreshIndexJob jobRunner; + + @BeforeEach + public void setup() { + MockitoAnnotations.openMocks(this); + jobRunner = OpenSearchRefreshIndexJob.getJobRunnerInstance(); + jobRunner.setClusterService(clusterService); + jobRunner.setThreadPool(threadPool); + jobRunner.setClient(client); + when(context.getLockService()).thenReturn(lockService); + } + + @Test + public void testRunJobWithCorrectParameter() { + OpenSearchRefreshIndexJobRequest jobParameter = + OpenSearchRefreshIndexJobRequest.builder() + .jobName("testJob") + .lastUpdateTime(Instant.now()) + .lockDurationSeconds(10L) + .build(); + + jobRunner.runJob(jobParameter, context); + + ArgumentCaptor captor = ArgumentCaptor.forClass(Runnable.class); + verify(threadPool.generic()).submit(captor.capture()); + + Runnable runnable = captor.getValue(); + runnable.run(); + + verify(lockService).acquireLock(eq(jobParameter), eq(context), any(ActionListener.class)); + } + + @Test + public void testRunJobWithIncorrectParameter() { + ScheduledJobParameter wrongParameter = mock(ScheduledJobParameter.class); + + try { + jobRunner.runJob(wrongParameter, context); + } catch (IllegalStateException e) { + assertEquals( + "Job parameter is not instance of OpenSearchRefreshIndexJobRequest, type: " + + wrongParameter.getClass().getCanonicalName(), + e.getMessage()); + } + } + + @Test + public void testRunJobWithUninitializedServices() { + OpenSearchRefreshIndexJobRequest jobParameter = + OpenSearchRefreshIndexJobRequest.builder() + .jobName("testJob") + .lastUpdateTime(Instant.now()) + .build(); + + OpenSearchRefreshIndexJob uninitializedJobRunner = + OpenSearchRefreshIndexJob.getJobRunnerInstance(); + + try { + uninitializedJobRunner.runJob(jobParameter, context); + } catch (IllegalStateException e) { + assertEquals("ClusterService is not initialized.", e.getMessage()); + } + + uninitializedJobRunner.setClusterService(clusterService); + + try { + uninitializedJobRunner.runJob(jobParameter, context); + } catch (IllegalStateException e) { + assertEquals("ThreadPool is not initialized.", e.getMessage()); + } + } +} diff --git a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/model/OpenSearchRefreshIndexJobRequestTest.java b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/model/OpenSearchRefreshIndexJobRequestTest.java new file mode 100644 index 0000000000..108f1acfd5 --- /dev/null +++ b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/model/OpenSearchRefreshIndexJobRequestTest.java @@ -0,0 +1,81 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.scheduler.model; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS; + +import java.io.IOException; +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import org.junit.jupiter.api.Test; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.jobscheduler.spi.schedule.IntervalSchedule; + +public class OpenSearchRefreshIndexJobRequestTest { + + @Test + public void testBuilderAndGetterMethods() { + Instant now = Instant.now(); + IntervalSchedule schedule = new IntervalSchedule(now, 1, ChronoUnit.MINUTES); + + OpenSearchRefreshIndexJobRequest jobRequest = + OpenSearchRefreshIndexJobRequest.builder() + .jobName("testJob") + .jobType("testType") + .schedule(schedule) + .enabled(true) + .lastUpdateTime(now) + .enabledTime(now) + .lockDurationSeconds(60L) + .jitter(0.1) + .build(); + + assertEquals("testJob", jobRequest.getName()); + assertEquals("testType", jobRequest.getJobType()); + assertEquals(schedule, jobRequest.getSchedule()); + assertTrue(jobRequest.isEnabled()); + assertEquals(now, jobRequest.getLastUpdateTime()); + assertEquals(now, jobRequest.getEnabledTime()); + assertEquals(60L, jobRequest.getLockDurationSeconds()); + assertEquals(0.1, jobRequest.getJitter()); + } + + @Test + public void testToXContent() throws IOException { + Instant now = Instant.now(); + IntervalSchedule schedule = new IntervalSchedule(now, 1, ChronoUnit.MINUTES); + + OpenSearchRefreshIndexJobRequest jobRequest = + OpenSearchRefreshIndexJobRequest.builder() + .jobName("testJob") + .jobType("testType") + .schedule(schedule) + .enabled(true) + .lastUpdateTime(now) + .enabledTime(now) + .lockDurationSeconds(60L) + .jitter(0.1) + .build(); + + XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint(); + jobRequest.toXContent(builder, EMPTY_PARAMS); + String jsonString = builder.toString(); + + assertTrue(jsonString.contains("\"jobName\" : \"testJob\"")); + assertTrue(jsonString.contains("\"jobType\" : \"testType\"")); + assertTrue(jsonString.contains("\"start_time\" : " + now.toEpochMilli())); + assertTrue(jsonString.contains("\"period\" : 1")); + assertTrue(jsonString.contains("\"unit\" : \"Minutes\"")); + assertTrue(jsonString.contains("\"enabled\" : true")); + assertTrue(jsonString.contains("\"lastUpdateTime\" : " + now.toEpochMilli())); + assertTrue(jsonString.contains("\"enabledTime\" : " + now.toEpochMilli())); + assertTrue(jsonString.contains("\"lockDurationSeconds\" : 60")); + assertTrue(jsonString.contains("\"jitter\" : 0.1")); + } +} From 8f84b6ccf51072514f48fc7b831423878edcbed5 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Wed, 17 Jul 2024 14:47:38 -0700 Subject: [PATCH 04/18] Remove test rest API Signed-off-by: Louis Chu --- .../scheduler/SampleExtensionRestHandler.java | 133 ------------------ .../org/opensearch/sql/plugin/SQLPlugin.java | 2 - 2 files changed, 135 deletions(-) delete mode 100644 async-query/src/main/java/org/opensearch/sql/spark/scheduler/SampleExtensionRestHandler.java diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/SampleExtensionRestHandler.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/SampleExtensionRestHandler.java deleted file mode 100644 index 4dcee97499..0000000000 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/SampleExtensionRestHandler.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.opensearch.sql.spark.scheduler; - -import static org.opensearch.core.rest.RestStatus.BAD_REQUEST; -import static org.opensearch.core.rest.RestStatus.OK; - -import java.time.Instant; -import java.time.temporal.ChronoUnit; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.opensearch.client.node.NodeClient; -import org.opensearch.common.xcontent.json.JsonXContent; -import org.opensearch.core.rest.RestStatus; -import org.opensearch.core.xcontent.XContentBuilder; -import org.opensearch.jobscheduler.spi.schedule.IntervalSchedule; -import org.opensearch.rest.BaseRestHandler; -import org.opensearch.rest.BytesRestResponse; -import org.opensearch.rest.RestRequest; -import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; - -/** A sample rest handler that supports schedule and deschedule job operation */ -public class SampleExtensionRestHandler extends BaseRestHandler { - private static final Logger LOG = LogManager.getLogger(SampleExtensionRestHandler.class); - - public static final String WATCH_INDEX_URI = "/_plugins/scheduler_sample/watch"; - private final OpenSearchAsyncQueryScheduler scheduler; - - public SampleExtensionRestHandler(OpenSearchAsyncQueryScheduler scheduler) { - this.scheduler = scheduler; - } - - @Override - public String getName() { - return "Sample JobScheduler extension handler"; - } - - @Override - public List routes() { - return Collections.unmodifiableList( - Arrays.asList( - new Route(RestRequest.Method.POST, WATCH_INDEX_URI), - new Route(RestRequest.Method.DELETE, WATCH_INDEX_URI))); - } - - @Override - protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) { - switch (request.method()) { - case POST: - return handlePostRequest(request); - case DELETE: - return handleDeleteRequest(request); - default: - return restChannel -> { - restChannel.sendResponse( - new BytesRestResponse( - RestStatus.METHOD_NOT_ALLOWED, request.method() + " is not allowed.")); - }; - } - } - - private RestChannelConsumer handlePostRequest(RestRequest request) { - String jobName = request.param("job_name"); - String interval = request.param("interval"); - - if (jobName == null || interval == null) { - return restChannel -> - restChannel.sendResponse( - new BytesRestResponse( - BAD_REQUEST, "Missing required parameters: id, index, job_name, interval")); - } - - return restChannel -> { - try { - Instant now = Instant.now(); - OpenSearchRefreshIndexJobRequest jobRequest = - OpenSearchRefreshIndexJobRequest.builder() - .jobName(jobName) - .schedule( - new IntervalSchedule( - now, - Integer.parseInt(interval), - ChronoUnit - .MINUTES)) // Assuming ScheduleParser can parse the interval directly - .enabled(true) - .enabledTime(now) - .lastUpdateTime(now) - .lockDurationSeconds(1L) - .build(); - scheduler.scheduleJob(jobRequest); - XContentBuilder builder = JsonXContent.contentBuilder(); - builder.startObject().field("message", "Scheduled job with name " + jobName).endObject(); - restChannel.sendResponse(new BytesRestResponse(OK, builder)); - } catch (Exception e) { - LOG.error("Failed to schedule job", e); - restChannel.sendResponse( - new BytesRestResponse(RestStatus.INTERNAL_SERVER_ERROR, e.getMessage())); - } - }; - } - - private RestChannelConsumer handleDeleteRequest(RestRequest request) { - String jobName = request.param("job_name"); - if (jobName == null) { - return restChannel -> - restChannel.sendResponse( - new BytesRestResponse(BAD_REQUEST, "Must specify jobName parameter")); - } - - return restChannel -> { - try { - scheduler.removeJob(jobName); - XContentBuilder builder = JsonXContent.contentBuilder(); - builder - .startObject() - .field("message", "Remove scheduled job with name " + jobName) - .endObject(); - restChannel.sendResponse(new BytesRestResponse(OK, builder)); - } catch (Exception e) { - LOG.error("Failed to schedule job", e); - restChannel.sendResponse( - new BytesRestResponse(RestStatus.INTERNAL_SERVER_ERROR, e.getMessage())); - } - }; - } -} 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 39f3c64e12..186a53f07a 100644 --- a/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java +++ b/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java @@ -95,7 +95,6 @@ import org.opensearch.sql.spark.flint.operation.FlintIndexOpFactory; import org.opensearch.sql.spark.rest.RestAsyncQueryManagementAction; import org.opensearch.sql.spark.scheduler.OpenSearchAsyncQueryScheduler; -import org.opensearch.sql.spark.scheduler.SampleExtensionRestHandler; import org.opensearch.sql.spark.storage.SparkStorageFactory; import org.opensearch.sql.spark.transport.TransportCancelAsyncQueryRequestAction; import org.opensearch.sql.spark.transport.TransportCreateAsyncQueryRequestAction; @@ -148,7 +147,6 @@ public List getRestHandlers( Metrics.getInstance().registerDefaultMetrics(); return Arrays.asList( - new SampleExtensionRestHandler(this.asyncQueryScheduler), new RestPPLQueryAction(), new RestSqlAction(settings, injector), new RestSqlStatsAction(settings, restController), From f9eeea08420bbd325f63551eded3f61e2ef68594 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Wed, 17 Jul 2024 18:00:50 -0700 Subject: [PATCH 05/18] Fix doc test Signed-off-by: Louis Chu --- .gitignore | 1 + doctest/build.gradle | 25 +++++++++++++++++++++++++ 2 files changed, 26 insertions(+) diff --git a/.gitignore b/.gitignore index 1b892036dd..b9775dea04 100644 --- a/.gitignore +++ b/.gitignore @@ -49,4 +49,5 @@ gen .worktrees http-client.env.json /doctest/sql-cli/ +/doctest/opensearch-job-scheduler/ .factorypath diff --git a/doctest/build.gradle b/doctest/build.gradle index ec5a26b52b..9f700ce07d 100644 --- a/doctest/build.gradle +++ b/doctest/build.gradle @@ -5,6 +5,8 @@ import org.opensearch.gradle.testclusters.RunTask +import java.util.concurrent.Callable + plugins { id 'base' id 'com.wiredforcode.spawn' @@ -109,6 +111,10 @@ if (version_tokens.length > 1) { String mlCommonsRemoteFile = 'https://ci.opensearch.org/ci/dbc/distribution-build-opensearch/' + opensearch_no_snapshot + '/latest/linux/x64/tar/builds/opensearch/plugins/opensearch-ml-' + opensearch_build + '.zip' String mlCommonsPlugin = 'opensearch-ml' +String bwcOpenSearchJSDownload = 'https://ci.opensearch.org/ci/dbc/distribution-build-opensearch/' + opensearch_no_snapshot + '/latest/linux/x64/tar/builds/' + + 'opensearch/plugins/opensearch-job-scheduler-' + opensearch_build + '.zip' +String jsPlugin = 'opensearch-job-scheduler' + testClusters { docTestCluster { // Disable loading of `ML-commons` plugin, because it might be unavailable (not released yet). @@ -133,6 +139,25 @@ testClusters { } })) */ + plugin(provider(new Callable(){ + @Override + RegularFile call() throws Exception { + return new RegularFile() { + @Override + File getAsFile() { + File dir = new File('./doctest/' + jsPlugin) + if (!dir.exists()) { + dir.mkdirs() + } + File f = new File(dir, jsPlugin + '-' + opensearch_build + '.zip') + if (!f.exists()) { + new URL(bwcOpenSearchJSDownload).withInputStream{ ins -> f.withOutputStream{ it << ins } } + } + return fileTree(jsPlugin).getSingleFile() + } + } + } + })) plugin ':opensearch-sql-plugin' testDistribution = 'archive' } From 36fcdef67d5828702f15827020011ae4abd722d6 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Thu, 18 Jul 2024 07:46:31 -0700 Subject: [PATCH 06/18] Add more tests Signed-off-by: Louis Chu --- .../src/main/antlr/SqlBaseParser.g4 | 8 --- .../OpenSearchAsyncQueryScheduler.java | 2 +- .../OpenSearchAsyncQuerySchedulerTest.java | 62 ++++++++++++++++--- doctest/build.gradle | 32 ++++++++-- 4 files changed, 81 insertions(+), 23 deletions(-) diff --git a/async-query-core/src/main/antlr/SqlBaseParser.g4 b/async-query-core/src/main/antlr/SqlBaseParser.g4 index 2d748642f0..a50051715e 100644 --- a/async-query-core/src/main/antlr/SqlBaseParser.g4 +++ b/async-query-core/src/main/antlr/SqlBaseParser.g4 @@ -83,14 +83,6 @@ endLabel : multipartIdentifier ; -beginLabel - : multipartIdentifier COLON - ; - -endLabel - : multipartIdentifier - ; - singleExpression : namedExpression EOF ; diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java index 573b409128..c2bbae8d36 100644 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java @@ -120,7 +120,7 @@ public void updateJob(OpenSearchRefreshIndexJobRequest request) throws IOExcepti ActionFuture updateResponseActionFuture = client.update(updateRequest); updateResponse = updateResponseActionFuture.actionGet(); } catch (DocumentMissingException exception) { - throw new IllegalArgumentException("Job with name: " + request.getName() + " doesn't exist"); + throw new IllegalArgumentException("Job: " + request.getName() + " doesn't exist"); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java index c2765d6de9..4e90d02239 100644 --- a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java +++ b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java @@ -36,6 +36,7 @@ import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.action.ActionFuture; +import org.opensearch.index.engine.DocumentMissingException; import org.opensearch.index.engine.VersionConflictEngineException; import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; import org.opensearch.threadpool.ThreadPool; @@ -44,6 +45,8 @@ public class OpenSearchAsyncQuerySchedulerTest { private static final String TEST_SCHEDULER_INDEX_NAME = "testQS"; + private static final String TEST_JOB_ID = "testJobId"; + @Mock(answer = Answers.RETURNS_DEEP_STUBS) private Client client; @@ -88,7 +91,7 @@ public void testScheduleJob() { OpenSearchRefreshIndexJobRequest request = OpenSearchRefreshIndexJobRequest.builder() - .jobName("testJob") + .jobName(TEST_JOB_ID) .lastUpdateTime(Instant.now()) .build(); @@ -112,7 +115,7 @@ public void testScheduleJobWithExistingJob() { OpenSearchRefreshIndexJobRequest request = OpenSearchRefreshIndexJobRequest.builder() - .jobName("testJob") + .jobName(TEST_JOB_ID) .lastUpdateTime(Instant.now()) .build(); @@ -131,8 +134,6 @@ public void testScheduleJobWithExistingJob() { @Test public void testUnscheduleJob() throws IOException { - String jobId = "testJob"; - when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); when(updateResponseActionFuture.actionGet()).thenReturn(updateResponse); @@ -140,13 +141,13 @@ public void testUnscheduleJob() throws IOException { when(client.update(any(UpdateRequest.class))).thenReturn(updateResponseActionFuture); - scheduler.unscheduleJob(jobId); + scheduler.unscheduleJob(TEST_JOB_ID); ArgumentCaptor captor = ArgumentCaptor.forClass(UpdateRequest.class); verify(client).update(captor.capture()); UpdateRequest capturedRequest = captor.getValue(); - assertEquals(jobId, capturedRequest.id()); + assertEquals(TEST_JOB_ID, capturedRequest.id()); assertEquals(WriteRequest.RefreshPolicy.IMMEDIATE, capturedRequest.getRefreshPolicy()); } @@ -154,7 +155,7 @@ public void testUnscheduleJob() throws IOException { public void testUpdateJob() throws IOException { OpenSearchRefreshIndexJobRequest request = OpenSearchRefreshIndexJobRequest.builder() - .jobName("testJob") + .jobName(TEST_JOB_ID) .lastUpdateTime(Instant.now()) .build(); @@ -177,8 +178,6 @@ public void testUpdateJob() throws IOException { @Test public void testRemoveJob() { - String jobId = "testJob"; - when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); DeleteResponse deleteResponse = mock(DeleteResponse.class); @@ -187,7 +186,7 @@ public void testRemoveJob() { when(client.delete(any(DeleteRequest.class))).thenReturn(deleteResponseActionFuture); - scheduler.removeJob(jobId); + scheduler.removeJob(TEST_JOB_ID); ArgumentCaptor captor = ArgumentCaptor.forClass(DeleteRequest.class); verify(client).delete(captor.capture()); @@ -235,4 +234,47 @@ public void testCreateAsyncQuerySchedulerIndexFailure() throws IOException { "Internal server error while creating .async-query-scheduler index: Error creating index", exception.getMessage()); } + + @Test + public void testUpdateJobNotFound() throws IOException { + OpenSearchRefreshIndexJobRequest request = + OpenSearchRefreshIndexJobRequest.builder() + .jobName(TEST_JOB_ID) + .lastUpdateTime(Instant.now()) + .build(); + + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); + + when(client.update(any(UpdateRequest.class))) + .thenThrow(new DocumentMissingException(null, null)); + + IllegalArgumentException exception = + assertThrows( + IllegalArgumentException.class, + () -> { + scheduler.updateJob(request); + }); + + assertEquals("Job: testJob doesn't exist", exception.getMessage()); + } + + @Test + public void testRemoveJobNotFound() { + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); + + DeleteResponse deleteResponse = mock(DeleteResponse.class); + when(deleteResponseActionFuture.actionGet()).thenReturn(deleteResponse); + when(deleteResponse.getResult()).thenReturn(DocWriteResponse.Result.NOT_FOUND); + + when(client.delete(any(DeleteRequest.class))).thenReturn(deleteResponseActionFuture); + + IllegalArgumentException exception = + assertThrows( + IllegalArgumentException.class, + () -> { + scheduler.removeJob(TEST_JOB_ID); + }); + + assertEquals("Job : testJob doesn't exist", exception.getMessage()); + } } diff --git a/doctest/build.gradle b/doctest/build.gradle index 9f700ce07d..804eafba92 100644 --- a/doctest/build.gradle +++ b/doctest/build.gradle @@ -145,15 +145,39 @@ testClusters { return new RegularFile() { @Override File getAsFile() { - File dir = new File('./doctest/' + jsPlugin) + // Use absolute paths + String basePath = new File('.').getCanonicalPath() + File dir = new File(basePath + File.separator + 'doctest' + File.separator + jsPlugin) + + // Log the directory path for debugging + println("Creating directory: " + dir.getAbsolutePath()) + + // Create directory if it doesn't exist if (!dir.exists()) { - dir.mkdirs() + if (!dir.mkdirs()) { + throw new IOException("Failed to create directory: " + dir.getAbsolutePath()) + } } + + // Define the file path File f = new File(dir, jsPlugin + '-' + opensearch_build + '.zip') + + // Download file if it doesn't exist if (!f.exists()) { - new URL(bwcOpenSearchJSDownload).withInputStream{ ins -> f.withOutputStream{ it << ins } } + println("Downloading file from: " + bwcOpenSearchJSDownload) + println("Saving to file: " + f.getAbsolutePath()) + + new URL(bwcOpenSearchJSDownload).withInputStream { ins -> + f.withOutputStream { it << ins } + } } - return fileTree(jsPlugin).getSingleFile() + + // Check if the file was created successfully + if (!f.exists()) { + throw new FileNotFoundException("File was not created: " + f.getAbsolutePath()) + } + + return fileTree(f.getParent()).matching { include f.getName() }.singleFile } } } From fc4cffce5bb911b896fd5a6e8c559da9b1b2d538 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Thu, 18 Jul 2024 08:23:31 -0700 Subject: [PATCH 07/18] Fix IT Signed-off-by: Louis Chu --- .../OpenSearchAsyncQuerySchedulerTest.java | 4 +- integ-test/build.gradle | 42 +++++++++++++++++++ 2 files changed, 44 insertions(+), 2 deletions(-) diff --git a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java index 4e90d02239..689d1f808e 100644 --- a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java +++ b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java @@ -45,7 +45,7 @@ public class OpenSearchAsyncQuerySchedulerTest { private static final String TEST_SCHEDULER_INDEX_NAME = "testQS"; - private static final String TEST_JOB_ID = "testJobId"; + private static final String TEST_JOB_ID = "testJob"; @Mock(answer = Answers.RETURNS_DEEP_STUBS) private Client client; @@ -192,7 +192,7 @@ public void testRemoveJob() { verify(client).delete(captor.capture()); DeleteRequest capturedRequest = captor.getValue(); - assertEquals(jobId, capturedRequest.id()); + assertEquals(TEST_JOB_ID, capturedRequest.id()); assertEquals(WriteRequest.RefreshPolicy.IMMEDIATE, capturedRequest.getRefreshPolicy()); } diff --git a/integ-test/build.gradle b/integ-test/build.gradle index 22b6d24005..c2312797bc 100644 --- a/integ-test/build.gradle +++ b/integ-test/build.gradle @@ -78,8 +78,23 @@ ext { return repo + "opensearch-security-${securitySnapshotVersion}.zip" } + getJobSchedulerPluginDownloadLink = { -> + var repo = "https://aws.oss.sonatype.org/content/repositories/snapshots/org/opensearch/plugin/" + + "opensearch-job-scheduler/$opensearch_build_snapshot/" + var metadataFile = Paths.get(projectDir.toString(), "build", "job-scheduler-maven-metadata.xml").toAbsolutePath().toFile() + download.run { + src repo + "maven-metadata.xml" + dest metadataFile + } + def metadata = new XmlParser().parse(metadataFile) + def jobSchedulerSnapshotVersion = metadata.versioning.snapshotVersions[0].snapshotVersion[0].value[0].text() + + return repo + "opensearch-job-scheduler-${jobSchedulerSnapshotVersion}.zip" + } + var projectAbsPath = projectDir.getAbsolutePath() File downloadedSecurityPlugin = Paths.get(projectAbsPath, 'bin', 'opensearch-security-snapshot.zip').toFile() + File downloadedJobSchedulerPlugin = Paths.get(projectAbsPath, 'bin', 'opensearch-job-scheduler-snapshot.zip').toFile() configureSecurityPlugin = { OpenSearchCluster cluster -> @@ -138,6 +153,21 @@ ext { cluster.plugin provider((Callable) (() -> (RegularFile) (() -> downloadedSecurityPlugin))) } + + configureJobSchedulerPlugin = { OpenSearchCluster cluster -> + + // add a check to avoid re-downloading multiple times during single test run + if (!downloadedJobSchedulerPlugin.exists()) { + download.run { + src getJobSchedulerPluginDownloadLink() + dest downloadedJobSchedulerPlugin + } + } else { + println "Job Scheduler Plugin File Already Exists" + } + + cluster.plugin provider((Callable) (() -> (RegularFile) (() -> downloadedJobSchedulerPlugin))) + } } tasks.withType(licenseHeaders.class) { @@ -191,6 +221,7 @@ dependencies { testCompileOnly 'org.apiguardian:apiguardian-api:1.1.2' // Needed for BWC tests + zipArchive group: 'org.opensearch.plugin', name:'opensearch-job-scheduler', version: "${opensearch_build}" zipArchive group: 'org.opensearch.plugin', name:'opensearch-sql-plugin', version: "${bwcVersion}-SNAPSHOT" } @@ -231,6 +262,16 @@ testClusters { } integTestWithSecurity { testDistribution = 'archive' + plugin(provider({ + new RegularFile() { + @Override + File getAsFile() { + return configurations.zipArchive.asFileTree.matching { + include '**/opensearch-job-scheduler*' + }.singleFile + } + } + })) plugin ":opensearch-sql-plugin" } remoteIntegTestWithSecurity { @@ -323,6 +364,7 @@ task integTestWithSecurity(type: RestIntegTestTask) { getClusters().stream().map(cluster -> cluster.getName()).collect(Collectors.joining(",")) getClusters().forEach { cluster -> + configureJobSchedulerPlugin(cluster) configureSecurityPlugin(cluster) } From 674925485fbe70bbc75934da289b99157d9c4d17 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Thu, 18 Jul 2024 16:45:39 -0700 Subject: [PATCH 08/18] Fix IT with security Signed-off-by: Louis Chu --- integ-test/build.gradle | 41 ++++++++++------------------------------- 1 file changed, 10 insertions(+), 31 deletions(-) diff --git a/integ-test/build.gradle b/integ-test/build.gradle index c2312797bc..33c137594d 100644 --- a/integ-test/build.gradle +++ b/integ-test/build.gradle @@ -78,23 +78,8 @@ ext { return repo + "opensearch-security-${securitySnapshotVersion}.zip" } - getJobSchedulerPluginDownloadLink = { -> - var repo = "https://aws.oss.sonatype.org/content/repositories/snapshots/org/opensearch/plugin/" + - "opensearch-job-scheduler/$opensearch_build_snapshot/" - var metadataFile = Paths.get(projectDir.toString(), "build", "job-scheduler-maven-metadata.xml").toAbsolutePath().toFile() - download.run { - src repo + "maven-metadata.xml" - dest metadataFile - } - def metadata = new XmlParser().parse(metadataFile) - def jobSchedulerSnapshotVersion = metadata.versioning.snapshotVersions[0].snapshotVersion[0].value[0].text() - - return repo + "opensearch-job-scheduler-${jobSchedulerSnapshotVersion}.zip" - } - var projectAbsPath = projectDir.getAbsolutePath() File downloadedSecurityPlugin = Paths.get(projectAbsPath, 'bin', 'opensearch-security-snapshot.zip').toFile() - File downloadedJobSchedulerPlugin = Paths.get(projectAbsPath, 'bin', 'opensearch-job-scheduler-snapshot.zip').toFile() configureSecurityPlugin = { OpenSearchCluster cluster -> @@ -153,21 +138,6 @@ ext { cluster.plugin provider((Callable) (() -> (RegularFile) (() -> downloadedSecurityPlugin))) } - - configureJobSchedulerPlugin = { OpenSearchCluster cluster -> - - // add a check to avoid re-downloading multiple times during single test run - if (!downloadedJobSchedulerPlugin.exists()) { - download.run { - src getJobSchedulerPluginDownloadLink() - dest downloadedJobSchedulerPlugin - } - } else { - println "Job Scheduler Plugin File Already Exists" - } - - cluster.plugin provider((Callable) (() -> (RegularFile) (() -> downloadedJobSchedulerPlugin))) - } } tasks.withType(licenseHeaders.class) { @@ -276,6 +246,16 @@ testClusters { } remoteIntegTestWithSecurity { testDistribution = 'archive' + plugin(provider({ + new RegularFile() { + @Override + File getAsFile() { + return configurations.zipArchive.asFileTree.matching { + include '**/opensearch-job-scheduler*' + }.singleFile + } + } + })) plugin ":opensearch-sql-plugin" } } @@ -364,7 +344,6 @@ task integTestWithSecurity(type: RestIntegTestTask) { getClusters().stream().map(cluster -> cluster.getName()).collect(Collectors.joining(",")) getClusters().forEach { cluster -> - configureJobSchedulerPlugin(cluster) configureSecurityPlugin(cluster) } From 981fcf38aaea785a0ace5999a5af2ae37c84b2d8 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Thu, 18 Jul 2024 17:22:15 -0700 Subject: [PATCH 09/18] Improve test coverage Signed-off-by: Louis Chu --- .../job/OpenSearchRefreshIndexJob.java | 40 +++------- .../OpenSearchAsyncQuerySchedulerTest.java | 59 ++++++++++++++- .../job/OpenSearchRefreshIndexJobTest.java | 73 +++++++++++-------- 3 files changed, 107 insertions(+), 65 deletions(-) diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java index 787eca9e0c..2e2f27989b 100644 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java @@ -9,11 +9,9 @@ import org.apache.logging.log4j.Logger; import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.core.action.ActionListener; import org.opensearch.jobscheduler.spi.JobExecutionContext; import org.opensearch.jobscheduler.spi.ScheduledJobParameter; import org.opensearch.jobscheduler.spi.ScheduledJobRunner; -import org.opensearch.jobscheduler.spi.utils.LockService; import org.opensearch.plugins.Plugin; import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; import org.opensearch.threadpool.ThreadPool; @@ -85,40 +83,20 @@ public void runJob(ScheduledJobParameter jobParameter, JobExecutionContext conte throw new IllegalStateException("ThreadPool is not initialized."); } - final LockService lockService = context.getLockService(); + if (this.client == null) { + throw new IllegalStateException("Client is not initialized."); + } Runnable runnable = () -> { - if (jobParameter.getLockDurationSeconds() != null) { - lockService.acquireLock( - jobParameter, - context, - ActionListener.wrap( - lock -> { - if (lock == null) { - return; - } - - // TODO: add logic to refresh index - log.info("Running job: " + jobParameter.getName()); - - lockService.release( - lock, - ActionListener.wrap( - released -> { - log.info("Released lock for job {}", jobParameter.getName()); - }, - exception -> { - throw new IllegalStateException( - "Failed to release lock.", exception); - })); - }, - exception -> { - throw new IllegalStateException("Failed to acquire lock.", exception); - })); - } + doRefresh(jobParameter.getName()); }; threadPool.generic().submit(runnable); } + + void doRefresh(String refreshIndex) { + // TODO: add logic to refresh index + log.info("Scheduled refresh index job on : " + refreshIndex); + } } diff --git a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java index 689d1f808e..7e86352091 100644 --- a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java +++ b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java @@ -132,6 +132,25 @@ public void testScheduleJobWithExistingJob() { assertEquals("A job already exists with name: testJob", exception.getMessage()); } + @Test + public void testScheduleJobWithException() { + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) + .thenReturn(Boolean.FALSE); + when(client.admin().indices().create(any(CreateIndexRequest.class))) + .thenReturn(createIndexResponseActionFuture); + when(createIndexResponseActionFuture.actionGet()) + .thenReturn(new CreateIndexResponse(true, true, TEST_SCHEDULER_INDEX_NAME)); + when(client.index(any(IndexRequest.class))).thenThrow(new RuntimeException("Test exception")); + + OpenSearchRefreshIndexJobRequest request = + OpenSearchRefreshIndexJobRequest.builder() + .jobName(TEST_JOB_ID) + .lastUpdateTime(Instant.now()) + .build(); + + assertThrows(RuntimeException.class, () -> scheduler.scheduleJob(request)); + } + @Test public void testUnscheduleJob() throws IOException { when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); @@ -151,6 +170,13 @@ public void testUnscheduleJob() throws IOException { assertEquals(WriteRequest.RefreshPolicy.IMMEDIATE, capturedRequest.getRefreshPolicy()); } + @Test + public void testUnscheduleJobWithIndexNotFound() { + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(false); + + assertThrows(IllegalArgumentException.class, () -> scheduler.unscheduleJob(TEST_JOB_ID)); + } + @Test public void testUpdateJob() throws IOException { OpenSearchRefreshIndexJobRequest request = @@ -176,6 +202,19 @@ public void testUpdateJob() throws IOException { assertEquals(WriteRequest.RefreshPolicy.IMMEDIATE, capturedRequest.getRefreshPolicy()); } + @Test + public void testUpdateJobWithIndexNotFound() { + OpenSearchRefreshIndexJobRequest request = + OpenSearchRefreshIndexJobRequest.builder() + .jobName(TEST_JOB_ID) + .lastUpdateTime(Instant.now()) + .build(); + + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(false); + + assertThrows(IllegalArgumentException.class, () -> scheduler.updateJob(request)); + } + @Test public void testRemoveJob() { when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); @@ -196,6 +235,13 @@ public void testRemoveJob() { assertEquals(WriteRequest.RefreshPolicy.IMMEDIATE, capturedRequest.getRefreshPolicy()); } + @Test + public void testRemoveJobWithIndexNotFound() { + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(false); + + assertThrows(IllegalArgumentException.class, () -> scheduler.removeJob(TEST_JOB_ID)); + } + @Test public void testCreateAsyncQuerySchedulerIndex() { when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(false); @@ -217,7 +263,7 @@ public void testCreateAsyncQuerySchedulerIndex() { } @Test - public void testCreateAsyncQuerySchedulerIndexFailure() throws IOException { + public void testCreateAsyncQuerySchedulerIndexFailure() { when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(false); when(client.admin().indices().create(any(CreateIndexRequest.class))) @@ -236,7 +282,7 @@ public void testCreateAsyncQuerySchedulerIndexFailure() throws IOException { } @Test - public void testUpdateJobNotFound() throws IOException { + public void testUpdateJobNotFound() { OpenSearchRefreshIndexJobRequest request = OpenSearchRefreshIndexJobRequest.builder() .jobName(TEST_JOB_ID) @@ -277,4 +323,13 @@ public void testRemoveJobNotFound() { assertEquals("Job : testJob doesn't exist", exception.getMessage()); } + + @Test + public void testRemoveJobWithException() { + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); + + when(client.delete(any(DeleteRequest.class))).thenThrow(new RuntimeException("Test exception")); + + assertThrows(RuntimeException.class, () -> scheduler.removeJob(TEST_JOB_ID)); + } } diff --git a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java index c19d696e0e..8061b1a217 100644 --- a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java +++ b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java @@ -6,11 +6,9 @@ package org.opensearch.sql.spark.scheduler.job; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.mockito.ArgumentMatchers.any; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; import java.time.Instant; import org.junit.jupiter.api.BeforeEach; @@ -21,10 +19,8 @@ import org.mockito.MockitoAnnotations; import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.core.action.ActionListener; import org.opensearch.jobscheduler.spi.JobExecutionContext; import org.opensearch.jobscheduler.spi.ScheduledJobParameter; -import org.opensearch.jobscheduler.spi.utils.LockService; import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; import org.opensearch.threadpool.ThreadPool; @@ -41,18 +37,18 @@ public class OpenSearchRefreshIndexJobTest { @Mock private JobExecutionContext context; - @Mock private LockService lockService; - private OpenSearchRefreshIndexJob jobRunner; + private OpenSearchRefreshIndexJob spyJobRunner; + @BeforeEach public void setup() { MockitoAnnotations.openMocks(this); jobRunner = OpenSearchRefreshIndexJob.getJobRunnerInstance(); - jobRunner.setClusterService(clusterService); - jobRunner.setThreadPool(threadPool); - jobRunner.setClient(client); - when(context.getLockService()).thenReturn(lockService); + spyJobRunner = spy(jobRunner); + spyJobRunner.setClusterService(clusterService); + spyJobRunner.setThreadPool(threadPool); + spyJobRunner.setClient(client); } @Test @@ -64,7 +60,7 @@ public void testRunJobWithCorrectParameter() { .lockDurationSeconds(10L) .build(); - jobRunner.runJob(jobParameter, context); + spyJobRunner.runJob(jobParameter, context); ArgumentCaptor captor = ArgumentCaptor.forClass(Runnable.class); verify(threadPool.generic()).submit(captor.capture()); @@ -72,21 +68,23 @@ public void testRunJobWithCorrectParameter() { Runnable runnable = captor.getValue(); runnable.run(); - verify(lockService).acquireLock(eq(jobParameter), eq(context), any(ActionListener.class)); + verify(spyJobRunner).doRefresh(eq(jobParameter.getName())); } @Test public void testRunJobWithIncorrectParameter() { ScheduledJobParameter wrongParameter = mock(ScheduledJobParameter.class); - try { - jobRunner.runJob(wrongParameter, context); - } catch (IllegalStateException e) { - assertEquals( - "Job parameter is not instance of OpenSearchRefreshIndexJobRequest, type: " - + wrongParameter.getClass().getCanonicalName(), - e.getMessage()); - } + IllegalStateException exception = + assertThrows( + IllegalStateException.class, + () -> jobRunner.runJob(wrongParameter, context), + "Expected IllegalStateException but no exception was thrown"); + + assertEquals( + "Job parameter is not instance of OpenSearchRefreshIndexJobRequest, type: " + + wrongParameter.getClass().getCanonicalName(), + exception.getMessage()); } @Test @@ -100,18 +98,29 @@ public void testRunJobWithUninitializedServices() { OpenSearchRefreshIndexJob uninitializedJobRunner = OpenSearchRefreshIndexJob.getJobRunnerInstance(); - try { - uninitializedJobRunner.runJob(jobParameter, context); - } catch (IllegalStateException e) { - assertEquals("ClusterService is not initialized.", e.getMessage()); - } + IllegalStateException exception = + assertThrows( + IllegalStateException.class, + () -> uninitializedJobRunner.runJob(jobParameter, context), + "Expected IllegalStateException but no exception was thrown"); + assertEquals("ClusterService is not initialized.", exception.getMessage()); uninitializedJobRunner.setClusterService(clusterService); - try { - uninitializedJobRunner.runJob(jobParameter, context); - } catch (IllegalStateException e) { - assertEquals("ThreadPool is not initialized.", e.getMessage()); - } + exception = + assertThrows( + IllegalStateException.class, + () -> uninitializedJobRunner.runJob(jobParameter, context), + "Expected IllegalStateException but no exception was thrown"); + assertEquals("ThreadPool is not initialized.", exception.getMessage()); + + uninitializedJobRunner.setThreadPool(threadPool); + + exception = + assertThrows( + IllegalStateException.class, + () -> uninitializedJobRunner.runJob(jobParameter, context), + "Expected IllegalStateException but no exception was thrown"); + assertEquals("Client is not initialized.", exception.getMessage()); } } From 5059dffc44d19f7346ea5eb6e77df67942c295c8 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Thu, 18 Jul 2024 17:58:01 -0700 Subject: [PATCH 10/18] Fix integTest cluster Signed-off-by: Louis Chu --- integ-test/build.gradle | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/integ-test/build.gradle b/integ-test/build.gradle index 33c137594d..d5f61650ea 100644 --- a/integ-test/build.gradle +++ b/integ-test/build.gradle @@ -223,11 +223,31 @@ testClusters.all { testClusters { integTest { testDistribution = 'archive' + plugin(provider({ + new RegularFile() { + @Override + File getAsFile() { + return configurations.zipArchive.asFileTree.matching { + include '**/opensearch-job-scheduler*' + }.singleFile + } + } + })) plugin ":opensearch-sql-plugin" setting "plugins.query.datasources.encryption.masterkey", "1234567812345678" } remoteCluster { testDistribution = 'archive' + plugin(provider({ + new RegularFile() { + @Override + File getAsFile() { + return configurations.zipArchive.asFileTree.matching { + include '**/opensearch-job-scheduler*' + }.singleFile + } + } + })) plugin ":opensearch-sql-plugin" } integTestWithSecurity { From 41c954f5df00bbf8ba9d84bd64d0d0139a5a7952 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Fri, 19 Jul 2024 09:12:47 -0700 Subject: [PATCH 11/18] Fix UT Signed-off-by: Louis Chu --- .../job/OpenSearchRefreshIndexJobTest.java | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java index 8061b1a217..072184821c 100644 --- a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java +++ b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java @@ -8,7 +8,9 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; import java.time.Instant; import org.junit.jupiter.api.BeforeEach; @@ -46,13 +48,17 @@ public void setup() { MockitoAnnotations.openMocks(this); jobRunner = OpenSearchRefreshIndexJob.getJobRunnerInstance(); spyJobRunner = spy(jobRunner); - spyJobRunner.setClusterService(clusterService); - spyJobRunner.setThreadPool(threadPool); - spyJobRunner.setClient(client); + spyJobRunner.setClusterService(null); + spyJobRunner.setThreadPool(null); + spyJobRunner.setClient(null); } @Test public void testRunJobWithCorrectParameter() { + spyJobRunner.setClusterService(clusterService); + spyJobRunner.setThreadPool(threadPool); + spyJobRunner.setClient(client); + OpenSearchRefreshIndexJobRequest jobParameter = OpenSearchRefreshIndexJobRequest.builder() .jobName("testJob") @@ -73,6 +79,10 @@ public void testRunJobWithCorrectParameter() { @Test public void testRunJobWithIncorrectParameter() { + spyJobRunner.setClusterService(clusterService); + spyJobRunner.setThreadPool(threadPool); + spyJobRunner.setClient(client); + ScheduledJobParameter wrongParameter = mock(ScheduledJobParameter.class); IllegalStateException exception = @@ -95,31 +105,28 @@ public void testRunJobWithUninitializedServices() { .lastUpdateTime(Instant.now()) .build(); - OpenSearchRefreshIndexJob uninitializedJobRunner = - OpenSearchRefreshIndexJob.getJobRunnerInstance(); - IllegalStateException exception = assertThrows( IllegalStateException.class, - () -> uninitializedJobRunner.runJob(jobParameter, context), + () -> jobRunner.runJob(jobParameter, context), "Expected IllegalStateException but no exception was thrown"); assertEquals("ClusterService is not initialized.", exception.getMessage()); - uninitializedJobRunner.setClusterService(clusterService); + jobRunner.setClusterService(clusterService); exception = assertThrows( IllegalStateException.class, - () -> uninitializedJobRunner.runJob(jobParameter, context), + () -> jobRunner.runJob(jobParameter, context), "Expected IllegalStateException but no exception was thrown"); assertEquals("ThreadPool is not initialized.", exception.getMessage()); - uninitializedJobRunner.setThreadPool(threadPool); + jobRunner.setThreadPool(threadPool); exception = assertThrows( IllegalStateException.class, - () -> uninitializedJobRunner.runJob(jobParameter, context), + () -> jobRunner.runJob(jobParameter, context), "Expected IllegalStateException but no exception was thrown"); assertEquals("Client is not initialized.", exception.getMessage()); } From 4519fe5b47a8ac94003724ab3afcd5e208669c61 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Mon, 22 Jul 2024 10:31:20 -0700 Subject: [PATCH 12/18] Update UT Signed-off-by: Louis Chu --- async-query/build.gradle | 1 + .../OpenSearchAsyncQueryScheduler.java | 59 ---------- ...penSearchRefreshIndexJobRequestParser.java | 71 ++++++++++++ .../job/OpenSearchRefreshIndexJob.java | 12 +- .../OpenSearchAsyncQuerySchedulerTest.java | 109 +++++++++++++++++- .../job/OpenSearchRefreshIndexJobTest.java | 53 +++++++-- .../org/opensearch/sql/plugin/SQLPlugin.java | 3 +- 7 files changed, 226 insertions(+), 82 deletions(-) create mode 100644 async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchRefreshIndexJobRequestParser.java diff --git a/async-query/build.gradle b/async-query/build.gradle index c885e2efb3..abda6161d3 100644 --- a/async-query/build.gradle +++ b/async-query/build.gradle @@ -99,6 +99,7 @@ jacocoTestCoverageVerification { // ignore because XContext IOException 'org.opensearch.sql.spark.execution.statestore.StateStore', 'org.opensearch.sql.spark.rest.*', + 'org.opensearch.sql.spark.scheduler.OpenSearchRefreshIndexJobRequestParser', 'org.opensearch.sql.spark.transport.model.*' ] limit { diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java index c2bbae8d36..5b49bf2647 100644 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java @@ -30,13 +30,9 @@ import org.opensearch.common.action.ActionFuture; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.xcontent.json.JsonXContent; -import org.opensearch.core.xcontent.XContentParser; -import org.opensearch.core.xcontent.XContentParserUtils; import org.opensearch.index.engine.DocumentMissingException; import org.opensearch.index.engine.VersionConflictEngineException; -import org.opensearch.jobscheduler.spi.ScheduledJobParser; import org.opensearch.jobscheduler.spi.ScheduledJobRunner; -import org.opensearch.jobscheduler.spi.schedule.ScheduleParser; import org.opensearch.sql.spark.scheduler.job.OpenSearchRefreshIndexJob; import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; import org.opensearch.threadpool.ThreadPool; @@ -194,59 +190,4 @@ void createAsyncQuerySchedulerIndex() { public static ScheduledJobRunner getJobRunner() { return OpenSearchRefreshIndexJob.getJobRunnerInstance(); } - - /** Returns the job parser instance for the scheduler. */ - public static ScheduledJobParser getJobParser() { - return (parser, id, jobDocVersion) -> { - OpenSearchRefreshIndexJobRequest.OpenSearchRefreshIndexJobRequestBuilder builder = - OpenSearchRefreshIndexJobRequest.builder(); - XContentParserUtils.ensureExpectedToken( - XContentParser.Token.START_OBJECT, parser.nextToken(), parser); - - while (!parser.nextToken().equals(XContentParser.Token.END_OBJECT)) { - String fieldName = parser.currentName(); - parser.nextToken(); - switch (fieldName) { - case OpenSearchRefreshIndexJobRequest.JOB_NAME_FIELD: - builder.jobName(parser.text()); - break; - case OpenSearchRefreshIndexJobRequest.JOB_TYPE_FIELD: - builder.jobType(parser.text()); - break; - case OpenSearchRefreshIndexJobRequest.ENABLED_FIELD: - builder.enabled(parser.booleanValue()); - break; - case OpenSearchRefreshIndexJobRequest.ENABLED_TIME_FIELD: - builder.enabledTime(parseInstantValue(parser)); - break; - case OpenSearchRefreshIndexJobRequest.LAST_UPDATE_TIME_FIELD: - builder.lastUpdateTime(parseInstantValue(parser)); - break; - case OpenSearchRefreshIndexJobRequest.SCHEDULE_FIELD: - builder.schedule(ScheduleParser.parse(parser)); - break; - case OpenSearchRefreshIndexJobRequest.LOCK_DURATION_SECONDS: - builder.lockDurationSeconds(parser.longValue()); - break; - case OpenSearchRefreshIndexJobRequest.JITTER: - builder.jitter(parser.doubleValue()); - break; - default: - XContentParserUtils.throwUnknownToken(parser.currentToken(), parser.getTokenLocation()); - } - } - return builder.build(); - }; - } - - private static Instant parseInstantValue(XContentParser parser) throws IOException { - if (XContentParser.Token.VALUE_NULL.equals(parser.currentToken())) { - return null; - } - if (parser.currentToken().isValue()) { - return Instant.ofEpochMilli(parser.longValue()); - } - XContentParserUtils.throwUnknownToken(parser.currentToken(), parser.getTokenLocation()); - return null; - } } diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchRefreshIndexJobRequestParser.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchRefreshIndexJobRequestParser.java new file mode 100644 index 0000000000..0422e7c015 --- /dev/null +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchRefreshIndexJobRequestParser.java @@ -0,0 +1,71 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.spark.scheduler; + +import java.io.IOException; +import java.time.Instant; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.core.xcontent.XContentParserUtils; +import org.opensearch.jobscheduler.spi.ScheduledJobParser; +import org.opensearch.jobscheduler.spi.schedule.ScheduleParser; +import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; + +public class OpenSearchRefreshIndexJobRequestParser { + + private static Instant parseInstantValue(XContentParser parser) throws IOException { + if (XContentParser.Token.VALUE_NULL.equals(parser.currentToken())) { + return null; + } + if (parser.currentToken().isValue()) { + return Instant.ofEpochMilli(parser.longValue()); + } + XContentParserUtils.throwUnknownToken(parser.currentToken(), parser.getTokenLocation()); + return null; + } + + public static ScheduledJobParser getJobParser() { + return (parser, id, jobDocVersion) -> { + OpenSearchRefreshIndexJobRequest.OpenSearchRefreshIndexJobRequestBuilder builder = + OpenSearchRefreshIndexJobRequest.builder(); + XContentParserUtils.ensureExpectedToken( + XContentParser.Token.START_OBJECT, parser.nextToken(), parser); + + while (!parser.nextToken().equals(XContentParser.Token.END_OBJECT)) { + String fieldName = parser.currentName(); + parser.nextToken(); + switch (fieldName) { + case OpenSearchRefreshIndexJobRequest.JOB_NAME_FIELD: + builder.jobName(parser.text()); + break; + case OpenSearchRefreshIndexJobRequest.JOB_TYPE_FIELD: + builder.jobType(parser.text()); + break; + case OpenSearchRefreshIndexJobRequest.ENABLED_FIELD: + builder.enabled(parser.booleanValue()); + break; + case OpenSearchRefreshIndexJobRequest.ENABLED_TIME_FIELD: + builder.enabledTime(parseInstantValue(parser)); + break; + case OpenSearchRefreshIndexJobRequest.LAST_UPDATE_TIME_FIELD: + builder.lastUpdateTime(parseInstantValue(parser)); + break; + case OpenSearchRefreshIndexJobRequest.SCHEDULE_FIELD: + builder.schedule(ScheduleParser.parse(parser)); + break; + case OpenSearchRefreshIndexJobRequest.LOCK_DURATION_SECONDS: + builder.lockDurationSeconds(parser.longValue()); + break; + case OpenSearchRefreshIndexJobRequest.JITTER: + builder.jitter(parser.doubleValue()); + break; + default: + XContentParserUtils.throwUnknownToken(parser.currentToken(), parser.getTokenLocation()); + } + } + return builder.build(); + }; + } +} diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java index 2e2f27989b..d15757777d 100644 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java @@ -33,18 +33,10 @@ public class OpenSearchRefreshIndexJob implements ScheduledJobRunner { private static final Logger log = LogManager.getLogger(OpenSearchRefreshIndexJob.class); - private static OpenSearchRefreshIndexJob INSTANCE; + public static OpenSearchRefreshIndexJob INSTANCE = new OpenSearchRefreshIndexJob(); public static OpenSearchRefreshIndexJob getJobRunnerInstance() { - if (INSTANCE != null) { - return INSTANCE; - } - synchronized (OpenSearchRefreshIndexJob.class) { - if (INSTANCE == null) { - INSTANCE = new OpenSearchRefreshIndexJob(); - } - return INSTANCE; - } + return INSTANCE; } private ClusterService clusterService; diff --git a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java index 7e86352091..b61069ca5b 100644 --- a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java +++ b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java @@ -6,15 +6,18 @@ package org.opensearch.sql.spark.scheduler; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.opensearch.sql.spark.scheduler.OpenSearchAsyncQueryScheduler.SCHEDULER_INDEX_NAME; import java.io.IOException; import java.time.Instant; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Answers; @@ -38,6 +41,7 @@ import org.opensearch.common.action.ActionFuture; import org.opensearch.index.engine.DocumentMissingException; import org.opensearch.index.engine.VersionConflictEngineException; +import org.opensearch.jobscheduler.spi.ScheduledJobRunner; import org.opensearch.sql.spark.scheduler.model.OpenSearchRefreshIndexJobRequest; import org.opensearch.threadpool.ThreadPool; @@ -86,6 +90,8 @@ public void testScheduleJob() { when(createIndexResponseActionFuture.actionGet()) .thenReturn(new CreateIndexResponse(true, true, TEST_SCHEDULER_INDEX_NAME)); when(client.index(any(IndexRequest.class))).thenReturn(indexResponseActionFuture); + + // Test the if case when(indexResponseActionFuture.actionGet()).thenReturn(indexResponse); when(indexResponse.getResult()).thenReturn(DocWriteResponse.Result.CREATED); @@ -98,11 +104,11 @@ public void testScheduleJob() { scheduler.scheduleJob(request); // Verify index created - verify(client.admin().indices(), Mockito.times(1)).create(ArgumentMatchers.any()); + verify(client.admin().indices(), times(1)).create(ArgumentMatchers.any()); // Verify doc indexed ArgumentCaptor captor = ArgumentCaptor.forClass(IndexRequest.class); - verify(client, Mockito.times(1)).index(captor.capture()); + verify(client, times(1)).index(captor.capture()); IndexRequest capturedRequest = captor.getValue(); assertEquals(request.getName(), capturedRequest.id()); assertEquals(WriteRequest.RefreshPolicy.IMMEDIATE, capturedRequest.getRefreshPolicy()); @@ -128,12 +134,12 @@ public void testScheduleJobWithExistingJob() { scheduler.scheduleJob(request); }); - verify(client, Mockito.times(1)).index(ArgumentCaptor.forClass(IndexRequest.class).capture()); + verify(client, times(1)).index(ArgumentCaptor.forClass(IndexRequest.class).capture()); assertEquals("A job already exists with name: testJob", exception.getMessage()); } @Test - public void testScheduleJobWithException() { + public void testScheduleJobWithExceptions() { when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) .thenReturn(Boolean.FALSE); when(client.admin().indices().create(any(CreateIndexRequest.class))) @@ -149,6 +155,14 @@ public void testScheduleJobWithException() { .build(); assertThrows(RuntimeException.class, () -> scheduler.scheduleJob(request)); + + when(client.index(any(IndexRequest.class))).thenReturn(indexResponseActionFuture); + when(indexResponseActionFuture.actionGet()).thenReturn(indexResponse); + when(indexResponse.getResult()).thenReturn(DocWriteResponse.Result.NOT_FOUND); + + RuntimeException exception = + assertThrows(RuntimeException.class, () -> scheduler.scheduleJob(request)); + assertEquals("Schedule job failed with result : not_found", exception.getMessage()); } @Test @@ -168,6 +182,17 @@ public void testUnscheduleJob() throws IOException { UpdateRequest capturedRequest = captor.getValue(); assertEquals(TEST_JOB_ID, capturedRequest.id()); assertEquals(WriteRequest.RefreshPolicy.IMMEDIATE, capturedRequest.getRefreshPolicy()); + + // Reset the captor for the next verification + captor = ArgumentCaptor.forClass(UpdateRequest.class); + + when(updateResponse.getResult()).thenReturn(DocWriteResponse.Result.NOOP); + scheduler.unscheduleJob(TEST_JOB_ID); + + verify(client, times(2)).update(captor.capture()); + capturedRequest = captor.getValue(); + assertEquals(TEST_JOB_ID, capturedRequest.id()); + assertEquals(WriteRequest.RefreshPolicy.IMMEDIATE, capturedRequest.getRefreshPolicy()); } @Test @@ -215,6 +240,47 @@ public void testUpdateJobWithIndexNotFound() { assertThrows(IllegalArgumentException.class, () -> scheduler.updateJob(request)); } + @Test + public void testUpdateJobWithExceptions() { + OpenSearchRefreshIndexJobRequest request = + OpenSearchRefreshIndexJobRequest.builder() + .jobName(TEST_JOB_ID) + .lastUpdateTime(Instant.now()) + .build(); + + when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); + when(client.update(any(UpdateRequest.class))) + .thenThrow(new DocumentMissingException(null, null)); + + IllegalArgumentException exception1 = + assertThrows( + IllegalArgumentException.class, + () -> { + scheduler.updateJob(request); + }); + + assertEquals("Job: testJob doesn't exist", exception1.getMessage()); + + when(client.update(any(UpdateRequest.class))).thenThrow(new RuntimeException("Test exception")); + + RuntimeException exception2 = + assertThrows( + RuntimeException.class, + () -> { + scheduler.updateJob(request); + }); + + assertEquals("java.lang.RuntimeException: Test exception", exception2.getMessage()); + + when(client.update(any(UpdateRequest.class))).thenReturn(updateResponseActionFuture); + when(updateResponseActionFuture.actionGet()).thenReturn(updateResponse); + when(updateResponse.getResult()).thenReturn(DocWriteResponse.Result.NOT_FOUND); + + RuntimeException exception = + assertThrows(RuntimeException.class, () -> scheduler.updateJob(request)); + assertEquals("Update job failed with result : not_found", exception.getMessage()); + } + @Test public void testRemoveJob() { when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); @@ -279,6 +345,24 @@ public void testCreateAsyncQuerySchedulerIndexFailure() { assertEquals( "Internal server error while creating .async-query-scheduler index: Error creating index", exception.getMessage()); + + when(client.admin().indices().create(any(CreateIndexRequest.class))) + .thenReturn(createIndexResponseActionFuture); + Mockito.when(createIndexResponseActionFuture.actionGet()) + .thenReturn(new CreateIndexResponse(false, false, SCHEDULER_INDEX_NAME)); + + OpenSearchRefreshIndexJobRequest request = + OpenSearchRefreshIndexJobRequest.builder() + .jobName(TEST_JOB_ID) + .lastUpdateTime(Instant.now()) + .build(); + + RuntimeException runtimeException = + Assertions.assertThrows(RuntimeException.class, () -> scheduler.scheduleJob(request)); + Assertions.assertEquals( + "Internal server error while creating .async-query-scheduler index: Index creation is not" + + " acknowledged.", + runtimeException.getMessage()); } @Test @@ -325,11 +409,26 @@ public void testRemoveJobNotFound() { } @Test - public void testRemoveJobWithException() { + public void testRemoveJobWithExceptions() { when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(true); when(client.delete(any(DeleteRequest.class))).thenThrow(new RuntimeException("Test exception")); assertThrows(RuntimeException.class, () -> scheduler.removeJob(TEST_JOB_ID)); + + DeleteResponse deleteResponse = mock(DeleteResponse.class); + when(client.delete(any(DeleteRequest.class))).thenReturn(deleteResponseActionFuture); + when(deleteResponseActionFuture.actionGet()).thenReturn(deleteResponse); + when(deleteResponse.getResult()).thenReturn(DocWriteResponse.Result.NOOP); + + RuntimeException runtimeException = + Assertions.assertThrows(RuntimeException.class, () -> scheduler.removeJob(TEST_JOB_ID)); + Assertions.assertEquals("Remove job failed with result : noop", runtimeException.getMessage()); + } + + @Test + public void testGetJobRunner() { + ScheduledJobRunner jobRunner = OpenSearchAsyncQueryScheduler.getJobRunner(); + assertNotNull(jobRunner); } } diff --git a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java index 072184821c..d5d432f3db 100644 --- a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java +++ b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java @@ -6,6 +6,8 @@ package org.opensearch.sql.spark.scheduler.job; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; @@ -13,6 +15,10 @@ import static org.mockito.Mockito.verify; import java.time.Instant; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Answers; @@ -47,14 +53,36 @@ public class OpenSearchRefreshIndexJobTest { public void setup() { MockitoAnnotations.openMocks(this); jobRunner = OpenSearchRefreshIndexJob.getJobRunnerInstance(); - spyJobRunner = spy(jobRunner); - spyJobRunner.setClusterService(null); - spyJobRunner.setThreadPool(null); - spyJobRunner.setClient(null); + jobRunner.setClient(null); + jobRunner.setClusterService(null); + jobRunner.setThreadPool(null); + } + + @Test + public void testGetJobRunnerInstanceCalledConcurrently() throws InterruptedException { + int threadCount = 3; + ExecutorService executorService = Executors.newFixedThreadPool(threadCount); + CountDownLatch latch = new CountDownLatch(threadCount); + + for (int i = 0; i < threadCount; i++) { + executorService.execute( + () -> { + try { + OpenSearchRefreshIndexJob instance = OpenSearchRefreshIndexJob.getJobRunnerInstance(); + assertNotNull(instance); + } finally { + latch.countDown(); + } + }); + } + + latch.await(5, TimeUnit.SECONDS); + executorService.shutdown(); } @Test public void testRunJobWithCorrectParameter() { + spyJobRunner = spy(jobRunner); spyJobRunner.setClusterService(clusterService); spyJobRunner.setThreadPool(threadPool); spyJobRunner.setClient(client); @@ -79,9 +107,10 @@ public void testRunJobWithCorrectParameter() { @Test public void testRunJobWithIncorrectParameter() { - spyJobRunner.setClusterService(clusterService); - spyJobRunner.setThreadPool(threadPool); - spyJobRunner.setClient(client); + jobRunner = OpenSearchRefreshIndexJob.getJobRunnerInstance(); + jobRunner.setClusterService(clusterService); + jobRunner.setThreadPool(threadPool); + jobRunner.setClient(client); ScheduledJobParameter wrongParameter = mock(ScheduledJobParameter.class); @@ -130,4 +159,14 @@ public void testRunJobWithUninitializedServices() { "Expected IllegalStateException but no exception was thrown"); assertEquals("Client is not initialized.", exception.getMessage()); } + + @Test + public void testGetJobRunnerInstanceMultipleCalls() { + OpenSearchRefreshIndexJob instance1 = OpenSearchRefreshIndexJob.getJobRunnerInstance(); + OpenSearchRefreshIndexJob instance2 = OpenSearchRefreshIndexJob.getJobRunnerInstance(); + OpenSearchRefreshIndexJob instance3 = OpenSearchRefreshIndexJob.getJobRunnerInstance(); + + assertSame(instance1, instance2); + assertSame(instance2, instance3); + } } 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 186a53f07a..3f4ab29568 100644 --- a/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java +++ b/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java @@ -95,6 +95,7 @@ import org.opensearch.sql.spark.flint.operation.FlintIndexOpFactory; import org.opensearch.sql.spark.rest.RestAsyncQueryManagementAction; import org.opensearch.sql.spark.scheduler.OpenSearchAsyncQueryScheduler; +import org.opensearch.sql.spark.scheduler.OpenSearchRefreshIndexJobRequestParser; import org.opensearch.sql.spark.storage.SparkStorageFactory; import org.opensearch.sql.spark.transport.TransportCancelAsyncQueryRequestAction; import org.opensearch.sql.spark.transport.TransportCreateAsyncQueryRequestAction; @@ -268,7 +269,7 @@ public ScheduledJobRunner getJobRunner() { @Override public ScheduledJobParser getJobParser() { - return OpenSearchAsyncQueryScheduler.getJobParser(); + return OpenSearchRefreshIndexJobRequestParser.getJobParser(); } @Override From ebe32c5b7365952952daa2f56988a6ec17bffb0d Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Mon, 22 Jul 2024 17:07:35 -0700 Subject: [PATCH 13/18] Fix bwc test Signed-off-by: Louis Chu --- integ-test/build.gradle | 23 ++++++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/integ-test/build.gradle b/integ-test/build.gradle index d5f61650ea..b65049ec0e 100644 --- a/integ-test/build.gradle +++ b/integ-test/build.gradle @@ -80,7 +80,6 @@ ext { var projectAbsPath = projectDir.getAbsolutePath() File downloadedSecurityPlugin = Paths.get(projectAbsPath, 'bin', 'opensearch-security-snapshot.zip').toFile() - configureSecurityPlugin = { OpenSearchCluster cluster -> cluster.getNodes().forEach { node -> @@ -138,6 +137,10 @@ ext { cluster.plugin provider((Callable) (() -> (RegularFile) (() -> downloadedSecurityPlugin))) } + + bwcOpenSearchJSDownload = 'https://ci.opensearch.org/ci/dbc/distribution-build-opensearch/' + baseVersion + '/latest/linux/x64/tar/builds/' + + 'opensearch/plugins/opensearch-job-scheduler-' + bwcVersion + '.zip' + bwcJobSchedulerPath = bwcFilePath + "job-scheduler/" } tasks.withType(licenseHeaders.class) { @@ -543,6 +546,24 @@ task comparisonTest(type: RestIntegTestTask) { testDistribution = "ARCHIVE" versions = [baseVersion, opensearch_version] numberOfNodes = 3 + plugin(provider(new Callable(){ + @Override + RegularFile call() throws Exception { + return new RegularFile() { + @Override + File getAsFile() { + if (new File("$project.rootDir/$bwcFilePath/job-scheduler/$bwcVersion").exists()) { + project.delete(files("$project.rootDir/$bwcFilePath/job-scheduler/$bwcVersion")) + } + project.mkdir bwcJobSchedulerPath + bwcVersion + ant.get(src: bwcOpenSearchJSDownload, + dest: bwcJobSchedulerPath + bwcVersion, + httpusecaches: false) + return fileTree(bwcJobSchedulerPath + bwcVersion).getSingleFile() + } + } + } + })) plugin(provider(new Callable(){ @Override RegularFile call() throws Exception { From 4fea61432a8f5853af7d69d9ba3f1f73d7419dcc Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Tue, 23 Jul 2024 10:09:21 -0700 Subject: [PATCH 14/18] Resolve comments Signed-off-by: Louis Chu --- .../OpenSearchAsyncQueryScheduler.java | 30 +++++++++++-------- .../job/OpenSearchRefreshIndexJob.java | 2 ++ 2 files changed, 19 insertions(+), 13 deletions(-) diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java index 5b49bf2647..a1ba286cab 100644 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java @@ -7,6 +7,7 @@ import static org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS; +import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; @@ -46,6 +47,7 @@ public class OpenSearchAsyncQueryScheduler { private static final String SCHEDULER_INDEX_SETTINGS_FILE_NAME = "async-query-scheduler-index-settings.yml"; private static final Logger LOG = LogManager.getLogger(); + private Client client; private ClusterService clusterService; @@ -69,15 +71,15 @@ public void scheduleJob(OpenSearchRefreshIndexJobRequest request) { indexRequest.id(request.getName()); indexRequest.opType(DocWriteRequest.OpType.CREATE); indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - ActionFuture indexResponseActionFuture; IndexResponse indexResponse; try { indexRequest.source(request.toXContent(JsonXContent.contentBuilder(), EMPTY_PARAMS)); - indexResponseActionFuture = client.index(indexRequest); + ActionFuture indexResponseActionFuture = client.index(indexRequest); indexResponse = indexResponseActionFuture.actionGet(); } catch (VersionConflictEngineException exception) { throw new IllegalArgumentException("A job already exists with name: " + request.getName()); - } catch (Exception e) { + } catch (Throwable e) { + LOG.error("Failed to schedule job : {}", request.getName(), e); throw new RuntimeException(e); } @@ -91,9 +93,7 @@ public void scheduleJob(OpenSearchRefreshIndexJobRequest request) { /** Unschedules a job by marking it as disabled and updating its last update time. */ public void unscheduleJob(String jobId) throws IOException { - if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { - throw new IllegalArgumentException("Job index does not exist."); - } + verifyIndexExists(); OpenSearchRefreshIndexJobRequest request = OpenSearchRefreshIndexJobRequest.builder() .jobName(jobId) @@ -105,9 +105,7 @@ public void unscheduleJob(String jobId) throws IOException { /** Updates an existing job with new parameters. */ public void updateJob(OpenSearchRefreshIndexJobRequest request) throws IOException { - if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { - throw new IllegalArgumentException("Job index does not exist."); - } + verifyIndexExists(); UpdateRequest updateRequest = new UpdateRequest(SCHEDULER_INDEX_NAME, request.getName()); updateRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); updateRequest.doc(request.toXContent(JsonXContent.contentBuilder(), EMPTY_PARAMS)); @@ -117,7 +115,8 @@ public void updateJob(OpenSearchRefreshIndexJobRequest request) throws IOExcepti updateResponse = updateResponseActionFuture.actionGet(); } catch (DocumentMissingException exception) { throw new IllegalArgumentException("Job: " + request.getName() + " doesn't exist"); - } catch (Exception e) { + } catch (Throwable e) { + LOG.error("Failed to update job : {}", request.getName(), e); throw new RuntimeException(e); } @@ -132,9 +131,7 @@ public void updateJob(OpenSearchRefreshIndexJobRequest request) throws IOExcepti /** Removes a job by deleting its document from the index. */ public void removeJob(String jobId) { - if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { - throw new IllegalArgumentException("Job index does not exist."); - } + verifyIndexExists(); DeleteRequest deleteRequest = new DeleteRequest(SCHEDULER_INDEX_NAME, jobId); deleteRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); ActionFuture deleteResponseActionFuture = client.delete(deleteRequest); @@ -151,6 +148,7 @@ public void removeJob(String jobId) { } /** Creates the async query scheduler index with specified mappings and settings. */ + @VisibleForTesting void createAsyncQuerySchedulerIndex() { try { InputStream mappingFileStream = @@ -186,6 +184,12 @@ void createAsyncQuerySchedulerIndex() { } } + private void verifyIndexExists() { + if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { + throw new IllegalArgumentException("Job index does not exist."); + } + } + /** Returns the job runner instance for the scheduler. */ public static ScheduledJobRunner getJobRunner() { return OpenSearchRefreshIndexJob.getJobRunnerInstance(); diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java index d15757777d..73e0b50fc4 100644 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java @@ -9,6 +9,7 @@ import org.apache.logging.log4j.Logger; import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.inject.Singleton; import org.opensearch.jobscheduler.spi.JobExecutionContext; import org.opensearch.jobscheduler.spi.ScheduledJobParameter; import org.opensearch.jobscheduler.spi.ScheduledJobRunner; @@ -29,6 +30,7 @@ * and using singleton job runner to ensure we register a usable job runner instance to JobScheduler * plugin. */ +@Singleton public class OpenSearchRefreshIndexJob implements ScheduledJobRunner { private static final Logger log = LogManager.getLogger(OpenSearchRefreshIndexJob.class); From b8a586e62dbaeb577c6a8551a24efd280773c453 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Tue, 23 Jul 2024 17:53:45 -0700 Subject: [PATCH 15/18] Fix bwc test Signed-off-by: Louis Chu --- integ-test/build.gradle | 75 +++++++++++++++++------------------------ 1 file changed, 30 insertions(+), 45 deletions(-) diff --git a/integ-test/build.gradle b/integ-test/build.gradle index b65049ec0e..3a71ccc82a 100644 --- a/integ-test/build.gradle +++ b/integ-test/build.gradle @@ -223,11 +223,11 @@ testClusters.all { } } -testClusters { - integTest { - testDistribution = 'archive' - plugin(provider({ - new RegularFile() { +def getJobSchedulerPlugin() { + provider(new Callable() { + @Override + RegularFile call() throws Exception { + return new RegularFile() { @Override File getAsFile() { return configurations.zipArchive.asFileTree.matching { @@ -235,50 +235,30 @@ testClusters { }.singleFile } } - })) + } + }) +} + +testClusters { + integTest { + testDistribution = 'archive' + plugin(getJobSchedulerPlugin()) plugin ":opensearch-sql-plugin" setting "plugins.query.datasources.encryption.masterkey", "1234567812345678" } remoteCluster { testDistribution = 'archive' - plugin(provider({ - new RegularFile() { - @Override - File getAsFile() { - return configurations.zipArchive.asFileTree.matching { - include '**/opensearch-job-scheduler*' - }.singleFile - } - } - })) + plugin(getJobSchedulerPlugin()) plugin ":opensearch-sql-plugin" } integTestWithSecurity { testDistribution = 'archive' - plugin(provider({ - new RegularFile() { - @Override - File getAsFile() { - return configurations.zipArchive.asFileTree.matching { - include '**/opensearch-job-scheduler*' - }.singleFile - } - } - })) + plugin(getJobSchedulerPlugin()) plugin ":opensearch-sql-plugin" } remoteIntegTestWithSecurity { testDistribution = 'archive' - plugin(provider({ - new RegularFile() { - @Override - File getAsFile() { - return configurations.zipArchive.asFileTree.matching { - include '**/opensearch-job-scheduler*' - }.singleFile - } - } - })) + plugin(getJobSchedulerPlugin()) plugin ":opensearch-sql-plugin" } } @@ -584,17 +564,18 @@ task comparisonTest(type: RestIntegTestTask) { } List> plugins = [ - provider(new Callable() { - @Override - RegularFile call() throws Exception { - return new RegularFile() { - @Override - File getAsFile() { - return fileTree(bwcFilePath + project.version).getSingleFile() + getJobSchedulerPlugin(), + provider(new Callable() { + @Override + RegularFile call() throws Exception { + return new RegularFile() { + @Override + File getAsFile() { + return fileTree(bwcFilePath + project.version).getSingleFile() + } } } - } - }) + }) ] // Creates 2 test clusters with 3 nodes of the old version. @@ -619,6 +600,7 @@ task "${baseName}#mixedClusterTask"(type: StandaloneRestIntegTestTask) { useCluster testClusters."${baseName}0" dependsOn "${baseName}#oldVersionClusterTask0" doFirst { + println "List of plugins: $plugins" testClusters."${baseName}0".upgradeNodeAndPluginToNextVersion(plugins) } filter { @@ -638,6 +620,7 @@ task "${baseName}#twoThirdsUpgradedClusterTask"(type: StandaloneRestIntegTestTas dependsOn "${baseName}#mixedClusterTask" useCluster testClusters."${baseName}0" doFirst { + println "List of plugins: $plugins" testClusters."${baseName}0".upgradeNodeAndPluginToNextVersion(plugins) } filter { @@ -657,6 +640,7 @@ task "${baseName}#rollingUpgradeClusterTask"(type: StandaloneRestIntegTestTask) dependsOn "${baseName}#twoThirdsUpgradedClusterTask" useCluster testClusters."${baseName}0" doFirst { + println "List of plugins: $plugins" testClusters."${baseName}0".upgradeNodeAndPluginToNextVersion(plugins) } filter { @@ -676,6 +660,7 @@ task "${baseName}#fullRestartClusterTask"(type: StandaloneRestIntegTestTask) { dependsOn "${baseName}#oldVersionClusterTask1" useCluster testClusters."${baseName}1" doFirst { + println "List of plugins: $plugins" testClusters."${baseName}1".upgradeAllNodesAndPluginsToNextVersion(plugins) } filter { From 2b69b66166dcc0b022404d7424717b6503265bef Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Wed, 24 Jul 2024 08:15:07 -0700 Subject: [PATCH 16/18] clean up doc test Signed-off-by: Louis Chu --- .../src/main/antlr/SqlBaseParser.g4 | 17 ++-- doctest/build.gradle | 90 ++++++++++--------- integ-test/build.gradle | 4 - 3 files changed, 58 insertions(+), 53 deletions(-) diff --git a/async-query-core/src/main/antlr/SqlBaseParser.g4 b/async-query-core/src/main/antlr/SqlBaseParser.g4 index a50051715e..c7aa56cf92 100644 --- a/async-query-core/src/main/antlr/SqlBaseParser.g4 +++ b/async-query-core/src/main/antlr/SqlBaseParser.g4 @@ -66,8 +66,8 @@ compoundStatement ; setStatementWithOptionalVarKeyword - : SET (VARIABLE | VAR)? assignmentList #setVariableWithOptionalKeyword - | SET (VARIABLE | VAR)? LEFT_PAREN multipartIdentifierList RIGHT_PAREN EQ + : SET variable? assignmentList #setVariableWithOptionalKeyword + | SET variable? LEFT_PAREN multipartIdentifierList RIGHT_PAREN EQ LEFT_PAREN query RIGHT_PAREN #setVariableWithOptionalKeyword ; @@ -215,9 +215,9 @@ statement routineCharacteristics RETURN (query | expression) #createUserDefinedFunction | DROP TEMPORARY? FUNCTION (IF EXISTS)? identifierReference #dropFunction - | DECLARE (OR REPLACE)? VARIABLE? + | DECLARE (OR REPLACE)? variable? identifierReference dataType? variableDefaultExpression? #createVariable - | DROP TEMPORARY VARIABLE (IF EXISTS)? identifierReference #dropVariable + | DROP TEMPORARY variable (IF EXISTS)? identifierReference #dropVariable | EXPLAIN (LOGICAL | FORMATTED | EXTENDED | CODEGEN | COST)? (statement|setResetStatement) #explain | SHOW TABLES ((FROM | IN) identifierReference)? @@ -272,8 +272,8 @@ setResetStatement | SET TIME ZONE interval #setTimeZone | SET TIME ZONE timezone #setTimeZone | SET TIME ZONE .*? #setTimeZone - | SET (VARIABLE | VAR) assignmentList #setVariable - | SET (VARIABLE | VAR) LEFT_PAREN multipartIdentifierList RIGHT_PAREN EQ + | SET variable assignmentList #setVariable + | SET variable LEFT_PAREN multipartIdentifierList RIGHT_PAREN EQ LEFT_PAREN query RIGHT_PAREN #setVariable | SET configKey EQ configValue #setQuotedConfiguration | SET configKey (EQ .*?)? #setConfiguration @@ -438,6 +438,11 @@ namespaces | SCHEMAS ; +variable + : VARIABLE + | VAR + ; + describeFuncName : identifierReference | stringLit diff --git a/doctest/build.gradle b/doctest/build.gradle index 804eafba92..a125a4f336 100644 --- a/doctest/build.gradle +++ b/doctest/build.gradle @@ -139,49 +139,7 @@ testClusters { } })) */ - plugin(provider(new Callable(){ - @Override - RegularFile call() throws Exception { - return new RegularFile() { - @Override - File getAsFile() { - // Use absolute paths - String basePath = new File('.').getCanonicalPath() - File dir = new File(basePath + File.separator + 'doctest' + File.separator + jsPlugin) - - // Log the directory path for debugging - println("Creating directory: " + dir.getAbsolutePath()) - - // Create directory if it doesn't exist - if (!dir.exists()) { - if (!dir.mkdirs()) { - throw new IOException("Failed to create directory: " + dir.getAbsolutePath()) - } - } - - // Define the file path - File f = new File(dir, jsPlugin + '-' + opensearch_build + '.zip') - - // Download file if it doesn't exist - if (!f.exists()) { - println("Downloading file from: " + bwcOpenSearchJSDownload) - println("Saving to file: " + f.getAbsolutePath()) - - new URL(bwcOpenSearchJSDownload).withInputStream { ins -> - f.withOutputStream { it << ins } - } - } - - // Check if the file was created successfully - if (!f.exists()) { - throw new FileNotFoundException("File was not created: " + f.getAbsolutePath()) - } - - return fileTree(f.getParent()).matching { include f.getName() }.singleFile - } - } - } - })) + plugin(getJobSchedulerPlugin(jsPlugin, bwcOpenSearchJSDownload)) plugin ':opensearch-sql-plugin' testDistribution = 'archive' } @@ -208,3 +166,49 @@ spotless { googleJavaFormat('1.17.0').reflowLongStrings().groupArtifact('com.google.googlejavaformat:google-java-format') } } + +def getJobSchedulerPlugin(String jsPlugin, String bwcOpenSearchJSDownload) { + return provider(new Callable() { + @Override + RegularFile call() throws Exception { + return new RegularFile() { + @Override + File getAsFile() { + // Use absolute paths + String basePath = new File('.').getCanonicalPath() + File dir = new File(basePath + File.separator + 'doctest' + File.separator + jsPlugin) + + // Log the directory path for debugging + println("Creating directory: " + dir.getAbsolutePath()) + + // Create directory if it doesn't exist + if (!dir.exists()) { + if (!dir.mkdirs()) { + throw new IOException("Failed to create directory: " + dir.getAbsolutePath()) + } + } + + // Define the file path + File f = new File(dir, jsPlugin + '-' + opensearch_build + '.zip') + + // Download file if it doesn't exist + if (!f.exists()) { + println("Downloading file from: " + bwcOpenSearchJSDownload) + println("Saving to file: " + f.getAbsolutePath()) + + new URL(bwcOpenSearchJSDownload).withInputStream { ins -> + f.withOutputStream { it << ins } + } + } + + // Check if the file was created successfully + if (!f.exists()) { + throw new FileNotFoundException("File was not created: " + f.getAbsolutePath()) + } + + return fileTree(f.getParent()).matching { include f.getName() }.singleFile + } + } + } + }) +} diff --git a/integ-test/build.gradle b/integ-test/build.gradle index 3a71ccc82a..bae2361041 100644 --- a/integ-test/build.gradle +++ b/integ-test/build.gradle @@ -600,7 +600,6 @@ task "${baseName}#mixedClusterTask"(type: StandaloneRestIntegTestTask) { useCluster testClusters."${baseName}0" dependsOn "${baseName}#oldVersionClusterTask0" doFirst { - println "List of plugins: $plugins" testClusters."${baseName}0".upgradeNodeAndPluginToNextVersion(plugins) } filter { @@ -620,7 +619,6 @@ task "${baseName}#twoThirdsUpgradedClusterTask"(type: StandaloneRestIntegTestTas dependsOn "${baseName}#mixedClusterTask" useCluster testClusters."${baseName}0" doFirst { - println "List of plugins: $plugins" testClusters."${baseName}0".upgradeNodeAndPluginToNextVersion(plugins) } filter { @@ -640,7 +638,6 @@ task "${baseName}#rollingUpgradeClusterTask"(type: StandaloneRestIntegTestTask) dependsOn "${baseName}#twoThirdsUpgradedClusterTask" useCluster testClusters."${baseName}0" doFirst { - println "List of plugins: $plugins" testClusters."${baseName}0".upgradeNodeAndPluginToNextVersion(plugins) } filter { @@ -660,7 +657,6 @@ task "${baseName}#fullRestartClusterTask"(type: StandaloneRestIntegTestTask) { dependsOn "${baseName}#oldVersionClusterTask1" useCluster testClusters."${baseName}1" doFirst { - println "List of plugins: $plugins" testClusters."${baseName}1".upgradeAllNodesAndPluginsToNextVersion(plugins) } filter { From 81c4c9384e0e29b22680f2c6f562752427464cf2 Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Wed, 24 Jul 2024 12:17:25 -0700 Subject: [PATCH 17/18] Resolve comments Signed-off-by: Louis Chu --- .../OpenSearchAsyncQueryScheduler.java | 10 +++---- .../job/OpenSearchRefreshIndexJob.java | 3 --- .../job/OpenSearchRefreshIndexJobTest.java | 27 ------------------- build.gradle | 3 ++- common/build.gradle | 4 +-- core/build.gradle | 2 +- integ-test/build.gradle | 2 +- legacy/build.gradle | 2 +- plugin/build.gradle | 4 +-- .../org/opensearch/sql/plugin/SQLPlugin.java | 3 ++- ppl/build.gradle | 2 +- protocol/build.gradle | 2 +- sql/build.gradle | 2 +- 13 files changed, 19 insertions(+), 47 deletions(-) diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java index a1ba286cab..c7a66fc6be 100644 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQueryScheduler.java @@ -93,7 +93,7 @@ public void scheduleJob(OpenSearchRefreshIndexJobRequest request) { /** Unschedules a job by marking it as disabled and updating its last update time. */ public void unscheduleJob(String jobId) throws IOException { - verifyIndexExists(); + assertIndexExists(); OpenSearchRefreshIndexJobRequest request = OpenSearchRefreshIndexJobRequest.builder() .jobName(jobId) @@ -105,7 +105,7 @@ public void unscheduleJob(String jobId) throws IOException { /** Updates an existing job with new parameters. */ public void updateJob(OpenSearchRefreshIndexJobRequest request) throws IOException { - verifyIndexExists(); + assertIndexExists(); UpdateRequest updateRequest = new UpdateRequest(SCHEDULER_INDEX_NAME, request.getName()); updateRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); updateRequest.doc(request.toXContent(JsonXContent.contentBuilder(), EMPTY_PARAMS)); @@ -131,7 +131,7 @@ public void updateJob(OpenSearchRefreshIndexJobRequest request) throws IOExcepti /** Removes a job by deleting its document from the index. */ public void removeJob(String jobId) { - verifyIndexExists(); + assertIndexExists(); DeleteRequest deleteRequest = new DeleteRequest(SCHEDULER_INDEX_NAME, jobId); deleteRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); ActionFuture deleteResponseActionFuture = client.delete(deleteRequest); @@ -184,9 +184,9 @@ void createAsyncQuerySchedulerIndex() { } } - private void verifyIndexExists() { + private void assertIndexExists() { if (!this.clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)) { - throw new IllegalArgumentException("Job index does not exist."); + throw new IllegalStateException("Job index does not exist."); } } diff --git a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java index 73e0b50fc4..e465a8790f 100644 --- a/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java +++ b/async-query/src/main/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJob.java @@ -9,7 +9,6 @@ import org.apache.logging.log4j.Logger; import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.inject.Singleton; import org.opensearch.jobscheduler.spi.JobExecutionContext; import org.opensearch.jobscheduler.spi.ScheduledJobParameter; import org.opensearch.jobscheduler.spi.ScheduledJobRunner; @@ -30,7 +29,6 @@ * and using singleton job runner to ensure we register a usable job runner instance to JobScheduler * plugin. */ -@Singleton public class OpenSearchRefreshIndexJob implements ScheduledJobRunner { private static final Logger log = LogManager.getLogger(OpenSearchRefreshIndexJob.class); @@ -85,7 +83,6 @@ public void runJob(ScheduledJobParameter jobParameter, JobExecutionContext conte () -> { doRefresh(jobParameter.getName()); }; - threadPool.generic().submit(runnable); } diff --git a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java index d5d432f3db..cbf137997e 100644 --- a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java +++ b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/job/OpenSearchRefreshIndexJobTest.java @@ -6,7 +6,6 @@ package org.opensearch.sql.spark.scheduler.job; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.eq; @@ -15,10 +14,6 @@ import static org.mockito.Mockito.verify; import java.time.Instant; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Answers; @@ -58,28 +53,6 @@ public void setup() { jobRunner.setThreadPool(null); } - @Test - public void testGetJobRunnerInstanceCalledConcurrently() throws InterruptedException { - int threadCount = 3; - ExecutorService executorService = Executors.newFixedThreadPool(threadCount); - CountDownLatch latch = new CountDownLatch(threadCount); - - for (int i = 0; i < threadCount; i++) { - executorService.execute( - () -> { - try { - OpenSearchRefreshIndexJob instance = OpenSearchRefreshIndexJob.getJobRunnerInstance(); - assertNotNull(instance); - } finally { - latch.countDown(); - } - }); - } - - latch.await(5, TimeUnit.SECONDS); - executorService.shutdown(); - } - @Test public void testRunJobWithCorrectParameter() { spyJobRunner = spy(jobRunner); diff --git a/build.gradle b/build.gradle index d088883022..702d6f478a 100644 --- a/build.gradle +++ b/build.gradle @@ -50,6 +50,7 @@ buildscript { return "https://github.com/prometheus/prometheus/releases/download/v${prometheus_binary_version}/prometheus-${prometheus_binary_version}."+ getOSFamilyType() + "-" + getArchType() + ".tar.gz" } aws_java_sdk_version = "1.12.651" + guava_version = "32.1.3-jre" } repositories { @@ -192,7 +193,7 @@ configurations.all { exclude group: "commons-logging", module: "commons-logging" // enforce 1.1.3, https://www.whitesourcesoftware.com/vulnerability-database/WS-2019-0379 resolutionStrategy.force 'commons-codec:commons-codec:1.13' - resolutionStrategy.force 'com.google.guava:guava:32.1.3-jre' + resolutionStrategy.force "com.google.guava:guava:${guava_version}" } // updateVersion: Task to auto increment to the next development iteration diff --git a/common/build.gradle b/common/build.gradle index 786629b027..15c48dd6b3 100644 --- a/common/build.gradle +++ b/common/build.gradle @@ -34,7 +34,7 @@ repositories { dependencies { api "org.antlr:antlr4-runtime:4.7.1" - api group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' + api group: 'com.google.guava', name: 'guava', version: "${guava_version}" api group: 'org.apache.logging.log4j', name: 'log4j-core', version:"${versions.log4j}" api group: 'org.apache.commons', name: 'commons-lang3', version: '3.12.0' api group: 'org.apache.commons', name: 'commons-text', version: '1.10.0' @@ -46,7 +46,7 @@ dependencies { testImplementation group: 'junit', name: 'junit', version: '4.13.2' testImplementation group: 'org.assertj', name: 'assertj-core', version: '3.9.1' - testImplementation group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' + testImplementation group: 'com.google.guava', name: 'guava', version: "${guava_version}" testImplementation group: 'org.hamcrest', name: 'hamcrest-library', version: '2.1' testImplementation('org.junit.jupiter:junit-jupiter:5.9.3') testImplementation group: 'org.mockito', name: 'mockito-core', version: '5.7.0' diff --git a/core/build.gradle b/core/build.gradle index 9a2f08f148..f36777030c 100644 --- a/core/build.gradle +++ b/core/build.gradle @@ -46,7 +46,7 @@ pitest { } dependencies { - api group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' + api group: 'com.google.guava', name: 'guava', version: "${guava_version}" api group: 'org.apache.commons', name: 'commons-lang3', version: '3.12.0' api group: 'org.apache.commons', name: 'commons-text', version: '1.10.0' api group: 'com.facebook.presto', name: 'presto-matching', version: '0.240' diff --git a/integ-test/build.gradle b/integ-test/build.gradle index bae2361041..1acacdb4a5 100644 --- a/integ-test/build.gradle +++ b/integ-test/build.gradle @@ -156,7 +156,6 @@ configurations.all { resolutionStrategy.force "commons-logging:commons-logging:1.2" // enforce 1.1.3, https://www.whitesourcesoftware.com/vulnerability-database/WS-2019-0379 resolutionStrategy.force 'commons-codec:commons-codec:1.13' - resolutionStrategy.force 'com.google.guava:guava:32.1.3-jre' resolutionStrategy.force "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" resolutionStrategy.force "com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:${versions.jackson}" resolutionStrategy.force "com.fasterxml.jackson.dataformat:jackson-dataformat-smile:${versions.jackson}" @@ -169,6 +168,7 @@ configurations.all { 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:${aws_java_sdk_version}" + resolutionStrategy.force "com.google.guava:guava:${guava_version}" } configurations { diff --git a/legacy/build.gradle b/legacy/build.gradle index 303387bdbe..e3ddf27066 100644 --- a/legacy/build.gradle +++ b/legacy/build.gradle @@ -107,7 +107,7 @@ dependencies { because 'https://www.whitesourcesoftware.com/vulnerability-database/WS-2019-0379' } } - implementation group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' + implementation group: 'com.google.guava', name: 'guava', version: "${guava_version}" implementation group: 'org.json', name: 'json', version:'20231013' implementation group: 'org.apache.commons', name: 'commons-lang3', version: '3.12.0' implementation group: 'org.apache.commons', name: 'commons-text', version: '1.10.0' diff --git a/plugin/build.gradle b/plugin/build.gradle index 7c7c882a22..7ebd0ad2d9 100644 --- a/plugin/build.gradle +++ b/plugin/build.gradle @@ -99,7 +99,7 @@ configurations.all { resolutionStrategy.force "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" // enforce 1.1.3, https://www.whitesourcesoftware.com/vulnerability-database/WS-2019-0379 resolutionStrategy.force 'commons-codec:commons-codec:1.13' - resolutionStrategy.force 'com.google.guava:guava:32.1.3-jre' + resolutionStrategy.force "com.google.guava:guava:${guava_version}" resolutionStrategy.force 'com.google.guava:failureaccess:1.0.2' resolutionStrategy.force "com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:${versions.jackson}" resolutionStrategy.force "com.fasterxml.jackson.dataformat:jackson-dataformat-smile:${versions.jackson}" @@ -142,7 +142,7 @@ spotless { dependencies { compileOnly "org.opensearch:opensearch-job-scheduler-spi:${opensearch_build}" - compileOnly 'com.google.guava:guava:32.1.3-jre' + compileOnly "com.google.guava:guava:${guava_version}" compileOnly 'com.google.guava:failureaccess:1.0.2' api "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" 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 3f4ab29568..a1b1e32955 100644 --- a/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java +++ b/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java @@ -96,6 +96,7 @@ import org.opensearch.sql.spark.rest.RestAsyncQueryManagementAction; import org.opensearch.sql.spark.scheduler.OpenSearchAsyncQueryScheduler; import org.opensearch.sql.spark.scheduler.OpenSearchRefreshIndexJobRequestParser; +import org.opensearch.sql.spark.scheduler.job.OpenSearchRefreshIndexJob; import org.opensearch.sql.spark.storage.SparkStorageFactory; import org.opensearch.sql.spark.transport.TransportCancelAsyncQueryRequestAction; import org.opensearch.sql.spark.transport.TransportCreateAsyncQueryRequestAction; @@ -264,7 +265,7 @@ public String getJobIndex() { @Override public ScheduledJobRunner getJobRunner() { - return OpenSearchAsyncQueryScheduler.getJobRunner(); + return OpenSearchRefreshIndexJob.getJobRunnerInstance(); } @Override diff --git a/ppl/build.gradle b/ppl/build.gradle index 39bed7a359..2a3d6bdbf9 100644 --- a/ppl/build.gradle +++ b/ppl/build.gradle @@ -48,7 +48,7 @@ dependencies { runtimeOnly group: 'org.reflections', name: 'reflections', version: '0.9.12' implementation "org.antlr:antlr4-runtime:4.7.1" - implementation group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' + implementation group: 'com.google.guava', name: 'guava', version: "${guava_version}" api group: 'org.json', name: 'json', version: '20231013' implementation group: 'org.apache.logging.log4j', name: 'log4j-core', version:"${versions.log4j}" api project(':common') diff --git a/protocol/build.gradle b/protocol/build.gradle index d706b480c3..b5d7929041 100644 --- a/protocol/build.gradle +++ b/protocol/build.gradle @@ -30,7 +30,7 @@ plugins { } dependencies { - implementation group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' + implementation group: 'com.google.guava', name: 'guava', version: "${guava_version}" implementation group: 'com.fasterxml.jackson.core', name: 'jackson-core', version: "${versions.jackson}" implementation group: 'com.fasterxml.jackson.core', name: 'jackson-databind', version: "${versions.jackson_databind}" implementation group: 'com.fasterxml.jackson.dataformat', name: 'jackson-dataformat-cbor', version: "${versions.jackson}" diff --git a/sql/build.gradle b/sql/build.gradle index 52ae0ac621..10bb4b24bb 100644 --- a/sql/build.gradle +++ b/sql/build.gradle @@ -46,7 +46,7 @@ dependencies { antlr "org.antlr:antlr4:4.7.1" implementation "org.antlr:antlr4-runtime:4.7.1" - implementation group: 'com.google.guava', name: 'guava', version: '32.1.3-jre' + implementation group: 'com.google.guava', name: 'guava', version: "${guava_version}" implementation group: 'org.json', name: 'json', version:'20231013' implementation project(':common') implementation project(':core') From c59f70bfd7918ca990289a532fadff59e124f88b Mon Sep 17 00:00:00 2001 From: Louis Chu Date: Wed, 24 Jul 2024 14:43:26 -0700 Subject: [PATCH 18/18] Fix UT Signed-off-by: Louis Chu --- .../spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java index b61069ca5b..de86f111f3 100644 --- a/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java +++ b/async-query/src/test/java/org/opensearch/sql/spark/scheduler/OpenSearchAsyncQuerySchedulerTest.java @@ -199,7 +199,7 @@ public void testUnscheduleJob() throws IOException { public void testUnscheduleJobWithIndexNotFound() { when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(false); - assertThrows(IllegalArgumentException.class, () -> scheduler.unscheduleJob(TEST_JOB_ID)); + assertThrows(IllegalStateException.class, () -> scheduler.unscheduleJob(TEST_JOB_ID)); } @Test @@ -237,7 +237,7 @@ public void testUpdateJobWithIndexNotFound() { when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(false); - assertThrows(IllegalArgumentException.class, () -> scheduler.updateJob(request)); + assertThrows(IllegalStateException.class, () -> scheduler.updateJob(request)); } @Test @@ -305,7 +305,7 @@ public void testRemoveJob() { public void testRemoveJobWithIndexNotFound() { when(clusterService.state().routingTable().hasIndex(SCHEDULER_INDEX_NAME)).thenReturn(false); - assertThrows(IllegalArgumentException.class, () -> scheduler.removeJob(TEST_JOB_ID)); + assertThrows(IllegalStateException.class, () -> scheduler.removeJob(TEST_JOB_ID)); } @Test