-
Notifications
You must be signed in to change notification settings - Fork 30
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
base: main
Are you sure you want to change the base?
Changes from all commits
9b1488c
e6013bb
67cec2a
b53a4b6
172e93a
f4109f7
a617d66
f0d7aab
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
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 | ||
|
||
``` |
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") { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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? |
||
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 |
---|---|---|
|
@@ -368,6 +368,7 @@ LIKE: 'LIKE'; | |
ISNULL: 'ISNULL'; | ||
ISNOTNULL: 'ISNOTNULL'; | ||
ISPRESENT: 'ISPRESENT'; | ||
BETWEEN: 'BETWEEN'; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Add |
||
|
||
// FLOWCONTROL FUNCTIONS | ||
IFNULL: 'IFNULL'; | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Move this new expression under |
||
; | ||
|
||
comparisonExpression | ||
|
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) | ||
} | ||
|
||
} |
There was a problem hiding this comment.
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 returnsTrue