Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add PPL Between functionality #758

Open
wants to merge 8 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions docs/ppl-lang/PPL-Example-Commands.md
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,8 @@ _- **Limitation: new field added by eval command with a function cannot be dropp
- `source = table | where isempty(a)`
- `source = table | where isblank(a)`
- `source = table | where case(length(a) > 6, 'True' else 'False') = 'True'`
- `source = table | where a between 1 and 4`
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we need explain the range of between is [1, 4] rather than (1, 4) in user doc. From the code implementation, I think it is [1, 4]. In anther word, when a = 1, the predicate returns True

- `source = table | where b not between '2024-09-10' and '2025-09-10'`

```sql
source = table | eval status_category =
Expand Down
20 changes: 20 additions & 0 deletions docs/ppl-lang/planning/ppl-between.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
## between syntax proposal

1. **Proposed syntax**
- `... | where expr1 [NOT] BETWEEN expr2 AND expr3`
- evaluate if expr1 is [not] in between expr2 and expr3
- `... | where a between 1 and 4`
- `... | where b not between '2024-09-10' and '2025-09-10'`

2. **Proposed impl**
- forward to sparks built-in function of between

### New syntax definition in ANTLR

```ANTLR

logicalExpression
...
| expr1 = functionArg NOT? BETWEEN expr2 = functionArg AND expr3 = functionArg # between

```
2 changes: 2 additions & 0 deletions docs/ppl-lang/ppl-where-command.md
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,8 @@ PPL query:
- `source = table | where isempty(a)`
- `source = table | where isblank(a)`
- `source = table | where case(length(a) > 6, 'True' else 'False') = 'True'`
- `source = table | where a between 1 and 4`
- `source = table | where b not between '2024-09-10' and '2025-09-10'`

- `source = table | eval status_category =
case(a >= 200 AND a < 300, 'Success',
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
/*
* Copyright OpenSearch Contributors
* SPDX-License-Identifier: Apache-2.0
*/

package org.opensearch.flint.spark.ppl

import org.apache.spark.sql.QueryTest
import org.apache.spark.sql.streaming.StreamTest

class FlintSparkPPLBetweenITSuite
extends QueryTest
with LogicalPlanTestUtils
with FlintPPLSuite
with StreamTest {

/** Test table and index name */
private val testTable = "spark_catalog.default.flint_ppl_test"

override def beforeAll(): Unit = {
super.beforeAll()

// Create test table
createPartitionedStateCountryTable(testTable)
}

protected override def afterEach(): Unit = {
super.afterEach()
// Stop all streaming jobs if any
spark.streams.active.foreach { job =>
job.stop()
job.awaitTermination()
}
}

test("test between should return records between two values") {
val frame = sql(s"""
| source = $testTable | where age between 20 and 30
| """.stripMargin)

val results = frame.collect()
assert(results.length == 3)
assert(frame.columns.length == 6)

results.foreach(row => {
val age = row.getAs[Int]("age")
assert(age >= 20 && age <= 30, s"Age $age is not between 20 and 30")
})
}

test("test between should return records NOT between two values") {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add two tests (between and not between) with Date/Time in condition?
I think you can use the table by createTimeSeriesTable

val frame = sql(s"""
| source = $testTable | where age NOT between 20 and 30
| """.stripMargin)

val results = frame.collect()
assert(results.length == 1)
assert(frame.columns.length == 6)

results.foreach(row => {
val age = row.getAs[Int]("age")
assert(age < 20 || age > 30, s"Age $age is not between 20 and 30")
})
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -368,6 +368,7 @@ LIKE: 'LIKE';
ISNULL: 'ISNULL';
ISNOTNULL: 'ISNOTNULL';
ISPRESENT: 'ISPRESENT';
BETWEEN: 'BETWEEN';
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add BETWEEN to keywordsCanBeId


// FLOWCONTROL FUNCTIONS
IFNULL: 'IFNULL';
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -371,6 +371,7 @@ logicalExpression
| left = logicalExpression OR right = logicalExpression # logicalOr
| left = logicalExpression XOR right = logicalExpression # logicalXor
| booleanExpression # booleanExpr
| expr1 = functionArg NOT? BETWEEN expr2 = functionArg AND expr3 = functionArg # between
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Move this new expression under booleanExpression

;

comparisonExpression
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,9 @@
import org.apache.spark.sql.catalyst.expressions.Descending$;
import org.apache.spark.sql.catalyst.expressions.Exists$;
import org.apache.spark.sql.catalyst.expressions.Expression;
import org.apache.spark.sql.catalyst.expressions.GreaterThanOrEqual;
import org.apache.spark.sql.catalyst.expressions.InSubquery$;
import org.apache.spark.sql.catalyst.expressions.LessThanOrEqual;
import org.apache.spark.sql.catalyst.expressions.ListQuery$;
import org.apache.spark.sql.catalyst.expressions.NamedExpression;
import org.apache.spark.sql.catalyst.expressions.Predicate;
Expand All @@ -34,6 +36,7 @@
import org.opensearch.sql.ast.expression.AllFields;
import org.opensearch.sql.ast.expression.And;
import org.opensearch.sql.ast.expression.Argument;
import org.opensearch.sql.ast.expression.Between;
import org.opensearch.sql.ast.expression.BinaryExpression;
import org.opensearch.sql.ast.expression.Case;
import org.opensearch.sql.ast.expression.Compare;
Expand Down Expand Up @@ -829,5 +832,14 @@ public Expression visitExistsSubquery(ExistsSubquery node, CatalystPlanContext c
Option.empty());
return context.getNamedParseExpressions().push(existsSubQuery);
}

@Override
public Expression visitBetween(Between node, CatalystPlanContext context) {
Expression value = analyze(node.getValue(), context);
Expression lower = analyze(node.getLowerBound(), context);
Expression upper = analyze(node.getUpperBound(), context);
context.retainAllNamedParseExpressions(p -> p);
return context.getNamedParseExpressions().push(new org.apache.spark.sql.catalyst.expressions.And(new GreaterThanOrEqual(value, lower), new LessThanOrEqual(value, upper)));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
import org.opensearch.sql.ast.expression.AllFields;
import org.opensearch.sql.ast.expression.And;
import org.opensearch.sql.ast.expression.Argument;
import org.opensearch.sql.ast.expression.Between;
import org.opensearch.sql.ast.expression.Case;
import org.opensearch.sql.ast.expression.Compare;
import org.opensearch.sql.ast.expression.DataType;
Expand Down Expand Up @@ -269,6 +270,12 @@ public UnresolvedExpression visitConvertedDataType(OpenSearchPPLParser.Converted
return new Literal(ctx.getText(), DataType.STRING);
}

@Override
public UnresolvedExpression visitBetween(OpenSearchPPLParser.BetweenContext ctx) {
UnresolvedExpression betweenExpr = new Between(visit(ctx.expr1),visit(ctx.expr2),visit(ctx.expr3));
return ctx.NOT() != null ? new Not(betweenExpr) : betweenExpr;
}

private Function buildFunction(
String functionName, List<OpenSearchPPLParser.FunctionArgContext> args) {
return new Function(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* Copyright OpenSearch Contributors
* SPDX-License-Identifier: Apache-2.0
*/

package org.opensearch.flint.spark.ppl

import org.opensearch.flint.spark.ppl.PlaneUtils.plan
import org.opensearch.sql.ppl.{CatalystPlanContext, CatalystQueryPlanVisitor}
import org.scalatest.matchers.should.Matchers

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation, UnresolvedStar}
import org.apache.spark.sql.catalyst.expressions.{And, GreaterThanOrEqual, LessThanOrEqual, Literal}
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical._

class PPLLogicalPlanBetweenExpressionTranslatorTestSuite
extends SparkFunSuite
with PlanTest
with LogicalPlanTestUtils
with Matchers {

private val planTransformer = new CatalystQueryPlanVisitor()
private val pplParser = new PPLSyntaxParser()

test("test between expression") {
// if successful build ppl logical plan and translate to catalyst logical plan
val context = new CatalystPlanContext
val logPlan = {
planTransformer.visit(
plan(
pplParser,
"source = table | where datetime_field between '2024-09-10' and '2024-09-15'"),
context)
}
// SQL: SELECT * FROM table WHERE datetime_field BETWEEN '2024-09-10' AND '2024-09-15'
val star = Seq(UnresolvedStar(None))

val datetime_field = UnresolvedAttribute("datetime_field")
val tableRelation = UnresolvedRelation(Seq("table"))

val lowerBound = Literal("2024-09-10")
val upperBound = Literal("2024-09-15")
val betweenCondition = And(
GreaterThanOrEqual(datetime_field, lowerBound),
LessThanOrEqual(datetime_field, upperBound))

val filterPlan = Filter(betweenCondition, tableRelation)
val expectedPlan = Project(star, filterPlan)

comparePlans(expectedPlan, logPlan, false)
}

}
Loading