Skip to content

Commit 17bdc36

Browse files
aokolnychyigatorsmile
authored andcommitted
[SPARK-21102][SQL] Refresh command is too aggressive in parsing
### Idea This PR adds validation to REFRESH sql statements. Currently, users can specify whatever they want as resource path. For example, spark.sql("REFRESH ! $ !") will be executed without any exceptions. ### Implementation I am not sure that my current implementation is the most optimal, so any feedback is appreciated. My first idea was to make the grammar as strict as possible. Unfortunately, there were some problems. I tried the approach below: SqlBase.g4 ``` ... | REFRESH TABLE tableIdentifier #refreshTable | REFRESH resourcePath #refreshResource ... resourcePath : STRING | (IDENTIFIER | number | nonReserved | '/' | '-')+ // other symbols can be added if needed ; ``` It is not flexible enough and requires to explicitly mention all possible symbols. Therefore, I came up with the current approach that is implemented in the code. Let me know your opinion on which one is better. Author: aokolnychyi <[email protected]> Closes #18368 from aokolnychyi/spark-21102.
1 parent eb7a5a6 commit 17bdc36

File tree

3 files changed

+38
-5
lines changed

3 files changed

+38
-5
lines changed

sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -149,7 +149,7 @@ statement
149149
| (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)?
150150
tableIdentifier partitionSpec? describeColName? #describeTable
151151
| REFRESH TABLE tableIdentifier #refreshTable
152-
| REFRESH .*? #refreshResource
152+
| REFRESH (STRING | .*?) #refreshResource
153153
| CACHE LAZY? TABLE tableIdentifier (AS? query)? #cacheTable
154154
| UNCACHE TABLE (IF EXISTS)? tableIdentifier #uncacheTable
155155
| CLEAR CACHE #clearCache

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala

Lines changed: 17 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -230,11 +230,25 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
230230
}
231231

232232
/**
233-
* Create a [[RefreshTable]] logical plan.
233+
* Create a [[RefreshResource]] logical plan.
234234
*/
235235
override def visitRefreshResource(ctx: RefreshResourceContext): LogicalPlan = withOrigin(ctx) {
236-
val resourcePath = remainder(ctx.REFRESH.getSymbol).trim
237-
RefreshResource(resourcePath)
236+
val path = if (ctx.STRING != null) string(ctx.STRING) else extractUnquotedResourcePath(ctx)
237+
RefreshResource(path)
238+
}
239+
240+
private def extractUnquotedResourcePath(ctx: RefreshResourceContext): String = withOrigin(ctx) {
241+
val unquotedPath = remainder(ctx.REFRESH.getSymbol).trim
242+
validate(
243+
unquotedPath != null && !unquotedPath.isEmpty,
244+
"Resource paths cannot be empty in REFRESH statements. Use / to match everything",
245+
ctx)
246+
val forbiddenSymbols = Seq(" ", "\n", "\r", "\t")
247+
validate(
248+
!forbiddenSymbols.exists(unquotedPath.contains(_)),
249+
"REFRESH statements cannot contain ' ', '\\n', '\\r', '\\t' inside unquoted resource paths",
250+
ctx)
251+
unquotedPath
238252
}
239253

240254
/**

sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala

Lines changed: 20 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Concat, SortOrder}
2525
import org.apache.spark.sql.catalyst.parser.ParseException
2626
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort}
2727
import org.apache.spark.sql.execution.command._
28-
import org.apache.spark.sql.execution.datasources.CreateTable
28+
import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource}
2929
import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
3030
import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType}
3131

@@ -66,6 +66,25 @@ class SparkSqlParserSuite extends AnalysisTest {
6666
}
6767
}
6868

69+
test("refresh resource") {
70+
assertEqual("REFRESH prefix_path", RefreshResource("prefix_path"))
71+
assertEqual("REFRESH /", RefreshResource("/"))
72+
assertEqual("REFRESH /path///a", RefreshResource("/path///a"))
73+
assertEqual("REFRESH pat1h/112/_1a", RefreshResource("pat1h/112/_1a"))
74+
assertEqual("REFRESH pat1h/112/_1a/a-1", RefreshResource("pat1h/112/_1a/a-1"))
75+
assertEqual("REFRESH path-with-dash", RefreshResource("path-with-dash"))
76+
assertEqual("REFRESH \'path with space\'", RefreshResource("path with space"))
77+
assertEqual("REFRESH \"path with space 2\"", RefreshResource("path with space 2"))
78+
intercept("REFRESH a b", "REFRESH statements cannot contain")
79+
intercept("REFRESH a\tb", "REFRESH statements cannot contain")
80+
intercept("REFRESH a\nb", "REFRESH statements cannot contain")
81+
intercept("REFRESH a\rb", "REFRESH statements cannot contain")
82+
intercept("REFRESH a\r\nb", "REFRESH statements cannot contain")
83+
intercept("REFRESH @ $a$", "REFRESH statements cannot contain")
84+
intercept("REFRESH ", "Resource paths cannot be empty in REFRESH statements")
85+
intercept("REFRESH", "Resource paths cannot be empty in REFRESH statements")
86+
}
87+
6988
test("show functions") {
7089
assertEqual("show functions", ShowFunctionsCommand(None, None, true, true))
7190
assertEqual("show all functions", ShowFunctionsCommand(None, None, true, true))

0 commit comments

Comments
 (0)