Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
[SPARK-21102][SQL] Make refresh resource command less aggressive in p…
…arsing
  • Loading branch information
aokolnychyi committed Jul 2, 2017
commit 8d820f39c04ab975b7720b83c309ff8e4aebdd64
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,7 @@ statement
| (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)?
tableIdentifier partitionSpec? describeColName? #describeTable
| REFRESH TABLE tableIdentifier #refreshTable
| REFRESH .*? #refreshResource
| REFRESH (STRING | .*?) #refreshResource
| CACHE LAZY? TABLE tableIdentifier (AS? query)? #cacheTable
| UNCACHE TABLE (IF EXISTS)? tableIdentifier #uncacheTable
| CLEAR CACHE #clearCache
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -230,11 +230,24 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
}

/**
* Create a [[RefreshTable]] logical plan.
* Create a [[RefreshResource]] logical plan.
*/
override def visitRefreshResource(ctx: RefreshResourceContext): LogicalPlan = withOrigin(ctx) {
val resourcePath = remainder(ctx.REFRESH.getSymbol).trim
RefreshResource(resourcePath)
val path = if (ctx.STRING != null) string(ctx.STRING) else extractUnquotedResourcePath(ctx)
RefreshResource(path)
}

private def extractUnquotedResourcePath(ctx: RefreshResourceContext): String = withOrigin(ctx) {
val unquotedPath = remainder(ctx.REFRESH.getSymbol).trim
validate(
unquotedPath != null && !unquotedPath.isEmpty,
"Resource paths cannot be empty in REFRESH statements. Use / to match everything",
ctx)
validate(
!unquotedPath.contains(' '),
Copy link
Member

Choose a reason for hiding this comment

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

How about the other space \r\n\t?

"It is not allowed to use spaces inside unquoted resource paths in REFRESH statements",
ctx)
unquotedPath
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Concat, SortOrder}
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort}
import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.execution.datasources.CreateTable
import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource}
import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType}

Expand Down Expand Up @@ -66,6 +66,29 @@ class SparkSqlParserSuite extends AnalysisTest {
}
}

test("refresh resource") {
assertEqual("REFRESH prefix_path", RefreshResource("prefix_path"))
assertEqual("REFRESH /", RefreshResource("/"))
assertEqual("REFRESH /path///a", RefreshResource("/path///a"))
assertEqual("REFRESH pat1h/112/_1a", RefreshResource("pat1h/112/_1a"))
assertEqual("REFRESH pat1h/112/_1a/a-1", RefreshResource("pat1h/112/_1a/a-1"))
assertEqual("REFRESH path-with-dash", RefreshResource("path-with-dash"))
assertEqual("REFRESH \'path with space\'", RefreshResource("path with space"))
assertEqual("REFRESH \"path with space 2\"", RefreshResource("path with space 2"))
intercept(
"REFRESH a b",
"It is not allowed to use spaces inside unquoted resource paths in REFRESH statements")
intercept(
"REFRESH @ $a$",
"It is not allowed to use spaces inside unquoted resource paths in REFRESH statements")
intercept(
"REFRESH ",
"Resource paths cannot be empty in REFRESH statements. Use / to match everything")
intercept(
"REFRESH",
"Resource paths cannot be empty in REFRESH statements. Use / to match everything")
}

test("show functions") {
assertEqual("show functions", ShowFunctionsCommand(None, None, true, true))
assertEqual("show all functions", ShowFunctionsCommand(None, None, true, true))
Expand Down