Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst._
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, ExpressionInfo, UpCast}
import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface}
import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException, ParserInterface}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, SubqueryAlias, View}
import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils}
import org.apache.spark.sql.connector.catalog.CatalogManager
Expand Down Expand Up @@ -877,7 +877,12 @@ class SessionCatalog(
}
val viewConfigs = metadata.viewSQLConfigs
val parsedPlan = SQLConf.withExistingConf(View.effectiveSQLConf(viewConfigs, isTempView)) {
parser.parsePlan(viewText)
try {
parser.parseQuery(viewText)
} catch {
case _: ParseException =>
throw QueryCompilationErrors.invalidViewText(viewText, metadata.qualifiedName)
}
}
val projectList = if (!isHiveCreatedView(metadata)) {
val viewColumnNames = if (metadata.viewQueryColumnNames.isEmpty) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,11 @@ abstract class AbstractSqlParser extends ParserInterface with SQLConfHelper with
astBuilder.visitSingleTableSchema(parser.singleTableSchema())
}

/** Creates LogicalPlan for a given SQL string of query. */
override def parseQuery(sqlText: String): LogicalPlan = parse(sqlText) { parser =>
astBuilder.visitQuery(parser.query())
}

/** Creates LogicalPlan for a given SQL string. */
override def parsePlan(sqlText: String): LogicalPlan = parse(sqlText) { parser =>
astBuilder.visitSingleStatement(parser.singleStatement()) match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,4 +70,10 @@ trait ParserInterface {
*/
@throws[ParseException]("Text cannot be parsed to a DataType")
def parseDataType(sqlText: String): DataType

/**
* Parse a query string to a [[LogicalPlan]].
*/
@throws[ParseException]("Text cannot be parsed to a LogicalPlan")
def parseQuery(sqlText: String): LogicalPlan
}
Original file line number Diff line number Diff line change
Expand Up @@ -2366,4 +2366,9 @@ object QueryCompilationErrors {
def tableIndexNotSupportedError(errorMessage: String): Throwable = {
new AnalysisException(errorMessage)
}

def invalidViewText(viewText: String, tableName: String): Throwable = {
new AnalysisException(
s"Invalid view text: $viewText. The view $tableName may have been tampered with")
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -431,6 +431,9 @@ case class MyParser(spark: SparkSession, delegate: ParserInterface) extends Pars

override def parseDataType(sqlText: String): DataType =
delegate.parseDataType(sqlText)

override def parseQuery(sqlText: String): LogicalPlan =
delegate.parseQuery(sqlText)
}

object MyExtensions {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -530,4 +530,20 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession {
spark.sessionState.conf.clear()
}
}

test("SPARK-37266: View text can only be SELECT queries") {
withView("v") {
sql("CREATE VIEW v AS SELECT 1")
val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("v"))
val dropView = "DROP VIEW v"
// Simulate the behavior of hackers
val tamperedTable = table.copy(viewText = Some(dropView))
spark.sessionState.catalog.alterTable(tamperedTable)
val message = intercept[AnalysisException] {
sql("SELECT * FROM v")
}.getMessage
assert(message.contains(s"Invalid view text: $dropView." +
s" The view ${table.qualifiedName} may have been tampered with"))
}
}
}