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

[SPARK-12689][SQL] Migrate DDL parsing to the newly absorbed parser #10723

Closed
wants to merge 20 commits into from

Conversation

viirya
Copy link
Member

@viirya viirya commented Jan 12, 2016

JIRA: https://issues.apache.org/jira/browse/SPARK-12689

DDLParser processes three commands: createTable, describeTable and refreshTable.
This patch migrates the three commands to newly absorbed parser.

@SparkQA
Copy link

SparkQA commented Jan 12, 2016

Test build #49237 has finished for PR 10723 at commit f886f67.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@rxin
Copy link
Contributor

rxin commented Jan 12, 2016

cc @hvanhovell

@viirya can you rename the title to "migrate describe table parsing to ..."

@viirya
Copy link
Member Author

viirya commented Jan 13, 2016

Because SQLContext still uses DDLParser, looks like I can't simply remove describeTable command from DDLParser. So I can't gradually migrate these three commands in three PRs.

@viirya viirya changed the title [SPARK-12689][SQL] Migrate DDL parsing to the newly absorbed parser: describe command [SPARK-12689][SQL][WIP] Migrate DDL parsing to the newly absorbed parser Jan 13, 2016
@SparkQA
Copy link

SparkQA commented Jan 13, 2016

Test build #49317 has finished for PR 10723 at commit c60cd9e.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@yhuai
Copy link
Contributor

yhuai commented Jan 13, 2016

@cloud-fan Can you also take a look? It is related to the work of adding DDL support for creating bucketed tables.

@@ -52,26 +57,30 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly
nodeToDescribeFallback(node)
} else {
tableType match {
case Token("TOK_TABTYPE", Token("TOK_TABNAME", nameParts :: Nil) :: Nil) =>
case Token("TOK_TABTYPE", Token("TOK_TABNAME", nameParts) :: Nil) =>
Copy link
Contributor

Choose a reason for hiding this comment

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

Why change this? You didn't touch the describe stuff in SparkSqlParser.g right?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yes. I think it is incorrect from beginning but not be tested it out because we don't reach here before. I've tested it locally. Once all three commands are migrated, we can see this passing tests.

Copy link
Contributor

Choose a reason for hiding this comment

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

if we parse the following SQL using the parse driver org.apache.spark.sql.catalyst.parser.ParseDriver.parsePlan("DESCRIBE EXTENDED tbl.a", null)

We would end up with the following AST:

TOK_DESCTABLE 1, 0, 6, 18
:- TOK_TABTYPE 1, 4, 6, 18 
:  +- TOK_TABNAME 1, 4, 6, 18 
:     :- tbl 1, 4, 4, 18 
:     +- a 1, 6, 6, 22 
+- EXTENDED 1, 2, 2, 9 

This change would pick this up, and old code didn't (I am sure I tested this though :S ). You can disable this in the DDL parser, to see if it works now.

Copy link
Contributor

Choose a reason for hiding this comment

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

Could we add a test for this? The Hive test suite apparently misses this one. I could also address in a different PR.

Copy link
Member Author

Choose a reason for hiding this comment

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

Actually we have test for describe table command in HiveQuerySuite. Do we need another test?

Conflicts:
	sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g
@viirya viirya changed the title [SPARK-12689][SQL][WIP] Migrate DDL parsing to the newly absorbed parser [SPARK-12689][SQL] Migrate DDL parsing to the newly absorbed parser Jan 18, 2016
@SparkQA
Copy link

SparkQA commented Jan 18, 2016

Test build #49585 has finished for PR 10723 at commit 5a6cc4a.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

// It is describing a column with the format like "describe db.table column".
nodeToDescribeFallback(node)
case tableName =>
case tableName :: Nil =>
// It is describing a table with the format like "describe table".
datasources.DescribeCommand(
UnresolvedRelation(TableIdentifier(tableName.text), None),
Copy link
Contributor

Choose a reason for hiding this comment

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

cleanIdentifier?

@@ -316,7 +316,7 @@ class HiveContext private[hive](
}

protected[sql] override def parseSql(sql: String): LogicalPlan = {
super.parseSql(substitutor.substitute(hiveconf, sql))
sqlParser.parsePlan(substitutor.substitute(hiveconf, sql))
Copy link
Contributor

Choose a reason for hiding this comment

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

How about gradually moving functionality from the DLL parser to SparkQl? That would allow us to test this in the meantime.

Copy link
Member Author

Choose a reason for hiding this comment

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

DDLParser is still used in SQLContext. Do we want to completely remove it? Because I already migrate three commands. I think we can test them all together.

case Token("TOK_TABLEOPTIONS", options) =>
options.map {
case Token("TOK_TABLEOPTION", Token(key, _) :: Token(value, _) :: Nil) =>
(key, value.replaceAll("^\'|^\"|\"$|\'$", ""))
Copy link
Contributor

Choose a reason for hiding this comment

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

Why not use unquoteString this does the same and is easier to read?

Copy link
Member Author

Choose a reason for hiding this comment

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

Don't know there is unquoteString. Thanks.

@SparkQA
Copy link

SparkQA commented Jan 18, 2016

Test build #49591 has finished for PR 10723 at commit 1c145eb.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Conflicts:
	sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
	sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala
looseNonReserved
: nonReserved | KW_FROM | KW_TO
;

Copy link
Member Author

Choose a reason for hiding this comment

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

We are allowed to use From and To in CreateTableUsing command's options (actually seems we can use any string as the option key). But we can't simply add them into nonReserved because by doing that we mess other existing rules. So we create a looseIdentifier and looseNonReserved here.

Copy link
Contributor

Choose a reason for hiding this comment

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

Why not add this to the option rule directly?

Copy link
Member Author

Choose a reason for hiding this comment

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

Because I don't know if we will add other reserved words later. If so, the option rule might be too long. I don't count if any keywords are not included in nonReserved.

Copy link
Member Author

Choose a reason for hiding this comment

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

Both (current approach or adding it to the option rule) are okay for me.

Copy link
Contributor

Choose a reason for hiding this comment

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

Could add your initial line commentaar as a comment in the code?

Copy link
Member Author

Choose a reason for hiding this comment

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

Thanks for reminding. I've added it.

@SparkQA
Copy link

SparkQA commented Jan 19, 2016

Test build #49673 has finished for PR 10723 at commit 838f701.

  • This patch fails MiMa tests.
  • This patch does not merge cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 19, 2016

Test build #49669 has finished for PR 10723 at commit d800c58.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 19, 2016

Test build #49675 has finished for PR 10723 at commit 7e0f218.

  • This patch fails MiMa tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

case Token(k, Nil) => k
}.mkString(".")
val value = unquoteString(keysAndValue.last.text)
(key, unquoteString(value))
Copy link
Contributor

Choose a reason for hiding this comment

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

Unquoting twice?

@hvanhovell
Copy link
Contributor

@viirya I have done another round.

Most things are minor, but I would to know why you want to change the treatment of quoted identifiers?

asfgit pushed a commit that referenced this pull request Jan 27, 2016
…Parser commands to new Parser

This PR moves all the functionality provided by the SparkSQLParser/ExtendedHiveQlParser to the new Parser hierarchy (SparkQl/HiveQl). This also improves the current SET command parsing: the current implementation swallows ```set role ...``` and ```set autocommit ...``` commands, this PR respects these commands (and passes them on to Hive).

This PR and #10723 end the use of Parser-Combinator parsers for SQL parsing. As a result we can also remove the ```AbstractSQLParser``` in Catalyst.

The PR is marked WIP as long as it doesn't pass all tests.

cc rxin viirya winningsix (this touches #10144)

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #10905 from hvanhovell/SPARK-12866.
Conflicts:
	sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g
	sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala
	sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@viirya
Copy link
Member Author

viirya commented Jan 28, 2016

@hvanhovell Thanks for reviewing this. I've updated this to address your comments. Please see if it is proper for you.

@SparkQA
Copy link

SparkQA commented Jan 28, 2016

Test build #50258 has finished for PR 10723 at commit 7350f07.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 28, 2016

Test build #50276 has finished for PR 10723 at commit 7d31844.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@hvanhovell
Copy link
Contributor

LGTM

Conflicts:
	sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala
@SparkQA
Copy link

SparkQA commented Jan 29, 2016

Test build #50355 has finished for PR 10723 at commit 8b7086e.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • case class SetDatabaseCommand(databaseName: String) extends RunnableCommand

@viirya
Copy link
Member Author

viirya commented Jan 29, 2016

retest this please.

@SparkQA
Copy link

SparkQA commented Jan 29, 2016

Test build #50366 has finished for PR 10723 at commit 8b7086e.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • case class SetDatabaseCommand(databaseName: String) extends RunnableCommand

@viirya
Copy link
Member Author

viirya commented Jan 29, 2016

It's weird.

@viirya
Copy link
Member Author

viirya commented Jan 29, 2016

retest this please.

@hvanhovell
Copy link
Contributor

It is, can't make sense of this either. Are tests passing locally?

@viirya
Copy link
Member Author

viirya commented Jan 29, 2016

Yeah, I think so. And I don't update codes since last successful test.

@viirya
Copy link
Member Author

viirya commented Jan 29, 2016

See how another round of test shows.

@viirya
Copy link
Member Author

viirya commented Jan 29, 2016

Many unrelated failures like can't find hive jar file.

@SparkQA
Copy link

SparkQA commented Jan 29, 2016

Test build #50377 has finished for PR 10723 at commit 8b7086e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • case class SetDatabaseCommand(databaseName: String) extends RunnableCommand

@viirya
Copy link
Member Author

viirya commented Jan 29, 2016

ping @rxin

@hvanhovell
Copy link
Contributor

@viirya I am gonna trigger another test to make sure things keep working.

@hvanhovell
Copy link
Contributor

retest this please

@viirya
Copy link
Member Author

viirya commented Jan 29, 2016

@hvanhovell ok, thanks.

@SparkQA
Copy link

SparkQA commented Jan 29, 2016

Test build #50382 has finished for PR 10723 at commit 8b7086e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • case class SetDatabaseCommand(databaseName: String) extends RunnableCommand

@viirya
Copy link
Member Author

viirya commented Jan 31, 2016

cc @rxin

@rxin
Copy link
Contributor

rxin commented Jan 31, 2016

Thanks - merging this in master.

@asfgit asfgit closed this in 0e6d92d Jan 31, 2016
@viirya viirya deleted the migrate-ddl-describe branch December 27, 2023 18:18
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
6 participants