-
Notifications
You must be signed in to change notification settings - Fork 30
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
separate unit-tests into a dedicated file per each test category
Signed-off-by: YANGDB <yang.db.dev@gmail.com>
- Loading branch information
Showing
6 changed files
with
281 additions
and
280 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
119 changes: 119 additions & 0 deletions
119
...cala/org/opensearch/flint/spark/ppl/PPLLogicalPlanComplexQueriesTranslatorTestSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,119 @@ | ||
/* | ||
* Copyright OpenSearch Contributors | ||
* SPDX-License-Identifier: Apache-2.0 | ||
*/ | ||
|
||
package org.opensearch.flint.spark.ppl | ||
|
||
import org.apache.spark.SparkFunSuite | ||
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} | ||
import org.apache.spark.sql.catalyst.expressions.NamedExpression | ||
import org.apache.spark.sql.catalyst.plans.logical._ | ||
import org.junit.Assert.assertEquals | ||
import org.opensearch.flint.spark.ppl.PlaneUtils.plan | ||
import org.opensearch.sql.ppl.{CatalystPlanContext, CatalystQueryPlanVisitor} | ||
import org.scalatest.matchers.should.Matchers | ||
|
||
class PPLLogicalPlanComplexQueriesTranslatorTestSuite | ||
extends SparkFunSuite | ||
with Matchers { | ||
|
||
private val planTrnasformer = new CatalystQueryPlanVisitor() | ||
private val pplParser = new PPLSyntaxParser() | ||
|
||
test("test simple search with only one table and no explicit fields (defaults to all fields)") { | ||
// if successful build ppl logical plan and translate to catalyst logical plan | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source=table", false), context) | ||
|
||
val projectList: Seq[NamedExpression] = Seq(UnresolvedStar(None)) | ||
val expectedPlan = Project(projectList, UnresolvedRelation(Seq("table"))) | ||
assertEquals(expectedPlan, context.getPlan) | ||
assertEquals(logPlan, "source=[table] | fields + *") | ||
|
||
} | ||
|
||
test("test simple search with schema.table and no explicit fields (defaults to all fields)") { | ||
// if successful build ppl logical plan and translate to catalyst logical plan | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source=schema.table", false), context) | ||
|
||
val projectList: Seq[NamedExpression] = Seq(UnresolvedStar(None)) | ||
val expectedPlan = Project(projectList, UnresolvedRelation(Seq("schema", "table"))) | ||
assertEquals(expectedPlan, context.getPlan) | ||
assertEquals(logPlan, "source=[schema.table] | fields + *") | ||
|
||
} | ||
|
||
test("test simple search with schema.table and one field projected") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source=schema.table | fields A", false), context) | ||
|
||
val projectList: Seq[NamedExpression] = Seq(UnresolvedAttribute("A")) | ||
val expectedPlan = Project(projectList, UnresolvedRelation(Seq("schema", "table"))) | ||
assertEquals(expectedPlan, context.getPlan) | ||
assertEquals(logPlan, "source=[schema.table] | fields + A") | ||
} | ||
|
||
test("test simple search with only one table with one field projected") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source=table | fields A", false), context) | ||
|
||
val projectList: Seq[NamedExpression] = Seq(UnresolvedAttribute("A")) | ||
val expectedPlan = Project(projectList, UnresolvedRelation(Seq("table"))) | ||
assertEquals(expectedPlan, context.getPlan) | ||
assertEquals(logPlan, "source=[table] | fields + A") | ||
} | ||
|
||
test("test simple search with only one table with two fields projected") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source=t | fields A, B", false), context) | ||
|
||
|
||
val table = UnresolvedRelation(Seq("t")) | ||
val projectList = Seq(UnresolvedAttribute("A"), UnresolvedAttribute("B")) | ||
val expectedPlan = Project(projectList, table) | ||
assertEquals(expectedPlan, context.getPlan) | ||
assertEquals(logPlan, "source=[t] | fields + A,B") | ||
} | ||
|
||
test("Search multiple tables - translated into union call - fields expected to exist in both tables ") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "search source = table1, table2 | fields A, B", false), context) | ||
|
||
|
||
val table1 = UnresolvedRelation(Seq("table1")) | ||
val table2 = UnresolvedRelation(Seq("table2")) | ||
|
||
val allFields1 = Seq(UnresolvedAttribute("A"), UnresolvedAttribute("B")) | ||
val allFields2 = Seq(UnresolvedAttribute("A"), UnresolvedAttribute("B")) | ||
|
||
val projectedTable1 = Project(allFields1, table1) | ||
val projectedTable2 = Project(allFields2, table2) | ||
|
||
val expectedPlan = Union(Seq(projectedTable1, projectedTable2), byName = true, allowMissingCol = true) | ||
|
||
assertEquals(logPlan, "source=[table1, table2] | fields + A,B") | ||
assertEquals(expectedPlan, context.getPlan) | ||
} | ||
|
||
test("Search multiple tables - translated into union call with fields") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source = table1, table2 ", false), context) | ||
|
||
|
||
val table1 = UnresolvedRelation(Seq("table1")) | ||
val table2 = UnresolvedRelation(Seq("table2")) | ||
|
||
val allFields1 = UnresolvedStar(None) | ||
val allFields2 = UnresolvedStar(None) | ||
|
||
val projectedTable1 = Project(Seq(allFields1), table1) | ||
val projectedTable2 = Project(Seq(allFields2), table2) | ||
|
||
val expectedPlan = Union(Seq(projectedTable1, projectedTable2), byName = true, allowMissingCol = true) | ||
|
||
assertEquals(logPlan, "source=[table1, table2] | fields + *") | ||
assertEquals(expectedPlan, context.getPlan) | ||
} | ||
} |
136 changes: 136 additions & 0 deletions
136
.../test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanFiltersTranslatorTestSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,136 @@ | ||
/* | ||
* Copyright OpenSearch Contributors | ||
* SPDX-License-Identifier: Apache-2.0 | ||
*/ | ||
|
||
package org.opensearch.flint.spark.ppl | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.spark.SparkFunSuite | ||
import org.apache.spark.sql.catalyst.TableIdentifier | ||
import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry, TableFunctionRegistry, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} | ||
import org.apache.spark.sql.catalyst.catalog._ | ||
import org.apache.spark.sql.catalyst.expressions.aggregate._ | ||
import org.apache.spark.sql.catalyst.expressions.{Alias, And, Descending, Divide, EqualTo, Floor, GreaterThan, GreaterThanOrEqual, LessThan, LessThanOrEqual, Like, Literal, NamedExpression, Not, SortOrder, UnixTimestamp} | ||
import org.apache.spark.sql.catalyst.parser.ParserInterface | ||
import org.apache.spark.sql.catalyst.plans.logical._ | ||
import org.apache.spark.sql.types.{IntegerType, StructField, StructType} | ||
import org.junit.Assert.assertEquals | ||
import org.mockito.Mockito.when | ||
import org.opensearch.flint.spark.ppl.PlaneUtils.plan | ||
import org.opensearch.sql.ppl.{CatalystPlanContext, CatalystQueryPlanVisitor} | ||
import org.scalatest.matchers.should.Matchers | ||
import org.scalatestplus.mockito.MockitoSugar.mock | ||
|
||
class PPLLogicalPlanFiltersTranslatorTestSuite | ||
extends SparkFunSuite | ||
with Matchers { | ||
|
||
private val planTrnasformer = new CatalystQueryPlanVisitor() | ||
private val pplParser = new PPLSyntaxParser() | ||
|
||
test("test simple search with only one table with one field literal filtered ") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source=t a = 1 ", false), context) | ||
|
||
val table = UnresolvedRelation(Seq("t")) | ||
val filterExpr = EqualTo(UnresolvedAttribute("a"), Literal(1)) | ||
val filterPlan = Filter(filterExpr, table) | ||
val projectList = Seq(UnresolvedStar(None)) | ||
val expectedPlan = Project(projectList, filterPlan) | ||
assertEquals(expectedPlan, context.getPlan) | ||
assertEquals(logPlan, "source=[t] | where a = 1 | fields + *") | ||
} | ||
|
||
test("test simple search with only one table with one field literal int equality filtered and one field projected") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source=t a = 1 | fields a", false), context) | ||
|
||
val table = UnresolvedRelation(Seq("t")) | ||
val filterExpr = EqualTo(UnresolvedAttribute("a"), Literal(1)) | ||
val filterPlan = Filter(filterExpr, table) | ||
val projectList = Seq(UnresolvedAttribute("a")) | ||
val expectedPlan = Project(projectList, filterPlan) | ||
assertEquals(expectedPlan, context.getPlan) | ||
assertEquals(logPlan, "source=[t] | where a = 1 | fields + a") | ||
} | ||
|
||
test("test simple search with only one table with one field literal string equality filtered and one field projected") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, """source=t a = 'hi' | fields a""", false), context) | ||
|
||
val table = UnresolvedRelation(Seq("t")) | ||
val filterExpr = EqualTo(UnresolvedAttribute("a"), Literal("'hi'")) | ||
val filterPlan = Filter(filterExpr, table) | ||
val projectList = Seq(UnresolvedAttribute("a")) | ||
val expectedPlan = Project(projectList, filterPlan) | ||
|
||
assertEquals(expectedPlan,context.getPlan) | ||
assertEquals(logPlan, "source=[t] | where a = 'hi' | fields + a") | ||
} | ||
|
||
test("test simple search with only one table with one field greater than filtered and one field projected") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source=t a > 1 | fields a", false), context) | ||
|
||
val table = UnresolvedRelation(Seq("t")) | ||
val filterExpr = GreaterThan(UnresolvedAttribute("a"), Literal(1)) | ||
val filterPlan = Filter(filterExpr, table) | ||
val projectList = Seq(UnresolvedAttribute("a")) | ||
val expectedPlan = Project(projectList, filterPlan) | ||
assertEquals(expectedPlan, context.getPlan) | ||
assertEquals(logPlan, "source=[t] | where a > 1 | fields + a") | ||
} | ||
|
||
test("test simple search with only one table with one field greater than equal filtered and one field projected") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source=t a >= 1 | fields a", false), context) | ||
|
||
val table = UnresolvedRelation(Seq("t")) | ||
val filterExpr = GreaterThanOrEqual(UnresolvedAttribute("a"), Literal(1)) | ||
val filterPlan = Filter(filterExpr, table) | ||
val projectList = Seq(UnresolvedAttribute("a")) | ||
val expectedPlan = Project(projectList, filterPlan) | ||
assertEquals(expectedPlan, context.getPlan) | ||
assertEquals(logPlan, "source=[t] | where a >= 1 | fields + a") | ||
} | ||
|
||
test("test simple search with only one table with one field lower than filtered and one field projected") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source=t a < 1 | fields a", false), context) | ||
|
||
val table = UnresolvedRelation(Seq("t")) | ||
val filterExpr = LessThan(UnresolvedAttribute("a"), Literal(1)) | ||
val filterPlan = Filter(filterExpr, table) | ||
val projectList = Seq(UnresolvedAttribute("a")) | ||
val expectedPlan = Project(projectList, filterPlan) | ||
assertEquals(expectedPlan, context.getPlan) | ||
assertEquals(logPlan, "source=[t] | where a < 1 | fields + a") | ||
} | ||
|
||
test("test simple search with only one table with one field lower than equal filtered and one field projected") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source=t a <= 1 | fields a", false), context) | ||
|
||
val table = UnresolvedRelation(Seq("t")) | ||
val filterExpr = LessThanOrEqual(UnresolvedAttribute("a"), Literal(1)) | ||
val filterPlan = Filter(filterExpr, table) | ||
val projectList = Seq(UnresolvedAttribute("a")) | ||
val expectedPlan = Project(projectList, filterPlan) | ||
assertEquals(expectedPlan, context.getPlan) | ||
assertEquals(logPlan, "source=[t] | where a <= 1 | fields + a") | ||
} | ||
|
||
test("test simple search with only one table with one field not equal filtered and one field projected") { | ||
val context = new CatalystPlanContext | ||
val logPlan = planTrnasformer.visit(plan(pplParser, "source=t a != 1 | fields a", false), context) | ||
|
||
val table = UnresolvedRelation(Seq("t")) | ||
val filterExpr = Not(EqualTo(UnresolvedAttribute("a"), Literal(1))) | ||
val filterPlan = Filter(filterExpr, table) | ||
val projectList = Seq(UnresolvedAttribute("a")) | ||
val expectedPlan = Project(projectList, filterPlan) | ||
assertEquals(expectedPlan, context.getPlan) | ||
assertEquals(logPlan, "source=[t] | where a != 1 | fields + a") | ||
} | ||
} |
Oops, something went wrong.