-
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
Describe show commands #533
Changes from 6 commits
21c92a0
74f261a
28e942a
cb42807
a0956f8
e34c0c6
b021076
a66a0dc
92d1e4c
439b12a
bdfb0ec
2735e2d
5f644ef
7d15613
1b867dc
fda684a
0bb2fbb
ad78b12
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 |
---|---|---|
|
@@ -6,9 +6,11 @@ | |
package org.opensearch.flint.spark.ppl | ||
|
||
import org.apache.spark.sql.{QueryTest, Row} | ||
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} | ||
import org.apache.spark.sql.catalyst.TableIdentifier | ||
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedTableOrView} | ||
import org.apache.spark.sql.catalyst.expressions.{Ascending, Literal, SortOrder} | ||
import org.apache.spark.sql.catalyst.plans.logical._ | ||
import org.apache.spark.sql.execution.command.DescribeTableCommand | ||
import org.apache.spark.sql.streaming.StreamTest | ||
|
||
class FlintSparkPPLBasicITSuite | ||
|
@@ -36,7 +38,42 @@ class FlintSparkPPLBasicITSuite | |
} | ||
} | ||
|
||
test("create ppl simple query test") { | ||
test("describe table query test") { | ||
val testTableQuoted = "`spark_catalog`.`default`.`flint_ppl_test`" | ||
Seq(testTable, testTableQuoted).foreach { table => | ||
val frame = sql(s""" | ||
describe flint_ppl_test | ||
""".stripMargin) | ||
|
||
// Retrieve the results | ||
val results: Array[Row] = frame.collect() | ||
// Define the expected results | ||
val expectedResults: Array[Row] = Array( | ||
Row("name", "string", null), | ||
Row("age", "int", null), | ||
Row("state", "string", null), | ||
Row("country", "string", null), | ||
Row("year", "int", null), | ||
Row("month", "int", null), | ||
Row("# Partition Information", "", ""), | ||
Row("# col_name", "data_type", "comment"), | ||
Row("year", "int", null), | ||
Row("month", "int", null) | ||
) | ||
// 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.commandExecuted.asInstanceOf[CommandResult].commandLogicalPlan | ||
// Define the expected logical plan | ||
val expectedPlan: LogicalPlan = | ||
DescribeTableCommand(TableIdentifier("default.flint_ppl_test"), null, isExtended = false, output = Seq()) | ||
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. @YANG-DB to make this IT test successful.
|
||
// Compare the two plans | ||
comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) | ||
} | ||
} | ||
|
||
test("create ppl simple query test") { | ||
val testTableQuoted = "`spark_catalog`.`default`.`flint_ppl_test`" | ||
Seq(testTable, testTableQuoted).foreach { table => | ||
val frame = sql(s""" | ||
|
@@ -208,7 +245,7 @@ class FlintSparkPPLBasicITSuite | |
val sortedPlan: LogicalPlan = | ||
Sort(Seq(SortOrder(UnresolvedAttribute("age"), Ascending)), global = true, limitPlan) | ||
|
||
val expectedPlan = Project(Seq(UnresolvedStar(None)), sortedPlan); | ||
val expectedPlan = Project(Seq(UnresolvedStar(None)), sortedPlan) | ||
// Compare the two plans | ||
assert(compareByString(expectedPlan) === compareByString(logicalPlan)) | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,6 +27,7 @@ queryStatement | |
// commands | ||
pplCommands | ||
: searchCommand | ||
| describeCommand | ||
; | ||
|
||
commands | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,17 @@ | ||
/* | ||
* Copyright OpenSearch Contributors | ||
* SPDX-License-Identifier: Apache-2.0 | ||
*/ | ||
|
||
package org.opensearch.sql.ast.tree; | ||
|
||
import org.opensearch.sql.ast.expression.UnresolvedExpression; | ||
|
||
/** | ||
* Extend Relation to describe the table itself | ||
*/ | ||
public class DescribeRelation extends Relation{ | ||
public DescribeRelation(UnresolvedExpression tableName) { | ||
super(tableName); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -5,6 +5,7 @@ | |
|
||
package org.opensearch.sql.ppl; | ||
|
||
import org.apache.spark.sql.catalyst.TableIdentifier; | ||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$; | ||
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation; | ||
import org.apache.spark.sql.catalyst.analysis.UnresolvedStar$; | ||
|
@@ -13,8 +14,10 @@ | |
import org.apache.spark.sql.catalyst.expressions.Predicate; | ||
import org.apache.spark.sql.catalyst.expressions.SortOrder; | ||
import org.apache.spark.sql.catalyst.plans.logical.Aggregate; | ||
import org.apache.spark.sql.catalyst.plans.logical.DescribeRelation$; | ||
import org.apache.spark.sql.catalyst.plans.logical.Limit; | ||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; | ||
import org.apache.spark.sql.execution.command.DescribeTableCommand; | ||
import org.apache.spark.sql.types.DataTypes; | ||
import org.apache.spark.sql.util.CaseInsensitiveStringMap; | ||
import org.opensearch.sql.ast.AbstractNodeVisitor; | ||
|
@@ -46,6 +49,7 @@ | |
import org.opensearch.sql.ast.tree.Aggregation; | ||
import org.opensearch.sql.ast.tree.Correlation; | ||
import org.opensearch.sql.ast.tree.Dedupe; | ||
import org.opensearch.sql.ast.tree.DescribeRelation; | ||
import org.opensearch.sql.ast.tree.Eval; | ||
import org.opensearch.sql.ast.tree.Filter; | ||
import org.opensearch.sql.ast.tree.Head; | ||
|
@@ -107,6 +111,15 @@ public LogicalPlan visitExplain(Explain node, CatalystPlanContext context) { | |
|
||
@Override | ||
public LogicalPlan visitRelation(Relation node, CatalystPlanContext context) { | ||
if (node instanceof DescribeRelation) { | ||
return context.with( | ||
new DescribeTableCommand( | ||
new TableIdentifier(node.getTableQualifiedName().toString()), | ||
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. I checked the spark sql reference. The DESCRIBE command syntax is
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. Then we can test on Seq("flint_ppl_test", "`flint_ppl_test`", "default.flint_ppl_test", "`default`.`flint_ppl_test`") 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. @LantaoJin thanks again |
||
scala.collection.immutable.Map$.MODULE$.<String, String>empty(), | ||
false, | ||
DescribeRelation$.MODULE$.getOutputAttrs())); | ||
} | ||
//regular sql algebraic relations | ||
node.getTableName().forEach(t -> | ||
// Resolving the qualifiedName which is composed of a datasource.schema.table | ||
context.with(new UnresolvedRelation(seq(of(t.split("\\."))), CaseInsensitiveStringMap.empty(), 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.
Oh, we miss testing the table_identifier with quoted.
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.
yes probably need to add some more fail based use cases...