-
Notifications
You must be signed in to change notification settings - Fork 33
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 rename PPL function #618
Merged
Merged
Changes from 6 commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
6ff8d92
add support for rename command
kt-eliatra 121590b
add tests
kt-eliatra 85d8814
add usage examples to README.md
kt-eliatra 2bc1d8d
fix failing test and add more tests
kt-eliatra 0df487c
fix after rebase
kt-eliatra cddb658
fix scalafmtCheck checkstyle
salyh 1420ab3
The rename command does not return the source column/field
lukasz-soszynski-eliatra File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
152 changes: 152 additions & 0 deletions
152
...est/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLRenameITSuite.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,152 @@ | ||
/* | ||
* Copyright OpenSearch Contributors | ||
* SPDX-License-Identifier: Apache-2.0 | ||
*/ | ||
|
||
package org.opensearch.flint.spark.ppl | ||
|
||
import org.opensearch.sql.ppl.utils.DataTypeTransformer.seq | ||
|
||
import org.apache.spark.sql.{QueryTest, Row} | ||
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar} | ||
import org.apache.spark.sql.catalyst.expressions.Alias | ||
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project} | ||
import org.apache.spark.sql.streaming.StreamTest | ||
|
||
class FlintSparkPPLRenameITSuite | ||
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 single renamed field in fields command") { | ||
val frame = sql(s""" | ||
| source = $testTable | rename age as renamed_age | fields name, renamed_age | ||
| """.stripMargin) | ||
|
||
// Retrieve the results | ||
val results: Array[Row] = frame.collect() | ||
// Define the expected results | ||
val expectedResults: Array[Row] = | ||
Array(Row("Jake", 70), Row("Hello", 30), Row("John", 25), Row("Jane", 20)) | ||
// Compare the results | ||
implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) | ||
assert(results.sorted.sameElements(expectedResults.sorted)) | ||
|
||
// Retrieve the logical plan | ||
val logicalPlan: LogicalPlan = frame.queryExecution.logical | ||
// Define the expected logical plan | ||
val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) | ||
val fieldsProjectList = Seq(UnresolvedAttribute("name"), UnresolvedAttribute("renamed_age")) | ||
val renameProjectList = | ||
Seq(UnresolvedStar(None), Alias(UnresolvedAttribute("age"), "renamed_age")()) | ||
val expectedPlan = Project(fieldsProjectList, Project(renameProjectList, table)) | ||
// Compare the two plans | ||
comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) | ||
} | ||
|
||
test("test multiple renamed fields in fields command") { | ||
val frame = sql(s""" | ||
| source = $testTable | rename name as renamed_name, country as renamed_country | fields renamed_name, age, renamed_country | ||
| """.stripMargin) | ||
|
||
val results: Array[Row] = frame.collect() | ||
// results.foreach(println(_)) | ||
val expectedResults: Array[Row] = | ||
Array( | ||
Row("Jake", 70, "USA"), | ||
Row("Hello", 30, "USA"), | ||
Row("John", 25, "Canada"), | ||
Row("Jane", 20, "Canada")) | ||
implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) | ||
assert(results.sorted.sameElements(expectedResults.sorted)) | ||
|
||
val logicalPlan: LogicalPlan = frame.queryExecution.logical | ||
val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) | ||
val fieldsProjectList = Seq( | ||
UnresolvedAttribute("renamed_name"), | ||
UnresolvedAttribute("age"), | ||
UnresolvedAttribute("renamed_country")) | ||
val renameProjectList = | ||
Seq( | ||
UnresolvedStar(None), | ||
Alias(UnresolvedAttribute("name"), "renamed_name")(), | ||
Alias(UnresolvedAttribute("country"), "renamed_country")()) | ||
val expectedPlan = Project(fieldsProjectList, Project(renameProjectList, table)) | ||
comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) | ||
} | ||
|
||
test("test renamed fields without fields command") { | ||
val frame = sql(s""" | ||
| source = $testTable | rename age as user_age, country as user_country | ||
| """.stripMargin) | ||
|
||
val results: Array[Row] = frame.collect() | ||
val expectedResults: Array[Row] = Array( | ||
Row("Jake", 70, "California", "USA", 2023, 4, 70, "USA"), | ||
Row("Hello", 30, "New York", "USA", 2023, 4, 30, "USA"), | ||
Row("John", 25, "Ontario", "Canada", 2023, 4, 25, "Canada"), | ||
Row("Jane", 20, "Quebec", "Canada", 2023, 4, 20, "Canada")) | ||
implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) | ||
assert(results.sorted.sameElements(expectedResults.sorted)) | ||
|
||
val logicalPlan: LogicalPlan = frame.queryExecution.logical | ||
val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) | ||
val renameProjectList = Seq( | ||
UnresolvedStar(None), | ||
Alias(UnresolvedAttribute("age"), "user_age")(), | ||
Alias(UnresolvedAttribute("country"), "user_country")()) | ||
val expectedPlan = Project(seq(UnresolvedStar(None)), Project(renameProjectList, table)) | ||
comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) | ||
} | ||
|
||
test("test renamed field used in aggregation") { | ||
val frame = sql(s""" | ||
| source = $testTable | rename age as user_age | stats avg(user_age) by country | ||
| """.stripMargin) | ||
|
||
val results: Array[Row] = frame.collect() | ||
val expectedResults: Array[Row] = Array(Row(22.5, "Canada"), Row(50.0, "USA")) | ||
implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, Double](_.getAs[Double](0)) | ||
assert(results.sorted.sameElements(expectedResults.sorted)) | ||
|
||
val logicalPlan: LogicalPlan = frame.queryExecution.logical | ||
val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) | ||
val renameProjectList = | ||
Seq(UnresolvedStar(None), Alias(UnresolvedAttribute("age"), "user_age")()) | ||
val aggregateExpressions = | ||
Seq( | ||
Alias( | ||
UnresolvedFunction( | ||
Seq("AVG"), | ||
Seq(UnresolvedAttribute("user_age")), | ||
isDistinct = false), | ||
"avg(user_age)")(), | ||
Alias(UnresolvedAttribute("country"), "country")()) | ||
val aggregatePlan = Aggregate( | ||
Seq(Alias(UnresolvedAttribute("country"), "country")()), | ||
aggregateExpressions, | ||
Project(renameProjectList, table)) | ||
val expectedPlan = Project(seq(UnresolvedStar(None)), aggregatePlan) | ||
comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) | ||
} | ||
} |
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
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -46,6 +46,7 @@ commands | |
| parseCommand | ||
| patternsCommand | ||
| lookupCommand | ||
| renameCommand | ||
; | ||
|
||
searchCommand | ||
|
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
116 changes: 116 additions & 0 deletions
116
...c/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanRenameTranslatorTestSuite.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,116 @@ | ||
/* | ||
* 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.opensearch.sql.ppl.utils.DataTypeTransformer.seq | ||
import org.scalatest.matchers.should.Matchers | ||
|
||
import org.apache.spark.SparkFunSuite | ||
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar} | ||
import org.apache.spark.sql.catalyst.expressions.{Alias, Descending, ExprId, NamedExpression, SortOrder} | ||
import org.apache.spark.sql.catalyst.plans.PlanTest | ||
import org.apache.spark.sql.catalyst.plans.logical.{Project, Sort} | ||
|
||
class PPLLogicalPlanRenameTranslatorTestSuite | ||
extends SparkFunSuite | ||
with PlanTest | ||
with LogicalPlanTestUtils | ||
with Matchers { | ||
|
||
private val planTransformer = new CatalystQueryPlanVisitor() | ||
private val pplParser = new PPLSyntaxParser() | ||
|
||
test("test renamed fields not included in fields expressions") { | ||
val context = new CatalystPlanContext | ||
val logPlan = | ||
planTransformer.visit( | ||
plan(pplParser, "source=t | rename a as r_a, b as r_b | fields c"), | ||
context) | ||
val renameProjectList: Seq[NamedExpression] = | ||
Seq( | ||
UnresolvedStar(None), | ||
Alias(UnresolvedAttribute("a"), "r_a")(), | ||
Alias(UnresolvedAttribute("b"), "r_b")()) | ||
val expectedPlan = Project( | ||
seq(UnresolvedAttribute("c")), | ||
Project(renameProjectList, UnresolvedRelation(Seq("t")))) | ||
comparePlans(expectedPlan, logPlan, checkAnalysis = false) | ||
} | ||
|
||
test("test renamed fields included in fields expression") { | ||
val context = new CatalystPlanContext | ||
val logPlan = | ||
planTransformer.visit( | ||
plan(pplParser, "source=t | rename a as r_a, b as r_b | fields r_a, r_b, c"), | ||
context) | ||
|
||
val renameProjectList: Seq[NamedExpression] = | ||
Seq( | ||
UnresolvedStar(None), | ||
Alias(UnresolvedAttribute("a"), "r_a")(), | ||
Alias(UnresolvedAttribute("b"), "r_b")()) | ||
val expectedPlan = Project( | ||
seq(UnresolvedAttribute("r_a"), UnresolvedAttribute("r_b"), UnresolvedAttribute("c")), | ||
Project(renameProjectList, UnresolvedRelation(Seq("t")))) | ||
comparePlans(expectedPlan, logPlan, checkAnalysis = false) | ||
} | ||
|
||
test("test renamed fields without fields command") { | ||
val context = new CatalystPlanContext | ||
val logPlan = | ||
planTransformer.visit(plan(pplParser, "source=t | rename a as r_a, b as r_b"), context) | ||
|
||
val renameProjectList: Seq[NamedExpression] = | ||
Seq( | ||
UnresolvedStar(None), | ||
Alias(UnresolvedAttribute("a"), "r_a")(), | ||
Alias(UnresolvedAttribute("b"), "r_b")()) | ||
val expectedPlan = | ||
Project(seq(UnresolvedStar(None)), Project(renameProjectList, UnresolvedRelation(Seq("t")))) | ||
comparePlans(expectedPlan, logPlan, checkAnalysis = false) | ||
} | ||
|
||
test("test renamed fields with sort") { | ||
val context = new CatalystPlanContext | ||
val logPlan = | ||
planTransformer.visit( | ||
plan(pplParser, "source=t | rename a as r_a, b as r_b | sort - r_a | fields r_b"), | ||
context) | ||
|
||
val renameProjectList: Seq[NamedExpression] = | ||
Seq( | ||
UnresolvedStar(None), | ||
Alias(UnresolvedAttribute("a"), "r_a")(), | ||
Alias(UnresolvedAttribute("b"), "r_b")()) | ||
val renameProject = Project(renameProjectList, UnresolvedRelation(Seq("t"))) | ||
val sortOrder = SortOrder(UnresolvedAttribute("r_a"), Descending, Seq.empty) | ||
val sort = Sort(seq(sortOrder), global = true, renameProject) | ||
val expectedPlan = Project(seq(UnresolvedAttribute("r_b")), sort) | ||
comparePlans(expectedPlan, logPlan, checkAnalysis = false) | ||
} | ||
|
||
test("test rename eval expression output") { | ||
val context = new CatalystPlanContext | ||
val logPlan = | ||
planTransformer.visit( | ||
plan(pplParser, "source=t | eval a = RAND() | rename a as eval_rand | fields eval_rand"), | ||
context) | ||
|
||
val evalProjectList: Seq[NamedExpression] = Seq( | ||
UnresolvedStar(None), | ||
Alias(UnresolvedFunction("rand", Seq.empty, isDistinct = false), "a")( | ||
exprId = ExprId(0), | ||
qualifier = Seq.empty)) | ||
val evalProject = Project(evalProjectList, UnresolvedRelation(Seq("t"))) | ||
val renameProjectList: Seq[NamedExpression] = | ||
Seq(UnresolvedStar(None), Alias(UnresolvedAttribute("a"), "eval_rand")()) | ||
val expectedPlan = | ||
Project(seq(UnresolvedAttribute("eval_rand")), Project(renameProjectList, evalProject)) | ||
comparePlans(expectedPlan, logPlan, checkAnalysis = false) | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Could you test the following case:
Ref #618 (comment)
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.
@lukasz-soszynski-eliatra is on it ...
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.
So far, the query
source = $testTable | rename age as renamed_age | fields age
has worked "correctly"; that is, the fieldage
was present in the query result. I introduced corrections and updated the tests. Currently, the execution of the previously mentioned query causes an error.