diff --git a/.github/workflows/sql-test-and-build-workflow.yml b/.github/workflows/sql-test-and-build-workflow.yml index 3d063a2bfc..25e0387cf3 100644 --- a/.github/workflows/sql-test-and-build-workflow.yml +++ b/.github/workflows/sql-test-and-build-workflow.yml @@ -25,10 +25,10 @@ jobs: matrix: entry: - { os: ubuntu-latest, java: 11 } - - { os: windows-latest, java: 11, os_build_args: -x doctest -x integTest -x jacocoTestReport -x compileJdbc} + - { os: windows-latest, java: 11, os_build_args: -x doctest -x integTest -x jacocoTestReport -x compileJdbc -PbuildPlatform=windows } - { os: macos-latest, java: 11, os_build_args: -x doctest -x integTest -x jacocoTestReport -x compileJdbc } - { os: ubuntu-latest, java: 17 } - - { os: windows-latest, java: 17, os_build_args: -x doctest -x integTest -x jacocoTestReport -x compileJdbc } + - { os: windows-latest, java: 17, os_build_args: -x doctest -x integTest -x jacocoTestReport -x compileJdbc -PbuildPlatform=windows } - { os: macos-latest, java: 17, os_build_args: -x doctest -x integTest -x jacocoTestReport -x compileJdbc } runs-on: ${{ matrix.entry.os }} diff --git a/core/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java b/core/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java index 60e7d6f06e..53ff93eec1 100644 --- a/core/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java +++ b/core/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java @@ -35,6 +35,9 @@ import org.opensearch.sql.ast.expression.When; import org.opensearch.sql.ast.expression.WindowFunction; import org.opensearch.sql.ast.expression.Xor; +import org.opensearch.sql.ast.statement.Explain; +import org.opensearch.sql.ast.statement.Query; +import org.opensearch.sql.ast.statement.Statement; import org.opensearch.sql.ast.tree.AD; import org.opensearch.sql.ast.tree.Aggregation; import org.opensearch.sql.ast.tree.Dedupe; @@ -274,4 +277,16 @@ public T visitML(ML node, C context) { public T visitHighlightFunction(HighlightFunction node, C context) { return visitChildren(node, context); } + + public T visitStatement(Statement node, C context) { + return visit(node, context); + } + + public T visitQuery(Query node, C context) { + return visitStatement(node, context); + } + + public T visitExplain(Explain node, C context) { + return visitStatement(node, context); + } } diff --git a/core/src/main/java/org/opensearch/sql/ast/statement/Explain.java b/core/src/main/java/org/opensearch/sql/ast/statement/Explain.java new file mode 100644 index 0000000000..d0f2e3b372 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/ast/statement/Explain.java @@ -0,0 +1,26 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.ast.statement; + +import lombok.Data; +import org.opensearch.sql.ast.AbstractNodeVisitor; + +/** + * Explain Statement. + */ +@Data +public class Explain extends Statement { + + private final Statement statement; + + @Override + public R accept(AbstractNodeVisitor visitor, C context) { + return visitor.visitExplain(this, context); + } +} diff --git a/core/src/main/java/org/opensearch/sql/ast/statement/Query.java b/core/src/main/java/org/opensearch/sql/ast/statement/Query.java new file mode 100644 index 0000000000..a7b547ed2a --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/ast/statement/Query.java @@ -0,0 +1,35 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.ast.statement; + +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import lombok.Setter; +import lombok.ToString; +import org.opensearch.sql.ast.AbstractNodeVisitor; +import org.opensearch.sql.ast.tree.UnresolvedPlan; + +/** + * Query Statement. + */ +@Getter +@Setter +@ToString +@EqualsAndHashCode(callSuper = false) +@RequiredArgsConstructor +public class Query extends Statement { + + private final UnresolvedPlan plan; + + @Override + public R accept(AbstractNodeVisitor visitor, C context) { + return visitor.visitQuery(this, context); + } +} diff --git a/core/src/main/java/org/opensearch/sql/ast/statement/Statement.java b/core/src/main/java/org/opensearch/sql/ast/statement/Statement.java new file mode 100644 index 0000000000..e32a8dbfd8 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/ast/statement/Statement.java @@ -0,0 +1,22 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.ast.statement; + +import org.opensearch.sql.ast.AbstractNodeVisitor; +import org.opensearch.sql.ast.Node; + +/** + * Statement is the high interface of core engine. + */ +public abstract class Statement extends Node { + @Override + public R accept(AbstractNodeVisitor visitor, C context) { + return visitor.visitStatement(this, context); + } +} diff --git a/core/src/main/java/org/opensearch/sql/executor/DefaultQueryManager.java b/core/src/main/java/org/opensearch/sql/executor/DefaultQueryManager.java new file mode 100644 index 0000000000..9ab3bd7486 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/DefaultQueryManager.java @@ -0,0 +1,24 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor; + +import org.opensearch.sql.executor.execution.AbstractPlan; + +/** + * Default QueryManager implementation which execute {@link AbstractPlan} on caller thread. + */ +public class DefaultQueryManager implements QueryManager { + + @Override + public QueryId submit(AbstractPlan queryExecution) { + queryExecution.execute(); + + return queryExecution.getQueryId(); + } +} diff --git a/core/src/main/java/org/opensearch/sql/executor/QueryId.java b/core/src/main/java/org/opensearch/sql/executor/QueryId.java new file mode 100644 index 0000000000..933cb5d82d --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/QueryId.java @@ -0,0 +1,36 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor; + +import lombok.Getter; +import org.apache.commons.lang3.RandomStringUtils; +import org.opensearch.sql.executor.execution.AbstractPlan; + +/** + * Query id of {@link AbstractPlan}. + */ +public class QueryId { + /** + * Query id. + */ + @Getter + private final String queryId; + + /** + * Generate {@link QueryId}. + * @return {@link QueryId}. + */ + public static QueryId queryId() { + return new QueryId(RandomStringUtils.random(10, true, true)); + } + + private QueryId(String queryId) { + this.queryId = queryId; + } +} diff --git a/core/src/main/java/org/opensearch/sql/executor/QueryManager.java b/core/src/main/java/org/opensearch/sql/executor/QueryManager.java new file mode 100644 index 0000000000..3a32e4c7e9 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/QueryManager.java @@ -0,0 +1,25 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor; + +import org.opensearch.sql.executor.execution.AbstractPlan; + +/** + * QueryManager is the high-level interface of core engine. + * Frontend submit {@link AbstractPlan} to QueryManager. + */ +public interface QueryManager { + + /** + * Submit {@link AbstractPlan}. + * @param queryPlan {@link AbstractPlan}. + * @return {@link QueryId}. + */ + QueryId submit(AbstractPlan queryPlan); +} diff --git a/core/src/main/java/org/opensearch/sql/executor/QueryService.java b/core/src/main/java/org/opensearch/sql/executor/QueryService.java new file mode 100644 index 0000000000..dcdf6bc010 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/QueryService.java @@ -0,0 +1,97 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor; + +import lombok.RequiredArgsConstructor; +import org.opensearch.sql.analysis.AnalysisContext; +import org.opensearch.sql.analysis.Analyzer; +import org.opensearch.sql.ast.tree.UnresolvedPlan; +import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.planner.PlanContext; +import org.opensearch.sql.planner.Planner; +import org.opensearch.sql.planner.logical.LogicalPlan; +import org.opensearch.sql.planner.physical.PhysicalPlan; + +/** + * The low level interface of core engine. + */ +@RequiredArgsConstructor +public class QueryService { + + private final Analyzer analyzer; + + private final ExecutionEngine executionEngine; + + private final Planner planner; + + /** + * Execute the {@link UnresolvedPlan}, using {@link ResponseListener} to get response. + * Todo. deprecated this interface after finalize {@link PlanContext}. + * + * @param plan {@link UnresolvedPlan} + * @param listener {@link ResponseListener} + */ + public void execute(UnresolvedPlan plan, + ResponseListener listener) { + try { + executePlan(analyze(plan), PlanContext.emptyPlanContext(), listener); + } catch (Exception e) { + listener.onFailure(e); + } + } + + /** + * Execute the {@link UnresolvedPlan}, with {@link PlanContext} and using {@link ResponseListener} + * to get response. + * Todo. Pass split from PlanContext to ExecutionEngine in following PR. + * + * @param plan {@link LogicalPlan} + * @param planContext {@link PlanContext} + * @param listener {@link ResponseListener} + */ + public void executePlan(LogicalPlan plan, + PlanContext planContext, + ResponseListener listener) { + try { + executionEngine.execute(plan(plan), listener); + } catch (Exception e) { + listener.onFailure(e); + } + } + + /** + * Explain the query in {@link UnresolvedPlan} using {@link ResponseListener} to + * get and format explain response. + * + * @param plan {@link UnresolvedPlan} + * @param listener {@link ResponseListener} for explain response + */ + public void explain(UnresolvedPlan plan, + ResponseListener listener) { + try { + executionEngine.explain(plan(analyze(plan)), listener); + } catch (Exception e) { + listener.onFailure(e); + } + } + + /** + * Analyze {@link UnresolvedPlan}. + */ + public LogicalPlan analyze(UnresolvedPlan plan) { + return analyzer.analyze(plan, new AnalysisContext()); + } + + /** + * Translate {@link LogicalPlan} to {@link PhysicalPlan}. + */ + public PhysicalPlan plan(LogicalPlan plan) { + return planner.plan(plan); + } +} diff --git a/core/src/main/java/org/opensearch/sql/executor/execution/AbstractPlan.java b/core/src/main/java/org/opensearch/sql/executor/execution/AbstractPlan.java new file mode 100644 index 0000000000..1654293c04 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/execution/AbstractPlan.java @@ -0,0 +1,41 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor.execution; + + +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.QueryId; + +/** + * AbstractPlan represent the execution entity of the Statement. + */ +@RequiredArgsConstructor +public abstract class AbstractPlan { + + /** + * Uniq query id. + */ + @Getter + private final QueryId queryId; + + /** + * Start query execution. + */ + public abstract void execute(); + + /** + * Explain query execution. + * + * @param listener query explain response listener. + */ + public abstract void explain(ResponseListener listener); +} diff --git a/core/src/main/java/org/opensearch/sql/executor/execution/ExplainPlan.java b/core/src/main/java/org/opensearch/sql/executor/execution/ExplainPlan.java new file mode 100644 index 0000000000..8c784f82ed --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/execution/ExplainPlan.java @@ -0,0 +1,44 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor.execution; + +import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.QueryId; + +/** + * Explain plan. + */ +public class ExplainPlan extends AbstractPlan { + + private final AbstractPlan plan; + + private final ResponseListener explainListener; + + /** + * Constructor. + */ + public ExplainPlan(QueryId queryId, + AbstractPlan plan, + ResponseListener explainListener) { + super(queryId); + this.plan = plan; + this.explainListener = explainListener; + } + + @Override + public void execute() { + plan.explain(explainListener); + } + + @Override + public void explain(ResponseListener listener) { + throw new UnsupportedOperationException("explain query can not been explained."); + } +} diff --git a/core/src/main/java/org/opensearch/sql/executor/execution/QueryPlan.java b/core/src/main/java/org/opensearch/sql/executor/execution/QueryPlan.java new file mode 100644 index 0000000000..02cb701a0b --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/execution/QueryPlan.java @@ -0,0 +1,57 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor.execution; + +import org.opensearch.sql.ast.tree.UnresolvedPlan; +import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.QueryId; +import org.opensearch.sql.executor.QueryService; + +/** + * Query plan. Which includes. + * + *

select query. + */ +public class QueryPlan extends AbstractPlan { + + /** + * The query plan ast. + */ + private final UnresolvedPlan plan; + + /** + * Query service. + */ + private final QueryService queryService; + + private final ResponseListener listener; + + /** constructor. */ + public QueryPlan( + QueryId queryId, + UnresolvedPlan plan, + QueryService queryService, + ResponseListener listener) { + super(queryId); + this.plan = plan; + this.queryService = queryService; + this.listener = listener; + } + + @Override + public void execute() { + queryService.execute(plan, listener); + } + + @Override + public void explain(ResponseListener listener) { + queryService.explain(plan, listener); + } +} diff --git a/core/src/main/java/org/opensearch/sql/executor/execution/QueryPlanFactory.java b/core/src/main/java/org/opensearch/sql/executor/execution/QueryPlanFactory.java new file mode 100644 index 0000000000..851381cc7a --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/execution/QueryPlanFactory.java @@ -0,0 +1,100 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor.execution; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import java.util.Optional; +import lombok.RequiredArgsConstructor; +import org.apache.commons.lang3.tuple.Pair; +import org.opensearch.sql.ast.AbstractNodeVisitor; +import org.opensearch.sql.ast.statement.Explain; +import org.opensearch.sql.ast.statement.Query; +import org.opensearch.sql.ast.statement.Statement; +import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.QueryId; +import org.opensearch.sql.executor.QueryService; + +/** + * QueryExecution Factory. + */ +@RequiredArgsConstructor +public class QueryPlanFactory + extends AbstractNodeVisitor< + AbstractPlan, + Pair< + Optional>, + Optional>>> { + + /** + * Query Service. + */ + private final QueryService queryService; + + /** + * NO_CONSUMER_RESPONSE_LISTENER should never been called. It is only used as constructor + * parameter of {@link QueryPlan}. + */ + @VisibleForTesting + protected static final ResponseListener + NO_CONSUMER_RESPONSE_LISTENER = + new ResponseListener<>() { + @Override + public void onResponse(ExecutionEngine.QueryResponse response) { + throw new IllegalStateException( + "[BUG] query response should not sent to unexpected channel"); + } + + @Override + public void onFailure(Exception e) { + throw new IllegalStateException( + "[BUG] exception response should not sent to unexpected channel"); + } + }; + + /** + * Create QueryExecution from Statement. + */ + public AbstractPlan create( + Statement statement, + Optional> queryListener, + Optional> explainListener) { + return statement.accept(this, Pair.of(queryListener, explainListener)); + } + + @Override + public AbstractPlan visitQuery( + Query node, + Pair< + Optional>, + Optional>> + context) { + Preconditions.checkArgument( + context.getLeft().isPresent(), "[BUG] query listener must be not null"); + + return new QueryPlan(QueryId.queryId(), node.getPlan(), queryService, context.getLeft().get()); + } + + @Override + public AbstractPlan visitExplain( + Explain node, + Pair< + Optional>, + Optional>> + context) { + Preconditions.checkArgument( + context.getRight().isPresent(), "[BUG] explain listener must be not null"); + + return new ExplainPlan( + QueryId.queryId(), + create(node.getStatement(), Optional.of(NO_CONSUMER_RESPONSE_LISTENER), Optional.empty()), + context.getRight().get()); + } +} diff --git a/core/src/main/java/org/opensearch/sql/executor/streaming/Batch.java b/core/src/main/java/org/opensearch/sql/executor/streaming/Batch.java new file mode 100644 index 0000000000..cd7d7dae5a --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/streaming/Batch.java @@ -0,0 +1,17 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.executor.streaming; + +import lombok.Data; +import org.opensearch.sql.storage.split.Split; + +/** + * A batch of streaming execution. + */ +@Data +public class Batch { + private final Split split; +} diff --git a/core/src/main/java/org/opensearch/sql/executor/streaming/DefaultMetadataLog.java b/core/src/main/java/org/opensearch/sql/executor/streaming/DefaultMetadataLog.java new file mode 100644 index 0000000000..e439d93f6c --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/streaming/DefaultMetadataLog.java @@ -0,0 +1,76 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor.streaming; + +import com.google.common.base.Preconditions; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.SortedMap; +import java.util.TreeMap; +import org.apache.commons.lang3.tuple.Pair; + +/** + * In memory implementation of {@link MetadataLog}. Todo. Current implementation does not guarantee + * thread safe. We will re-evaluate it when adding pipeline execution. + * + * @param type of metadata type. + */ +public class DefaultMetadataLog implements MetadataLog { + + private static final long MIN_ACCEPTABLE_ID = 0L; + + private SortedMap metadataMap = new TreeMap<>(); + + @Override + public boolean add(Long batchId, T metadata) { + Preconditions.checkArgument(batchId >= MIN_ACCEPTABLE_ID, "batch id must large or equal 0"); + + if (metadataMap.containsKey(batchId)) { + return false; + } + metadataMap.put(batchId, metadata); + return true; + } + + @Override + public Optional get(Long batchId) { + if (!metadataMap.containsKey(batchId)) { + return Optional.empty(); + } else { + return Optional.of(metadataMap.get(batchId)); + } + } + + @Override + public List get(Optional startBatchId, Optional endBatchId) { + if (startBatchId.isEmpty() && endBatchId.isEmpty()) { + return new ArrayList<>(metadataMap.values()); + } else { + Long s = startBatchId.orElse(MIN_ACCEPTABLE_ID); + Long e = endBatchId.map(i -> i + 1).orElse(Long.MAX_VALUE); + return new ArrayList<>(metadataMap.subMap(s, e).values()); + } + } + + @Override + public Optional> getLatest() { + if (metadataMap.isEmpty()) { + return Optional.empty(); + } else { + Long latestId = metadataMap.lastKey(); + return Optional.of(Pair.of(latestId, metadataMap.get(latestId))); + } + } + + @Override + public void purge(Long batchId) { + metadataMap.headMap(batchId).clear(); + } +} diff --git a/core/src/main/java/org/opensearch/sql/executor/streaming/MetadataLog.java b/core/src/main/java/org/opensearch/sql/executor/streaming/MetadataLog.java new file mode 100644 index 0000000000..d6bb9bacd6 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/streaming/MetadataLog.java @@ -0,0 +1,61 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor.streaming; + +import java.util.List; +import java.util.Optional; +import org.apache.commons.lang3.tuple.Pair; + +/** + * Write-ahead Log (WAL). Which allow client write metadata associate with id. + * + * @param type of metadata type. + */ +public interface MetadataLog { + + /** + * add metadata to WAL. + * + * @param id metadata index in WAL. + * @param metadata metadata. + * @return true if add success, otherwise return false. + */ + boolean add(Long id, T metadata); + + /** + * get metadata from WAL. + * + * @param id metadata index in WAL. + * @return metadata. + */ + Optional get(Long id); + + /** + * Return metadata for id between [startId, endId]. + * + * @param startId If startId is empty, return all metadata before endId (inclusive). + * @param endId If end is empty, return all batches after endId (inclusive). + * @return a list of metadata sorted by id (nature order). + */ + List get(Optional startId, Optional endId); + + /** + * Get latest batchId and metadata. + * + * @return pair of id and metadata if not empty. + */ + Optional> getLatest(); + + /** + * Remove all the metadata less then id (exclusive). + * + * @param id smallest batchId should keep. + */ + void purge(Long id); +} diff --git a/core/src/main/java/org/opensearch/sql/executor/streaming/MicroBatchStreamingExecution.java b/core/src/main/java/org/opensearch/sql/executor/streaming/MicroBatchStreamingExecution.java new file mode 100644 index 0000000000..4f25b9433f --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/streaming/MicroBatchStreamingExecution.java @@ -0,0 +1,130 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor.streaming; + +import com.google.common.base.Preconditions; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.QueryService; +import org.opensearch.sql.planner.PlanContext; +import org.opensearch.sql.planner.logical.LogicalPlan; + +/** + * Micro batch streaming execution. + */ +public class MicroBatchStreamingExecution { + + private static final Logger log = LogManager.getLogger(MicroBatchStreamingExecution.class); + + static final long INITIAL_LATEST_BATCH_ID = -1L; + + private final StreamingSource source; + + private final LogicalPlan batchPlan; + + private final QueryService queryService; + + /** + * A write-ahead-log that records the offsets that are present in each batch. In order to ensure + * that a given batch will always consist of the same data, we write to this log before any + * processing is done. Thus, the Nth record in this log indicated data that is currently being + * processed and the N-1th entry indicates which offsets have been durably committed to the sink. + */ + private final MetadataLog offsetLog; + + /** keep track the latest commit batchId. */ + private final MetadataLog committedLog; + + /** + * Constructor. + */ + public MicroBatchStreamingExecution( + StreamingSource source, + LogicalPlan batchPlan, + QueryService queryService, + MetadataLog offsetLog, + MetadataLog committedLog) { + this.source = source; + this.batchPlan = batchPlan; + this.queryService = queryService; + // todo. add offsetLog and committedLog offset recovery. + this.offsetLog = offsetLog; + this.committedLog = committedLog; + } + + /** + * Pull the {@link Batch} from {@link StreamingSource} and execute the {@link Batch}. + */ + public void execute() { + Long latestBatchId = offsetLog.getLatest().map(Pair::getKey).orElse(INITIAL_LATEST_BATCH_ID); + Long latestCommittedBatchId = + committedLog.getLatest().map(Pair::getKey).orElse(INITIAL_LATEST_BATCH_ID); + Optional committedOffset = offsetLog.get(latestCommittedBatchId); + AtomicLong currentBatchId = new AtomicLong(INITIAL_LATEST_BATCH_ID); + + if (latestBatchId.equals(latestCommittedBatchId)) { + // there are no unhandled Offset. + currentBatchId.set(latestCommittedBatchId + 1L); + } else { + Preconditions.checkArgument( + latestBatchId.equals(latestCommittedBatchId + 1L), + "[BUG] Expected latestBatchId - latestCommittedBatchId = 0 or 1, " + + "but latestBatchId=%d, latestCommittedBatchId=%d", + latestBatchId, + latestCommittedBatchId); + + // latestBatchId is not committed yet. + currentBatchId.set(latestBatchId); + } + + Optional availableOffsets = source.getLatestOffset(); + if (hasNewData(availableOffsets, committedOffset)) { + Batch batch = source.getBatch(committedOffset, availableOffsets.get()); + offsetLog.add(currentBatchId.get(), availableOffsets.get()); + queryService.executePlan( + batchPlan, + new PlanContext(batch.getSplit()), + new ResponseListener<>() { + @Override + public void onResponse(ExecutionEngine.QueryResponse response) { + long finalBatchId = currentBatchId.get(); + Offset finalAvailableOffsets = availableOffsets.get(); + committedLog.add(finalBatchId, finalAvailableOffsets); + } + + @Override + public void onFailure(Exception e) { + log.error("streaming processing failed. source = {}", source); + } + }); + } + } + + private boolean hasNewData(Optional availableOffsets, Optional committedOffset) { + if (availableOffsets.equals(committedOffset)) { + log.debug("source does not have new data, exit. source = {}", source); + return false; + } else { + Preconditions.checkArgument( + availableOffsets.isPresent(), "[BUG] available offsets must be no empty"); + + log.debug( + "source has new data. source = {}, availableOffsets:{}, committedOffset:{}", + source, + availableOffsets, + committedOffset); + return true; + } + } +} diff --git a/core/src/main/java/org/opensearch/sql/executor/streaming/Offset.java b/core/src/main/java/org/opensearch/sql/executor/streaming/Offset.java new file mode 100644 index 0000000000..00f040e437 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/streaming/Offset.java @@ -0,0 +1,17 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.executor.streaming; + +import lombok.Data; + +/** + * Offset. + */ +@Data +public class Offset { + + private final Long offset; +} diff --git a/core/src/main/java/org/opensearch/sql/executor/streaming/StreamingSource.java b/core/src/main/java/org/opensearch/sql/executor/streaming/StreamingSource.java new file mode 100644 index 0000000000..ebd3fa714b --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/executor/streaming/StreamingSource.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.executor.streaming; + +import java.util.Optional; + +/** + * Streaming source. + */ +public interface StreamingSource { + /** + * Get current {@link Offset} of stream data. + * + * @return empty if the stream does not has new data. + */ + Optional getLatestOffset(); + + /** + * Get a {@link Batch} from source between (start, end]. + * + * @param start start offset. + * @param end end offset. + * @return @link Batch}. + */ + Batch getBatch(Optional start, Offset end); +} diff --git a/core/src/main/java/org/opensearch/sql/expression/config/ExpressionConfig.java b/core/src/main/java/org/opensearch/sql/expression/config/ExpressionConfig.java index c68086ab4d..9335512b11 100644 --- a/core/src/main/java/org/opensearch/sql/expression/config/ExpressionConfig.java +++ b/core/src/main/java/org/opensearch/sql/expression/config/ExpressionConfig.java @@ -21,8 +21,10 @@ import org.opensearch.sql.expression.system.SystemFunctions; import org.opensearch.sql.expression.text.TextFunction; import org.opensearch.sql.expression.window.WindowFunctions; +import org.springframework.beans.factory.config.ConfigurableBeanFactory; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; +import org.springframework.context.annotation.Scope; /** * Expression Config for Spring IoC. @@ -33,6 +35,7 @@ public class ExpressionConfig { * BuiltinFunctionRepository constructor. */ @Bean + @Scope(value = ConfigurableBeanFactory.SCOPE_PROTOTYPE) public BuiltinFunctionRepository functionRepository() { BuiltinFunctionRepository builtinFunctionRepository = new BuiltinFunctionRepository(new HashMap<>()); @@ -52,6 +55,7 @@ public BuiltinFunctionRepository functionRepository() { } @Bean + @Scope(value = ConfigurableBeanFactory.SCOPE_PROTOTYPE) public DSL dsl(BuiltinFunctionRepository repository) { return new DSL(repository); } diff --git a/core/src/main/java/org/opensearch/sql/expression/span/SpanExpression.java b/core/src/main/java/org/opensearch/sql/expression/span/SpanExpression.java index 715c911b13..aff114145e 100644 --- a/core/src/main/java/org/opensearch/sql/expression/span/SpanExpression.java +++ b/core/src/main/java/org/opensearch/sql/expression/span/SpanExpression.java @@ -7,7 +7,6 @@ import lombok.EqualsAndHashCode; import lombok.Getter; -import lombok.RequiredArgsConstructor; import lombok.ToString; import org.opensearch.sql.ast.expression.SpanUnit; import org.opensearch.sql.data.model.ExprValue; @@ -15,8 +14,8 @@ import org.opensearch.sql.expression.Expression; import org.opensearch.sql.expression.ExpressionNodeVisitor; import org.opensearch.sql.expression.env.Environment; +import org.opensearch.sql.planner.physical.collector.Rounding; -@RequiredArgsConstructor @Getter @ToString @EqualsAndHashCode @@ -25,9 +24,19 @@ public class SpanExpression implements Expression { private final Expression value; private final SpanUnit unit; + /** + * Construct a span expression by field and span interval expression. + */ + public SpanExpression(Expression field, Expression value, SpanUnit unit) { + this.field = field; + this.value = value; + this.unit = unit; + } + @Override public ExprValue valueOf(Environment valueEnv) { - return value.valueOf(valueEnv); + Rounding rounding = Rounding.createRounding(this); //TODO: will integrate with WindowAssigner + return rounding.round(field.valueOf(valueEnv)); } /** diff --git a/core/src/main/java/org/opensearch/sql/planner/PlanContext.java b/core/src/main/java/org/opensearch/sql/planner/PlanContext.java new file mode 100644 index 0000000000..3d43c02d61 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/planner/PlanContext.java @@ -0,0 +1,31 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner; + +import java.util.Optional; +import lombok.Getter; +import org.opensearch.sql.storage.split.Split; + +/** + * Plan context hold planning related information. + */ +public class PlanContext { + + @Getter + private final Optional split; + + public PlanContext(Split split) { + this.split = Optional.of(split); + } + + private PlanContext(Optional split) { + this.split = split; + } + + public static PlanContext emptyPlanContext() { + return new PlanContext(Optional.empty()); + } +} diff --git a/core/src/main/java/org/opensearch/sql/planner/physical/AggregationOperator.java b/core/src/main/java/org/opensearch/sql/planner/physical/AggregationOperator.java index 3cf823d0e2..1d9523464b 100644 --- a/core/src/main/java/org/opensearch/sql/planner/physical/AggregationOperator.java +++ b/core/src/main/java/org/opensearch/sql/planner/physical/AggregationOperator.java @@ -34,8 +34,7 @@ public class AggregationOperator extends PhysicalPlan { private final List aggregatorList; @Getter private final List groupByExprList; - @Getter - private final NamedExpression span; + /** * {@link BindingTuple} Collector. */ @@ -56,18 +55,7 @@ public AggregationOperator(PhysicalPlan input, List aggregatorL this.input = input; this.aggregatorList = aggregatorList; this.groupByExprList = groupByExprList; - if (hasSpan(groupByExprList)) { - // span expression is always the first expression in group list if exist. - this.span = groupByExprList.get(0); - this.collector = - Collector.Builder.build( - this.span, groupByExprList.subList(1, groupByExprList.size()), this.aggregatorList); - - } else { - this.span = null; - this.collector = - Collector.Builder.build(this.span, this.groupByExprList, this.aggregatorList); - } + this.collector = Collector.Builder.build(groupByExprList, this.aggregatorList); } @Override @@ -99,9 +87,4 @@ public void open() { } iterator = collector.results().iterator(); } - - private boolean hasSpan(List namedExpressionList) { - return !namedExpressionList.isEmpty() - && namedExpressionList.get(0).getDelegated() instanceof SpanExpression; - } } diff --git a/core/src/main/java/org/opensearch/sql/planner/physical/collector/BucketCollector.java b/core/src/main/java/org/opensearch/sql/planner/physical/collector/BucketCollector.java index c3a803ca9e..a00fd3cdf3 100644 --- a/core/src/main/java/org/opensearch/sql/planner/physical/collector/BucketCollector.java +++ b/core/src/main/java/org/opensearch/sql/planner/physical/collector/BucketCollector.java @@ -7,11 +7,11 @@ import com.google.common.collect.ImmutableList; import java.util.Arrays; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.TreeMap; import java.util.function.Supplier; import java.util.stream.Collectors; import lombok.RequiredArgsConstructor; @@ -38,9 +38,10 @@ public class BucketCollector implements Collector { private final Supplier supplier; /** - * Map between bucketKey and collector in the bucket. + * Map from bucketKey to nested collector sorted by key to make sure + * final result is in order after traversal. */ - private final Map collectorMap = new HashMap<>(); + private final Map collectorMap = new TreeMap<>(); /** * Bucket Index. diff --git a/core/src/main/java/org/opensearch/sql/planner/physical/collector/Collector.java b/core/src/main/java/org/opensearch/sql/planner/physical/collector/Collector.java index 66eba7440b..a2b3a41a27 100644 --- a/core/src/main/java/org/opensearch/sql/planner/physical/collector/Collector.java +++ b/core/src/main/java/org/opensearch/sql/planner/physical/collector/Collector.java @@ -40,17 +40,14 @@ class Builder { /** * build {@link Collector}. */ - public static Collector build( - NamedExpression span, List buckets, List aggregators) { - if (span == null && buckets.isEmpty()) { + public static Collector build(List buckets, + List aggregators) { + if (buckets.isEmpty()) { return new MetricCollector(aggregators); - } else if (span != null) { - return new SpanCollector(span, () -> build(null, buckets, aggregators)); } else { return new BucketCollector( buckets.get(0), - () -> - build(null, ImmutableList.copyOf(buckets.subList(1, buckets.size())), aggregators)); + () -> build(ImmutableList.copyOf(buckets.subList(1, buckets.size())), aggregators)); } } } diff --git a/core/src/main/java/org/opensearch/sql/planner/physical/collector/Rounding.java b/core/src/main/java/org/opensearch/sql/planner/physical/collector/Rounding.java index fd65453c5d..71ae5dc7c3 100644 --- a/core/src/main/java/org/opensearch/sql/planner/physical/collector/Rounding.java +++ b/core/src/main/java/org/opensearch/sql/planner/physical/collector/Rounding.java @@ -17,7 +17,6 @@ import java.time.LocalDateTime; import java.time.LocalTime; import java.time.ZoneId; -import java.time.ZonedDateTime; import java.time.temporal.ChronoField; import java.util.Arrays; import java.util.concurrent.TimeUnit; @@ -40,10 +39,6 @@ */ @EqualsAndHashCode public abstract class Rounding { - @Getter - protected T maxRounded; - @Getter - protected T minRounded; /** * Create Rounding instance. @@ -75,10 +70,6 @@ public static Rounding createRounding(SpanExpression span) { public abstract ExprValue round(ExprValue value); - public abstract Integer locate(ExprValue value); - - public abstract ExprValue[] createBuckets(); - static class TimestampRounding extends Rounding { private final ExprValue interval; @@ -93,50 +84,8 @@ public TimestampRounding(ExprValue interval, String unit) { public ExprValue round(ExprValue var) { Instant instant = Instant.ofEpochMilli(dateTimeUnit.round(var.timestampValue() .toEpochMilli(), interval.integerValue())); - updateRounded(instant); return new ExprTimestampValue(instant); } - - @Override - public ExprValue[] createBuckets() { - if (dateTimeUnit.isMillisBased) { - int size = (int) ((maxRounded.toEpochMilli() - minRounded.toEpochMilli()) / (interval - .integerValue() * dateTimeUnit.ratio)) + 1; - return new ExprValue[size]; - } else { - ZonedDateTime maxZonedDateTime = maxRounded.atZone(ZoneId.of("UTC")); - ZonedDateTime minZonedDateTime = minRounded.atZone(ZoneId.of("UTC")); - int monthDiff = (maxZonedDateTime.getYear() - minZonedDateTime - .getYear()) * 12 + maxZonedDateTime.getMonthValue() - minZonedDateTime.getMonthValue(); - int size = monthDiff / ((int) dateTimeUnit.ratio * interval.integerValue()) + 1; - return new ExprValue[size]; - } - } - - @Override - public Integer locate(ExprValue value) { - if (dateTimeUnit.isMillisBased) { - long intervalInEpochMillis = dateTimeUnit.ratio; - return Long.valueOf((value.timestampValue() - .atZone(ZoneId.of("UTC")).toInstant().toEpochMilli() - minRounded - .atZone(ZoneId.of("UTC")).toInstant().toEpochMilli()) / (intervalInEpochMillis - * interval.integerValue())).intValue(); - } else { - int monthDiff = (value.dateValue().getYear() - minRounded.atZone(ZoneId.of("UTC")) - .getYear()) * 12 + value.dateValue().getMonthValue() - minRounded - .atZone(ZoneId.of("UTC")).getMonthValue(); - return (int) (monthDiff / (dateTimeUnit.ratio * interval.integerValue())); - } - } - - private void updateRounded(Instant value) { - if (maxRounded == null || value.isAfter(maxRounded)) { - maxRounded = value; - } - if (minRounded == null || value.isBefore(minRounded)) { - minRounded = value; - } - } } @@ -153,52 +102,8 @@ public DatetimeRounding(ExprValue interval, String unit) { public ExprValue round(ExprValue var) { Instant instant = Instant.ofEpochMilli(dateTimeUnit.round(var.datetimeValue() .atZone(ZoneId.of("UTC")).toInstant().toEpochMilli(), interval.integerValue())); - updateRounded(instant); return new ExprDatetimeValue(instant.atZone(ZoneId.of("UTC")).toLocalDateTime()); } - - @Override - public ExprValue[] createBuckets() { - if (dateTimeUnit.isMillisBased) { - int size = (int) ((maxRounded.atZone(ZoneId.of("UTC")).toInstant() - .toEpochMilli() - minRounded.atZone(ZoneId.of("UTC")).toInstant() - .toEpochMilli()) / (interval.integerValue() * dateTimeUnit.ratio)) + 1; - return new ExprValue[size]; - } else { - ZonedDateTime maxZonedDateTime = maxRounded.atZone(ZoneId.of("UTC")); - ZonedDateTime minZonedDateTime = minRounded.atZone(ZoneId.of("UTC")); - int monthDiff = (maxZonedDateTime.getYear() - minZonedDateTime - .getYear()) * 12 + maxZonedDateTime.getMonthValue() - minZonedDateTime.getMonthValue(); - int size = monthDiff / ((int) dateTimeUnit.ratio * interval.integerValue()) + 1; - return new ExprValue[size]; - } - } - - @Override - public Integer locate(ExprValue value) { - if (dateTimeUnit.isMillisBased) { - long intervalInEpochMillis = dateTimeUnit.ratio; - return Long.valueOf((value.datetimeValue() - .atZone(ZoneId.of("UTC")).toInstant().toEpochMilli() - minRounded - .atZone(ZoneId.of("UTC")).toInstant().toEpochMilli()) / (intervalInEpochMillis - * interval.integerValue())).intValue(); - } else { - int monthDiff = (value.datetimeValue().getYear() - minRounded.getYear()) * 12 - + value.dateValue().getMonthValue() - minRounded.getMonthValue(); - return (int) (monthDiff / (dateTimeUnit.ratio * interval.integerValue())); - } - } - - private void updateRounded(Instant value) { - if (maxRounded == null || value.isAfter(maxRounded - .atZone(ZoneId.of("UTC")).toInstant())) { - maxRounded = value.atZone(ZoneId.of("UTC")).toLocalDateTime(); - } - if (minRounded == null || value.isBefore(minRounded - .atZone(ZoneId.of("UTC")).toInstant())) { - minRounded = value.atZone(ZoneId.of("UTC")).toLocalDateTime(); - } - } } @@ -215,52 +120,8 @@ public DateRounding(ExprValue interval, String unit) { public ExprValue round(ExprValue var) { Instant instant = Instant.ofEpochMilli(dateTimeUnit.round(var.dateValue().atStartOfDay() .atZone(ZoneId.of("UTC")).toInstant().toEpochMilli(), interval.integerValue())); - updateRounded(instant); return new ExprDateValue(instant.atZone(ZoneId.of("UTC")).toLocalDate()); } - - @Override - public ExprValue[] createBuckets() { - if (dateTimeUnit.isMillisBased) { - int size = (int) ((maxRounded.atStartOfDay().atZone(ZoneId.of("UTC")).toInstant() - .toEpochMilli() - minRounded.atStartOfDay().atZone(ZoneId.of("UTC")).toInstant() - .toEpochMilli()) / (interval.integerValue() * dateTimeUnit.ratio)) + 1; - return new ExprValue[size]; - } else { - ZonedDateTime maxZonedDateTime = maxRounded.atStartOfDay().atZone(ZoneId.of("UTC")); - ZonedDateTime minZonedDateTime = minRounded.atStartOfDay().atZone(ZoneId.of("UTC")); - int monthDiff = (maxZonedDateTime.getYear() - minZonedDateTime - .getYear()) * 12 + maxZonedDateTime.getMonthValue() - minZonedDateTime.getMonthValue(); - int size = monthDiff / ((int) dateTimeUnit.ratio * interval.integerValue()) + 1; - return new ExprValue[size]; - } - } - - @Override - public Integer locate(ExprValue value) { - if (dateTimeUnit.isMillisBased) { - long intervalInEpochMillis = dateTimeUnit.ratio; - return Long.valueOf((value.dateValue().atStartOfDay() - .atZone(ZoneId.of("UTC")).toInstant().toEpochMilli() - minRounded.atStartOfDay() - .atZone(ZoneId.of("UTC")).toInstant().toEpochMilli()) / (intervalInEpochMillis - * interval.integerValue())).intValue(); - } else { - int monthDiff = (value.dateValue().getYear() - minRounded.getYear()) * 12 - + value.dateValue().getMonthValue() - minRounded.getMonthValue(); - return (int) (monthDiff / (dateTimeUnit.ratio * interval.integerValue())); - } - } - - private void updateRounded(Instant value) { - if (maxRounded == null || value.isAfter(maxRounded.atStartOfDay() - .atZone(ZoneId.of("UTC")).toInstant())) { - maxRounded = value.atZone(ZoneId.of("UTC")).toLocalDate(); - } - if (minRounded == null || value.isBefore(minRounded.atStartOfDay() - .atZone(ZoneId.of("UTC")).toInstant())) { - minRounded = value.atZone(ZoneId.of("UTC")).toLocalDate(); - } - } } static class TimeRounding extends Rounding { @@ -281,39 +142,8 @@ public ExprValue round(ExprValue var) { Instant instant = Instant.ofEpochMilli(dateTimeUnit.round(var.timeValue().getLong( ChronoField.MILLI_OF_DAY), interval.integerValue())); - updateRounded(instant); return new ExprTimeValue(instant.atZone(ZoneId.of("UTC")).toLocalTime()); } - - @Override - public ExprValue[] createBuckets() { - // local time is converted to timestamp on 1970-01-01 for aggregations - int size = (int) ((maxRounded.atDate(LocalDate.of(1970, 1, 1)) - .atZone(ZoneId.of("UTC")).toInstant().toEpochMilli() - minRounded - .atDate(LocalDate.of(1970, 1, 1)).atZone(ZoneId.of("UTC")).toInstant() - .toEpochMilli()) / (interval.integerValue() * dateTimeUnit.ratio)) + 1; - return new ExprValue[size]; - } - - @Override - public Integer locate(ExprValue value) { - long intervalInEpochMillis = dateTimeUnit.ratio; - return Long.valueOf((value.timeValue().atDate(LocalDate.of(1970, 1, 1)) - .atZone(ZoneId.of("UTC")).toInstant().toEpochMilli() - minRounded - .atDate(LocalDate.of(1970, 1, 1)) - .atZone(ZoneId.of("UTC")).toInstant().toEpochMilli()) / (intervalInEpochMillis * interval - .integerValue())).intValue(); - } - - private void updateRounded(Instant value) { - if (maxRounded == null || value.isAfter(maxRounded.atDate(LocalDate.of(1970, 1, 1)) - .atZone(ZoneId.of("UTC")).toInstant())) { - maxRounded = value.atZone(ZoneId.of("UTC")).toLocalTime(); - } - if (minRounded == null) { - minRounded = value.atZone(ZoneId.of("UTC")).toLocalTime(); - } - } } @@ -327,29 +157,8 @@ protected LongRounding(ExprValue interval) { @Override public ExprValue round(ExprValue value) { long rounded = Math.floorDiv(value.longValue(), longInterval) * longInterval; - updateRounded(rounded); return ExprValueUtils.longValue(rounded); } - - @Override - public Integer locate(ExprValue value) { - return Long.valueOf((value.longValue() - minRounded) / longInterval).intValue(); - } - - @Override - public ExprValue[] createBuckets() { - int size = Long.valueOf((maxRounded - minRounded) / longInterval).intValue() + 1; - return new ExprValue[size]; - } - - private void updateRounded(Long value) { - if (maxRounded == null || value > maxRounded) { - maxRounded = value; - } - if (minRounded == null || value < minRounded) { - minRounded = value; - } - } } @@ -364,29 +173,8 @@ protected DoubleRounding(ExprValue interval) { public ExprValue round(ExprValue value) { double rounded = Double .valueOf(value.doubleValue() / doubleInterval).intValue() * doubleInterval; - updateRounded(rounded); return ExprValueUtils.doubleValue(rounded); } - - @Override - public Integer locate(ExprValue value) { - return Double.valueOf((value.doubleValue() - minRounded) / doubleInterval).intValue(); - } - - @Override - public ExprValue[] createBuckets() { - int size = Double.valueOf((maxRounded - minRounded) / doubleInterval).intValue() + 1; - return new ExprValue[size]; - } - - private void updateRounded(Double value) { - if (maxRounded == null || value > maxRounded) { - maxRounded = value; - } - if (minRounded == null || value < minRounded) { - minRounded = value; - } - } } @@ -396,16 +184,6 @@ static class UnknownRounding extends Rounding { public ExprValue round(ExprValue var) { return null; } - - @Override - public Integer locate(ExprValue value) { - return null; - } - - @Override - public ExprValue[] createBuckets() { - return new ExprValue[0]; - } } diff --git a/core/src/main/java/org/opensearch/sql/planner/physical/collector/SpanCollector.java b/core/src/main/java/org/opensearch/sql/planner/physical/collector/SpanCollector.java deleted file mode 100644 index 092d79bd81..0000000000 --- a/core/src/main/java/org/opensearch/sql/planner/physical/collector/SpanCollector.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.sql.planner.physical.collector; - -import java.util.function.Supplier; -import org.opensearch.sql.data.model.ExprValue; -import org.opensearch.sql.expression.NamedExpression; -import org.opensearch.sql.expression.span.SpanExpression; -import org.opensearch.sql.storage.bindingtuple.BindingTuple; - -/** - * Span Collector. - */ -public class SpanCollector extends BucketCollector { - - /** - * Span Expression. - */ - private final SpanExpression spanExpr; - - /** - * Rounding. - */ - private final Rounding rounding; - - /** - * Constructor. - */ - public SpanCollector(NamedExpression bucketExpr, Supplier supplier) { - super(bucketExpr, supplier); - this.spanExpr = (SpanExpression) bucketExpr.getDelegated(); - this.rounding = Rounding.createRounding(spanExpr); - } - - /** - * Rounding bucket value. - * - * @param tuple {@link BindingTuple}. - * @return {@link ExprValue}. - */ - @Override - protected ExprValue bucketKey(BindingTuple tuple) { - return rounding.round(spanExpr.getField().valueOf(tuple)); - } - - /** - * Allocates Buckets for building results. - * - * @return buckets. - */ - @Override - protected ExprValue[] allocateBuckets() { - return rounding.createBuckets(); - } - - /** - * Current Bucket index in allocated buckets. - * - * @param value bucket key. - * @return index. - */ - @Override - protected int locateBucket(ExprValue value) { - return rounding.locate(value); - } -} diff --git a/core/src/main/java/org/opensearch/sql/planner/streaming/StreamContext.java b/core/src/main/java/org/opensearch/sql/planner/streaming/StreamContext.java new file mode 100644 index 0000000000..18eb10f19d --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/planner/streaming/StreamContext.java @@ -0,0 +1,19 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming; + +import lombok.Data; + +/** + * Stream context required by stream processing components and can be + * stored and restored between executions. + */ +@Data +public class StreamContext { + + /** Current watermark timestamp. */ + private long watermark; +} diff --git a/core/src/main/java/org/opensearch/sql/planner/streaming/watermark/BoundedOutOfOrderWatermarkGenerator.java b/core/src/main/java/org/opensearch/sql/planner/streaming/watermark/BoundedOutOfOrderWatermarkGenerator.java new file mode 100644 index 0000000000..63d6a5b163 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/planner/streaming/watermark/BoundedOutOfOrderWatermarkGenerator.java @@ -0,0 +1,27 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.watermark; + +import lombok.RequiredArgsConstructor; + +/** + * Watermark generator that generates watermark with bounded out-of-order delay. + */ +@RequiredArgsConstructor +public class BoundedOutOfOrderWatermarkGenerator implements WatermarkGenerator { + + /** The maximum out-of-order allowed in millisecond. */ + private final long maxOutOfOrderAllowed; + + /** The maximum timestamp seen so far in millisecond. */ + private long maxTimestamp; + + @Override + public long generate(long timestamp) { + maxTimestamp = Math.max(maxTimestamp, timestamp); + return (maxTimestamp - maxOutOfOrderAllowed - 1); + } +} diff --git a/core/src/main/java/org/opensearch/sql/planner/streaming/watermark/WatermarkGenerator.java b/core/src/main/java/org/opensearch/sql/planner/streaming/watermark/WatermarkGenerator.java new file mode 100644 index 0000000000..4f4c9a8a00 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/planner/streaming/watermark/WatermarkGenerator.java @@ -0,0 +1,22 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.watermark; + +/** + * A watermark generator generates watermark timestamp based on some strategy which is defined + * in implementation class. + */ +public interface WatermarkGenerator { + + /** + * Generate watermark timestamp on the given event timestamp. + * + * @param timestamp event timestamp in millisecond + * @return watermark timestamp in millisecond + */ + long generate(long timestamp); + +} diff --git a/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/Window.java b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/Window.java new file mode 100644 index 0000000000..2a85ea391c --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/Window.java @@ -0,0 +1,28 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.windowing; + +import lombok.Data; + +/** + * A time window is a window of time interval with inclusive start time and exclusive end time. + */ +@Data +public class Window { + + /** Start timestamp (inclusive) of the time window. */ + private final long startTime; + + /** End timestamp (exclusive) of the time window. */ + private final long endTime; + + /** + * Return the maximum timestamp (inclusive) of the window. + */ + public long maxTimestamp() { + return endTime - 1; + } +} diff --git a/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/assigner/SlidingWindowAssigner.java b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/assigner/SlidingWindowAssigner.java new file mode 100644 index 0000000000..f0f47fd575 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/assigner/SlidingWindowAssigner.java @@ -0,0 +1,57 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.windowing.assigner; + +import com.google.common.base.Preconditions; +import java.util.LinkedList; +import java.util.List; +import org.opensearch.sql.planner.streaming.windowing.Window; +import org.opensearch.sql.utils.DateTimeUtils; + +/** + * A sliding window assigner assigns multiple overlapped window per event timestamp. + * The overlap size is determined by the given slide interval. + */ +public class SlidingWindowAssigner implements WindowAssigner { + + /** Window size in millisecond. */ + private final long windowSize; + + /** Slide size in millisecond. */ + private final long slideSize; + + /** + * Create sliding window assigner with the given window and slide size in millisecond. + * + * @param windowSize window size in millisecond + * @param slideSize slide size in millisecond + */ + public SlidingWindowAssigner(long windowSize, long slideSize) { + Preconditions.checkArgument(windowSize > 0, + "Window size [%s] must be positive number", windowSize); + Preconditions.checkArgument(slideSize > 0, + "Slide size [%s] must be positive number", slideSize); + this.windowSize = windowSize; + this.slideSize = slideSize; + } + + @Override + public List assign(long timestamp) { + LinkedList windows = new LinkedList<>(); + + // Assign window from the last start time to the first until timestamp outside current window + long startTime = DateTimeUtils.getWindowStartTime(timestamp, slideSize); + for (Window win = window(startTime); win.maxTimestamp() >= timestamp; win = window(startTime)) { + windows.addFirst(win); + startTime -= slideSize; + } + return windows; + } + + private Window window(long startTime) { + return new Window(startTime, startTime + windowSize); + } +} diff --git a/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/assigner/TumblingWindowAssigner.java b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/assigner/TumblingWindowAssigner.java new file mode 100644 index 0000000000..192bb6c429 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/assigner/TumblingWindowAssigner.java @@ -0,0 +1,38 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.windowing.assigner; + +import com.google.common.base.Preconditions; +import java.util.Collections; +import java.util.List; +import org.opensearch.sql.planner.streaming.windowing.Window; +import org.opensearch.sql.utils.DateTimeUtils; + +/** + * A tumbling window assigner assigns a single window per event timestamp without overlap. + */ +public class TumblingWindowAssigner implements WindowAssigner { + + /** Window size in millisecond. */ + private final long windowSize; + + /** + * Create tumbling window assigner with the given window size. + * + * @param windowSize window size in millisecond + */ + public TumblingWindowAssigner(long windowSize) { + Preconditions.checkArgument(windowSize > 0, + "Window size [%s] must be positive number", windowSize); + this.windowSize = windowSize; + } + + @Override + public List assign(long timestamp) { + long startTime = DateTimeUtils.getWindowStartTime(timestamp, windowSize); + return Collections.singletonList(new Window(startTime, startTime + windowSize)); + } +} diff --git a/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/assigner/WindowAssigner.java b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/assigner/WindowAssigner.java new file mode 100644 index 0000000000..dac882c5ff --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/assigner/WindowAssigner.java @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.windowing.assigner; + +import java.util.List; +import org.opensearch.sql.planner.streaming.windowing.Window; + +/** + * A window assigner assigns zero or more window to an event timestamp + * based on different windowing approach. + */ +public interface WindowAssigner { + + /** + * Return window(s) assigned to the timestamp. + * @param timestamp given event timestamp + * @return windows assigned + */ + List assign(long timestamp); + +} diff --git a/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/trigger/AfterWatermarkWindowTrigger.java b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/trigger/AfterWatermarkWindowTrigger.java new file mode 100644 index 0000000000..1801880961 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/trigger/AfterWatermarkWindowTrigger.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.windowing.trigger; + +import lombok.RequiredArgsConstructor; +import org.opensearch.sql.planner.streaming.StreamContext; +import org.opensearch.sql.planner.streaming.windowing.Window; + +/** + * After watermark window trigger fires window state output once a window is below watermark. + * Precisely speaking, after watermark means the window boundary (max timestamp) is equal to + * or less than the current watermark timestamp. + */ +@RequiredArgsConstructor +public class AfterWatermarkWindowTrigger implements WindowTrigger { + + /** Stream context that contains the current watermark. */ + private final StreamContext context; + + @Override + public TriggerResult trigger(Window window) { + if (window.maxTimestamp() <= context.getWatermark()) { + return TriggerResult.FIRE; + } + return TriggerResult.CONTINUE; + } +} diff --git a/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/trigger/TriggerResult.java b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/trigger/TriggerResult.java new file mode 100644 index 0000000000..465f0aa9eb --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/trigger/TriggerResult.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.windowing.trigger; + +import lombok.Getter; +import lombok.RequiredArgsConstructor; + +/** + * Result determined by a trigger for what should happen to the window. + */ +@Getter +@RequiredArgsConstructor +public enum TriggerResult { + + /** Continue without any operation. */ + CONTINUE(false, false), + + /** Fire and purge window state by default. */ + FIRE(true, true); + + /** If window should be fired to output. */ + private final boolean fire; + + /** If the window state should be discarded. */ + private final boolean purge; +} diff --git a/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/trigger/WindowTrigger.java b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/trigger/WindowTrigger.java new file mode 100644 index 0000000000..f6c2eba50f --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/planner/streaming/windowing/trigger/WindowTrigger.java @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.windowing.trigger; + +import org.opensearch.sql.planner.streaming.windowing.Window; + +/** + * A window trigger determines if the current window state should be evaluated to emit output. + * Typically, trigger strategy works with downstream Sink operator together to meet the semantic + * requirements. For example, per-event trigger can work with Sink for materialized view semantic. + */ +public interface WindowTrigger { + + /** + * Return trigger result for a window. + * + * @param window given window + * @return trigger result + */ + TriggerResult trigger(Window window); +} diff --git a/core/src/main/java/org/opensearch/sql/storage/Table.java b/core/src/main/java/org/opensearch/sql/storage/Table.java index 731cf878c6..34e6ece30b 100644 --- a/core/src/main/java/org/opensearch/sql/storage/Table.java +++ b/core/src/main/java/org/opensearch/sql/storage/Table.java @@ -16,6 +16,24 @@ */ public interface Table { + /** + * Check if current table exists. + * + * @return true if exists, otherwise false + */ + default boolean exists() { + throw new UnsupportedOperationException("Unsupported Operation"); + } + + /** + * Create table given table schema. + * + * @param schema table schema + */ + default void create(Map schema) { + throw new UnsupportedOperationException("Unsupported Operation"); + } + /** * Get the {@link ExprType} for each field in the table. */ diff --git a/core/src/main/java/org/opensearch/sql/storage/split/Split.java b/core/src/main/java/org/opensearch/sql/storage/split/Split.java new file mode 100644 index 0000000000..e9e0c6fcc1 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/storage/split/Split.java @@ -0,0 +1,21 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.storage.split; + +import org.opensearch.sql.storage.StorageEngine; + +/** + * Split is a sections of a data set. Each {@link StorageEngine} should have specific + * implementation of Split. + */ +public interface Split { + + /** + * Get the split id. + * @return split id. + */ + String getSplitId(); +} diff --git a/core/src/main/java/org/opensearch/sql/utils/DateTimeUtils.java b/core/src/main/java/org/opensearch/sql/utils/DateTimeUtils.java index fbcf7deca4..5a99af3f83 100644 --- a/core/src/main/java/org/opensearch/sql/utils/DateTimeUtils.java +++ b/core/src/main/java/org/opensearch/sql/utils/DateTimeUtils.java @@ -85,6 +85,16 @@ public static long roundYear(long utcMillis, int interval) { return initDateTime.plusYears(yearToAdd).toInstant().toEpochMilli(); } + /** + * Get window start time which aligns with the given size. + * + * @param timestamp event timestamp + * @param size defines a window's start time to align with + * @return start timestamp of the window + */ + public long getWindowStartTime(long timestamp, long size) { + return timestamp - timestamp % size; + } /** * isValidMySqlTimeZoneId for timezones which match timezone the range set by MySQL. diff --git a/core/src/test/java/org/opensearch/sql/analysis/AnalyzerTestBase.java b/core/src/test/java/org/opensearch/sql/analysis/AnalyzerTestBase.java index 3523fed98e..447802c963 100644 --- a/core/src/test/java/org/opensearch/sql/analysis/AnalyzerTestBase.java +++ b/core/src/test/java/org/opensearch/sql/analysis/AnalyzerTestBase.java @@ -58,6 +58,16 @@ protected StorageEngine storageEngine() { @Bean protected Table table() { return new Table() { + @Override + public boolean exists() { + return true; + } + + @Override + public void create(Map schema) { + throw new UnsupportedOperationException("Create table is not supported"); + } + @Override public Map getFieldTypes() { return typeMapping(); diff --git a/core/src/test/java/org/opensearch/sql/config/TestConfig.java b/core/src/test/java/org/opensearch/sql/config/TestConfig.java index c41bff91f2..7475f577a6 100644 --- a/core/src/test/java/org/opensearch/sql/config/TestConfig.java +++ b/core/src/test/java/org/opensearch/sql/config/TestConfig.java @@ -65,6 +65,16 @@ protected StorageEngine storageEngine() { @Override public Table getTable(CatalogSchemaName catalogSchemaName, String name) { return new Table() { + @Override + public boolean exists() { + return true; + } + + @Override + public void create(Map schema) { + throw new UnsupportedOperationException("Create table is not supported"); + } + @Override public Map getFieldTypes() { return typeMapping; diff --git a/core/src/test/java/org/opensearch/sql/executor/DefaultQueryManagerTest.java b/core/src/test/java/org/opensearch/sql/executor/DefaultQueryManagerTest.java new file mode 100644 index 0000000000..988b41657d --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/executor/DefaultQueryManagerTest.java @@ -0,0 +1,41 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor; + + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.sql.executor.execution.AbstractPlan; + +@ExtendWith(MockitoExtension.class) +class DefaultQueryManagerTest { + + @Mock + private AbstractPlan plan; + + @Mock + private QueryId queryId; + + @Test + public void submitQuery() { + when(plan.getQueryId()).thenReturn(queryId); + + QueryId actualQueryId = new DefaultQueryManager().submit(plan); + + assertEquals(queryId, actualQueryId); + verify(plan, times(1)).execute(); + } +} diff --git a/core/src/test/java/org/opensearch/sql/executor/QueryIdTest.java b/core/src/test/java/org/opensearch/sql/executor/QueryIdTest.java new file mode 100644 index 0000000000..7d837c3e24 --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/executor/QueryIdTest.java @@ -0,0 +1,22 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor; + +import static org.junit.jupiter.api.Assertions.assertFalse; + +import com.google.common.base.Strings; +import org.junit.jupiter.api.Test; + +class QueryIdTest { + + @Test + public void createQueryId() { + assertFalse(Strings.isNullOrEmpty(QueryId.queryId().getQueryId())); + } +} diff --git a/core/src/test/java/org/opensearch/sql/executor/QueryServiceTest.java b/core/src/test/java/org/opensearch/sql/executor/QueryServiceTest.java new file mode 100644 index 0000000000..d1ffa51fcc --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/executor/QueryServiceTest.java @@ -0,0 +1,217 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor; + +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.when; + +import java.util.Collections; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.sql.analysis.Analyzer; +import org.opensearch.sql.ast.tree.UnresolvedPlan; +import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.planner.PlanContext; +import org.opensearch.sql.planner.Planner; +import org.opensearch.sql.planner.logical.LogicalPlan; +import org.opensearch.sql.planner.physical.PhysicalPlan; +import org.springframework.context.annotation.AnnotationConfigApplicationContext; + +@ExtendWith(MockitoExtension.class) +class QueryServiceTest { + private AnnotationConfigApplicationContext context = new AnnotationConfigApplicationContext(); + + private QueryService queryService; + + @Mock + private ExecutionEngine executionEngine; + + @Mock + private Analyzer analyzer; + + @Mock + private Planner planner; + + @Mock + private UnresolvedPlan ast; + + @Mock + private LogicalPlan logicalPlan; + + @Mock + private PhysicalPlan plan; + + @Mock + private ExecutionEngine.Schema schema; + + @Mock + private PlanContext planContext; + + @BeforeEach + public void setUp() { + lenient().when(analyzer.analyze(any(), any())).thenReturn(logicalPlan); + lenient().when(planner.plan(any())).thenReturn(plan); + + queryService = new QueryService(analyzer, executionEngine, planner); + } + + @Test + public void testExecuteShouldPass() { + doAnswer( + invocation -> { + ResponseListener listener = invocation.getArgument(1); + listener.onResponse( + new ExecutionEngine.QueryResponse(schema, Collections.emptyList())); + return null; + }) + .when(executionEngine) + .execute(any(), any()); + + queryService.execute( + ast, + new ResponseListener<>() { + @Override + public void onResponse(ExecutionEngine.QueryResponse pplQueryResponse) { + assertNotNull(pplQueryResponse); + } + + @Override + public void onFailure(Exception e) { + fail(); + } + }); + } + + @Test + public void testExplainShouldPass() { + doAnswer( + invocation -> { + ResponseListener listener = + invocation.getArgument(1); + listener.onResponse( + new ExecutionEngine.ExplainResponse( + new ExecutionEngine.ExplainResponseNode("test"))); + return null; + }) + .when(executionEngine) + .explain(any(), any()); + + queryService.explain( + ast, + new ResponseListener<>() { + @Override + public void onResponse(ExecutionEngine.ExplainResponse pplQueryResponse) { + assertNotNull(pplQueryResponse); + } + + @Override + public void onFailure(Exception e) { + fail(); + } + }); + } + + @Test + public void testExecuteWithExceptionShouldBeCaughtByHandler() { + doThrow(new IllegalStateException("illegal state exception")) + .when(executionEngine) + .execute(any(), any()); + + queryService.execute( + ast, + new ResponseListener<>() { + @Override + public void onResponse(ExecutionEngine.QueryResponse pplQueryResponse) { + fail(); + } + + @Override + public void onFailure(Exception e) { + assertTrue(e instanceof IllegalStateException); + } + }); + } + + @Test + public void testExecuteWithIllegalQueryShouldBeCaughtByHandler() { + doThrow(new IllegalStateException("illegal state exception")) + .when(executionEngine) + .explain(any(), any()); + + queryService.explain( + ast, + new ResponseListener<>() { + @Override + public void onResponse(ExecutionEngine.ExplainResponse pplQueryResponse) { + fail(); + } + + @Override + public void onFailure(Exception e) { + assertTrue(e instanceof IllegalStateException); + } + }); + } + + @Test + public void testExecutePlanShouldPass() { + doAnswer( + invocation -> { + ResponseListener listener = invocation.getArgument(1); + listener.onResponse( + new ExecutionEngine.QueryResponse(schema, Collections.emptyList())); + return null; + }) + .when(executionEngine) + .execute(any(), any()); + + queryService.executePlan( + logicalPlan, + planContext, + new ResponseListener<>() { + @Override + public void onResponse(ExecutionEngine.QueryResponse pplQueryResponse) { + assertNotNull(pplQueryResponse); + } + + @Override + public void onFailure(Exception e) { + fail(); + } + }); + } + + @Test + public void analyzeExceptionShouldBeCached() { + when(analyzer.analyze(any(), any())).thenThrow(IllegalStateException.class); + + queryService.execute( + ast, + new ResponseListener<>() { + @Override + public void onResponse(ExecutionEngine.QueryResponse pplQueryResponse) { + fail(); + } + + @Override + public void onFailure(Exception e) { + assertTrue(e instanceof IllegalStateException); + } + }); + } +} diff --git a/core/src/test/java/org/opensearch/sql/executor/execution/ExplainPlanTest.java b/core/src/test/java/org/opensearch/sql/executor/execution/ExplainPlanTest.java new file mode 100644 index 0000000000..54b4f24db0 --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/executor/execution/ExplainPlanTest.java @@ -0,0 +1,59 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor.execution; + +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.doNothing; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.QueryId; + +@ExtendWith(MockitoExtension.class) +public class ExplainPlanTest { + @Mock + private QueryId queryId; + + @Mock + private QueryPlan queryPlan; + + @Mock + private ResponseListener explainListener; + + @Test + public void execute() { + doNothing().when(queryPlan).explain(any()); + + ExplainPlan explainPlan = new ExplainPlan(queryId, queryPlan, explainListener); + explainPlan.execute(); + + verify(queryPlan, times(1)).explain(explainListener); + } + + @Test + public void explainThrowException() { + ExplainPlan explainPlan = new ExplainPlan(queryId, queryPlan, explainListener); + + UnsupportedOperationException unsupportedExplainException = + assertThrows( + UnsupportedOperationException.class, + () -> { + explainPlan.explain(explainListener); + }); + assertEquals("explain query can not been explained.", unsupportedExplainException.getMessage()); + } +} diff --git a/core/src/test/java/org/opensearch/sql/executor/execution/QueryPlanFactoryTest.java b/core/src/test/java/org/opensearch/sql/executor/execution/QueryPlanFactoryTest.java new file mode 100644 index 0000000000..cc4bf070fb --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/executor/execution/QueryPlanFactoryTest.java @@ -0,0 +1,107 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor.execution; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.opensearch.sql.executor.execution.QueryPlanFactory.NO_CONSUMER_RESPONSE_LISTENER; + +import java.util.Optional; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.sql.ast.statement.Explain; +import org.opensearch.sql.ast.statement.Query; +import org.opensearch.sql.ast.statement.Statement; +import org.opensearch.sql.ast.tree.UnresolvedPlan; +import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.QueryService; + +@ExtendWith(MockitoExtension.class) +class QueryPlanFactoryTest { + + @Mock + private UnresolvedPlan plan; + + @Mock + private QueryService queryService; + + @Mock + private ResponseListener queryListener; + + @Mock + private ResponseListener explainListener; + + @Mock + private ExecutionEngine.QueryResponse queryResponse; + + private QueryPlanFactory factory; + + @BeforeEach + void init() { + factory = new QueryPlanFactory(queryService); + } + + @Test + public void createFromQueryShouldSuccess() { + Statement query = new Query(plan); + AbstractPlan queryExecution = + factory.create(query, Optional.of(queryListener), Optional.empty()); + assertTrue(queryExecution instanceof QueryPlan); + } + + @Test + public void createFromExplainShouldSuccess() { + Statement query = new Explain(new Query(plan)); + AbstractPlan queryExecution = + factory.create(query, Optional.empty(), Optional.of(explainListener)); + assertTrue(queryExecution instanceof ExplainPlan); + } + + @Test + public void createFromQueryWithoutQueryListenerShouldThrowException() { + Statement query = new Query(plan); + + IllegalArgumentException exception = + assertThrows(IllegalArgumentException.class, () -> factory.create(query, + Optional.empty(), Optional.empty())); + assertEquals("[BUG] query listener must be not null", exception.getMessage()); + } + + @Test + public void createFromExplainWithoutExplainListenerShouldThrowException() { + Statement query = new Explain(new Query(plan)); + + IllegalArgumentException exception = + assertThrows(IllegalArgumentException.class, () -> factory.create(query, + Optional.empty(), Optional.empty())); + assertEquals("[BUG] explain listener must be not null", exception.getMessage()); + } + + @Test + public void noConsumerResponseChannel() { + IllegalStateException exception = + assertThrows( + IllegalStateException.class, + () -> NO_CONSUMER_RESPONSE_LISTENER.onResponse(queryResponse)); + assertEquals( + "[BUG] query response should not sent to unexpected channel", exception.getMessage()); + + exception = + assertThrows( + IllegalStateException.class, + () -> NO_CONSUMER_RESPONSE_LISTENER.onFailure(new RuntimeException())); + assertEquals( + "[BUG] exception response should not sent to unexpected channel", exception.getMessage()); + } +} diff --git a/core/src/test/java/org/opensearch/sql/executor/execution/QueryPlanTest.java b/core/src/test/java/org/opensearch/sql/executor/execution/QueryPlanTest.java new file mode 100644 index 0000000000..834db76996 --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/executor/execution/QueryPlanTest.java @@ -0,0 +1,58 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor.execution; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.sql.ast.tree.UnresolvedPlan; +import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.QueryId; +import org.opensearch.sql.executor.QueryService; + +@ExtendWith(MockitoExtension.class) +class QueryPlanTest { + + @Mock + private QueryId queryId; + + @Mock + private UnresolvedPlan plan; + + @Mock + private QueryService queryService; + + @Mock + private ResponseListener explainListener; + + @Mock + private ResponseListener queryListener; + + @Test + public void execute() { + QueryPlan query = new QueryPlan(queryId, plan, queryService, queryListener); + query.execute(); + + verify(queryService, times(1)).execute(any(), any()); + } + + @Test + public void explain() { + QueryPlan query = new QueryPlan(queryId, plan, queryService, queryListener); + query.explain(explainListener); + + verify(queryService, times(1)).explain(plan, explainListener); + } +} diff --git a/core/src/test/java/org/opensearch/sql/executor/streaming/DefaultMetadataLogTest.java b/core/src/test/java/org/opensearch/sql/executor/streaming/DefaultMetadataLogTest.java new file mode 100644 index 0000000000..4d8c4f3e93 --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/executor/streaming/DefaultMetadataLogTest.java @@ -0,0 +1,126 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.executor.streaming; + + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.Arrays; +import java.util.Optional; +import org.apache.commons.lang3.tuple.Pair; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; + +@ExtendWith(MockitoExtension.class) +class DefaultMetadataLogTest { + + private DefaultMetadataLog metadataLog; + + @BeforeEach + void setup() { + metadataLog = new DefaultMetadataLog<>(); + } + + @Test + void addMetadataShouldSuccess() { + assertTrue(metadataLog.add(0L, 0L)); + assertTrue(metadataLog.add(1L, 1L)); + } + + @Test + void addMetadataWithSameBatchIdShouldFail() { + assertTrue(metadataLog.add(0L, 0L)); + assertFalse(metadataLog.add(0L, 1L)); + } + + @Test + void addMetadataWithInvalidIdShouldThrowException() { + IllegalArgumentException exception = + assertThrows(IllegalArgumentException.class, () -> metadataLog.add(-1L, 0L)); + assertEquals("batch id must large or equal 0", exception.getMessage()); + } + + @Test + void getWithIdReturnMetadata() { + metadataLog.add(0L, 0L); + + assertTrue(metadataLog.get(0L).isPresent()); + assertEquals(0L, metadataLog.get(0L).get()); + } + + @Test + void getWithNotExistShouldReturnEmtpy() { + metadataLog.add(0L, 0L); + + assertTrue(metadataLog.get(1L).isEmpty()); + assertTrue(metadataLog.get(-1L).isEmpty()); + } + + @Test + void getWithIdInRangeShouldReturnMetadataList() { + metadataLog.add(0L, 0L); + metadataLog.add(1L, 1L); + metadataLog.add(2L, 2L); + + assertEquals(Arrays.asList(0L, 1L, 2L), metadataLog.get(Optional.of(0L), Optional.of(2L))); + assertEquals(Arrays.asList(0L, 1L, 2L), metadataLog.get(Optional.of(0L), Optional.of(4L))); + assertEquals(Arrays.asList(0L, 1L, 2L), metadataLog.get(Optional.of(-1L), Optional.of(4L))); + assertEquals(Arrays.asList(0L, 1L), metadataLog.get(Optional.of(0L), Optional.of(1L))); + assertEquals(Arrays.asList(1L, 2L), metadataLog.get(Optional.of(1L), Optional.empty())); + assertEquals(Arrays.asList(0L, 1L), metadataLog.get(Optional.empty(), Optional.of(1L))); + assertEquals(Arrays.asList(0L, 1L, 2L), metadataLog.get(Optional.empty(), Optional.empty())); + } + + @Test + void getWithIdOutOfRangeShouldReturnEmpty() { + metadataLog.add(0L, 0L); + metadataLog.add(1L, 1L); + metadataLog.add(2L, 2L); + + assertTrue(metadataLog.get(Optional.of(3L), Optional.of(5L)).isEmpty()); + } + + @Test + void getLatestShouldReturnMetadata() { + metadataLog.add(0L, 10L); + metadataLog.add(1L, 11L); + + Optional> latest = metadataLog.getLatest(); + assertTrue(latest.isPresent()); + assertEquals(1L, latest.get().getLeft()); + assertEquals(11L, latest.get().getRight()); + } + + @Test + void getLatestFromEmptyWALShouldReturnEmpty() { + Optional> latest = metadataLog.getLatest(); + assertTrue(latest.isEmpty()); + } + + @Test + void purgeLatestShouldOnlyKeepLatest() { + metadataLog.add(0L, 10L); + metadataLog.add(1L, 11L); + metadataLog.add(2L, 12L); + + Optional> latest = metadataLog.getLatest(); + assertTrue(latest.isPresent()); + metadataLog.purge(latest.get().getLeft()); + + latest = metadataLog.getLatest(); + assertTrue(latest.isPresent()); + assertEquals(2L, latest.get().getLeft()); + assertEquals(12L, latest.get().getRight()); + } +} diff --git a/core/src/test/java/org/opensearch/sql/executor/streaming/MicroBatchStreamingExecutionTest.java b/core/src/test/java/org/opensearch/sql/executor/streaming/MicroBatchStreamingExecutionTest.java new file mode 100644 index 0000000000..1a2b6e3f2a --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/executor/streaming/MicroBatchStreamingExecutionTest.java @@ -0,0 +1,284 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.executor.streaming; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.lenient; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import lombok.EqualsAndHashCode; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mockito; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.QueryService; +import org.opensearch.sql.planner.PlanContext; +import org.opensearch.sql.planner.logical.LogicalPlan; +import org.opensearch.sql.storage.split.Split; + +@ExtendWith(MockitoExtension.class) +class MicroBatchStreamingExecutionTest { + + @Test + void executedSuccess() { + streamingQuery() + .addData() + .executeSuccess(0L) + .latestOffsetLogShouldBe(0L) + .latestCommittedLogShouldBe(0L); + } + + @Test + void executedFailed() { + streamingQuery() + .addData() + .executeFailed() + .latestOffsetLogShouldBe(0L) + .noCommittedLog(); + } + + @Test + void noDataInSource() { + streamingQuery() + .neverProcess() + .noOffsetLog() + .noCommittedLog(); + } + + @Test + void noNewDataInSource() { + streamingQuery() + .addData() + .executeSuccess(0L) + .latestOffsetLogShouldBe(0L) + .latestCommittedLogShouldBe(0L) + .neverProcess() + .latestOffsetLogShouldBe(0L) + .latestCommittedLogShouldBe(0L); + } + + @Test + void addNewDataInSequenceAllExecuteSuccess() { + streamingQuery() + .addData() + .executeSuccess(0L) + .latestOffsetLogShouldBe(0L) + .latestCommittedLogShouldBe(0L) + .addData() + .executeSuccess(1L) + .latestOffsetLogShouldBe(1L) + .latestCommittedLogShouldBe(1L); + } + + @Test + void addNewDataInSequenceExecuteFailedInBetween() { + streamingQuery() + .addData() + .executeSuccess(0L) + .latestOffsetLogShouldBe(0L) + .latestCommittedLogShouldBe(0L) + .addData() + .executeFailed() + .latestOffsetLogShouldBe(1L) + .latestCommittedLogShouldBe(0L) + .executeSuccess(1L) + .latestOffsetLogShouldBe(1L) + .latestCommittedLogShouldBe(1L); + } + + @Test + void addNewDataInSequenceExecuteFailed() { + streamingQuery() + .addData() + .executeSuccess(0L) + .latestOffsetLogShouldBe(0L) + .latestCommittedLogShouldBe(0L) + .addData() + .executeFailed() + .latestOffsetLogShouldBe(1L) + .latestCommittedLogShouldBe(0L) + .executeFailed() + .latestOffsetLogShouldBe(1L) + .latestCommittedLogShouldBe(0L); + } + + Helper streamingQuery() { + return new Helper(); + } + + private static class Helper { + + private final MicroBatchStreamingExecution execution; + + private final MetadataLog offsetLog; + + private final MetadataLog committedLog; + + private final LogicalPlan batchPlan; + + private final QueryService queryService; + + private final TestStreamingSource source = new TestStreamingSource(); + + public Helper() { + this.offsetLog = new DefaultMetadataLog<>(); + this.committedLog = new DefaultMetadataLog<>(); + this.batchPlan = Mockito.mock(LogicalPlan.class); + this.queryService = Mockito.mock(QueryService.class); + this.execution = + new MicroBatchStreamingExecution( + source, batchPlan, queryService, offsetLog, committedLog); + } + + Helper addData() { + source.addData(); + return this; + } + + Helper neverProcess() { + lenient() + .doAnswer( + invocation -> { + fail(); + return null; + }) + .when(queryService) + .executePlan(any(), any(), any()); + execution.execute(); + return this; + } + + Helper executeSuccess(Long... offsets) { + lenient() + .doAnswer( + invocation -> { + ResponseListener listener = + invocation.getArgument(2); + listener.onResponse( + new ExecutionEngine.QueryResponse(null, Collections.emptyList())); + + PlanContext planContext = invocation.getArgument(1); + assertTrue(planContext.getSplit().isPresent()); + assertEquals(new TestOffsetSplit(offsets), planContext.getSplit().get()); + + return null; + }) + .when(queryService) + .executePlan(any(), any(), any()); + execution.execute(); + + return this; + } + + Helper executeFailed() { + lenient() + .doAnswer( + invocation -> { + ResponseListener listener = + invocation.getArgument(2); + listener.onFailure(new RuntimeException()); + + return null; + }) + .when(queryService) + .executePlan(any(), any(), any()); + execution.execute(); + + return this; + } + + Helper noCommittedLog() { + assertTrue(committedLog.getLatest().isEmpty()); + return this; + } + + Helper latestCommittedLogShouldBe(Long offsetId) { + assertTrue(committedLog.getLatest().isPresent()); + assertEquals(offsetId, committedLog.getLatest().get().getRight().getOffset()); + return this; + } + + Helper noOffsetLog() { + assertTrue(offsetLog.getLatest().isEmpty()); + return this; + } + + Helper latestOffsetLogShouldBe(Long offsetId) { + assertTrue(offsetLog.getLatest().isPresent()); + assertEquals(offsetId, offsetLog.getLatest().get().getRight().getOffset()); + return this; + } + } + + /** + * StreamingSource impl only for testing. + * + *

initially, offset is -1, getLatestOffset() will return Optional.emtpy(). + * + *

call addData() add offset by one. + */ + static class TestStreamingSource implements StreamingSource { + + private final AtomicLong offset = new AtomicLong(-1L); + + /** add offset by one. */ + void addData() { + offset.incrementAndGet(); + } + + /** return offset if addData was called. */ + @Override + public Optional getLatestOffset() { + if (offset.get() == -1) { + return Optional.empty(); + } else { + return Optional.of(new Offset(offset.get())); + } + } + + /** always return `empty` Batch regardless start and end offset. */ + @Override + public Batch getBatch(Optional start, Offset end) { + return new Batch( + new TestOffsetSplit( + start.map(v -> v.getOffset() + 1).orElse(0L), Long.min(offset.get(), + end.getOffset()))); + } + } + + @EqualsAndHashCode + static class TestOffsetSplit implements Split { + + private final List offsets; + + public TestOffsetSplit(Long start, Long end) { + this.offsets = new ArrayList<>(); + for (long l = start; l <= end; l++) { + this.offsets.add(l); + } + } + + public TestOffsetSplit(Long... offsets) { + this.offsets = Arrays.stream(offsets).collect(Collectors.toList()); + } + + @Override + public String getSplitId() { + return "id"; + } + } +} diff --git a/core/src/test/java/org/opensearch/sql/expression/span/SpanExpressionTest.java b/core/src/test/java/org/opensearch/sql/expression/span/SpanExpressionTest.java index a554a93153..f311c0147b 100644 --- a/core/src/test/java/org/opensearch/sql/expression/span/SpanExpressionTest.java +++ b/core/src/test/java/org/opensearch/sql/expression/span/SpanExpressionTest.java @@ -8,7 +8,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.opensearch.sql.data.type.ExprCoreType.DOUBLE; import static org.opensearch.sql.data.type.ExprCoreType.INTEGER; -import static org.opensearch.sql.data.type.ExprCoreType.TIMESTAMP; import org.junit.jupiter.api.DisplayNameGeneration; import org.junit.jupiter.api.DisplayNameGenerator; @@ -20,22 +19,17 @@ @DisplayNameGeneration(DisplayNameGenerator.ReplaceUnderscores.class) public class SpanExpressionTest extends ExpressionTestBase { @Test - void span() { + void testSpanByNumeric() { SpanExpression span = DSL.span(DSL.ref("integer_value", INTEGER), DSL.literal(1), ""); assertEquals(INTEGER, span.type()); assertEquals(ExprValueUtils.integerValue(1), span.valueOf(valueEnv())); span = DSL.span(DSL.ref("integer_value", INTEGER), DSL.literal(1.5), ""); assertEquals(DOUBLE, span.type()); - assertEquals(ExprValueUtils.doubleValue(1.5), span.valueOf(valueEnv())); + assertEquals(ExprValueUtils.doubleValue(0.0), span.valueOf(valueEnv())); span = DSL.span(DSL.ref("double_value", DOUBLE), DSL.literal(1), ""); assertEquals(DOUBLE, span.type()); assertEquals(ExprValueUtils.doubleValue(1.0), span.valueOf(valueEnv())); - - span = DSL.span(DSL.ref("timestamp_value", TIMESTAMP), DSL.literal(1), "d"); - assertEquals(TIMESTAMP, span.type()); - assertEquals(ExprValueUtils.integerValue(1), span.valueOf(valueEnv())); } - } diff --git a/core/src/test/java/org/opensearch/sql/planner/PlanContextTest.java b/core/src/test/java/org/opensearch/sql/planner/PlanContextTest.java new file mode 100644 index 0000000000..77ae78f77e --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/planner/PlanContextTest.java @@ -0,0 +1,31 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.sql.storage.split.Split; + +@ExtendWith(MockitoExtension.class) +class PlanContextTest { + + @Mock + private Split split; + + @Test + void createEmptyPlanContext() { + assertTrue(PlanContext.emptyPlanContext().getSplit().isEmpty()); + } + + @Test + void createPlanContextWithSplit() { + assertTrue(new PlanContext(split).getSplit().isPresent()); + } +} diff --git a/core/src/test/java/org/opensearch/sql/planner/PlannerTest.java b/core/src/test/java/org/opensearch/sql/planner/PlannerTest.java index 91f4e7ba45..4207c7d31b 100644 --- a/core/src/test/java/org/opensearch/sql/planner/PlannerTest.java +++ b/core/src/test/java/org/opensearch/sql/planner/PlannerTest.java @@ -124,6 +124,16 @@ protected PhysicalPlan analyze(LogicalPlan logicalPlan) { protected class MockTable extends LogicalPlanNodeVisitor implements Table { + @Override + public boolean exists() { + return true; + } + + @Override + public void create(Map schema) { + throw new UnsupportedOperationException("Create table is not supported"); + } + @Override public Map getFieldTypes() { throw new UnsupportedOperationException(); diff --git a/core/src/test/java/org/opensearch/sql/planner/physical/AggregationOperatorTest.java b/core/src/test/java/org/opensearch/sql/planner/physical/AggregationOperatorTest.java index 318499c075..46b9821752 100644 --- a/core/src/test/java/org/opensearch/sql/planner/physical/AggregationOperatorTest.java +++ b/core/src/test/java/org/opensearch/sql/planner/physical/AggregationOperatorTest.java @@ -28,6 +28,7 @@ import org.junit.jupiter.api.Test; import org.opensearch.sql.data.model.ExprDateValue; import org.opensearch.sql.data.model.ExprDatetimeValue; +import org.opensearch.sql.data.model.ExprStringValue; import org.opensearch.sql.data.model.ExprTimeValue; import org.opensearch.sql.data.model.ExprTimestampValue; import org.opensearch.sql.data.model.ExprValue; @@ -496,6 +497,96 @@ public void twoBucketsSpanAndLong() { )); } + @Test + public void aggregate_with_two_groups_with_windowing() { + PhysicalPlan plan = new AggregationOperator(testScan(compoundInputs), + Collections.singletonList(DSL.named("sum", dsl.sum(DSL.ref("errors", INTEGER)))), + Arrays.asList( + DSL.named("host", DSL.ref("host", STRING)), + DSL.named("span", DSL.span(DSL.ref("day", DATE), DSL.literal(1), "d")))); + + List result = execute(plan); + assertEquals(7, result.size()); + assertThat(result, containsInRelativeOrder( + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h1"), + "span", new ExprDateValue("2021-01-03"), + "sum", 2)), + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h1"), + "span", new ExprDateValue("2021-01-04"), + "sum", 1)), + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h1"), + "span", new ExprDateValue("2021-01-06"), + "sum", 1)), + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h1"), + "span", new ExprDateValue("2021-01-07"), + "sum", 6)), + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h2"), + "span", new ExprDateValue("2021-01-03"), + "sum", 3)), + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h2"), + "span", new ExprDateValue("2021-01-04"), + "sum", 10)), + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h2"), + "span", new ExprDateValue("2021-01-07"), + "sum", 8)))); + } + + @Test + public void aggregate_with_three_groups_with_windowing() { + PhysicalPlan plan = new AggregationOperator(testScan(compoundInputs), + Collections.singletonList(DSL.named("sum", dsl.sum(DSL.ref("errors", INTEGER)))), + Arrays.asList( + DSL.named("host", DSL.ref("host", STRING)), + DSL.named("span", DSL.span(DSL.ref("day", DATE), DSL.literal(1), "d")), + DSL.named("region", DSL.ref("region", STRING)))); + + List result = execute(plan); + assertEquals(7, result.size()); + assertThat(result, containsInRelativeOrder( + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h1"), + "span", new ExprDateValue("2021-01-03"), + "region", new ExprStringValue("iad"), + "sum", 2)), + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h1"), + "span", new ExprDateValue("2021-01-04"), + "region", new ExprStringValue("iad"), + "sum", 1)), + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h1"), + "span", new ExprDateValue("2021-01-06"), + "region", new ExprStringValue("iad"), + "sum", 1)), + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h1"), + "span", new ExprDateValue("2021-01-07"), + "region", new ExprStringValue("iad"), + "sum", 6)), + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h2"), + "span", new ExprDateValue("2021-01-03"), + "region", new ExprStringValue("iad"), + "sum", 3)), + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h2"), + "span", new ExprDateValue("2021-01-04"), + "region", new ExprStringValue("iad"), + "sum", 10)), + ExprValueUtils.tupleValue(ImmutableMap.of( + "host", new ExprStringValue("h2"), + "span", new ExprDateValue("2021-01-07"), + "region", new ExprStringValue("iad"), + "sum", 8)))); + } + @Test public void copyOfAggregationOperatorShouldSame() { AggregationOperator plan = new AggregationOperator(testScan(datetimeInputs), diff --git a/core/src/test/java/org/opensearch/sql/planner/physical/catalog/CatalogTableTest.java b/core/src/test/java/org/opensearch/sql/planner/physical/catalog/CatalogTableTest.java index def92bf5e3..1c069005f0 100644 --- a/core/src/test/java/org/opensearch/sql/planner/physical/catalog/CatalogTableTest.java +++ b/core/src/test/java/org/opensearch/sql/planner/physical/catalog/CatalogTableTest.java @@ -8,6 +8,7 @@ package org.opensearch.sql.planner.physical.catalog; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.HashMap; @@ -46,4 +47,21 @@ void testImplement() { assertTrue(physicalPlan instanceof CatalogTableScan); } + // todo. temporary added for code coverage. remove if required. + @Test + void testExist() { + UnsupportedOperationException exception = + assertThrows(UnsupportedOperationException.class, + () -> new CatalogTable(catalogService).exists()); + assertEquals("Unsupported Operation", exception.getMessage()); + } + + // todo. temporary added for code coverage. remove if required. + @Test + void testCreateTable() { + UnsupportedOperationException exception = + assertThrows(UnsupportedOperationException.class, + () -> new CatalogTable(catalogService).create(new HashMap<>())); + assertEquals("Unsupported Operation", exception.getMessage()); + } } diff --git a/core/src/test/java/org/opensearch/sql/planner/physical/collector/RoundingTest.java b/core/src/test/java/org/opensearch/sql/planner/physical/collector/RoundingTest.java index 41b3ea5d6b..f40e5c058b 100644 --- a/core/src/test/java/org/opensearch/sql/planner/physical/collector/RoundingTest.java +++ b/core/src/test/java/org/opensearch/sql/planner/physical/collector/RoundingTest.java @@ -5,16 +5,13 @@ package org.opensearch.sql.planner.physical.collector; -import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.opensearch.sql.data.type.ExprCoreType.STRING; import static org.opensearch.sql.data.type.ExprCoreType.TIME; -import com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; import org.opensearch.sql.data.model.ExprTimeValue; -import org.opensearch.sql.data.model.ExprValue; import org.opensearch.sql.data.model.ExprValueUtils; import org.opensearch.sql.exception.ExpressionEvaluationException; import org.opensearch.sql.expression.DSL; @@ -34,8 +31,6 @@ void round_unknown_type() { SpanExpression span = DSL.span(DSL.ref("unknown", STRING), DSL.literal(1), ""); Rounding rounding = Rounding.createRounding(span); assertNull(rounding.round(ExprValueUtils.integerValue(1))); - assertNull(rounding.locate(ExprValueUtils.integerValue(1))); - assertEquals(0, rounding.createBuckets().length); } @Test diff --git a/core/src/test/java/org/opensearch/sql/planner/streaming/watermark/BoundedOutOfOrderWatermarkGeneratorTest.java b/core/src/test/java/org/opensearch/sql/planner/streaming/watermark/BoundedOutOfOrderWatermarkGeneratorTest.java new file mode 100644 index 0000000000..1d18a16f2a --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/planner/streaming/watermark/BoundedOutOfOrderWatermarkGeneratorTest.java @@ -0,0 +1,61 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.watermark; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import org.junit.jupiter.api.Test; + +class BoundedOutOfOrderWatermarkGeneratorTest { + + @Test + void shouldAdvanceWatermarkIfNewerEvent() { + assertWatermarkGenerator() + .thatAllowMaxDelay(100) + .afterSeenEventTime(1000) + .shouldGenerateWatermark(899) + .afterSeenEventTime(2000) + .shouldGenerateWatermark(1899); + } + + @Test + void shouldNotAdvanceWatermarkIfLateEvent() { + assertWatermarkGenerator() + .thatAllowMaxDelay(100) + .afterSeenEventTime(1000) + .shouldGenerateWatermark(899) + .afterSeenEventTime(500) + .shouldGenerateWatermark(899) + .afterSeenEventTime(999) + .shouldGenerateWatermark(899); + } + + private static AssertionHelper assertWatermarkGenerator() { + return new AssertionHelper(); + } + + private static class AssertionHelper { + + private WatermarkGenerator generator; + + private long actualResult; + + public AssertionHelper thatAllowMaxDelay(long delay) { + this.generator = new BoundedOutOfOrderWatermarkGenerator(delay); + return this; + } + + public AssertionHelper afterSeenEventTime(long timestamp) { + this.actualResult = generator.generate(timestamp); + return this; + } + + public AssertionHelper shouldGenerateWatermark(long expected) { + assertEquals(expected, actualResult); + return this; + } + } +} \ No newline at end of file diff --git a/core/src/test/java/org/opensearch/sql/planner/streaming/windowing/WindowTest.java b/core/src/test/java/org/opensearch/sql/planner/streaming/windowing/WindowTest.java new file mode 100644 index 0000000000..9b9aafa933 --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/planner/streaming/windowing/WindowTest.java @@ -0,0 +1,21 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.windowing; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import org.junit.jupiter.api.Test; + +class WindowTest { + + @Test + void test() { + Window window = new Window(1000, 2000); + assertEquals(1000, window.getStartTime()); + assertEquals(2000, window.getEndTime()); + assertEquals(1999, window.maxTimestamp()); + } +} \ No newline at end of file diff --git a/core/src/test/java/org/opensearch/sql/planner/streaming/windowing/assigner/SlidingWindowAssignerTest.java b/core/src/test/java/org/opensearch/sql/planner/streaming/windowing/assigner/SlidingWindowAssignerTest.java new file mode 100644 index 0000000000..fd69065742 --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/planner/streaming/windowing/assigner/SlidingWindowAssignerTest.java @@ -0,0 +1,52 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.windowing.assigner; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import java.util.List; +import org.junit.jupiter.api.Test; +import org.opensearch.sql.planner.streaming.windowing.Window; + +class SlidingWindowAssignerTest { + + @Test + void testAssignWindows() { + long windowSize = 1000; + long slideSize = 500; + SlidingWindowAssigner assigner = new SlidingWindowAssigner(windowSize, slideSize); + + assertEquals( + List.of( + new Window(0, 1000), + new Window(500, 1500)), + assigner.assign(500)); + + assertEquals( + List.of( + new Window(0, 1000), + new Window(500, 1500)), + assigner.assign(999)); + + assertEquals( + List.of( + new Window(500, 1500), + new Window(1000, 2000)), + assigner.assign(1000)); + } + + @Test + void testConstructWithIllegalArguments() { + IllegalArgumentException error1 = assertThrows(IllegalArgumentException.class, + () -> new SlidingWindowAssigner(-1, 100)); + assertEquals("Window size [-1] must be positive number", error1.getMessage()); + + IllegalArgumentException error2 = assertThrows(IllegalArgumentException.class, + () -> new SlidingWindowAssigner(1000, 0)); + assertEquals("Slide size [0] must be positive number", error2.getMessage()); + } +} \ No newline at end of file diff --git a/core/src/test/java/org/opensearch/sql/planner/streaming/windowing/assigner/TumblingWindowAssignerTest.java b/core/src/test/java/org/opensearch/sql/planner/streaming/windowing/assigner/TumblingWindowAssignerTest.java new file mode 100644 index 0000000000..4c98c40f7a --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/planner/streaming/windowing/assigner/TumblingWindowAssignerTest.java @@ -0,0 +1,39 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.windowing.assigner; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import java.util.Collections; +import org.junit.jupiter.api.Test; +import org.opensearch.sql.planner.streaming.windowing.Window; + +class TumblingWindowAssignerTest { + + @Test + void testAssignWindow() { + long windowSize = 1000; + TumblingWindowAssigner assigner = new TumblingWindowAssigner(windowSize); + + assertEquals( + Collections.singletonList(new Window(0, 1000)), + assigner.assign(500)); + assertEquals( + Collections.singletonList(new Window(1000, 2000)), + assigner.assign(1999)); + assertEquals( + Collections.singletonList(new Window(2000, 3000)), + assigner.assign(2000)); + } + + @Test + void testConstructWithIllegalWindowSize() { + IllegalArgumentException error = assertThrows(IllegalArgumentException.class, + () -> new TumblingWindowAssigner(-1)); + assertEquals("Window size [-1] must be positive number", error.getMessage()); + } +} \ No newline at end of file diff --git a/core/src/test/java/org/opensearch/sql/planner/streaming/windowing/trigger/AfterWatermarkWindowTriggerTest.java b/core/src/test/java/org/opensearch/sql/planner/streaming/windowing/trigger/AfterWatermarkWindowTriggerTest.java new file mode 100644 index 0000000000..3ef6907c38 --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/planner/streaming/windowing/trigger/AfterWatermarkWindowTriggerTest.java @@ -0,0 +1,34 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.planner.streaming.windowing.trigger; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import org.junit.jupiter.api.Test; +import org.opensearch.sql.planner.streaming.StreamContext; +import org.opensearch.sql.planner.streaming.windowing.Window; + +class AfterWatermarkWindowTriggerTest { + + private final StreamContext context = new StreamContext(); + + private final AfterWatermarkWindowTrigger trigger = new AfterWatermarkWindowTrigger(context); + + @Test + void shouldNotFireWindowAboveWatermark() { + context.setWatermark(999); + assertEquals(TriggerResult.CONTINUE, trigger.trigger(new Window(500, 1500))); + assertEquals(TriggerResult.CONTINUE, trigger.trigger(new Window(500, 1001))); + assertEquals(TriggerResult.CONTINUE, trigger.trigger(new Window(1000, 1500))); + } + + @Test + void shouldFireWindowBelowWatermark() { + context.setWatermark(999); + assertEquals(TriggerResult.FIRE, trigger.trigger(new Window(500, 800))); + assertEquals(TriggerResult.FIRE, trigger.trigger(new Window(500, 1000))); + } +} \ No newline at end of file diff --git a/doctest/build.gradle b/doctest/build.gradle index aa58cf9893..8ab939cb42 100644 --- a/doctest/build.gradle +++ b/doctest/build.gradle @@ -93,7 +93,7 @@ doctest.dependsOn startOpenSearch startOpenSearch.dependsOn startPrometheus doctest.finalizedBy stopOpenSearch stopOpenSearch.finalizedBy stopPrometheus -build.dependsOn doctest +check.dependsOn doctest clean.dependsOn(cleanBootstrap) // 2.0.0-alpha1-SNAPSHOT -> 2.0.0.0-alpha1-SNAPSHOT diff --git a/filesystem/build.gradle b/filesystem/build.gradle new file mode 100644 index 0000000000..0571088132 --- /dev/null +++ b/filesystem/build.gradle @@ -0,0 +1,129 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +plugins { + id 'java-library' + id "io.freefair.lombok" + id 'jacoco' +} + +ext { + hadoop = "3.3.4" + aws = "1.12.330" +} + +configurations.all { + resolutionStrategy.force "commons-io:commons-io:2.8.0" +} + +dependencies { + implementation project(':core') + // required by hadoop filesystem https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/filesystem/index.html. + implementation("org.apache.hadoop:hadoop-common:${hadoop}") { + exclude group: 'org.apache.zookeeper' + exclude group: 'org.eclipse.jetty' + exclude group: 'com.sun.jersey' + exclude group: 'javax.servlet.jsp' + exclude group: 'javax.servlet' + exclude group: 'org.apache.kerby' + exclude group: 'org.apache.curator' + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'com.nimbusds', module: 'nimbus-jose-jwt' + // enforce version. + exclude group: 'com.fasterxml.woodstox', module: 'woodstox-core' + exclude group: 'commons-io', module: 'commons-io' + exclude group: 'ch.qos.reload4j', module: 'reload4j' + exclude group: 'org.apache.httpcomponents', module: 'httpcore' + } + implementation('com.fasterxml.woodstox:woodstox-core') + constraints { + implementation('com.fasterxml.woodstox:woodstox-core:6.4.0') { + because 'https://www.mend.io/vulnerability-database/CVE-2022-40156' + } + } + implementation('commons-io:commons-io') + constraints { + implementation('commons-io:commons-io:2.8.0') { + because 'between versions 2.8.0 and 2.5' + } + } + implementation('ch.qos.reload4j:reload4j') + constraints { + implementation('ch.qos.reload4j:reload4j:1.2.22') { + because 'between versions 1.2.22 and 1.2.19' + } + } + implementation('org.apache.httpcomponents:httpcore') + constraints { + implementation('org.apache.httpcomponents:httpcore:4.4.15') { + because 'between versions 4.4.15 and 4.4.13' + } + } + + testImplementation('org.junit.jupiter:junit-jupiter:5.6.2') + testImplementation group: 'org.hamcrest', name: 'hamcrest-library', version: '2.1' + testImplementation group: 'org.mockito', name: 'mockito-core', version: '3.12.4' + testImplementation group: 'org.mockito', name: 'mockito-junit-jupiter', version: '3.12.4' +} + +test { + useJUnitPlatform() + testLogging { + events "passed", "skipped", "failed" + exceptionFormat "full" + } + + // hadoop-fs depend on native library which is missing on windows. + // https://hadoop.apache.org/docs/r3.3.4/hadoop-project-dist/hadoop-common/NativeLibraries.html#Native_Hadoop_Library + if ('windows' == project.getProperties().getOrDefault('buildPlatform', 'linux')) { + excludes = [ + '**/FileSystemStreamSourceTest.class' + ] + } +} + +jacocoTestReport { + reports { + html.enabled true + xml.enabled true + } + afterEvaluate { + classDirectories.setFrom(files(classDirectories.files.collect { + fileTree(dir: it) + })) + } +} +test.finalizedBy(project.tasks.jacocoTestReport) + +jacocoTestCoverageVerification { + violationRules { + rule { + // hadoop-fs depend on native library which is missing on windows. + // https://hadoop.apache.org/docs/r3.3.4/hadoop-project-dist/hadoop-common/NativeLibraries.html#Native_Hadoop_Library + if ('windows' == project.getProperties().getOrDefault('buildPlatform', 'linux')) { + excludes = [ + 'org.opensearch.sql.filesystem.streaming.FileSystemStreamSource' + ] + } + element = 'CLASS' + limit { + counter = 'LINE' + minimum = 1.0 + } + limit { + counter = 'BRANCH' + minimum = 1.0 + } + } + } + afterEvaluate { + classDirectories.setFrom(files(classDirectories.files.collect { + fileTree(dir: it) + })) + } +} +check.dependsOn jacocoTestCoverageVerification +jacocoTestCoverageVerification.dependsOn jacocoTestReport diff --git a/filesystem/src/main/java/org/opensearch/sql/filesystem/storage/split/FileSystemSplit.java b/filesystem/src/main/java/org/opensearch/sql/filesystem/storage/split/FileSystemSplit.java new file mode 100644 index 0000000000..7fefb11a85 --- /dev/null +++ b/filesystem/src/main/java/org/opensearch/sql/filesystem/storage/split/FileSystemSplit.java @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.filesystem.storage.split; + +import java.util.Set; +import java.util.UUID; +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import org.apache.hadoop.fs.Path; +import org.opensearch.sql.storage.split.Split; + +@Data +public class FileSystemSplit implements Split { + + @Getter + @EqualsAndHashCode.Exclude + private final String splitId = UUID.randomUUID().toString(); + + private final Set paths; +} diff --git a/filesystem/src/main/java/org/opensearch/sql/filesystem/streaming/FileMetaData.java b/filesystem/src/main/java/org/opensearch/sql/filesystem/streaming/FileMetaData.java new file mode 100644 index 0000000000..6a8c90ee80 --- /dev/null +++ b/filesystem/src/main/java/org/opensearch/sql/filesystem/streaming/FileMetaData.java @@ -0,0 +1,21 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.filesystem.streaming; + +import java.util.Set; +import lombok.Data; +import org.apache.hadoop.fs.Path; + +/** + * File metadata. Batch id associate with the set of {@link Path}. + */ +@Data +public class FileMetaData { + + private final Long batchId; + + private final Set paths; +} diff --git a/filesystem/src/main/java/org/opensearch/sql/filesystem/streaming/FileSystemStreamSource.java b/filesystem/src/main/java/org/opensearch/sql/filesystem/streaming/FileSystemStreamSource.java new file mode 100644 index 0000000000..6a9639bdcb --- /dev/null +++ b/filesystem/src/main/java/org/opensearch/sql/filesystem/streaming/FileSystemStreamSource.java @@ -0,0 +1,105 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.filesystem.streaming; + +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import lombok.SneakyThrows; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.sql.executor.streaming.Batch; +import org.opensearch.sql.executor.streaming.DefaultMetadataLog; +import org.opensearch.sql.executor.streaming.MetadataLog; +import org.opensearch.sql.executor.streaming.Offset; +import org.opensearch.sql.executor.streaming.StreamingSource; +import org.opensearch.sql.filesystem.storage.split.FileSystemSplit; + +/** + * FileSystem Streaming Source use Hadoop FileSystem. + */ +public class FileSystemStreamSource implements StreamingSource { + + private static final Logger log = LogManager.getLogger(FileSystemStreamSource.class); + + private final MetadataLog fileMetaDataLog; + + private Set seenFiles; + + private final FileSystem fs; + + private final Path basePath; + + /** + * Constructor of FileSystemStreamSource. + */ + public FileSystemStreamSource(FileSystem fs, Path basePath) { + this.fs = fs; + this.basePath = basePath; + // todo, need to add state recovery + this.fileMetaDataLog = new DefaultMetadataLog<>(); + // todo, need to add state recovery + this.seenFiles = new HashSet<>(); + } + + @SneakyThrows(value = IOException.class) + @Override + public Optional getLatestOffset() { + // list all files. todo. improvement list performance. + Set allFiles = + Arrays.stream(fs.listStatus(basePath)) + .filter(status -> !status.isDirectory()) + .map(FileStatus::getPath) + .collect(Collectors.toSet()); + + // find unread files. + log.debug("all files {}", allFiles); + Set unread = Sets.difference(allFiles, seenFiles); + + // update seenFiles. + seenFiles = allFiles; + log.debug("seen files {}", seenFiles); + + Optional latestBatchIdOptional = fileMetaDataLog.getLatest().map(Pair::getKey); + if (!unread.isEmpty()) { + long latestBatchId = latestBatchIdOptional.map(id -> id + 1).orElse(0L); + fileMetaDataLog.add(latestBatchId, new FileMetaData(latestBatchId, unread)); + log.debug("latestBatchId {}", latestBatchId); + return Optional.of(new Offset(latestBatchId)); + } else { + log.debug("no unread data"); + Optional offset = + latestBatchIdOptional.isEmpty() + ? Optional.empty() + : Optional.of(new Offset(latestBatchIdOptional.get())); + log.debug("return empty offset {}", offset); + return offset; + } + } + + @Override + public Batch getBatch(Optional start, Offset end) { + Long startBatchId = start.map(Offset::getOffset).map(id -> id + 1).orElse(0L); + Long endBatchId = end.getOffset(); + + Set paths = + fileMetaDataLog.get(Optional.of(startBatchId), Optional.of(endBatchId)).stream() + .map(FileMetaData::getPaths) + .flatMap(Set::stream) + .collect(Collectors.toSet()); + + log.debug("fetch files {} with id from: {} to: {}.", paths, start, end); + return new Batch(new FileSystemSplit(paths)); + } +} diff --git a/filesystem/src/test/java/org/opensearch/sql/filesystem/streaming/FileSystemStreamSourceTest.java b/filesystem/src/test/java/org/opensearch/sql/filesystem/streaming/FileSystemStreamSourceTest.java new file mode 100644 index 0000000000..fba038f6a3 --- /dev/null +++ b/filesystem/src/test/java/org/opensearch/sql/filesystem/streaming/FileSystemStreamSourceTest.java @@ -0,0 +1,166 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.filesystem.streaming; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Optional; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mockito; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.sql.executor.streaming.Offset; +import org.opensearch.sql.filesystem.storage.split.FileSystemSplit; +import org.opensearch.sql.storage.split.Split; + +@ExtendWith(MockitoExtension.class) +class FileSystemStreamSourceTest { + + @TempDir Path perTestTempDir; + + FileSystemStreamSource streamSource; + + /** + * use hadoop default filesystem. it only works on unix-like system. for running on windows, it + * require native library. Reference. + * https://hadoop.apache.org/docs/r3.3.4/hadoop-project-dist/hadoop-common/NativeLibraries.html + */ + @BeforeEach + void setup() throws IOException { + streamSource = + new FileSystemStreamSource( + FileSystem.get(new Configuration()), + new org.apache.hadoop.fs.Path(perTestTempDir.toUri())); + } + + @Test + void addOneFileToSource() throws IOException { + emptySource().addFile("log1").latestOffsetShouldBe(0L).batchFromStart("log1"); + } + + @Test + void addMultipleFileInSequence() throws IOException { + emptySource() + .addFile("log1") + .latestOffsetShouldBe(0L) + .batchFromStart("log1") + .addFile("log2") + .latestOffsetShouldBe(1L) + .batchFromStart("log1", "log2") + .batchInBetween(0L, 1L, "log2"); + } + + @Test + void latestOffsetShouldSameIfNoNewFileAdded() throws IOException { + emptySource() + .addFile("log1") + .latestOffsetShouldBe(0L) + .batchFromStart("log1") + .latestOffsetShouldBe(0L) + .batchFromStart("log1"); + } + + @Test + void latestOffsetIsEmptyIfNoFilesInSource() { + emptySource().noOffset(); + } + + @Test + void dirIsFiltered() throws IOException { + emptySource() + .addFile("log1") + .latestOffsetShouldBe(0L) + .addDir("dir1") + .latestOffsetShouldBe(0L) + .batchFromStart("log1"); + } + + @Test + void sneakThrowException() throws IOException { + FileSystem fs = Mockito.mock(FileSystem.class); + doThrow(IOException.class).when(fs).listStatus(any(org.apache.hadoop.fs.Path.class)); + + streamSource = + new FileSystemStreamSource(fs, + new org.apache.hadoop.fs.Path(perTestTempDir.toUri())); + assertThrows(IOException.class, () -> streamSource.getLatestOffset()); + } + + StreamSource emptySource() { + return new StreamSource(); + } + + private class StreamSource { + + StreamSource addFile(String filename) throws IOException { + Path file = Files.createFile(perTestTempDir.resolve(filename)); + assertTrue(file.toFile().exists()); + + return this; + } + + StreamSource addDir(String dirname) throws IOException { + Path dir = Files.createDirectory(perTestTempDir.resolve(dirname)); + assertTrue(dir.toFile().isDirectory()); + + return this; + } + + StreamSource noOffset() { + assertFalse(streamSource.getLatestOffset().isPresent()); + + return this; + } + + StreamSource latestOffsetShouldBe(Long offset) { + Optional latestOffset = streamSource.getLatestOffset(); + assertTrue(latestOffset.isPresent()); + assertEquals(new Offset(offset), latestOffset.get()); + + return this; + } + + StreamSource batchFromStart(String... uris) { + assertTrue(streamSource.getLatestOffset().isPresent()); + internalBatchInBetween(Optional.empty(), streamSource.getLatestOffset().get(), uris); + + return this; + } + + StreamSource batchInBetween(Long start, Long end, String... uris) { + internalBatchInBetween(Optional.of(new Offset(start)), new Offset(end), uris); + + return this; + } + + private StreamSource internalBatchInBetween( + Optional start, Offset end, String... uris) { + Split split = streamSource.getBatch(start, end).getSplit(); + assertThat( + ((FileSystemSplit) split).getPaths(), + containsInAnyOrder( + Arrays.stream(uris) + .map(name -> new org.apache.hadoop.fs.Path(perTestTempDir.resolve(name).toUri())) + .toArray())); + return this; + } + } +} diff --git a/integ-test/build.gradle b/integ-test/build.gradle index 454d67eca7..d8d4ab3844 100644 --- a/integ-test/build.gradle +++ b/integ-test/build.gradle @@ -72,6 +72,10 @@ configurations.all { resolutionStrategy.force "org.jetbrains.kotlin:kotlin-stdlib:1.6.0" resolutionStrategy.force "org.jetbrains.kotlin:kotlin-stdlib-common:1.6.0" resolutionStrategy.force "com.squareup.okhttp3:okhttp:4.9.3" + resolutionStrategy.force "org.apache.commons:commons-math3:3.6.1" + resolutionStrategy.force "org.apache.commons:commons-lang3:3.12.0" + resolutionStrategy.force "joda-time:joda-time:2.10.12" + resolutionStrategy.force "org.slf4j:slf4j-api:1.7.36" } dependencies { diff --git a/integ-test/src/test/java/org/opensearch/sql/ppl/StandaloneIT.java b/integ-test/src/test/java/org/opensearch/sql/ppl/StandaloneIT.java index 94cafef35c..bcd0c0ffb8 100644 --- a/integ-test/src/test/java/org/opensearch/sql/ppl/StandaloneIT.java +++ b/integ-test/src/test/java/org/opensearch/sql/ppl/StandaloneIT.java @@ -18,23 +18,40 @@ import org.opensearch.client.Request; import org.opensearch.client.RestClient; import org.opensearch.client.RestHighLevelClient; +import org.opensearch.sql.analysis.Analyzer; +import org.opensearch.sql.analysis.ExpressionAnalyzer; import org.opensearch.sql.catalog.CatalogService; import org.opensearch.sql.common.response.ResponseListener; import org.opensearch.sql.common.setting.Settings; +import org.opensearch.sql.executor.DefaultQueryManager; import org.opensearch.sql.executor.ExecutionEngine; import org.opensearch.sql.executor.ExecutionEngine.QueryResponse; +import org.opensearch.sql.executor.QueryManager; +import org.opensearch.sql.executor.QueryService; +import org.opensearch.sql.executor.execution.QueryPlanFactory; +import org.opensearch.sql.expression.DSL; +import org.opensearch.sql.expression.config.ExpressionConfig; +import org.opensearch.sql.expression.function.BuiltinFunctionRepository; import org.opensearch.sql.monitor.AlwaysHealthyMonitor; import org.opensearch.sql.opensearch.client.OpenSearchClient; import org.opensearch.sql.opensearch.client.OpenSearchRestClient; import org.opensearch.sql.opensearch.executor.OpenSearchExecutionEngine; import org.opensearch.sql.opensearch.executor.protector.OpenSearchExecutionProtector; import org.opensearch.sql.opensearch.storage.OpenSearchStorageEngine; +import org.opensearch.sql.planner.Planner; +import org.opensearch.sql.planner.optimizer.LogicalPlanOptimizer; import org.opensearch.sql.plugin.catalog.CatalogServiceImpl; import org.opensearch.sql.ppl.config.PPLServiceConfig; import org.opensearch.sql.ppl.domain.PPLQueryRequest; import org.opensearch.sql.protocol.response.QueryResult; import org.opensearch.sql.protocol.response.format.SimpleJsonResponseFormatter; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.beans.factory.config.ConfigurableBeanFactory; import org.springframework.context.annotation.AnnotationConfigApplicationContext; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; +import org.springframework.context.annotation.Import; +import org.springframework.context.annotation.Scope; /** * Run PPL with query engine outside OpenSearch cluster. This IT doesn't require our plugin @@ -56,10 +73,13 @@ public void init() { AnnotationConfigApplicationContext context = new AnnotationConfigApplicationContext(); context.registerBean(ExecutionEngine.class, () -> new OpenSearchExecutionEngine(client, new OpenSearchExecutionProtector(new AlwaysHealthyMonitor()))); - context.register(PPLServiceConfig.class); + context.registerBean(OpenSearchClient.class, () -> client); + context.registerBean(Settings.class, () -> defaultSettings()); OpenSearchStorageEngine openSearchStorageEngine = new OpenSearchStorageEngine(client, defaultSettings()); CatalogServiceImpl.getInstance().registerDefaultOpenSearchCatalog(openSearchStorageEngine); context.registerBean(CatalogService.class, CatalogServiceImpl::getInstance); + context.register(StandaloneConfig.class); + context.register(PPLServiceConfig.class); context.refresh(); pplService = context.getBean(PPLService.class); @@ -144,4 +164,39 @@ public InternalRestHighLevelClient(RestClient restClient) { super(restClient, RestClient::close, Collections.emptyList()); } } + + @Configuration + @Import({ExpressionConfig.class}) + static class StandaloneConfig { + @Autowired + private CatalogService catalogService; + + @Autowired + private ExecutionEngine executionEngine; + + @Bean + QueryManager queryManager() { + return new DefaultQueryManager(); + } + + @Bean + @Scope(value = ConfigurableBeanFactory.SCOPE_PROTOTYPE) + QueryPlanFactory queryExecutionFactory(BuiltinFunctionRepository functionRepository) { + catalogService + .getCatalogs() + .forEach( + catalog -> + catalog + .getStorageEngine() + .getFunctions() + .forEach( + functionResolver -> + functionRepository.register(catalog.getName(), functionResolver))); + Analyzer analyzer = new Analyzer(new ExpressionAnalyzer(functionRepository), + catalogService, functionRepository); + Planner planner = + new Planner(LogicalPlanOptimizer.create(new DSL(functionRepository))); + return new QueryPlanFactory(new QueryService(analyzer, executionEngine, planner)); + } + } } diff --git a/legacy/src/main/java/org/opensearch/sql/legacy/plugin/OpenSearchSQLPluginConfig.java b/legacy/src/main/java/org/opensearch/sql/legacy/plugin/OpenSearchSQLPluginConfig.java deleted file mode 100644 index b396d896b0..0000000000 --- a/legacy/src/main/java/org/opensearch/sql/legacy/plugin/OpenSearchSQLPluginConfig.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - - -package org.opensearch.sql.legacy.plugin; - -import org.opensearch.client.node.NodeClient; -import org.opensearch.sql.common.setting.Settings; -import org.opensearch.sql.executor.ExecutionEngine; -import org.opensearch.sql.expression.config.ExpressionConfig; -import org.opensearch.sql.expression.function.BuiltinFunctionRepository; -import org.opensearch.sql.expression.function.OpenSearchFunctions; -import org.opensearch.sql.monitor.ResourceMonitor; -import org.opensearch.sql.opensearch.client.OpenSearchClient; -import org.opensearch.sql.opensearch.client.OpenSearchNodeClient; -import org.opensearch.sql.opensearch.executor.OpenSearchExecutionEngine; -import org.opensearch.sql.opensearch.executor.protector.ExecutionProtector; -import org.opensearch.sql.opensearch.executor.protector.OpenSearchExecutionProtector; -import org.opensearch.sql.opensearch.monitor.OpenSearchMemoryHealthy; -import org.opensearch.sql.opensearch.monitor.OpenSearchResourceMonitor; -import org.opensearch.sql.opensearch.storage.OpenSearchStorageEngine; -import org.opensearch.sql.storage.StorageEngine; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.context.annotation.Bean; -import org.springframework.context.annotation.Configuration; -import org.springframework.context.annotation.Import; - -/** - * OpenSearch Plugin Config for SQL. - */ -@Configuration -@Import({ExpressionConfig.class}) -public class OpenSearchSQLPluginConfig { - - @Autowired - private NodeClient nodeClient; - - @Autowired - private Settings settings; - - @Autowired - private BuiltinFunctionRepository functionRepository; - - @Bean - public OpenSearchClient client() { - return new OpenSearchNodeClient(nodeClient); - } - - @Bean - public StorageEngine storageEngine() { - return new OpenSearchStorageEngine(client(), settings); - } - - @Bean - public ExecutionEngine executionEngine() { - OpenSearchFunctions.register(functionRepository); - return new OpenSearchExecutionEngine(client(), protector()); - } - - @Bean - public ResourceMonitor resourceMonitor() { - return new OpenSearchResourceMonitor(settings, new OpenSearchMemoryHealthy()); - } - - @Bean - public ExecutionProtector protector() { - return new OpenSearchExecutionProtector(resourceMonitor()); - } -} diff --git a/legacy/src/main/java/org/opensearch/sql/legacy/plugin/RestSQLQueryAction.java b/legacy/src/main/java/org/opensearch/sql/legacy/plugin/RestSQLQueryAction.java index 0db08398b8..a5a3ac5a4f 100644 --- a/legacy/src/main/java/org/opensearch/sql/legacy/plugin/RestSQLQueryAction.java +++ b/legacy/src/main/java/org/opensearch/sql/legacy/plugin/RestSQLQueryAction.java @@ -6,33 +6,27 @@ package org.opensearch.sql.legacy.plugin; -import static org.opensearch.rest.RestStatus.INTERNAL_SERVER_ERROR; import static org.opensearch.rest.RestStatus.OK; import static org.opensearch.sql.executor.ExecutionEngine.QueryResponse; import static org.opensearch.sql.protocol.response.format.JsonResponseFormatter.Style.PRETTY; -import java.io.IOException; -import java.security.PrivilegedExceptionAction; import java.util.List; -import javax.xml.catalog.Catalog; +import java.util.function.BiConsumer; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.client.node.NodeClient; -import org.opensearch.cluster.service.ClusterService; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.BytesRestResponse; import org.opensearch.rest.RestChannel; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestStatus; -import org.opensearch.sql.catalog.CatalogService; import org.opensearch.sql.common.antlr.SyntaxCheckException; import org.opensearch.sql.common.response.ResponseListener; -import org.opensearch.sql.common.setting.Settings; +import org.opensearch.sql.common.utils.QueryContext; import org.opensearch.sql.executor.ExecutionEngine.ExplainResponse; import org.opensearch.sql.legacy.metrics.MetricName; import org.opensearch.sql.legacy.metrics.Metrics; import org.opensearch.sql.opensearch.security.SecurityAccess; -import org.opensearch.sql.planner.physical.PhysicalPlan; import org.opensearch.sql.protocol.response.QueryResult; import org.opensearch.sql.protocol.response.format.CsvResponseFormatter; import org.opensearch.sql.protocol.response.format.Format; @@ -41,7 +35,6 @@ import org.opensearch.sql.protocol.response.format.RawResponseFormatter; import org.opensearch.sql.protocol.response.format.ResponseFormatter; import org.opensearch.sql.sql.SQLService; -import org.opensearch.sql.sql.config.SQLServiceConfig; import org.opensearch.sql.sql.domain.SQLQueryRequest; import org.springframework.context.annotation.AnnotationConfigApplicationContext; @@ -56,23 +49,14 @@ public class RestSQLQueryAction extends BaseRestHandler { public static final RestChannelConsumer NOT_SUPPORTED_YET = null; - private final ClusterService clusterService; - - /** - * Settings required by been initialization. - */ - private final Settings pluginSettings; - - private final CatalogService catalogService; + private final AnnotationConfigApplicationContext applicationContext; /** * Constructor of RestSQLQueryAction. */ - public RestSQLQueryAction(ClusterService clusterService, Settings pluginSettings, CatalogService catalogService) { + public RestSQLQueryAction(AnnotationConfigApplicationContext applicationContext) { super(); - this.clusterService = clusterService; - this.pluginSettings = pluginSettings; - this.catalogService = catalogService; + this.applicationContext = applicationContext; } @Override @@ -92,52 +76,67 @@ protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient nod /** * Prepare REST channel consumer for a SQL query request. - * @param request SQL request - * @param nodeClient node client - * @return channel consumer + * + * @param request SQL request + * @param fallbackHandler handle request fallback to legacy engine. + * @param executionErrorHandler handle error response during new engine execution. + * @return {@link RestChannelConsumer} */ - public RestChannelConsumer prepareRequest(SQLQueryRequest request, NodeClient nodeClient) { + public RestChannelConsumer prepareRequest( + SQLQueryRequest request, + BiConsumer fallbackHandler, + BiConsumer executionErrorHandler) { if (!request.isSupported()) { - return NOT_SUPPORTED_YET; + return channel -> fallbackHandler.accept(channel, new IllegalStateException("not supported")); } - SQLService sqlService = createSQLService(nodeClient); - PhysicalPlan plan; - try { - // For now analyzing and planning stage may throw syntax exception as well - // which hints the fallback to legacy code is necessary here. - plan = sqlService.plan( - sqlService.analyze( - sqlService.parse(request.getQuery()))); - } catch (SyntaxCheckException e) { - // When explain, print info log for what unsupported syntax is causing fallback to old engine - if (request.isExplainRequest()) { - LOG.info("Request is falling back to old SQL engine due to: " + e.getMessage()); - } - return NOT_SUPPORTED_YET; - } + SQLService sqlService = + SecurityAccess.doPrivileged(() -> applicationContext.getBean(SQLService.class)); if (request.isExplainRequest()) { - return channel -> sqlService.explain(plan, createExplainResponseListener(channel)); + return channel -> + sqlService.explain( + request, + fallBackListener( + channel, + createExplainResponseListener(channel, executionErrorHandler), + fallbackHandler)); + } else { + return channel -> + sqlService.execute( + request, + fallBackListener( + channel, + createQueryResponseListener(channel, request, executionErrorHandler), + fallbackHandler)); } - return channel -> sqlService.execute(plan, createQueryResponseListener(channel, request)); } - private SQLService createSQLService(NodeClient client) { - return doPrivileged(() -> { - AnnotationConfigApplicationContext context = new AnnotationConfigApplicationContext(); - context.registerBean(ClusterService.class, () -> clusterService); - context.registerBean(NodeClient.class, () -> client); - context.registerBean(Settings.class, () -> pluginSettings); - context.registerBean(CatalogService.class, () -> catalogService); - context.register(OpenSearchSQLPluginConfig.class); - context.register(SQLServiceConfig.class); - context.refresh(); - return context.getBean(SQLService.class); - }); + private ResponseListener fallBackListener( + RestChannel channel, + ResponseListener next, + BiConsumer fallBackHandler) { + return new ResponseListener() { + @Override + public void onResponse(T response) { + LOG.error("[{}] Request is handled by new SQL query engine", + QueryContext.getRequestId()); + next.onResponse(response); + } + + @Override + public void onFailure(Exception e) { + if (e instanceof SyntaxCheckException) { + fallBackHandler.accept(channel, e); + } else { + next.onFailure(e); + } + } + }; } - private ResponseListener createExplainResponseListener(RestChannel channel) { + private ResponseListener createExplainResponseListener( + RestChannel channel, BiConsumer errorHandler) { return new ResponseListener() { @Override public void onResponse(ExplainResponse response) { @@ -151,15 +150,15 @@ protected Object buildJsonObject(ExplainResponse response) { @Override public void onFailure(Exception e) { - LOG.error("Error happened during explain", e); - logAndPublishMetrics(e); - sendResponse(channel, INTERNAL_SERVER_ERROR, - "Failed to explain the query due to error: " + e.getMessage()); + errorHandler.accept(channel, e); } }; } - private ResponseListener createQueryResponseListener(RestChannel channel, SQLQueryRequest request) { + private ResponseListener createQueryResponseListener( + RestChannel channel, + SQLQueryRequest request, + BiConsumer errorHandler) { Format format = request.format(); ResponseFormatter formatter; if (format.equals(Format.CSV)) { @@ -178,21 +177,11 @@ public void onResponse(QueryResponse response) { @Override public void onFailure(Exception e) { - LOG.error("Error happened during query handling", e); - logAndPublishMetrics(e); - sendResponse(channel, INTERNAL_SERVER_ERROR, formatter.format(e)); + errorHandler.accept(channel, e); } }; } - private T doPrivileged(PrivilegedExceptionAction action) { - try { - return SecurityAccess.doPrivileged(action); - } catch (IOException e) { - throw new IllegalStateException("Failed to perform privileged action", e); - } - } - private void sendResponse(RestChannel channel, RestStatus status, String content) { channel.sendResponse(new BytesRestResponse( status, "application/json; charset=UTF-8", content)); diff --git a/legacy/src/main/java/org/opensearch/sql/legacy/plugin/RestSqlAction.java b/legacy/src/main/java/org/opensearch/sql/legacy/plugin/RestSqlAction.java index ab146404f8..de09bcee1a 100644 --- a/legacy/src/main/java/org/opensearch/sql/legacy/plugin/RestSqlAction.java +++ b/legacy/src/main/java/org/opensearch/sql/legacy/plugin/RestSqlAction.java @@ -9,7 +9,6 @@ import static org.opensearch.rest.RestStatus.BAD_REQUEST; import static org.opensearch.rest.RestStatus.OK; import static org.opensearch.rest.RestStatus.SERVICE_UNAVAILABLE; -import static org.opensearch.sql.opensearch.executor.Scheduler.schedule; import com.alibaba.druid.sql.parser.ParserException; import com.google.common.collect.ImmutableList; @@ -27,7 +26,6 @@ import org.apache.logging.log4j.Logger; import org.opensearch.client.Client; import org.opensearch.client.node.NodeClient; -import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; import org.opensearch.index.IndexNotFoundException; import org.opensearch.rest.BaseRestHandler; @@ -35,7 +33,6 @@ import org.opensearch.rest.RestChannel; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestStatus; -import org.opensearch.sql.catalog.CatalogService; import org.opensearch.sql.common.antlr.SyntaxCheckException; import org.opensearch.sql.common.utils.QueryContext; import org.opensearch.sql.exception.ExpressionEvaluationException; @@ -65,6 +62,7 @@ import org.opensearch.sql.legacy.utils.JsonPrettyFormatter; import org.opensearch.sql.legacy.utils.QueryDataAnonymizer; import org.opensearch.sql.sql.domain.SQLQueryRequest; +import org.springframework.context.annotation.AnnotationConfigApplicationContext; public class RestSqlAction extends BaseRestHandler { @@ -89,12 +87,16 @@ public class RestSqlAction extends BaseRestHandler { */ private final RestSQLQueryAction newSqlQueryHandler; - public RestSqlAction(Settings settings, ClusterService clusterService, - org.opensearch.sql.common.setting.Settings pluginSettings, - CatalogService catalogService) { + /** + * Application context used to create SQLService for each request. + */ + private final AnnotationConfigApplicationContext applicationContext; + + public RestSqlAction(Settings settings, AnnotationConfigApplicationContext applicationContext) { super(); this.allowExplicitIndex = MULTI_ALLOW_EXPLICIT_INDEX.get(settings); - this.newSqlQueryHandler = new RestSQLQueryAction(clusterService, pluginSettings, catalogService); + this.newSqlQueryHandler = new RestSQLQueryAction(applicationContext); + this.applicationContext = applicationContext; } @Override @@ -150,27 +152,29 @@ protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient cli Format format = SqlRequestParam.getFormat(request.params()); - return channel -> schedule(client, () -> { - try { - // Route request to new query engine if it's supported already - SQLQueryRequest newSqlRequest = new SQLQueryRequest(sqlRequest.getJsonContent(), - sqlRequest.getSql(), request.path(), request.params()); - RestChannelConsumer result = newSqlQueryHandler.prepareRequest(newSqlRequest, client); - if (result != RestSQLQueryAction.NOT_SUPPORTED_YET) { - LOG.info("[{}] Request is handled by new SQL query engine", - QueryContext.getRequestId()); - result.accept(channel); - } else { - LOG.debug("[{}] Request {} is not supported and falling back to old SQL engine", - QueryContext.getRequestId(), newSqlRequest); - QueryAction queryAction = explainRequest(client, sqlRequest, format); - executeSqlRequest(request, queryAction, client, channel); - } - } catch (Exception e) { - logAndPublishMetrics(e); - reportError(channel, e, isClientError(e) ? BAD_REQUEST : SERVICE_UNAVAILABLE); - } - }); + // Route request to new query engine if it's supported already + SQLQueryRequest newSqlRequest = new SQLQueryRequest(sqlRequest.getJsonContent(), + sqlRequest.getSql(), request.path(), request.params()); + return newSqlQueryHandler.prepareRequest(newSqlRequest, + (restChannel, exception) -> { + try{ + if (newSqlRequest.isExplainRequest()) { + LOG.info("Request is falling back to old SQL engine due to: " + exception.getMessage()); + } + LOG.debug("[{}] Request {} is not supported and falling back to old SQL engine", + QueryContext.getRequestId(), newSqlRequest); + QueryAction queryAction = explainRequest(client, sqlRequest, format); + executeSqlRequest(request, queryAction, client, restChannel); + } catch (Exception e) { + logAndPublishMetrics(e); + reportError(restChannel, e, isClientError(e) ? BAD_REQUEST : SERVICE_UNAVAILABLE); + } + }, + (restChannel, exception) -> { + logAndPublishMetrics(exception); + reportError(restChannel, exception, isClientError(exception) ? + BAD_REQUEST : SERVICE_UNAVAILABLE); + }); } catch (Exception e) { logAndPublishMetrics(e); return channel -> reportError(channel, e, isClientError(e) ? BAD_REQUEST : SERVICE_UNAVAILABLE); diff --git a/legacy/src/test/java/org/opensearch/sql/legacy/plugin/RestSQLQueryActionTest.java b/legacy/src/test/java/org/opensearch/sql/legacy/plugin/RestSQLQueryActionTest.java index 56d153eb9d..4c9afe802e 100644 --- a/legacy/src/test/java/org/opensearch/sql/legacy/plugin/RestSQLQueryActionTest.java +++ b/legacy/src/test/java/org/opensearch/sql/legacy/plugin/RestSQLQueryActionTest.java @@ -6,32 +6,40 @@ package org.opensearch.sql.legacy.plugin; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertSame; -import static org.mockito.Mockito.when; -import static org.opensearch.sql.legacy.plugin.RestSQLQueryAction.NOT_SUPPORTED_YET; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; import static org.opensearch.sql.legacy.plugin.RestSqlAction.EXPLAIN_API_ENDPOINT; import static org.opensearch.sql.legacy.plugin.RestSqlAction.QUERY_API_ENDPOINT; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; import org.json.JSONObject; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; +import org.mockito.Mockito; import org.mockito.junit.MockitoJUnitRunner; import org.opensearch.client.node.NodeClient; -import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.rest.BaseRestHandler; +import org.opensearch.rest.RestChannel; +import org.opensearch.rest.RestRequest; import org.opensearch.sql.catalog.CatalogService; -import org.opensearch.sql.common.setting.Settings; +import org.opensearch.sql.common.antlr.SyntaxCheckException; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.QueryManager; +import org.opensearch.sql.executor.execution.QueryPlanFactory; +import org.opensearch.sql.sql.config.SQLServiceConfig; import org.opensearch.sql.sql.domain.SQLQueryRequest; +import org.opensearch.sql.storage.StorageEngine; import org.opensearch.threadpool.ThreadPool; +import org.springframework.context.annotation.AnnotationConfigApplicationContext; @RunWith(MockitoJUnitRunner.class) -public class RestSQLQueryActionTest { - - @Mock - private ClusterService clusterService; +public class RestSQLQueryActionTest extends BaseRestHandler { private NodeClient nodeClient; @@ -39,44 +47,68 @@ public class RestSQLQueryActionTest { private ThreadPool threadPool; @Mock - private Settings settings; + private QueryManager queryManager; + + @Mock + private QueryPlanFactory factory; + + @Mock + private ExecutionEngine.Schema schema; @Mock - private CatalogService catalogService; + private RestChannel restChannel; + + private AnnotationConfigApplicationContext context; @Before public void setup() { nodeClient = new NodeClient(org.opensearch.common.settings.Settings.EMPTY, threadPool); - when(threadPool.getThreadContext()) + context = new AnnotationConfigApplicationContext(); + context.registerBean(StorageEngine.class, () -> Mockito.mock(StorageEngine.class)); + context.registerBean(ExecutionEngine.class, () -> Mockito.mock(ExecutionEngine.class)); + context.registerBean(CatalogService.class, () -> Mockito.mock(CatalogService.class)); + context.registerBean(QueryManager.class, () -> queryManager); + context.registerBean(QueryPlanFactory.class, () -> factory); + context.register(SQLServiceConfig.class); + context.refresh(); + Mockito.lenient().when(threadPool.getThreadContext()) .thenReturn(new ThreadContext(org.opensearch.common.settings.Settings.EMPTY)); } @Test - public void handleQueryThatCanSupport() { + public void handleQueryThatCanSupport() throws Exception { SQLQueryRequest request = new SQLQueryRequest( new JSONObject("{\"query\": \"SELECT -123\"}"), "SELECT -123", QUERY_API_ENDPOINT, ""); - RestSQLQueryAction queryAction = new RestSQLQueryAction(clusterService, settings, catalogService); - assertNotSame(NOT_SUPPORTED_YET, queryAction.prepareRequest(request, nodeClient)); + RestSQLQueryAction queryAction = new RestSQLQueryAction(context); + queryAction.prepareRequest(request, (channel, exception) -> { + fail(); + }, (channel, exception) -> { + fail(); + }).accept(restChannel); } @Test - public void handleExplainThatCanSupport() { + public void handleExplainThatCanSupport() throws Exception { SQLQueryRequest request = new SQLQueryRequest( new JSONObject("{\"query\": \"SELECT -123\"}"), "SELECT -123", EXPLAIN_API_ENDPOINT, ""); - RestSQLQueryAction queryAction = new RestSQLQueryAction(clusterService, settings, catalogService); - assertNotSame(NOT_SUPPORTED_YET, queryAction.prepareRequest(request, nodeClient)); + RestSQLQueryAction queryAction = new RestSQLQueryAction(context); + queryAction.prepareRequest(request, (channel, exception) -> { + fail(); + }, (channel, exception) -> { + fail(); + }).accept(restChannel); } @Test - public void skipQueryThatNotSupport() { + public void queryThatNotSupportIsHandledByFallbackHandler() throws Exception { SQLQueryRequest request = new SQLQueryRequest( new JSONObject( "{\"query\": \"SELECT name FROM test1 JOIN test2 ON test1.name = test2.name\"}"), @@ -84,8 +116,53 @@ public void skipQueryThatNotSupport() { QUERY_API_ENDPOINT, ""); - RestSQLQueryAction queryAction = new RestSQLQueryAction(clusterService, settings, catalogService); - assertSame(NOT_SUPPORTED_YET, queryAction.prepareRequest(request, nodeClient)); + AtomicBoolean fallback = new AtomicBoolean(false); + RestSQLQueryAction queryAction = new RestSQLQueryAction(context); + queryAction.prepareRequest(request, (channel, exception) -> { + fallback.set(true); + assertTrue(exception instanceof SyntaxCheckException); + }, (channel, exception) -> { + fail(); + }).accept(restChannel); + + assertTrue(fallback.get()); + } + + @Test + public void queryExecutionFailedIsHandledByExecutionErrorHandler() throws Exception { + SQLQueryRequest request = new SQLQueryRequest( + new JSONObject( + "{\"query\": \"SELECT -123\"}"), + "SELECT -123", + QUERY_API_ENDPOINT, + ""); + + doThrow(new IllegalStateException("execution exception")) + .when(queryManager) + .submit(any()); + + AtomicBoolean executionErrorHandler = new AtomicBoolean(false); + RestSQLQueryAction queryAction = new RestSQLQueryAction(context); + queryAction.prepareRequest(request, (channel, exception) -> { + assertTrue(exception instanceof SyntaxCheckException); + }, (channel, exception) -> { + executionErrorHandler.set(true); + assertTrue(exception instanceof IllegalStateException); + }).accept(restChannel); + + assertTrue(executionErrorHandler.get()); } + @Override + public String getName() { + // do nothing, RestChannelConsumer is protected which required to extend BaseRestHandler + return null; + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient nodeClient) + throws IOException { + // do nothing, RestChannelConsumer is protected which required to extend BaseRestHandler + return null; + } } diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/client/OpenSearchClient.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/client/OpenSearchClient.java index 09a83f65a5..dc6e72bd91 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/client/OpenSearchClient.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/client/OpenSearchClient.java @@ -22,6 +22,20 @@ public interface OpenSearchClient { String META_CLUSTER_NAME = "CLUSTER_NAME"; + /** + * Check if the given index exists. + * @param indexName index name + * @return true if exists, otherwise false + */ + boolean exists(String indexName); + + /** + * Create OpenSearch index based on the given mappings. + * @param indexName index name + * @param mappings index mappings + */ + void createIndex(String indexName, Map mappings); + /** * Fetch index mapping(s) according to index expression given. * diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/client/OpenSearchNodeClient.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/client/OpenSearchNodeClient.java index 80a2fb8604..13fb0f6391 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/client/OpenSearchNodeClient.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/client/OpenSearchNodeClient.java @@ -18,6 +18,9 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.opensearch.action.admin.indices.create.CreateIndexRequest; +import org.opensearch.action.admin.indices.exists.indices.IndicesExistsRequest; +import org.opensearch.action.admin.indices.exists.indices.IndicesExistsResponse; import org.opensearch.action.admin.indices.get.GetIndexResponse; import org.opensearch.action.admin.indices.mapping.get.GetMappingsResponse; import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse; @@ -50,6 +53,28 @@ public OpenSearchNodeClient(NodeClient client) { this.resolver = new IndexNameExpressionResolver(client.threadPool().getThreadContext()); } + @Override + public boolean exists(String indexName) { + try { + IndicesExistsResponse checkExistResponse = client.admin().indices() + .exists(new IndicesExistsRequest(indexName)).actionGet(); + return checkExistResponse.isExists(); + } catch (Exception e) { + throw new IllegalStateException("Failed to check if index [" + indexName + "] exists", e); + } + } + + @Override + public void createIndex(String indexName, Map mappings) { + try { + // TODO: 1.pass index settings (the number of primary shards, etc); 2.check response? + CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName).mapping(mappings); + client.admin().indices().create(createIndexRequest).actionGet(); + } catch (Exception e) { + throw new IllegalStateException("Failed to create index [" + indexName + "]", e); + } + } + /** * Get field mappings of index by an index expression. Majority is copied from legacy * LocalClusterState. diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/client/OpenSearchRestClient.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/client/OpenSearchRestClient.java index f354215e05..d9f9dbbe5d 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/client/OpenSearchRestClient.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/client/OpenSearchRestClient.java @@ -23,6 +23,7 @@ import org.opensearch.action.search.ClearScrollRequest; import org.opensearch.client.RequestOptions; import org.opensearch.client.RestHighLevelClient; +import org.opensearch.client.indices.CreateIndexRequest; import org.opensearch.client.indices.GetIndexRequest; import org.opensearch.client.indices.GetIndexResponse; import org.opensearch.client.indices.GetMappingsRequest; @@ -46,6 +47,26 @@ public class OpenSearchRestClient implements OpenSearchClient { /** OpenSearch high level REST client. */ private final RestHighLevelClient client; + @Override + public boolean exists(String indexName) { + try { + return client.indices().exists( + new GetIndexRequest(indexName), RequestOptions.DEFAULT); + } catch (IOException e) { + throw new IllegalStateException("Failed to check if index [" + indexName + "] exist", e); + } + } + + @Override + public void createIndex(String indexName, Map mappings) { + try { + client.indices().create( + new CreateIndexRequest(indexName).mapping(mappings), RequestOptions.DEFAULT); + } catch (IOException e) { + throw new IllegalStateException("Failed to create index [" + indexName + "]", e); + } + } + @Override public Map getIndexMappings(String... indexExpression) { GetMappingsRequest request = new GetMappingsRequest().indices(indexExpression); diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/data/type/OpenSearchDataType.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/data/type/OpenSearchDataType.java index f620ae3aaf..05b80bfa23 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/data/type/OpenSearchDataType.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/data/type/OpenSearchDataType.java @@ -9,12 +9,15 @@ import static org.opensearch.sql.data.type.ExprCoreType.STRING; import static org.opensearch.sql.data.type.ExprCoreType.UNKNOWN; +import com.google.common.collect.BiMap; +import com.google.common.collect.ImmutableBiMap; import com.google.common.collect.ImmutableMap; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; import lombok.RequiredArgsConstructor; +import org.opensearch.sql.data.type.ExprCoreType; import org.opensearch.sql.data.type.ExprType; /** @@ -52,6 +55,39 @@ public boolean shouldCast(ExprType other) { OPENSEARCH_BINARY(Arrays.asList(UNKNOWN), "binary"); + /** + * Bidirectional mapping between OpenSearch type name and ExprType. + */ + private static final BiMap OPENSEARCH_TYPE_TO_EXPR_TYPE_MAPPING = + ImmutableBiMap.builder() + .put("text", OPENSEARCH_TEXT) + .put("text_keyword", OPENSEARCH_TEXT_KEYWORD) + .put("keyword", ExprCoreType.STRING) + .put("byte", ExprCoreType.BYTE) + .put("short", ExprCoreType.SHORT) + .put("integer", ExprCoreType.INTEGER) + .put("long", ExprCoreType.LONG) + .put("float", ExprCoreType.FLOAT) + .put("double", ExprCoreType.DOUBLE) + .put("boolean", ExprCoreType.BOOLEAN) + .put("nested", ExprCoreType.ARRAY) + .put("object", ExprCoreType.STRUCT) + .put("date", ExprCoreType.TIMESTAMP) + .put("ip", OPENSEARCH_IP) + .put("geo_point", OPENSEARCH_GEO_POINT) + .put("binary", OPENSEARCH_BINARY) + .build(); + + /** + * Mapping from extra OpenSearch type name which may map to same ExprType as above. + */ + private static final Map EXTRA_OPENSEARCH_TYPE_TO_EXPR_TYPE_MAPPING = + ImmutableMap.builder() + .put("half_float", ExprCoreType.FLOAT) + .put("scaled_float", ExprCoreType.DOUBLE) + .put("date_nanos", ExprCoreType.TIMESTAMP) + .build(); + /** * The mapping between Type and legacy JDBC type name. */ @@ -70,6 +106,27 @@ public boolean shouldCast(ExprType other) { */ private final String jdbcType; + /** + * Convert OpenSearch type string to ExprType. + * @param openSearchType OpenSearch type string + * @return expr type + */ + public static ExprType getExprType(String openSearchType) { + if (OPENSEARCH_TYPE_TO_EXPR_TYPE_MAPPING.containsKey(openSearchType)) { + return OPENSEARCH_TYPE_TO_EXPR_TYPE_MAPPING.get(openSearchType); + } + return EXTRA_OPENSEARCH_TYPE_TO_EXPR_TYPE_MAPPING.getOrDefault(openSearchType, UNKNOWN); + } + + /** + * Convert ExprType to OpenSearch type string. + * @param type expr type + * @return OpenSearch type string + */ + public static String getOpenSearchType(ExprType type) { + return OPENSEARCH_TYPE_TO_EXPR_TYPE_MAPPING.inverse().get(type); + } + @Override public List getParent() { return parents; diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/executor/OpenSearchQueryManager.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/executor/OpenSearchQueryManager.java new file mode 100644 index 0000000000..9c6fcdb825 --- /dev/null +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/executor/OpenSearchQueryManager.java @@ -0,0 +1,50 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.opensearch.executor; + +import java.util.Map; +import lombok.RequiredArgsConstructor; +import org.apache.logging.log4j.ThreadContext; +import org.opensearch.client.node.NodeClient; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.sql.executor.QueryId; +import org.opensearch.sql.executor.QueryManager; +import org.opensearch.sql.executor.execution.AbstractPlan; +import org.opensearch.threadpool.ThreadPool; + +/** + * QueryManager implemented in OpenSearch cluster. + */ +@RequiredArgsConstructor +public class OpenSearchQueryManager implements QueryManager { + + private final NodeClient nodeClient; + + private static final String SQL_WORKER_THREAD_POOL_NAME = "sql-worker"; + + @Override + public QueryId submit(AbstractPlan queryPlan) { + schedule(nodeClient, () -> queryPlan.execute()); + + return queryPlan.getQueryId(); + } + + private void schedule(NodeClient client, Runnable task) { + ThreadPool threadPool = client.threadPool(); + threadPool.schedule(withCurrentContext(task), new TimeValue(0), SQL_WORKER_THREAD_POOL_NAME); + } + + private Runnable withCurrentContext(final Runnable task) { + final Map currentContext = ThreadContext.getImmutableContext(); + return () -> { + ThreadContext.putAll(currentContext); + task.run(); + }; + } +} diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/executor/Scheduler.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/executor/Scheduler.java deleted file mode 100644 index 5567d1f9b2..0000000000 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/executor/Scheduler.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.sql.opensearch.executor; - -import java.util.Map; -import lombok.experimental.UtilityClass; -import org.apache.logging.log4j.ThreadContext; -import org.opensearch.client.node.NodeClient; -import org.opensearch.common.unit.TimeValue; -import org.opensearch.threadpool.ThreadPool; - -/** The scheduler which schedule the task run in sql-worker thread pool. */ -@UtilityClass -public class Scheduler { - - public static final String SQL_WORKER_THREAD_POOL_NAME = "sql-worker"; - - public static void schedule(NodeClient client, Runnable task) { - ThreadPool threadPool = client.threadPool(); - threadPool.schedule(withCurrentContext(task), new TimeValue(0), SQL_WORKER_THREAD_POOL_NAME); - } - - private static Runnable withCurrentContext(final Runnable task) { - final Map currentContext = ThreadContext.getImmutableContext(); - return () -> { - ThreadContext.putAll(currentContext); - task.run(); - }; - } -} diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/request/system/OpenSearchDescribeIndexRequest.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/request/system/OpenSearchDescribeIndexRequest.java index f321497099..50402fc75b 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/request/system/OpenSearchDescribeIndexRequest.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/request/system/OpenSearchDescribeIndexRequest.java @@ -10,7 +10,6 @@ import static org.opensearch.sql.data.model.ExprValueUtils.stringValue; import static org.opensearch.sql.opensearch.client.OpenSearchClient.META_CLUSTER_NAME; -import com.google.common.collect.ImmutableMap; import java.util.ArrayList; import java.util.HashMap; import java.util.LinkedHashMap; @@ -39,33 +38,6 @@ public class OpenSearchDescribeIndexRequest implements OpenSearchSystemRequest { private static final String DEFAULT_IS_AUTOINCREMENT = "NO"; - /** - * Type mapping from OpenSearch data type to expression type in our type system in query - * engine. TODO: geo, ip etc. - */ - private static final Map OPENSEARCH_TYPE_TO_EXPR_TYPE_MAPPING = - ImmutableMap.builder() - .put("text", OpenSearchDataType.OPENSEARCH_TEXT) - .put("text_keyword", OpenSearchDataType.OPENSEARCH_TEXT_KEYWORD) - .put("keyword", ExprCoreType.STRING) - .put("byte", ExprCoreType.BYTE) - .put("short", ExprCoreType.SHORT) - .put("integer", ExprCoreType.INTEGER) - .put("long", ExprCoreType.LONG) - .put("float", ExprCoreType.FLOAT) - .put("half_float", ExprCoreType.FLOAT) - .put("scaled_float", ExprCoreType.DOUBLE) - .put("double", ExprCoreType.DOUBLE) - .put("boolean", ExprCoreType.BOOLEAN) - .put("nested", ExprCoreType.ARRAY) - .put("object", ExprCoreType.STRUCT) - .put("date", ExprCoreType.TIMESTAMP) - .put("date_nanos", ExprCoreType.TIMESTAMP) - .put("ip", OpenSearchDataType.OPENSEARCH_IP) - .put("geo_point", OpenSearchDataType.OPENSEARCH_GEO_POINT) - .put("binary", OpenSearchDataType.OPENSEARCH_BINARY) - .build(); - /** * OpenSearch client connection. */ @@ -132,7 +104,7 @@ public Integer getMaxResultWindow() { } private ExprType transformESTypeToExprType(String openSearchType) { - return OPENSEARCH_TYPE_TO_EXPR_TYPE_MAPPING.getOrDefault(openSearchType, ExprCoreType.UNKNOWN); + return OpenSearchDataType.getExprType(openSearchType); } private ExprTupleValue row(String fieldName, String fieldType, int position, String clusterName) { diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/security/SecurityAccess.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/security/SecurityAccess.java index cff219578f..0c1b2e58b1 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/security/SecurityAccess.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/security/SecurityAccess.java @@ -6,7 +6,6 @@ package org.opensearch.sql.opensearch.security; -import java.io.IOException; import java.security.AccessController; import java.security.PrivilegedActionException; import java.security.PrivilegedExceptionAction; @@ -21,13 +20,12 @@ public class SecurityAccess { /** * Execute the operation in privileged mode. */ - public static T doPrivileged(final PrivilegedExceptionAction operation) - throws IOException { + public static T doPrivileged(final PrivilegedExceptionAction operation) { SpecialPermission.check(); try { return AccessController.doPrivileged(operation); } catch (final PrivilegedActionException e) { - throw (IOException) e.getCause(); + throw new IllegalStateException("Failed to perform privileged action", e); } } } diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/OpenSearchIndex.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/OpenSearchIndex.java index 9ebdc12ba2..26082abed1 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/OpenSearchIndex.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/OpenSearchIndex.java @@ -7,6 +7,7 @@ package org.opensearch.sql.opensearch.storage; import com.google.common.annotations.VisibleForTesting; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -18,6 +19,7 @@ import org.opensearch.sql.common.utils.StringUtils; import org.opensearch.sql.data.type.ExprType; import org.opensearch.sql.opensearch.client.OpenSearchClient; +import org.opensearch.sql.opensearch.data.type.OpenSearchDataType; import org.opensearch.sql.opensearch.data.value.OpenSearchExprValueFactory; import org.opensearch.sql.opensearch.planner.logical.OpenSearchLogicalIndexAgg; import org.opensearch.sql.opensearch.planner.logical.OpenSearchLogicalIndexScan; @@ -74,6 +76,23 @@ public OpenSearchIndex(OpenSearchClient client, Settings settings, String indexN this.indexName = new OpenSearchRequest.IndexName(indexName); } + @Override + public boolean exists() { + return client.exists(indexName.toString()); + } + + @Override + public void create(Map schema) { + Map mappings = new HashMap<>(); + Map properties = new HashMap<>(); + mappings.put("properties", properties); + + for (Map.Entry colType : schema.entrySet()) { + properties.put(colType.getKey(), OpenSearchDataType.getOpenSearchType(colType.getValue())); + } + client.createIndex(indexName.toString(), mappings); + } + /* * TODO: Assume indexName doesn't have wildcard. * Need to either handle field name conflicts diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/system/OpenSearchSystemIndex.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/system/OpenSearchSystemIndex.java index edd5593f4d..7b6efeeba4 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/system/OpenSearchSystemIndex.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/system/OpenSearchSystemIndex.java @@ -38,6 +38,17 @@ public OpenSearchSystemIndex( this.systemIndexBundle = buildIndexBundle(client, indexName); } + @Override + public boolean exists() { + return true; // TODO: implement for system index later + } + + @Override + public void create(Map schema) { + throw new UnsupportedOperationException( + "OpenSearch system index is predefined and cannot be created"); + } + @Override public Map getFieldTypes() { return systemIndexBundle.getLeft().getMapping(); diff --git a/opensearch/src/test/java/org/opensearch/sql/opensearch/client/OpenSearchNodeClientTest.java b/opensearch/src/test/java/org/opensearch/sql/opensearch/client/OpenSearchNodeClientTest.java index ad26d792ed..c3cd30a530 100644 --- a/opensearch/src/test/java/org/opensearch/sql/opensearch/client/OpenSearchNodeClientTest.java +++ b/opensearch/src/test/java/org/opensearch/sql/opensearch/client/OpenSearchNodeClientTest.java @@ -32,12 +32,17 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.lucene.search.TotalHits; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.InOrder; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.action.admin.indices.create.CreateIndexRequest; +import org.opensearch.action.admin.indices.create.CreateIndexResponse; +import org.opensearch.action.admin.indices.exists.indices.IndicesExistsRequest; +import org.opensearch.action.admin.indices.exists.indices.IndicesExistsResponse; import org.opensearch.action.admin.indices.get.GetIndexResponse; import org.opensearch.action.admin.indices.mapping.get.GetMappingsResponse; import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse; @@ -88,12 +93,66 @@ class OpenSearchNodeClientTest { private ExprTupleValue exprTupleValue = ExprTupleValue.fromExprValueMap(ImmutableMap.of("id", new ExprIntegerValue(1))); + private OpenSearchClient client; + + @BeforeEach + void setUp() { + this.client = new OpenSearchNodeClient(nodeClient); + } + + @Test + void isIndexExist() { + when(nodeClient.admin().indices() + .exists(any(IndicesExistsRequest.class)).actionGet()) + .thenReturn(new IndicesExistsResponse(true)); + + assertTrue(client.exists("test")); + } + + @Test + void isIndexNotExist() { + String indexName = "test"; + when(nodeClient.admin().indices() + .exists(any(IndicesExistsRequest.class)).actionGet()) + .thenReturn(new IndicesExistsResponse(false)); + + assertFalse(client.exists(indexName)); + } + + @Test + void isIndexExistWithException() { + when(nodeClient.admin().indices().exists(any())).thenThrow(RuntimeException.class); + + assertThrows(IllegalStateException.class, () -> client.exists("test")); + } + @Test - public void getIndexMappings() throws IOException { + void createIndex() { + String indexName = "test"; + Map mappings = ImmutableMap.of( + "properties", + ImmutableMap.of("name", "text")); + when(nodeClient.admin().indices() + .create(any(CreateIndexRequest.class)).actionGet()) + .thenReturn(new CreateIndexResponse(true, true, indexName)); + + client.createIndex(indexName, mappings); + } + + @Test + void createIndexWithException() { + when(nodeClient.admin().indices().create(any())).thenThrow(RuntimeException.class); + + assertThrows(IllegalStateException.class, + () -> client.createIndex("test", ImmutableMap.of())); + } + + @Test + void getIndexMappings() throws IOException { URL url = Resources.getResource(TEST_MAPPING_FILE); String mappings = Resources.toString(url, Charsets.UTF_8); String indexName = "test"; - OpenSearchNodeClient client = mockClient(indexName, mappings); + mockNodeClientIndicesMappings(indexName, mappings); Map indexMappings = client.getIndexMappings(indexName); assertEquals(1, indexMappings.size()); @@ -121,9 +180,9 @@ public void getIndexMappings() throws IOException { } @Test - public void getIndexMappingsWithEmptyMapping() { + void getIndexMappingsWithEmptyMapping() { String indexName = "test"; - OpenSearchNodeClient client = mockClient(indexName, ""); + mockNodeClientIndicesMappings(indexName, ""); Map indexMappings = client.getIndexMappings(indexName); assertEquals(1, indexMappings.size()); @@ -132,28 +191,25 @@ public void getIndexMappingsWithEmptyMapping() { } @Test - public void getIndexMappingsWithIOException() { + void getIndexMappingsWithIOException() { String indexName = "test"; when(nodeClient.admin().indices()).thenThrow(RuntimeException.class); - OpenSearchNodeClient client = new OpenSearchNodeClient(nodeClient); assertThrows(IllegalStateException.class, () -> client.getIndexMappings(indexName)); } @Test - public void getIndexMappingsWithNonExistIndex() { - OpenSearchNodeClient client = - new OpenSearchNodeClient(mockNodeClient("test")); + void getIndexMappingsWithNonExistIndex() { + mockNodeClient("test"); assertTrue(client.getIndexMappings("non_exist_index").isEmpty()); } @Test - public void getIndexMaxResultWindows() throws IOException { + void getIndexMaxResultWindows() throws IOException { URL url = Resources.getResource(TEST_MAPPING_SETTINGS_FILE); String indexMetadata = Resources.toString(url, Charsets.UTF_8); String indexName = "accounts"; - OpenSearchNodeClient client = - new OpenSearchNodeClient(mockNodeClientSettings(indexName, indexMetadata)); + mockNodeClientSettings(indexName, indexMetadata); Map indexMaxResultWindows = client.getIndexMaxResultWindows(indexName); assertEquals(1, indexMaxResultWindows.size()); @@ -163,12 +219,11 @@ public void getIndexMaxResultWindows() throws IOException { } @Test - public void getIndexMaxResultWindowsWithDefaultSettings() throws IOException { + void getIndexMaxResultWindowsWithDefaultSettings() throws IOException { URL url = Resources.getResource(TEST_MAPPING_FILE); String indexMetadata = Resources.toString(url, Charsets.UTF_8); String indexName = "accounts"; - OpenSearchNodeClient client = - new OpenSearchNodeClient(mockNodeClientSettings(indexName, indexMetadata)); + mockNodeClientSettings(indexName, indexMetadata); Map indexMaxResultWindows = client.getIndexMaxResultWindows(indexName); assertEquals(1, indexMaxResultWindows.size()); @@ -178,25 +233,21 @@ public void getIndexMaxResultWindowsWithDefaultSettings() throws IOException { } @Test - public void getIndexMaxResultWindowsWithIOException() { + void getIndexMaxResultWindowsWithIOException() { String indexName = "test"; when(nodeClient.admin().indices()).thenThrow(RuntimeException.class); - OpenSearchNodeClient client = new OpenSearchNodeClient(nodeClient); assertThrows(IllegalStateException.class, () -> client.getIndexMaxResultWindows(indexName)); } /** Jacoco enforce this constant lambda be tested. */ @Test - public void testAllFieldsPredicate() { + void testAllFieldsPredicate() { assertTrue(OpenSearchNodeClient.ALL_FIELDS.apply("any_index").test("any_field")); } @Test - public void search() { - OpenSearchNodeClient client = - new OpenSearchNodeClient(nodeClient); - + void search() { // Mock first scroll request SearchResponse searchResponse = mock(SearchResponse.class); when(nodeClient.search(any()).actionGet()).thenReturn(searchResponse); @@ -233,7 +284,6 @@ public void search() { @Test void schedule() { - OpenSearchNodeClient client = new OpenSearchNodeClient(nodeClient); AtomicBoolean isRun = new AtomicBoolean(false); client.schedule( () -> { @@ -249,7 +299,6 @@ void cleanup() { when(requestBuilder.addScrollId(any())).thenReturn(requestBuilder); when(requestBuilder.get()).thenReturn(null); - OpenSearchNodeClient client = new OpenSearchNodeClient(nodeClient); OpenSearchScrollRequest request = new OpenSearchScrollRequest("test", factory); request.setScrollId("scroll123"); client.cleanup(request); @@ -263,8 +312,6 @@ void cleanup() { @Test void cleanupWithoutScrollId() { - OpenSearchNodeClient client = new OpenSearchNodeClient(nodeClient); - OpenSearchScrollRequest request = new OpenSearchScrollRequest("test", factory); client.cleanup(request); verify(nodeClient, never()).prepareClearScroll(); @@ -284,7 +331,6 @@ void getIndices() { when(indexResponse.getIndices()).thenReturn(new String[] {"index"}); when(indexResponse.aliases()).thenReturn(openMap); - OpenSearchNodeClient client = new OpenSearchNodeClient(nodeClient); final List indices = client.indices(); assertEquals(2, indices.size()); } @@ -295,22 +341,15 @@ void meta() { when(nodeClient.settings()).thenReturn(settings); when(settings.get(anyString(), anyString())).thenReturn("cluster-name"); - OpenSearchNodeClient client = new OpenSearchNodeClient(nodeClient); final Map meta = client.meta(); assertEquals("cluster-name", meta.get(META_CLUSTER_NAME)); } @Test void ml() { - OpenSearchNodeClient client = new OpenSearchNodeClient(nodeClient); assertNotNull(client.getNodeClient()); } - private OpenSearchNodeClient mockClient(String indexName, String mappings) { - mockNodeClientIndicesMappings(indexName, mappings); - return new OpenSearchNodeClient(nodeClient); - } - public void mockNodeClientIndicesMappings(String indexName, String mappings) { GetMappingsResponse mockResponse = mock(GetMappingsResponse.class); MappingMetadata emptyMapping = mock(MappingMetadata.class); @@ -336,17 +375,16 @@ public void mockNodeClientIndicesMappings(String indexName, String mappings) { } } - public NodeClient mockNodeClient(String indexName) { + public void mockNodeClient(String indexName) { GetMappingsResponse mockResponse = mock(GetMappingsResponse.class); when(nodeClient.admin().indices() .prepareGetMappings(any()) .setLocal(anyBoolean()) .get()).thenReturn(mockResponse); when(mockResponse.mappings()).thenReturn(ImmutableOpenMap.of()); - return nodeClient; } - private NodeClient mockNodeClientSettings(String indexName, String indexMetadata) + private void mockNodeClientSettings(String indexName, String indexMetadata) throws IOException { GetSettingsResponse mockResponse = mock(GetSettingsResponse.class); when(nodeClient.admin().indices().prepareGetSettings(any()).setLocal(anyBoolean()).get()) @@ -357,7 +395,6 @@ private NodeClient mockNodeClientSettings(String indexName, String indexMetadata .build(); when(mockResponse.getIndexToSettings()).thenReturn(metadata); - return nodeClient; } private XContentParser createParser(String mappings) throws IOException { diff --git a/opensearch/src/test/java/org/opensearch/sql/opensearch/client/OpenSearchRestClientTest.java b/opensearch/src/test/java/org/opensearch/sql/opensearch/client/OpenSearchRestClientTest.java index bc334aaf39..25cfd6b35c 100644 --- a/opensearch/src/test/java/org/opensearch/sql/opensearch/client/OpenSearchRestClientTest.java +++ b/opensearch/src/test/java/org/opensearch/sql/opensearch/client/OpenSearchRestClientTest.java @@ -39,6 +39,7 @@ import org.opensearch.action.search.SearchResponse; import org.opensearch.client.RequestOptions; import org.opensearch.client.RestHighLevelClient; +import org.opensearch.client.indices.CreateIndexResponse; import org.opensearch.client.indices.GetIndexRequest; import org.opensearch.client.indices.GetIndexResponse; import org.opensearch.client.indices.GetMappingsRequest; @@ -69,7 +70,7 @@ class OpenSearchRestClientTest { @Mock(answer = RETURNS_DEEP_STUBS) private RestHighLevelClient restClient; - private OpenSearchRestClient client; + private OpenSearchClient client; @Mock private OpenSearchExprValueFactory factory; @@ -88,6 +89,52 @@ void setUp() { client = new OpenSearchRestClient(restClient); } + @Test + void isIndexExist() throws IOException { + when(restClient.indices() + .exists(any(), any())) // use any() because missing equals() in GetIndexRequest + .thenReturn(true); + + assertTrue(client.exists("test")); + } + + @Test + void isIndexNotExist() throws IOException { + when(restClient.indices() + .exists(any(), any())) // use any() because missing equals() in GetIndexRequest + .thenReturn(false); + + assertFalse(client.exists("test")); + } + + @Test + void isIndexExistWithException() throws IOException { + when(restClient.indices().exists(any(), any())).thenThrow(IOException.class); + + assertThrows(IllegalStateException.class, () -> client.exists("test")); + } + + @Test + void createIndex() throws IOException { + String indexName = "test"; + Map mappings = ImmutableMap.of( + "properties", + ImmutableMap.of("name", "text")); + when(restClient.indices() + .create(any(), any())) + .thenReturn(new CreateIndexResponse(true, true, indexName)); + + client.createIndex(indexName, mappings); + } + + @Test + void createIndexWithIOException() throws IOException { + when(restClient.indices().create(any(), any())).thenThrow(IOException.class); + + assertThrows(IllegalStateException.class, + () -> client.createIndex("test", ImmutableMap.of())); + } + @Test void getIndexMappings() throws IOException { URL url = Resources.getResource(TEST_MAPPING_FILE); diff --git a/opensearch/src/test/java/org/opensearch/sql/opensearch/data/type/OpenSearchDataTypeTest.java b/opensearch/src/test/java/org/opensearch/sql/opensearch/data/type/OpenSearchDataTypeTest.java index e55fff0e33..57bfcd1ea8 100644 --- a/opensearch/src/test/java/org/opensearch/sql/opensearch/data/type/OpenSearchDataTypeTest.java +++ b/opensearch/src/test/java/org/opensearch/sql/opensearch/data/type/OpenSearchDataTypeTest.java @@ -9,7 +9,10 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.opensearch.sql.data.type.ExprCoreType.DOUBLE; +import static org.opensearch.sql.data.type.ExprCoreType.FLOAT; import static org.opensearch.sql.data.type.ExprCoreType.STRING; +import static org.opensearch.sql.data.type.ExprCoreType.TIMESTAMP; import static org.opensearch.sql.opensearch.data.type.OpenSearchDataType.OPENSEARCH_TEXT; import static org.opensearch.sql.opensearch.data.type.OpenSearchDataType.OPENSEARCH_TEXT_KEYWORD; @@ -42,4 +45,23 @@ public void testShouldCast() { assertFalse(OPENSEARCH_TEXT.shouldCast(STRING)); assertFalse(OPENSEARCH_TEXT_KEYWORD.shouldCast(STRING)); } + + @Test + public void testGetExprType() { + assertEquals(OPENSEARCH_TEXT, OpenSearchDataType.getExprType("text")); + assertEquals(FLOAT, OpenSearchDataType.getExprType("float")); + assertEquals(FLOAT, OpenSearchDataType.getExprType("half_float")); + assertEquals(DOUBLE, OpenSearchDataType.getExprType("double")); + assertEquals(DOUBLE, OpenSearchDataType.getExprType("scaled_float")); + assertEquals(TIMESTAMP, OpenSearchDataType.getExprType("date")); + assertEquals(TIMESTAMP, OpenSearchDataType.getExprType("date_nanos")); + } + + @Test + public void testGetOpenSearchType() { + assertEquals("text", OpenSearchDataType.getOpenSearchType(OPENSEARCH_TEXT)); + assertEquals("float", OpenSearchDataType.getOpenSearchType(FLOAT)); + assertEquals("double", OpenSearchDataType.getOpenSearchType(DOUBLE)); + assertEquals("date", OpenSearchDataType.getOpenSearchType(TIMESTAMP)); + } } diff --git a/opensearch/src/test/java/org/opensearch/sql/opensearch/executor/SchedulerTest.java b/opensearch/src/test/java/org/opensearch/sql/opensearch/executor/OpenSearchQueryManagerTest.java similarity index 51% rename from opensearch/src/test/java/org/opensearch/sql/opensearch/executor/SchedulerTest.java rename to opensearch/src/test/java/org/opensearch/sql/opensearch/executor/OpenSearchQueryManagerTest.java index f14bda7a95..6d2b9b13ce 100644 --- a/opensearch/src/test/java/org/opensearch/sql/opensearch/executor/SchedulerTest.java +++ b/opensearch/src/test/java/org/opensearch/sql/opensearch/executor/OpenSearchQueryManagerTest.java @@ -1,6 +1,9 @@ /* - * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. */ package org.opensearch.sql.opensearch.executor; @@ -14,18 +17,47 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.client.node.NodeClient; +import org.opensearch.sql.ast.tree.UnresolvedPlan; +import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.QueryId; +import org.opensearch.sql.executor.QueryService; +import org.opensearch.sql.executor.execution.AbstractPlan; +import org.opensearch.sql.executor.execution.QueryPlan; import org.opensearch.threadpool.ThreadPool; @ExtendWith(MockitoExtension.class) -class SchedulerTest { +class OpenSearchQueryManagerTest { + + @Mock + private QueryId queryId; + + @Mock + private QueryService queryService; + + @Mock + private UnresolvedPlan plan; + + @Mock + private ResponseListener listener; + @Test - public void schedule() { + public void submitQuery() { NodeClient nodeClient = mock(NodeClient.class); ThreadPool threadPool = mock(ThreadPool.class); when(nodeClient.threadPool()).thenReturn(threadPool); + AtomicBoolean isRun = new AtomicBoolean(false); + AbstractPlan queryPlan = new QueryPlan(queryId, plan, queryService, listener) { + @Override + public void execute() { + isRun.set(true); + } + }; + doAnswer( invocation -> { Runnable task = invocation.getArgument(0); @@ -34,8 +66,8 @@ public void schedule() { }) .when(threadPool) .schedule(any(), any(), any()); - AtomicBoolean isRun = new AtomicBoolean(false); - Scheduler.schedule(nodeClient, () -> isRun.set(true)); + new OpenSearchQueryManager(nodeClient).submit(queryPlan); + assertTrue(isRun.get()); } } diff --git a/opensearch/src/test/java/org/opensearch/sql/opensearch/storage/OpenSearchIndexTest.java b/opensearch/src/test/java/org/opensearch/sql/opensearch/storage/OpenSearchIndexTest.java index 82ac3991ac..f7860403c7 100644 --- a/opensearch/src/test/java/org/opensearch/sql/opensearch/storage/OpenSearchIndexTest.java +++ b/opensearch/src/test/java/org/opensearch/sql/opensearch/storage/OpenSearchIndexTest.java @@ -14,6 +14,7 @@ import static org.hamcrest.Matchers.hasEntry; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.when; import static org.opensearch.sql.data.type.ExprCoreType.DOUBLE; import static org.opensearch.sql.data.type.ExprCoreType.INTEGER; @@ -21,6 +22,7 @@ import static org.opensearch.sql.expression.DSL.literal; import static org.opensearch.sql.expression.DSL.named; import static org.opensearch.sql.expression.DSL.ref; +import static org.opensearch.sql.opensearch.data.type.OpenSearchDataType.OPENSEARCH_TEXT_KEYWORD; import static org.opensearch.sql.opensearch.utils.Utils.indexScan; import static org.opensearch.sql.opensearch.utils.Utils.indexScanAgg; import static org.opensearch.sql.opensearch.utils.Utils.noProjects; @@ -37,10 +39,12 @@ import com.google.common.collect.ImmutableMap; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; @@ -77,6 +81,8 @@ class OpenSearchIndexTest { private final DSL dsl = new ExpressionConfig().dsl(new ExpressionConfig().functionRepository()); + private final String indexName = "test"; + @Mock private OpenSearchClient client; @@ -89,6 +95,35 @@ class OpenSearchIndexTest { @Mock private Table table; + private OpenSearchIndex index; + + @BeforeEach + void setUp() { + this.index = new OpenSearchIndex(client, settings, indexName); + } + + @Test + void isExist() { + when(client.exists(indexName)).thenReturn(true); + + assertTrue(index.exists()); + } + + @Test + void createIndex() { + Map mappings = ImmutableMap.of( + "properties", + ImmutableMap.of( + "name", "text_keyword", + "age", "integer")); + doNothing().when(client).createIndex(indexName, mappings); + + Map schema = new HashMap<>(); + schema.put("name", OPENSEARCH_TEXT_KEYWORD); + schema.put("age", INTEGER); + index.create(schema); + } + @Test void getFieldTypes() { when(client.getIndexMappings("test")) @@ -112,7 +147,6 @@ void getFieldTypes() { .put("blob", "binary") .build()))); - OpenSearchIndex index = new OpenSearchIndex(client, settings, "test"); Map fieldTypes = index.getFieldTypes(); assertThat( fieldTypes, @@ -139,9 +173,7 @@ void implementRelationOperatorOnly() { when(settings.getSettingValue(Settings.Key.QUERY_SIZE_LIMIT)).thenReturn(200); when(client.getIndexMaxResultWindows("test")).thenReturn(Map.of("test", 10000)); - String indexName = "test"; LogicalPlan plan = relation(indexName, table); - OpenSearchIndex index = new OpenSearchIndex(client, settings, indexName); Integer maxResultWindow = index.getMaxResultWindow(); assertEquals( new OpenSearchIndexScan(client, settings, indexName, maxResultWindow, exprValueFactory), @@ -153,9 +185,7 @@ void implementRelationOperatorWithOptimization() { when(settings.getSettingValue(Settings.Key.QUERY_SIZE_LIMIT)).thenReturn(200); when(client.getIndexMaxResultWindows("test")).thenReturn(Map.of("test", 10000)); - String indexName = "test"; LogicalPlan plan = relation(indexName, table); - OpenSearchIndex index = new OpenSearchIndex(client, settings, indexName); Integer maxResultWindow = index.getMaxResultWindow(); assertEquals( new OpenSearchIndexScan(client, settings, indexName, maxResultWindow, exprValueFactory), @@ -167,7 +197,6 @@ void implementOtherLogicalOperators() { when(settings.getSettingValue(Settings.Key.QUERY_SIZE_LIMIT)).thenReturn(200); when(client.getIndexMaxResultWindows("test")).thenReturn(Map.of("test", 10000)); - String indexName = "test"; NamedExpression include = named("age", ref("age", INTEGER)); ReferenceExpression exclude = ref("name", STRING); ReferenceExpression dedupeField = ref("name", STRING); @@ -199,7 +228,6 @@ void implementOtherLogicalOperators() { dedupeField), include); - OpenSearchIndex index = new OpenSearchIndex(client, settings, indexName); Integer maxResultWindow = index.getMaxResultWindow(); assertEquals( PhysicalPlanDSL.project( @@ -228,8 +256,6 @@ void shouldImplLogicalIndexScan() { NamedExpression named = named("n", field); Expression filterExpr = dsl.equal(field, literal("John")); - String indexName = "test"; - OpenSearchIndex index = new OpenSearchIndex(client, settings, indexName); PhysicalPlan plan = index.implement( project( indexScan( @@ -254,8 +280,6 @@ void shouldNotPushDownFilterFarFromRelation() { Arrays.asList(named("avg(age)", new AvgAggregator(Arrays.asList(ref("age", INTEGER)), DOUBLE))); - String indexName = "test"; - OpenSearchIndex index = new OpenSearchIndex(client, settings, indexName); PhysicalPlan plan = index.implement( filter( aggregation( @@ -280,9 +304,6 @@ void shouldImplLogicalIndexScanAgg() { Arrays.asList(named("avg(age)", new AvgAggregator(Arrays.asList(ref("age", INTEGER)), DOUBLE))); - String indexName = "test"; - OpenSearchIndex index = new OpenSearchIndex(client, settings, indexName); - // IndexScanAgg without Filter PhysicalPlan plan = index.implement( filter( @@ -317,9 +338,6 @@ void shouldNotPushDownAggregationFarFromRelation() { Arrays.asList(named("avg(age)", new AvgAggregator(Arrays.asList(ref("age", INTEGER)), DOUBLE))); - String indexName = "test"; - OpenSearchIndex index = new OpenSearchIndex(client, settings, indexName); - PhysicalPlan plan = index.implement( aggregation( filter(filter( @@ -339,8 +357,6 @@ void shouldImplIndexScanWithSort() { NamedExpression named = named("n", field); Expression sortExpr = ref("name", STRING); - String indexName = "test"; - OpenSearchIndex index = new OpenSearchIndex(client, settings, indexName); PhysicalPlan plan = index.implement( project( indexScan( @@ -361,8 +377,6 @@ void shouldImplIndexScanWithLimit() { ReferenceExpression field = ref("name", STRING); NamedExpression named = named("n", field); - String indexName = "test"; - OpenSearchIndex index = new OpenSearchIndex(client, settings, indexName); PhysicalPlan plan = index.implement( project( indexScan( @@ -384,8 +398,6 @@ void shouldImplIndexScanWithSortAndLimit() { NamedExpression named = named("n", field); Expression sortExpr = ref("name", STRING); - String indexName = "test"; - OpenSearchIndex index = new OpenSearchIndex(client, settings, indexName); PhysicalPlan plan = index.implement( project( indexScan( @@ -405,8 +417,6 @@ void shouldNotPushDownLimitFarFromRelationButUpdateScanSize() { when(settings.getSettingValue(Settings.Key.QUERY_SIZE_LIMIT)).thenReturn(200); when(client.getIndexMaxResultWindows("test")).thenReturn(Map.of("test", 10000)); - String indexName = "test"; - OpenSearchIndex index = new OpenSearchIndex(client, settings, indexName); PhysicalPlan plan = index.implement(index.optimize( project( limit( @@ -430,8 +440,6 @@ void shouldPushDownProjects() { when(settings.getSettingValue(Settings.Key.QUERY_SIZE_LIMIT)).thenReturn(200); when(client.getIndexMaxResultWindows("test")).thenReturn(Map.of("test", 10000)); - String indexName = "test"; - OpenSearchIndex index = new OpenSearchIndex(client, settings, indexName); PhysicalPlan plan = index.implement( project( indexScan( diff --git a/opensearch/src/test/java/org/opensearch/sql/opensearch/storage/system/OpenSearchSystemIndexTest.java b/opensearch/src/test/java/org/opensearch/sql/opensearch/storage/system/OpenSearchSystemIndexTest.java index e2efff22cb..a483f2dad8 100644 --- a/opensearch/src/test/java/org/opensearch/sql/opensearch/storage/system/OpenSearchSystemIndexTest.java +++ b/opensearch/src/test/java/org/opensearch/sql/opensearch/storage/system/OpenSearchSystemIndexTest.java @@ -9,6 +9,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.hasEntry; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.opensearch.sql.data.type.ExprCoreType.STRING; import static org.opensearch.sql.expression.DSL.named; @@ -18,6 +19,7 @@ import static org.opensearch.sql.utils.SystemIndexUtils.TABLE_INFO; import static org.opensearch.sql.utils.SystemIndexUtils.mappingTable; +import com.google.common.collect.ImmutableMap; import java.util.Map; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -58,6 +60,19 @@ void testGetFieldTypesOfMappingTable() { )); } + @Test + void testIsExist() { + Table systemIndex = new OpenSearchSystemIndex(client, TABLE_INFO); + assertTrue(systemIndex.exists()); + } + + @Test + void testCreateTable() { + Table systemIndex = new OpenSearchSystemIndex(client, TABLE_INFO); + assertThrows(UnsupportedOperationException.class, + () -> systemIndex.create(ImmutableMap.of())); + } + @Test void implement() { OpenSearchSystemIndex systemIndex = new OpenSearchSystemIndex(client, TABLE_INFO); diff --git a/plugin/build.gradle b/plugin/build.gradle index 6a0900c3cc..4754292216 100644 --- a/plugin/build.gradle +++ b/plugin/build.gradle @@ -93,6 +93,10 @@ configurations.all { resolutionStrategy.force "org.jetbrains.kotlin:kotlin-stdlib:1.6.0" resolutionStrategy.force "org.jetbrains.kotlin:kotlin-stdlib-common:1.6.0" resolutionStrategy.force "com.squareup.okhttp3:okhttp:4.9.3" + resolutionStrategy.force "org.apache.commons:commons-math3:3.6.1" + resolutionStrategy.force "org.apache.commons:commons-lang3:3.12.0" + resolutionStrategy.force "joda-time:joda-time:2.10.12" + resolutionStrategy.force "org.slf4j:slf4j-api:1.7.36" } compileJava { options.compilerArgs.addAll(["-processor", 'lombok.launch.AnnotationProcessorHider$AnnotationProcessor']) @@ -108,11 +112,11 @@ dependencies { api "com.fasterxml.jackson.core:jackson-databind:${jackson_databind_version}" api "com.fasterxml.jackson.core:jackson-annotations:${jackson_version}" - api project(":ppl") api project(':legacy') api project(':opensearch') api project(':prometheus') + api project(':filesystem') } test { 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 9234668ea1..e1d29a5775 100644 --- a/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java +++ b/plugin/src/main/java/org/opensearch/sql/plugin/SQLPlugin.java @@ -40,12 +40,14 @@ import org.opensearch.script.ScriptContext; import org.opensearch.script.ScriptEngine; import org.opensearch.script.ScriptService; +import org.opensearch.sql.catalog.CatalogService; import org.opensearch.sql.legacy.esdomain.LocalClusterState; import org.opensearch.sql.legacy.executor.AsyncRestExecutor; import org.opensearch.sql.legacy.metrics.Metrics; import org.opensearch.sql.legacy.plugin.RestSqlAction; import org.opensearch.sql.legacy.plugin.RestSqlStatsAction; import org.opensearch.sql.opensearch.client.OpenSearchNodeClient; +import org.opensearch.sql.opensearch.security.SecurityAccess; import org.opensearch.sql.opensearch.setting.LegacyOpenDistroSettings; import org.opensearch.sql.opensearch.setting.OpenSearchSettings; import org.opensearch.sql.opensearch.storage.OpenSearchStorageEngine; @@ -53,17 +55,21 @@ import org.opensearch.sql.opensearch.storage.serialization.DefaultExpressionSerializer; import org.opensearch.sql.plugin.catalog.CatalogServiceImpl; import org.opensearch.sql.plugin.catalog.CatalogSettings; +import org.opensearch.sql.plugin.config.OpenSearchPluginConfig; import org.opensearch.sql.plugin.rest.RestPPLQueryAction; import org.opensearch.sql.plugin.rest.RestPPLStatsAction; import org.opensearch.sql.plugin.rest.RestQuerySettingsAction; import org.opensearch.sql.plugin.transport.PPLQueryAction; import org.opensearch.sql.plugin.transport.TransportPPLQueryAction; import org.opensearch.sql.plugin.transport.TransportPPLQueryResponse; +import org.opensearch.sql.ppl.config.PPLServiceConfig; +import org.opensearch.sql.sql.config.SQLServiceConfig; import org.opensearch.sql.storage.StorageEngine; import org.opensearch.threadpool.ExecutorBuilder; import org.opensearch.threadpool.FixedExecutorBuilder; import org.opensearch.threadpool.ThreadPool; import org.opensearch.watcher.ResourceWatcherService; +import org.springframework.context.annotation.AnnotationConfigApplicationContext; public class SQLPlugin extends Plugin implements ActionPlugin, ScriptPlugin, ReloadablePlugin { @@ -76,6 +82,8 @@ public class SQLPlugin extends Plugin implements ActionPlugin, ScriptPlugin, Rel private NodeClient client; + private AnnotationConfigApplicationContext applicationContext; + public String name() { return "sql"; } @@ -101,8 +109,7 @@ public List getRestHandlers( return Arrays.asList( new RestPPLQueryAction(pluginSettings, settings), - new RestSqlAction(settings, clusterService, pluginSettings, - CatalogServiceImpl.getInstance()), + new RestSqlAction(settings, applicationContext), new RestSqlStatsAction(settings, restController), new RestPPLStatsAction(settings, restController), new RestQuerySettingsAction(settings, restController)); @@ -140,18 +147,25 @@ public Collection createComponents( LocalClusterState.state().setClusterService(clusterService); LocalClusterState.state().setPluginSettings((OpenSearchSettings) pluginSettings); - return super.createComponents( - client, - clusterService, - threadPool, - resourceWatcherService, - scriptService, - contentRegistry, - environment, - nodeEnvironment, - namedWriteableRegistry, - indexNameResolver, - repositoriesServiceSupplier); + this.applicationContext = new AnnotationConfigApplicationContext(); + SecurityAccess.doPrivileged( + () -> { + applicationContext.registerBean(ClusterService.class, () -> clusterService); + applicationContext.registerBean(NodeClient.class, () -> (NodeClient) client); + applicationContext.registerBean( + org.opensearch.sql.common.setting.Settings.class, () -> pluginSettings); + applicationContext.registerBean( + CatalogService.class, () -> CatalogServiceImpl.getInstance()); + applicationContext.register(OpenSearchPluginConfig.class); + applicationContext.register(PPLServiceConfig.class); + applicationContext.register(SQLServiceConfig.class); + applicationContext.refresh(); + return null; + }); + + // return objects used by Guice to inject dependencies for e.g., + // transport action handler constructors + return ImmutableList.of(applicationContext); } @Override @@ -186,8 +200,7 @@ public void reload(Settings settings) { } private StorageEngine openSearchStorageEngine() { - return new OpenSearchStorageEngine(new OpenSearchNodeClient(client), - pluginSettings); + return new OpenSearchStorageEngine(new OpenSearchNodeClient(client), pluginSettings); } } diff --git a/plugin/src/main/java/org/opensearch/sql/plugin/catalog/CatalogServiceImpl.java b/plugin/src/main/java/org/opensearch/sql/plugin/catalog/CatalogServiceImpl.java index 2e88a49f35..3c6e0e5281 100644 --- a/plugin/src/main/java/org/opensearch/sql/plugin/catalog/CatalogServiceImpl.java +++ b/plugin/src/main/java/org/opensearch/sql/plugin/catalog/CatalogServiceImpl.java @@ -12,7 +12,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.io.InputStream; -import java.security.PrivilegedExceptionAction; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -65,7 +64,7 @@ private CatalogServiceImpl() { * @param settings settings. */ public void loadConnectors(Settings settings) { - doPrivileged(() -> { + SecurityAccess.doPrivileged(() -> { InputStream inputStream = CatalogSettings.CATALOG_CONFIG.get(settings); if (inputStream != null) { ObjectMapper objectMapper = new ObjectMapper(); @@ -110,14 +109,6 @@ public void registerDefaultOpenSearchCatalog(StorageEngine storageEngine) { new Catalog(DEFAULT_CATALOG_NAME, ConnectorType.OPENSEARCH, storageEngine)); } - private T doPrivileged(PrivilegedExceptionAction action) { - try { - return SecurityAccess.doPrivileged(action); - } catch (IOException e) { - throw new IllegalStateException("Failed to perform privileged action", e); - } - } - private StorageEngine createStorageEngine(CatalogMetadata catalog) { ConnectorType connector = catalog.getConnector(); switch (connector) { diff --git a/plugin/src/main/java/org/opensearch/sql/plugin/config/OpenSearchPluginConfig.java b/plugin/src/main/java/org/opensearch/sql/plugin/config/OpenSearchPluginConfig.java new file mode 100644 index 0000000000..596296522c --- /dev/null +++ b/plugin/src/main/java/org/opensearch/sql/plugin/config/OpenSearchPluginConfig.java @@ -0,0 +1,121 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + + +package org.opensearch.sql.plugin.config; + +import org.opensearch.client.node.NodeClient; +import org.opensearch.sql.analysis.Analyzer; +import org.opensearch.sql.analysis.ExpressionAnalyzer; +import org.opensearch.sql.catalog.CatalogService; +import org.opensearch.sql.common.setting.Settings; +import org.opensearch.sql.executor.ExecutionEngine; +import org.opensearch.sql.executor.QueryManager; +import org.opensearch.sql.executor.QueryService; +import org.opensearch.sql.executor.execution.QueryPlanFactory; +import org.opensearch.sql.expression.DSL; +import org.opensearch.sql.expression.config.ExpressionConfig; +import org.opensearch.sql.expression.function.BuiltinFunctionRepository; +import org.opensearch.sql.monitor.ResourceMonitor; +import org.opensearch.sql.opensearch.client.OpenSearchClient; +import org.opensearch.sql.opensearch.client.OpenSearchNodeClient; +import org.opensearch.sql.opensearch.executor.OpenSearchExecutionEngine; +import org.opensearch.sql.opensearch.executor.OpenSearchQueryManager; +import org.opensearch.sql.opensearch.executor.protector.ExecutionProtector; +import org.opensearch.sql.opensearch.executor.protector.OpenSearchExecutionProtector; +import org.opensearch.sql.opensearch.monitor.OpenSearchMemoryHealthy; +import org.opensearch.sql.opensearch.monitor.OpenSearchResourceMonitor; +import org.opensearch.sql.opensearch.storage.OpenSearchStorageEngine; +import org.opensearch.sql.planner.Planner; +import org.opensearch.sql.planner.optimizer.LogicalPlanOptimizer; +import org.opensearch.sql.storage.StorageEngine; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.beans.factory.config.ConfigurableBeanFactory; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; +import org.springframework.context.annotation.Import; +import org.springframework.context.annotation.Scope; + +/** + * OpenSearch plugin config that injects cluster service and node client from plugin + * and initialize OpenSearch storage and execution engine. + */ +@Configuration +@Import({ExpressionConfig.class}) +public class OpenSearchPluginConfig { + + @Autowired + private NodeClient nodeClient; + + @Autowired + private Settings settings; + + @Autowired + private CatalogService catalogService; + + @Bean + @Scope(value = ConfigurableBeanFactory.SCOPE_PROTOTYPE) + public OpenSearchClient client() { + return new OpenSearchNodeClient(nodeClient); + } + + @Bean + @Scope(value = ConfigurableBeanFactory.SCOPE_PROTOTYPE) + public StorageEngine storageEngine() { + return new OpenSearchStorageEngine(client(), settings); + } + + @Bean + @Scope(value = ConfigurableBeanFactory.SCOPE_PROTOTYPE) + public ExecutionEngine executionEngine() { + return new OpenSearchExecutionEngine(client(), protector()); + } + + @Bean + @Scope(value = ConfigurableBeanFactory.SCOPE_PROTOTYPE) + public ResourceMonitor resourceMonitor() { + return new OpenSearchResourceMonitor(settings, new OpenSearchMemoryHealthy()); + } + + @Bean + @Scope(value = ConfigurableBeanFactory.SCOPE_PROTOTYPE) + public ExecutionProtector protector() { + return new OpenSearchExecutionProtector(resourceMonitor()); + } + + /** + * Per node singleton object. + */ + @Bean + public QueryManager queryManager() { + return new OpenSearchQueryManager(nodeClient); + } + + /** + * QueryPlanFactory. + */ + @Bean + @Scope(value = ConfigurableBeanFactory.SCOPE_PROTOTYPE) + public QueryPlanFactory queryExecutionFactory(BuiltinFunctionRepository functionRepository) { + catalogService + .getCatalogs() + .forEach( + catalog -> + catalog + .getStorageEngine() + .getFunctions() + .forEach( + functionResolver -> + functionRepository.register(catalog.getName(), functionResolver))); + Analyzer analyzer = new Analyzer(new ExpressionAnalyzer(functionRepository), + catalogService, functionRepository); + Planner planner = + new Planner(LogicalPlanOptimizer.create(new DSL(functionRepository))); + return new QueryPlanFactory(new QueryService(analyzer, executionEngine(), planner)); + } +} diff --git a/plugin/src/main/java/org/opensearch/sql/plugin/rest/OpenSearchPluginConfig.java b/plugin/src/main/java/org/opensearch/sql/plugin/rest/OpenSearchPluginConfig.java deleted file mode 100644 index 24d7e4e7f5..0000000000 --- a/plugin/src/main/java/org/opensearch/sql/plugin/rest/OpenSearchPluginConfig.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - - -package org.opensearch.sql.plugin.rest; - -import org.opensearch.client.node.NodeClient; -import org.opensearch.sql.common.setting.Settings; -import org.opensearch.sql.executor.ExecutionEngine; -import org.opensearch.sql.monitor.ResourceMonitor; -import org.opensearch.sql.opensearch.client.OpenSearchClient; -import org.opensearch.sql.opensearch.client.OpenSearchNodeClient; -import org.opensearch.sql.opensearch.executor.OpenSearchExecutionEngine; -import org.opensearch.sql.opensearch.executor.protector.ExecutionProtector; -import org.opensearch.sql.opensearch.executor.protector.OpenSearchExecutionProtector; -import org.opensearch.sql.opensearch.monitor.OpenSearchMemoryHealthy; -import org.opensearch.sql.opensearch.monitor.OpenSearchResourceMonitor; -import org.opensearch.sql.opensearch.storage.OpenSearchStorageEngine; -import org.opensearch.sql.storage.StorageEngine; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.context.annotation.Bean; -import org.springframework.context.annotation.Configuration; - -/** - * OpenSearch plugin config that injects cluster service and node client from plugin - * and initialize OpenSearch storage and execution engine. - */ -@Configuration -public class OpenSearchPluginConfig { - - @Autowired - private NodeClient nodeClient; - - @Autowired - private Settings settings; - - @Bean - public OpenSearchClient client() { - return new OpenSearchNodeClient(nodeClient); - } - - @Bean - public ExecutionEngine executionEngine() { - return new OpenSearchExecutionEngine(client(), protector()); - } - - @Bean - public ResourceMonitor resourceMonitor() { - return new OpenSearchResourceMonitor(settings, new OpenSearchMemoryHealthy()); - } - - @Bean - public ExecutionProtector protector() { - return new OpenSearchExecutionProtector(resourceMonitor()); - } -} diff --git a/plugin/src/main/java/org/opensearch/sql/plugin/rest/RestPPLQueryAction.java b/plugin/src/main/java/org/opensearch/sql/plugin/rest/RestPPLQueryAction.java index e9202d96e8..0f4d2f7d0c 100644 --- a/plugin/src/main/java/org/opensearch/sql/plugin/rest/RestPPLQueryAction.java +++ b/plugin/src/main/java/org/opensearch/sql/plugin/rest/RestPPLQueryAction.java @@ -9,7 +9,6 @@ import static org.opensearch.rest.RestStatus.INTERNAL_SERVER_ERROR; import static org.opensearch.rest.RestStatus.OK; import static org.opensearch.rest.RestStatus.SERVICE_UNAVAILABLE; -import static org.opensearch.sql.opensearch.executor.Scheduler.schedule; import com.google.common.collect.ImmutableList; import java.util.Arrays; @@ -39,7 +38,6 @@ import org.opensearch.sql.plugin.transport.PPLQueryAction; import org.opensearch.sql.plugin.transport.TransportPPLQueryRequest; import org.opensearch.sql.plugin.transport.TransportPPLQueryResponse; -import org.opensearch.sql.ppl.domain.PPLQueryRequest; public class RestPPLQueryAction extends BaseRestHandler { public static final String QUERY_API_ENDPOINT = "/_plugins/_ppl"; @@ -113,43 +111,42 @@ protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient nod PPLQueryRequestFactory.getPPLRequest(request) ); - return channel -> schedule(nodeClient, () -> - nodeClient.execute( - PPLQueryAction.INSTANCE, - transportPPLQueryRequest, - new ActionListener<>() { - @Override - public void onResponse(TransportPPLQueryResponse response) { - sendResponse(channel, OK, response.getResult()); - } - - @Override - public void onFailure(Exception e) { - if (transportPPLQueryRequest.isExplainRequest()) { - LOG.error("Error happened during explain", e); - sendResponse( - channel, - INTERNAL_SERVER_ERROR, - "Failed to explain the query due to error: " + e.getMessage()); - } else if (e instanceof IllegalAccessException) { - reportError(channel, e, BAD_REQUEST); - } else { - LOG.error("Error happened during query handling", e); - if (isClientError(e)) { - Metrics.getInstance() - .getNumericalMetric(MetricName.PPL_FAILED_REQ_COUNT_CUS) - .increment(); + return channel -> + nodeClient.execute( + PPLQueryAction.INSTANCE, + transportPPLQueryRequest, + new ActionListener<>() { + @Override + public void onResponse(TransportPPLQueryResponse response) { + sendResponse(channel, OK, response.getResult()); + } + + @Override + public void onFailure(Exception e) { + if (transportPPLQueryRequest.isExplainRequest()) { + LOG.error("Error happened during explain", e); + sendResponse( + channel, + INTERNAL_SERVER_ERROR, + "Failed to explain the query due to error: " + e.getMessage()); + } else if (e instanceof IllegalAccessException) { reportError(channel, e, BAD_REQUEST); } else { - Metrics.getInstance() - .getNumericalMetric(MetricName.PPL_FAILED_REQ_COUNT_SYS) - .increment(); - reportError(channel, e, SERVICE_UNAVAILABLE); + LOG.error("Error happened during query handling", e); + if (isClientError(e)) { + Metrics.getInstance() + .getNumericalMetric(MetricName.PPL_FAILED_REQ_COUNT_CUS) + .increment(); + reportError(channel, e, BAD_REQUEST); + } else { + Metrics.getInstance() + .getNumericalMetric(MetricName.PPL_FAILED_REQ_COUNT_SYS) + .increment(); + reportError(channel, e, SERVICE_UNAVAILABLE); + } } } - } - } - )); + }); } private void sendResponse(RestChannel channel, RestStatus status, String content) { diff --git a/plugin/src/main/java/org/opensearch/sql/plugin/transport/TransportPPLQueryAction.java b/plugin/src/main/java/org/opensearch/sql/plugin/transport/TransportPPLQueryAction.java index eaad009216..af57c91e5c 100644 --- a/plugin/src/main/java/org/opensearch/sql/plugin/transport/TransportPPLQueryAction.java +++ b/plugin/src/main/java/org/opensearch/sql/plugin/transport/TransportPPLQueryAction.java @@ -7,8 +7,6 @@ import static org.opensearch.sql.protocol.response.format.JsonResponseFormatter.Style.PRETTY; -import java.io.IOException; -import java.security.PrivilegedExceptionAction; import java.util.Locale; import java.util.Optional; import org.opensearch.action.ActionListener; @@ -18,7 +16,6 @@ import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.inject.Inject; -import org.opensearch.sql.catalog.CatalogService; import org.opensearch.sql.common.response.ResponseListener; import org.opensearch.sql.common.setting.Settings; import org.opensearch.sql.common.utils.QueryContext; @@ -27,10 +24,7 @@ import org.opensearch.sql.legacy.metrics.Metrics; import org.opensearch.sql.opensearch.security.SecurityAccess; import org.opensearch.sql.opensearch.setting.OpenSearchSettings; -import org.opensearch.sql.plugin.catalog.CatalogServiceImpl; -import org.opensearch.sql.plugin.rest.OpenSearchPluginConfig; import org.opensearch.sql.ppl.PPLService; -import org.opensearch.sql.ppl.config.PPLServiceConfig; import org.opensearch.sql.ppl.domain.PPLQueryRequest; import org.opensearch.sql.protocol.response.QueryResult; import org.opensearch.sql.protocol.response.format.CsvResponseFormatter; @@ -55,6 +49,8 @@ public class TransportPPLQueryAction /** Settings required by been initialization. */ private final Settings pluginSettings; + private final AnnotationConfigApplicationContext applicationContext; + /** Constructor of TransportPPLQueryAction. */ @Inject @@ -63,11 +59,12 @@ public TransportPPLQueryAction( ActionFilters actionFilters, NodeClient client, ClusterService clusterService, - org.opensearch.common.settings.Settings clusterSettings) { + AnnotationConfigApplicationContext applicationContext) { super(PPLQueryAction.NAME, transportService, actionFilters, TransportPPLQueryRequest::new); this.client = client; this.clusterService = clusterService; this.pluginSettings = new OpenSearchSettings(clusterService.getClusterSettings()); + this.applicationContext = applicationContext; } /** @@ -82,7 +79,8 @@ protected void doExecute( QueryContext.addRequestId(); - PPLService pplService = createPPLService(client); + PPLService pplService = + SecurityAccess.doPrivileged(() -> applicationContext.getBean(PPLService.class)); TransportPPLQueryRequest transportRequest = TransportPPLQueryRequest.fromActionRequest(request); // in order to use PPL service, we need to convert TransportPPLQueryRequest to PPLQueryRequest PPLQueryRequest transformedRequest = transportRequest.toPPLQueryRequest(); @@ -94,29 +92,6 @@ protected void doExecute( } } - private PPLService createPPLService(NodeClient client) { - return doPrivileged( - () -> { - AnnotationConfigApplicationContext context = new AnnotationConfigApplicationContext(); - context.registerBean(ClusterService.class, () -> clusterService); - context.registerBean(NodeClient.class, () -> client); - context.registerBean(Settings.class, () -> pluginSettings); - context.registerBean(CatalogService.class, CatalogServiceImpl::getInstance); - context.register(OpenSearchPluginConfig.class); - context.register(PPLServiceConfig.class); - context.refresh(); - return context.getBean(PPLService.class); - }); - } - - private T doPrivileged(PrivilegedExceptionAction action) { - try { - return SecurityAccess.doPrivileged(action); - } catch (IOException e) { - throw new IllegalStateException("Failed to perform privileged action", e); - } - } - /** * TODO: need to extract an interface for both SQL and PPL action handler and move these common * methods to the interface. This is not easy to do now because SQL action handler is still in diff --git a/ppl/build.gradle b/ppl/build.gradle index 12b0787efc..2c3c648478 100644 --- a/ppl/build.gradle +++ b/ppl/build.gradle @@ -46,7 +46,6 @@ dependencies { implementation "org.antlr:antlr4-runtime:4.7.1" implementation group: 'com.google.guava', name: 'guava', version: '31.0.1-jre' - api group: 'org.opensearch', name: 'opensearch-x-content', version: "${opensearch_version}" api group: 'org.json', name: 'json', version: '20180813' implementation group: 'org.springframework', name: 'spring-context', version: "${spring_version}" implementation group: 'org.springframework', name: 'spring-beans', version: "${spring_version}" diff --git a/ppl/src/main/antlr/OpenSearchPPLParser.g4 b/ppl/src/main/antlr/OpenSearchPPLParser.g4 index 14ebbe717a..7de8e19aa0 100644 --- a/ppl/src/main/antlr/OpenSearchPPLParser.g4 +++ b/ppl/src/main/antlr/OpenSearchPPLParser.g4 @@ -14,6 +14,14 @@ root /** statement */ pplStatement + : dmlStatement + ; + +dmlStatement + : queryStatement + ; + +queryStatement : pplCommands (PIPE commands)* ; diff --git a/ppl/src/main/java/org/opensearch/sql/ppl/PPLService.java b/ppl/src/main/java/org/opensearch/sql/ppl/PPLService.java index e6cbbb92f5..e11edc1646 100644 --- a/ppl/src/main/java/org/opensearch/sql/ppl/PPLService.java +++ b/ppl/src/main/java/org/opensearch/sql/ppl/PPLService.java @@ -8,41 +8,35 @@ import static org.opensearch.sql.executor.ExecutionEngine.QueryResponse; +import java.util.Optional; import lombok.RequiredArgsConstructor; import org.antlr.v4.runtime.tree.ParseTree; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.opensearch.sql.analysis.AnalysisContext; -import org.opensearch.sql.analysis.Analyzer; -import org.opensearch.sql.analysis.ExpressionAnalyzer; -import org.opensearch.sql.ast.tree.UnresolvedPlan; -import org.opensearch.sql.catalog.CatalogService; +import org.opensearch.sql.ast.statement.Statement; import org.opensearch.sql.common.response.ResponseListener; import org.opensearch.sql.common.utils.QueryContext; -import org.opensearch.sql.executor.ExecutionEngine; import org.opensearch.sql.executor.ExecutionEngine.ExplainResponse; -import org.opensearch.sql.expression.DSL; -import org.opensearch.sql.expression.function.BuiltinFunctionRepository; -import org.opensearch.sql.planner.Planner; -import org.opensearch.sql.planner.logical.LogicalPlan; -import org.opensearch.sql.planner.optimizer.LogicalPlanOptimizer; -import org.opensearch.sql.planner.physical.PhysicalPlan; +import org.opensearch.sql.executor.QueryManager; +import org.opensearch.sql.executor.execution.AbstractPlan; +import org.opensearch.sql.executor.execution.QueryPlanFactory; import org.opensearch.sql.ppl.antlr.PPLSyntaxParser; import org.opensearch.sql.ppl.domain.PPLQueryRequest; import org.opensearch.sql.ppl.parser.AstBuilder; import org.opensearch.sql.ppl.parser.AstExpressionBuilder; +import org.opensearch.sql.ppl.parser.AstStatementBuilder; import org.opensearch.sql.ppl.utils.PPLQueryDataAnonymizer; -import org.opensearch.sql.ppl.utils.UnresolvedPlanHelper; +/** + * PPLService. + */ @RequiredArgsConstructor public class PPLService { private final PPLSyntaxParser parser; - private final ExecutionEngine openSearchExecutionEngine; - - private final BuiltinFunctionRepository repository; + private final QueryManager queryManager; - private final CatalogService catalogService; + private final QueryPlanFactory queryExecutionFactory; private final PPLQueryDataAnonymizer anonymizer = new PPLQueryDataAnonymizer(); @@ -56,7 +50,7 @@ public class PPLService { */ public void execute(PPLQueryRequest request, ResponseListener listener) { try { - openSearchExecutionEngine.execute(plan(request), listener); + queryManager.submit(plan(request, Optional.of(listener), Optional.empty())); } catch (Exception e) { listener.onFailure(e); } @@ -71,28 +65,31 @@ public void execute(PPLQueryRequest request, ResponseListener lis */ public void explain(PPLQueryRequest request, ResponseListener listener) { try { - openSearchExecutionEngine.explain(plan(request), listener); + queryManager.submit(plan(request, Optional.empty(), Optional.of(listener))); } catch (Exception e) { listener.onFailure(e); } } - private PhysicalPlan plan(PPLQueryRequest request) { + private AbstractPlan plan( + PPLQueryRequest request, + Optional> queryListener, + Optional> explainListener) { // 1.Parse query and convert parse tree (CST) to abstract syntax tree (AST) ParseTree cst = parser.parse(request.getRequest()); - UnresolvedPlan ast = cst.accept( - new AstBuilder(new AstExpressionBuilder(), request.getRequest())); - LOG.info("[{}] Incoming request {}", QueryContext.getRequestId(), - anonymizer.anonymizeData(ast)); - // 2.Analyze abstract syntax to generate logical plan - LogicalPlan logicalPlan = - new Analyzer(new ExpressionAnalyzer(repository), catalogService, repository).analyze( - UnresolvedPlanHelper.addSelectAll(ast), - new AnalysisContext()); + Statement statement = + cst.accept( + new AstStatementBuilder( + new AstBuilder(new AstExpressionBuilder(), request.getRequest()), + AstStatementBuilder.StatementBuilderContext.builder() + .isExplain(request.isExplainRequest()) + .build())); - // 3.Generate optimal physical plan from logical plan - return new Planner(LogicalPlanOptimizer.create(new DSL(repository))) - .plan(logicalPlan); - } + LOG.info( + "[{}] Incoming request {}", + QueryContext.getRequestId(), + anonymizer.anonymizeStatement(statement)); + return queryExecutionFactory.create(statement, queryListener, explainListener); + } } diff --git a/ppl/src/main/java/org/opensearch/sql/ppl/config/PPLServiceConfig.java b/ppl/src/main/java/org/opensearch/sql/ppl/config/PPLServiceConfig.java index fb20fa2713..1067bbaa6b 100644 --- a/ppl/src/main/java/org/opensearch/sql/ppl/config/PPLServiceConfig.java +++ b/ppl/src/main/java/org/opensearch/sql/ppl/config/PPLServiceConfig.java @@ -6,29 +6,24 @@ package org.opensearch.sql.ppl.config; -import org.opensearch.sql.catalog.CatalogService; -import org.opensearch.sql.executor.ExecutionEngine; -import org.opensearch.sql.expression.config.ExpressionConfig; -import org.opensearch.sql.expression.function.BuiltinFunctionRepository; +import org.opensearch.sql.executor.QueryManager; +import org.opensearch.sql.executor.execution.QueryPlanFactory; import org.opensearch.sql.ppl.PPLService; import org.opensearch.sql.ppl.antlr.PPLSyntaxParser; import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.beans.factory.config.ConfigurableBeanFactory; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; -import org.springframework.context.annotation.Import; +import org.springframework.context.annotation.Scope; @Configuration -@Import({ExpressionConfig.class}) public class PPLServiceConfig { @Autowired - private ExecutionEngine executionEngine; + private QueryManager queryManager; @Autowired - private CatalogService catalogService; - - @Autowired - private BuiltinFunctionRepository functionRepository; + private QueryPlanFactory queryPlanFactory; /** * The registration of OpenSearch storage engine happens here because @@ -37,13 +32,9 @@ public class PPLServiceConfig { * @return PPLService. */ @Bean + @Scope(value = ConfigurableBeanFactory.SCOPE_PROTOTYPE) public PPLService pplService() { - catalogService.getCatalogs() - .forEach(catalog -> catalog.getStorageEngine().getFunctions() - .forEach(functionResolver -> functionRepository - .register(catalog.getName(), functionResolver))); - return new PPLService(new PPLSyntaxParser(), executionEngine, - functionRepository, catalogService); + return new PPLService(new PPLSyntaxParser(), queryManager, queryPlanFactory); } } diff --git a/ppl/src/main/java/org/opensearch/sql/ppl/domain/PPLQueryRequest.java b/ppl/src/main/java/org/opensearch/sql/ppl/domain/PPLQueryRequest.java index 0d8a4c63d1..87532e01d0 100644 --- a/ppl/src/main/java/org/opensearch/sql/ppl/domain/PPLQueryRequest.java +++ b/ppl/src/main/java/org/opensearch/sql/ppl/domain/PPLQueryRequest.java @@ -9,16 +9,17 @@ import java.util.Locale; import java.util.Optional; import lombok.Getter; -import lombok.RequiredArgsConstructor; import lombok.Setter; import lombok.experimental.Accessors; import org.json.JSONObject; import org.opensearch.sql.protocol.response.format.Format; import org.opensearch.sql.protocol.response.format.JsonResponseFormatter; -@RequiredArgsConstructor public class PPLQueryRequest { - public static final PPLQueryRequest NULL = new PPLQueryRequest("", null, "", ""); + + private static final String DEFAULT_PPL_PATH = "/_plugins/_ppl"; + + public static final PPLQueryRequest NULL = new PPLQueryRequest("", null, DEFAULT_PPL_PATH, ""); private final String pplQuery; @Getter @@ -38,13 +39,17 @@ public class PPLQueryRequest { @Accessors(fluent = true) private JsonResponseFormatter.Style style = JsonResponseFormatter.Style.COMPACT; + public PPLQueryRequest(String pplQuery, JSONObject jsonContent, String path) { + this(pplQuery, jsonContent, path, ""); + } + /** * Constructor of PPLQueryRequest. */ public PPLQueryRequest(String pplQuery, JSONObject jsonContent, String path, String format) { this.pplQuery = pplQuery; this.jsonContent = jsonContent; - this.path = path; + this.path = Optional.ofNullable(path).orElse(DEFAULT_PPL_PATH); this.format = format; } diff --git a/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java b/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java index 2638fc9a42..c72b638645 100644 --- a/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java +++ b/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java @@ -13,7 +13,6 @@ import static org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser.FieldsCommandContext; import static org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser.FromClauseContext; import static org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser.HeadCommandContext; -import static org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser.PplStatementContext; import static org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser.RareCommandContext; import static org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser.RenameCommandContext; import static org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser.SearchFilterFromContext; @@ -91,7 +90,7 @@ public class AstBuilder extends OpenSearchPPLParserBaseVisitor { private final String query; @Override - public UnresolvedPlan visitPplStatement(PplStatementContext ctx) { + public UnresolvedPlan visitQueryStatement(OpenSearchPPLParser.QueryStatementContext ctx) { UnresolvedPlan pplCommand = visit(ctx.pplCommands()); return ctx.commands() .stream() diff --git a/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstStatementBuilder.java b/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstStatementBuilder.java new file mode 100644 index 0000000000..e4f40e9a11 --- /dev/null +++ b/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstStatementBuilder.java @@ -0,0 +1,58 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.ppl.parser; + +import com.google.common.collect.ImmutableList; +import lombok.Builder; +import lombok.Data; +import lombok.RequiredArgsConstructor; +import org.opensearch.sql.ast.expression.AllFields; +import org.opensearch.sql.ast.statement.Explain; +import org.opensearch.sql.ast.statement.Query; +import org.opensearch.sql.ast.statement.Statement; +import org.opensearch.sql.ast.tree.Project; +import org.opensearch.sql.ast.tree.UnresolvedPlan; +import org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser; +import org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParserBaseVisitor; + +/** + * Build {@link Statement} from PPL Query. + */ +@RequiredArgsConstructor +public class AstStatementBuilder extends OpenSearchPPLParserBaseVisitor { + + private final AstBuilder astBuilder; + + private final StatementBuilderContext context; + + @Override + public Statement visitDmlStatement(OpenSearchPPLParser.DmlStatementContext ctx) { + Query query = new Query(addSelectAll(astBuilder.visit(ctx))); + return context.isExplain ? new Explain(query) : query; + } + + @Override + protected Statement aggregateResult(Statement aggregate, Statement nextResult) { + return nextResult != null ? nextResult : aggregate; + } + + @Data + @Builder + public static class StatementBuilderContext { + private final boolean isExplain; + } + + private UnresolvedPlan addSelectAll(UnresolvedPlan plan) { + if ((plan instanceof Project) && !((Project) plan).isExcluded()) { + return plan; + } else { + return new Project(ImmutableList.of(AllFields.of())).attach(plan); + } + } +} diff --git a/ppl/src/main/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizer.java b/ppl/src/main/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizer.java index be81fcb06c..504469a4b2 100644 --- a/ppl/src/main/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizer.java +++ b/ppl/src/main/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizer.java @@ -30,6 +30,9 @@ import org.opensearch.sql.ast.expression.Or; import org.opensearch.sql.ast.expression.UnresolvedExpression; import org.opensearch.sql.ast.expression.Xor; +import org.opensearch.sql.ast.statement.Explain; +import org.opensearch.sql.ast.statement.Query; +import org.opensearch.sql.ast.statement.Statement; import org.opensearch.sql.ast.tree.Aggregation; import org.opensearch.sql.ast.tree.Dedupe; import org.opensearch.sql.ast.tree.Eval; @@ -75,6 +78,23 @@ public String anonymizeData(UnresolvedPlan plan) { return plan.accept(this, null); } + public String anonymizeStatement(Statement plan) { + return plan.accept(this, null); + } + + /** + * Handle Query Statement. + */ + @Override + public String visitQuery(Query node, String context) { + return node.getPlan().accept(this, null); + } + + @Override + public String visitExplain(Explain node, String context) { + return node.getStatement().accept(this, null); + } + @Override public String visitRelation(Relation node, String context) { return StringUtils.format("source=%s", node.getTableName()); diff --git a/ppl/src/test/java/org/opensearch/sql/ppl/PPLServiceTest.java b/ppl/src/test/java/org/opensearch/sql/ppl/PPLServiceTest.java index f53f333b17..ef8ec25df8 100644 --- a/ppl/src/test/java/org/opensearch/sql/ppl/PPLServiceTest.java +++ b/ppl/src/test/java/org/opensearch/sql/ppl/PPLServiceTest.java @@ -8,11 +8,8 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.when; -import com.google.common.collect.ImmutableMap; import java.util.Collections; -import java.util.Set; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -20,71 +17,53 @@ import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; import org.opensearch.sql.catalog.CatalogService; -import org.opensearch.sql.catalog.model.Catalog; -import org.opensearch.sql.catalog.model.ConnectorType; import org.opensearch.sql.common.response.ResponseListener; -import org.opensearch.sql.data.type.ExprCoreType; +import org.opensearch.sql.executor.DefaultQueryManager; import org.opensearch.sql.executor.ExecutionEngine; import org.opensearch.sql.executor.ExecutionEngine.ExplainResponse; import org.opensearch.sql.executor.ExecutionEngine.ExplainResponseNode; import org.opensearch.sql.executor.ExecutionEngine.QueryResponse; -import org.opensearch.sql.expression.DSL; -import org.opensearch.sql.expression.function.BuiltinFunctionRepository; -import org.opensearch.sql.expression.function.FunctionResolver; -import org.opensearch.sql.planner.physical.PhysicalPlan; +import org.opensearch.sql.executor.QueryManager; +import org.opensearch.sql.executor.QueryService; +import org.opensearch.sql.executor.execution.QueryPlanFactory; import org.opensearch.sql.ppl.config.PPLServiceConfig; import org.opensearch.sql.ppl.domain.PPLQueryRequest; import org.opensearch.sql.storage.StorageEngine; -import org.opensearch.sql.storage.Table; import org.springframework.context.annotation.AnnotationConfigApplicationContext; @RunWith(MockitoJUnitRunner.class) public class PPLServiceTest { - private AnnotationConfigApplicationContext context = new AnnotationConfigApplicationContext(); - private PPLService pplService; + private static String QUERY = "/_plugins/_ppl"; - @Mock - private StorageEngine storageEngine; + private static String EXPLAIN = "/_plugins/_ppl/_explain"; - @Mock - private ExecutionEngine executionEngine; + private AnnotationConfigApplicationContext context = new AnnotationConfigApplicationContext(); - @Mock - private CatalogService catalogService; + private PPLService pplService; @Mock - private BuiltinFunctionRepository functionRepository; + private QueryService queryService; @Mock - private DSL dsl; + private StorageEngine storageEngine; @Mock - private Table table; + private ExecutionEngine executionEngine; @Mock - private PhysicalPlan plan; + private CatalogService catalogService; @Mock private ExecutionEngine.Schema schema; - @Mock - private FunctionResolver functionResolver; - /** * Setup the test context. */ @Before public void setUp() { - when(table.getFieldTypes()).thenReturn(ImmutableMap.of("a", ExprCoreType.INTEGER)); - when(table.implement(any())).thenReturn(plan); - when(storageEngine.getTable(any(), any())).thenReturn(table); - when(catalogService.getCatalogs()) - .thenReturn(Set.of(new Catalog("prometheus", ConnectorType.PROMETHEUS, storageEngine))); - when(catalogService.getCatalog(any())) - .thenReturn(new Catalog("prometheus", ConnectorType.PROMETHEUS, storageEngine)); - when(storageEngine.getFunctions()).thenReturn(Collections.singleton(functionResolver)); - + context.registerBean(QueryManager.class, DefaultQueryManager::new); + context.registerBean(QueryPlanFactory.class, () -> new QueryPlanFactory(queryService)); context.registerBean(StorageEngine.class, () -> storageEngine); context.registerBean(ExecutionEngine.class, () -> executionEngine); context.registerBean(CatalogService.class, () -> catalogService); @@ -99,9 +78,9 @@ public void testExecuteShouldPass() { ResponseListener listener = invocation.getArgument(1); listener.onResponse(new QueryResponse(schema, Collections.emptyList())); return null; - }).when(executionEngine).execute(any(), any()); + }).when(queryService).execute(any(), any()); - pplService.execute(new PPLQueryRequest("search source=t a=1", null, null), + pplService.execute(new PPLQueryRequest("search source=t a=1", null, QUERY), new ResponseListener() { @Override public void onResponse(QueryResponse pplQueryResponse) { @@ -121,9 +100,9 @@ public void testExecuteCsvFormatShouldPass() { ResponseListener listener = invocation.getArgument(1); listener.onResponse(new QueryResponse(schema, Collections.emptyList())); return null; - }).when(executionEngine).execute(any(), any()); + }).when(queryService).execute(any(), any()); - pplService.execute(new PPLQueryRequest("search source=t a=1", null, "/_plugins/_ppl", "csv"), + pplService.execute(new PPLQueryRequest("search source=t a=1", null, QUERY, "csv"), new ResponseListener() { @Override public void onResponse(QueryResponse pplQueryResponse) { @@ -138,15 +117,13 @@ public void onFailure(Exception e) { @Test public void testExplainShouldPass() { - when(catalogService.getCatalog(any())) - .thenReturn(new Catalog("prometheus", ConnectorType.PROMETHEUS, storageEngine)); doAnswer(invocation -> { ResponseListener listener = invocation.getArgument(1); listener.onResponse(new ExplainResponse(new ExplainResponseNode("test"))); return null; - }).when(executionEngine).explain(any(), any()); + }).when(queryService).explain(any(), any()); - pplService.explain(new PPLQueryRequest("search source=t a=1", null, null), + pplService.explain(new PPLQueryRequest("search source=t a=1", null, EXPLAIN), new ResponseListener() { @Override public void onResponse(ExplainResponse pplQueryResponse) { @@ -161,7 +138,7 @@ public void onFailure(Exception e) { @Test public void testExecuteWithIllegalQueryShouldBeCaughtByHandler() { - pplService.execute(new PPLQueryRequest("search", null, null), + pplService.execute(new PPLQueryRequest("search", null, QUERY), new ResponseListener() { @Override public void onResponse(QueryResponse pplQueryResponse) { @@ -177,7 +154,7 @@ public void onFailure(Exception e) { @Test public void testExplainWithIllegalQueryShouldBeCaughtByHandler() { - pplService.explain(new PPLQueryRequest("search", null, null), + pplService.explain(new PPLQueryRequest("search", null, QUERY), new ResponseListener<>() { @Override public void onResponse(ExplainResponse pplQueryResponse) { @@ -197,9 +174,9 @@ public void testPrometheusQuery() { ResponseListener listener = invocation.getArgument(1); listener.onResponse(new QueryResponse(schema, Collections.emptyList())); return null; - }).when(executionEngine).execute(any(), any()); + }).when(queryService).execute(any(), any()); - pplService.execute(new PPLQueryRequest("source = prometheus.http_requests_total", null, null), + pplService.execute(new PPLQueryRequest("source = prometheus.http_requests_total", null, QUERY), new ResponseListener<>() { @Override public void onResponse(QueryResponse pplQueryResponse) { @@ -214,8 +191,8 @@ public void onFailure(Exception e) { } @Test - public void test() { - pplService.execute(new PPLQueryRequest("search", null, null), + public void testInvalidPPLQuery() { + pplService.execute(new PPLQueryRequest("search", null, QUERY), new ResponseListener() { @Override public void onResponse(QueryResponse pplQueryResponse) { diff --git a/ppl/src/test/java/org/opensearch/sql/ppl/parser/AstStatementBuilderTest.java b/ppl/src/test/java/org/opensearch/sql/ppl/parser/AstStatementBuilderTest.java new file mode 100644 index 0000000000..4760024692 --- /dev/null +++ b/ppl/src/test/java/org/opensearch/sql/ppl/parser/AstStatementBuilderTest.java @@ -0,0 +1,72 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.ppl.parser; + +import static org.junit.Assert.assertEquals; +import static org.opensearch.sql.ast.dsl.AstDSL.compare; +import static org.opensearch.sql.ast.dsl.AstDSL.field; +import static org.opensearch.sql.ast.dsl.AstDSL.filter; +import static org.opensearch.sql.ast.dsl.AstDSL.intLiteral; +import static org.opensearch.sql.ast.dsl.AstDSL.project; +import static org.opensearch.sql.ast.dsl.AstDSL.relation; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.opensearch.sql.ast.Node; +import org.opensearch.sql.ast.expression.AllFields; +import org.opensearch.sql.ast.statement.Explain; +import org.opensearch.sql.ast.statement.Query; +import org.opensearch.sql.ast.statement.Statement; +import org.opensearch.sql.ppl.antlr.PPLSyntaxParser; + +public class AstStatementBuilderTest { + + @Rule + public ExpectedException exceptionRule = ExpectedException.none(); + + private PPLSyntaxParser parser = new PPLSyntaxParser(); + + @Test + public void buildQueryStatement() { + assertEqual( + "search source=t a=1", + new Query( + project( + filter(relation("t"), compare("=", field("a"), intLiteral(1))), AllFields.of()))); + } + + @Test + public void buildExplainStatement() { + assertExplainEqual( + "search source=t a=1", + new Explain( + new Query( + project( + filter(relation("t"), compare("=", field("a"), intLiteral(1))), + AllFields.of())))); + } + + private void assertEqual(String query, Statement expectedStatement) { + Node actualPlan = plan(query, false); + assertEquals(expectedStatement, actualPlan); + } + + private void assertExplainEqual(String query, Statement expectedStatement) { + Node actualPlan = plan(query, true); + assertEquals(expectedStatement, actualPlan); + } + + private Node plan(String query, boolean isExplain) { + final AstStatementBuilder builder = + new AstStatementBuilder(new AstBuilder(new AstExpressionBuilder(), query), + AstStatementBuilder.StatementBuilderContext.builder().isExplain(isExplain).build()); + return builder.visit(parser.parse(query)); + } +} diff --git a/ppl/src/test/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizerTest.java b/ppl/src/test/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizerTest.java index 4a4c0a5f22..1e4af28ecf 100644 --- a/ppl/src/test/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizerTest.java +++ b/ppl/src/test/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizerTest.java @@ -7,25 +7,21 @@ package org.opensearch.sql.ppl.utils; import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.when; import static org.opensearch.sql.ast.dsl.AstDSL.field; import static org.opensearch.sql.ast.dsl.AstDSL.projectWithArg; import static org.opensearch.sql.ast.dsl.AstDSL.relation; -import com.google.common.collect.ImmutableSet; import java.util.Collections; -import org.junit.Before; import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.Mockito; import org.mockito.junit.MockitoJUnitRunner; +import org.opensearch.sql.ast.statement.Statement; import org.opensearch.sql.ast.tree.UnresolvedPlan; -import org.opensearch.sql.catalog.CatalogService; import org.opensearch.sql.ppl.antlr.PPLSyntaxParser; import org.opensearch.sql.ppl.parser.AstBuilder; import org.opensearch.sql.ppl.parser.AstExpressionBuilder; +import org.opensearch.sql.ppl.parser.AstStatementBuilder; @RunWith(MockitoJUnitRunner.class) public class PPLQueryDataAnonymizerTest { @@ -176,6 +172,20 @@ public void testDateFunction() { ); } + @Test + public void testExplain() { + assertEquals("source=t | fields + a", + anonymizeStatement("source=t | fields a", true) + ); + } + + @Test + public void testQuery() { + assertEquals("source=t | fields + a", + anonymizeStatement("source=t | fields a", false) + ); + } + @Test public void anonymizeFieldsNoArg() { assertEquals("source=t | fields + f", @@ -192,4 +202,14 @@ private String anonymize(UnresolvedPlan plan) { final PPLQueryDataAnonymizer anonymize = new PPLQueryDataAnonymizer(); return anonymize.anonymizeData(plan); } + + private String anonymizeStatement(String query, boolean isExplain) { + AstStatementBuilder builder = + new AstStatementBuilder( + new AstBuilder(new AstExpressionBuilder(), query), + AstStatementBuilder.StatementBuilderContext.builder().isExplain(isExplain).build()); + Statement statement = builder.visit(parser.parse(query)); + PPLQueryDataAnonymizer anonymize = new PPLQueryDataAnonymizer(); + return anonymize.anonymizeStatement(statement); + } } diff --git a/prometheus/src/main/java/org/opensearch/sql/prometheus/storage/PrometheusMetricTable.java b/prometheus/src/main/java/org/opensearch/sql/prometheus/storage/PrometheusMetricTable.java index b81314d936..83384ff760 100644 --- a/prometheus/src/main/java/org/opensearch/sql/prometheus/storage/PrometheusMetricTable.java +++ b/prometheus/src/main/java/org/opensearch/sql/prometheus/storage/PrometheusMetricTable.java @@ -63,6 +63,18 @@ public PrometheusMetricTable(PrometheusClient prometheusService, this.prometheusQueryRequest = prometheusQueryRequest; } + @Override + public boolean exists() { + throw new UnsupportedOperationException( + "Prometheus metric exists operation is not supported"); + } + + @Override + public void create(Map schema) { + throw new UnsupportedOperationException( + "Prometheus metric create operation is not supported"); + } + @Override public Map getFieldTypes() { if (cachedFieldTypes == null) { diff --git a/prometheus/src/test/java/org/opensearch/sql/prometheus/storage/PrometheusMetricTableTest.java b/prometheus/src/test/java/org/opensearch/sql/prometheus/storage/PrometheusMetricTableTest.java index 960c3f3b2d..e106fa225b 100644 --- a/prometheus/src/test/java/org/opensearch/sql/prometheus/storage/PrometheusMetricTableTest.java +++ b/prometheus/src/test/java/org/opensearch/sql/prometheus/storage/PrometheusMetricTableTest.java @@ -34,6 +34,7 @@ import java.text.DateFormat; import java.text.SimpleDateFormat; import java.util.ArrayList; +import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.List; @@ -739,6 +740,17 @@ void testOptimize() { assertEquals(inputPlan, optimizedPlan); } + @Test + void testUnsupportedOperation() { + PrometheusQueryRequest prometheusQueryRequest = new PrometheusQueryRequest(); + PrometheusMetricTable prometheusMetricTable = + new PrometheusMetricTable(client, prometheusQueryRequest); + + assertThrows(UnsupportedOperationException.class, prometheusMetricTable::exists); + assertThrows(UnsupportedOperationException.class, + () -> prometheusMetricTable.create(Collections.emptyMap())); + } + @Test void testImplementPrometheusQueryWithBackQuotedFieldNamesInStatsQuery() { @@ -765,5 +777,4 @@ void testImplementPrometheusQueryWithBackQuotedFieldNamesInStatsQuery() { prometheusQueryRequest.getPromQl()); } - } diff --git a/settings.gradle b/settings.gradle index 2f850f422b..7650959451 100644 --- a/settings.gradle +++ b/settings.gradle @@ -18,4 +18,5 @@ include 'doctest' include 'legacy' include 'sql' include 'prometheus' +include 'filesystem' diff --git a/sql/src/main/java/org/opensearch/sql/sql/SQLService.java b/sql/src/main/java/org/opensearch/sql/sql/SQLService.java index 76de0f6249..082a3e9581 100644 --- a/sql/src/main/java/org/opensearch/sql/sql/SQLService.java +++ b/sql/src/main/java/org/opensearch/sql/sql/SQLService.java @@ -6,25 +6,20 @@ package org.opensearch.sql.sql; +import java.util.Optional; import lombok.RequiredArgsConstructor; import org.antlr.v4.runtime.tree.ParseTree; -import org.opensearch.sql.analysis.AnalysisContext; -import org.opensearch.sql.analysis.Analyzer; -import org.opensearch.sql.ast.tree.UnresolvedPlan; +import org.opensearch.sql.ast.statement.Statement; import org.opensearch.sql.common.response.ResponseListener; -import org.opensearch.sql.executor.ExecutionEngine; import org.opensearch.sql.executor.ExecutionEngine.ExplainResponse; import org.opensearch.sql.executor.ExecutionEngine.QueryResponse; -import org.opensearch.sql.expression.DSL; -import org.opensearch.sql.expression.function.BuiltinFunctionRepository; -import org.opensearch.sql.planner.Planner; -import org.opensearch.sql.planner.logical.LogicalPlan; -import org.opensearch.sql.planner.optimizer.LogicalPlanOptimizer; -import org.opensearch.sql.planner.physical.PhysicalPlan; +import org.opensearch.sql.executor.QueryManager; +import org.opensearch.sql.executor.execution.AbstractPlan; +import org.opensearch.sql.executor.execution.QueryPlanFactory; import org.opensearch.sql.sql.antlr.SQLSyntaxParser; import org.opensearch.sql.sql.domain.SQLQueryRequest; import org.opensearch.sql.sql.parser.AstBuilder; -import org.opensearch.sql.storage.StorageEngine; +import org.opensearch.sql.sql.parser.AstStatementBuilder; /** * SQL service. @@ -34,75 +29,52 @@ public class SQLService { private final SQLSyntaxParser parser; - private final Analyzer analyzer; + private final QueryManager queryManager; - private final ExecutionEngine executionEngine; - - private final BuiltinFunctionRepository repository; + private final QueryPlanFactory queryExecutionFactory; /** - * Parse, analyze, plan and execute the query. - * @param request SQL query request - * @param listener callback listener + * Given {@link SQLQueryRequest}, execute it. Using listener to listen result. + * + * @param request {@link SQLQueryRequest} + * @param listener callback listener */ public void execute(SQLQueryRequest request, ResponseListener listener) { try { - executionEngine.execute( - plan( - analyze( - parse(request.getQuery()))), listener); + queryManager.submit(plan(request, Optional.of(listener), Optional.empty())); } catch (Exception e) { listener.onFailure(e); } } /** - * Given physical plan, execute it and listen on response. - * @param plan physical plan - * @param listener callback listener + * Given {@link SQLQueryRequest}, explain it. Using listener to listen result. + * + * @param request {@link SQLQueryRequest} + * @param listener callback listener */ - public void execute(PhysicalPlan plan, ResponseListener listener) { + public void explain(SQLQueryRequest request, ResponseListener listener) { try { - executionEngine.execute(plan, listener); + queryManager.submit(plan(request, Optional.empty(), Optional.of(listener))); } catch (Exception e) { listener.onFailure(e); } } - /** - * Given physical plan, explain it. - * @param plan physical plan - * @param listener callback listener - */ - public void explain(PhysicalPlan plan, ResponseListener listener) { - try { - executionEngine.explain(plan, listener); - } catch (Exception e) { - listener.onFailure(e); - } - } + private AbstractPlan plan( + SQLQueryRequest request, + Optional> queryListener, + Optional> explainListener) { + // 1.Parse query and convert parse tree (CST) to abstract syntax tree (AST) + ParseTree cst = parser.parse(request.getQuery()); + Statement statement = + cst.accept( + new AstStatementBuilder( + new AstBuilder(request.getQuery()), + AstStatementBuilder.StatementBuilderContext.builder() + .isExplain(request.isExplainRequest()) + .build())); - /** - * Parse query and convert parse tree (CST) to abstract syntax tree (AST). - */ - public UnresolvedPlan parse(String query) { - ParseTree cst = parser.parse(query); - return cst.accept(new AstBuilder(query)); + return queryExecutionFactory.create(statement, queryListener, explainListener); } - - /** - * Analyze abstract syntax to generate logical plan. - */ - public LogicalPlan analyze(UnresolvedPlan ast) { - return analyzer.analyze(ast, new AnalysisContext()); - } - - /** - * Generate optimal physical plan from logical plan. - */ - public PhysicalPlan plan(LogicalPlan logicalPlan) { - return new Planner(LogicalPlanOptimizer.create(new DSL(repository))) - .plan(logicalPlan); - } - } diff --git a/sql/src/main/java/org/opensearch/sql/sql/config/SQLServiceConfig.java b/sql/src/main/java/org/opensearch/sql/sql/config/SQLServiceConfig.java index 5bac8a5aa0..4287883c34 100644 --- a/sql/src/main/java/org/opensearch/sql/sql/config/SQLServiceConfig.java +++ b/sql/src/main/java/org/opensearch/sql/sql/config/SQLServiceConfig.java @@ -6,40 +6,27 @@ package org.opensearch.sql.sql.config; -import org.opensearch.sql.analysis.Analyzer; -import org.opensearch.sql.analysis.ExpressionAnalyzer; -import org.opensearch.sql.catalog.CatalogService; -import org.opensearch.sql.executor.ExecutionEngine; -import org.opensearch.sql.expression.config.ExpressionConfig; -import org.opensearch.sql.expression.function.BuiltinFunctionRepository; +import org.opensearch.sql.executor.QueryManager; +import org.opensearch.sql.executor.execution.QueryPlanFactory; import org.opensearch.sql.sql.SQLService; import org.opensearch.sql.sql.antlr.SQLSyntaxParser; import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.beans.factory.config.ConfigurableBeanFactory; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; -import org.springframework.context.annotation.Import; +import org.springframework.context.annotation.Scope; /** * SQL service configuration for Spring container initialization. */ @Configuration -@Import({ExpressionConfig.class}) public class SQLServiceConfig { @Autowired - private ExecutionEngine executionEngine; + private QueryManager queryManager; @Autowired - private CatalogService catalogService; - - @Autowired - private BuiltinFunctionRepository functionRepository; - - @Bean - public Analyzer analyzer() { - return new Analyzer(new ExpressionAnalyzer(functionRepository), catalogService, - functionRepository); - } + private QueryPlanFactory queryExecutionFactory; /** * The registration of OpenSearch storage engine happens here because @@ -48,9 +35,12 @@ public Analyzer analyzer() { * @return SQLService. */ @Bean + @Scope(value = ConfigurableBeanFactory.SCOPE_PROTOTYPE) public SQLService sqlService() { - return new SQLService(new SQLSyntaxParser(), analyzer(), executionEngine, - functionRepository); + return new SQLService( + new SQLSyntaxParser(), + queryManager, + queryExecutionFactory); } } diff --git a/sql/src/main/java/org/opensearch/sql/sql/parser/AstStatementBuilder.java b/sql/src/main/java/org/opensearch/sql/sql/parser/AstStatementBuilder.java new file mode 100644 index 0000000000..40d549764a --- /dev/null +++ b/sql/src/main/java/org/opensearch/sql/sql/parser/AstStatementBuilder.java @@ -0,0 +1,43 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.sql.sql.parser; + +import lombok.Builder; +import lombok.Data; +import lombok.RequiredArgsConstructor; +import org.opensearch.sql.ast.statement.Explain; +import org.opensearch.sql.ast.statement.Query; +import org.opensearch.sql.ast.statement.Statement; +import org.opensearch.sql.sql.antlr.parser.OpenSearchSQLParser; +import org.opensearch.sql.sql.antlr.parser.OpenSearchSQLParserBaseVisitor; + +@RequiredArgsConstructor +public class AstStatementBuilder extends OpenSearchSQLParserBaseVisitor { + + private final AstBuilder astBuilder; + + private final StatementBuilderContext context; + + @Override + public Statement visitSqlStatement(OpenSearchSQLParser.SqlStatementContext ctx) { + Query query = new Query(astBuilder.visit(ctx)); + return context.isExplain ? new Explain(query) : query; + } + + @Override + protected Statement aggregateResult(Statement aggregate, Statement nextResult) { + return nextResult != null ? nextResult : aggregate; + } + + @Data + @Builder + public static class StatementBuilderContext { + private final boolean isExplain; + } +} diff --git a/sql/src/test/java/org/opensearch/sql/sql/SQLServiceTest.java b/sql/src/test/java/org/opensearch/sql/sql/SQLServiceTest.java index 774c5e2d52..f1d2c5293d 100644 --- a/sql/src/test/java/org/opensearch/sql/sql/SQLServiceTest.java +++ b/sql/src/test/java/org/opensearch/sql/sql/SQLServiceTest.java @@ -10,8 +10,6 @@ import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; import static org.opensearch.sql.executor.ExecutionEngine.QueryResponse; import java.util.Collections; @@ -21,41 +19,39 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.sql.catalog.CatalogService; import org.opensearch.sql.common.response.ResponseListener; +import org.opensearch.sql.executor.DefaultQueryManager; import org.opensearch.sql.executor.ExecutionEngine; import org.opensearch.sql.executor.ExecutionEngine.ExplainResponse; import org.opensearch.sql.executor.ExecutionEngine.ExplainResponseNode; -import org.opensearch.sql.planner.physical.PhysicalPlan; +import org.opensearch.sql.executor.QueryManager; +import org.opensearch.sql.executor.QueryService; +import org.opensearch.sql.executor.execution.QueryPlanFactory; import org.opensearch.sql.sql.config.SQLServiceConfig; import org.opensearch.sql.sql.domain.SQLQueryRequest; -import org.opensearch.sql.storage.StorageEngine; import org.springframework.context.annotation.AnnotationConfigApplicationContext; @ExtendWith(MockitoExtension.class) class SQLServiceTest { - private AnnotationConfigApplicationContext context = new AnnotationConfigApplicationContext(); + private static String QUERY = "/_plugins/_sql"; - private SQLService sqlService; + private static String EXPLAIN = "/_plugins/_sql/_explain"; - @Mock - private StorageEngine storageEngine; + private AnnotationConfigApplicationContext context = new AnnotationConfigApplicationContext(); - @Mock - private ExecutionEngine executionEngine; + private SQLService sqlService; @Mock - private CatalogService catalogService; + private QueryService queryService; @Mock private ExecutionEngine.Schema schema; @BeforeEach public void setUp() { - context.registerBean(StorageEngine.class, () -> storageEngine); - context.registerBean(ExecutionEngine.class, () -> executionEngine); - context.registerBean(CatalogService.class, () -> catalogService); + context.registerBean(QueryManager.class, DefaultQueryManager::new); + context.registerBean(QueryPlanFactory.class, () -> new QueryPlanFactory(queryService)); context.register(SQLServiceConfig.class); context.refresh(); sqlService = context.getBean(SQLService.class); @@ -67,10 +63,10 @@ public void canExecuteSqlQuery() { ResponseListener listener = invocation.getArgument(1); listener.onResponse(new QueryResponse(schema, Collections.emptyList())); return null; - }).when(executionEngine).execute(any(), any()); + }).when(queryService).execute(any(), any()); sqlService.execute( - new SQLQueryRequest(new JSONObject(), "SELECT 123", "_plugins/_sql", "jdbc"), + new SQLQueryRequest(new JSONObject(), "SELECT 123", QUERY, "jdbc"), new ResponseListener() { @Override public void onResponse(QueryResponse response) { @@ -90,10 +86,10 @@ public void canExecuteCsvFormatRequest() { ResponseListener listener = invocation.getArgument(1); listener.onResponse(new QueryResponse(schema, Collections.emptyList())); return null; - }).when(executionEngine).execute(any(), any()); + }).when(queryService).execute(any(), any()); sqlService.execute( - new SQLQueryRequest(new JSONObject(), "SELECT 123", "_plugins/_sql", "csv"), + new SQLQueryRequest(new JSONObject(), "SELECT 123", QUERY, "csv"), new ResponseListener() { @Override public void onResponse(QueryResponse response) { @@ -113,9 +109,9 @@ public void canExplainSqlQuery() { ResponseListener listener = invocation.getArgument(1); listener.onResponse(new ExplainResponse(new ExplainResponseNode("Test"))); return null; - }).when(executionEngine).explain(any(), any()); + }).when(queryService).explain(any(), any()); - sqlService.explain(mock(PhysicalPlan.class), + sqlService.explain(new SQLQueryRequest(new JSONObject(), "SELECT 123", EXPLAIN, "csv"), new ResponseListener() { @Override public void onResponse(ExplainResponse response) { @@ -129,50 +125,10 @@ public void onFailure(Exception e) { }); } - @Test - public void canExecuteFromPhysicalPlan() { - doAnswer(invocation -> { - ResponseListener listener = invocation.getArgument(1); - listener.onResponse(new QueryResponse(schema, Collections.emptyList())); - return null; - }).when(executionEngine).execute(any(), any()); - - sqlService.execute(mock(PhysicalPlan.class), - new ResponseListener() { - @Override - public void onResponse(QueryResponse response) { - assertNotNull(response); - } - - @Override - public void onFailure(Exception e) { - fail(e); - } - }); - } - @Test public void canCaptureErrorDuringExecution() { sqlService.execute( - new SQLQueryRequest(new JSONObject(), "SELECT", "_plugins/_sql", ""), - new ResponseListener() { - @Override - public void onResponse(QueryResponse response) { - fail(); - } - - @Override - public void onFailure(Exception e) { - assertNotNull(e); - } - }); - } - - @Test - public void canCaptureErrorDuringExecutionFromPhysicalPlan() { - doThrow(new RuntimeException()).when(executionEngine).execute(any(), any()); - - sqlService.execute(mock(PhysicalPlan.class), + new SQLQueryRequest(new JSONObject(), "SELECT", QUERY, ""), new ResponseListener() { @Override public void onResponse(QueryResponse response) { @@ -188,9 +144,8 @@ public void onFailure(Exception e) { @Test public void canCaptureErrorDuringExplain() { - doThrow(new RuntimeException()).when(executionEngine).explain(any(), any()); - - sqlService.explain(mock(PhysicalPlan.class), + sqlService.explain( + new SQLQueryRequest(new JSONObject(), "SELECT", EXPLAIN, ""), new ResponseListener() { @Override public void onResponse(ExplainResponse response) {