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 @@ -4142,7 +4142,7 @@ object SQLConf {
"flow and error handling.")
.version("4.0.0")
.booleanConf
.createWithDefault(false)
.createWithDefault(true)

val CONCAT_BINARY_AS_STRING = buildConf("spark.sql.function.concatBinaryAsString")
.doc("When this option is set to false and all inputs are binary, `functions.concat` returns " +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,22 +23,10 @@ import org.apache.spark.sql.catalyst.plans.SQLHelper
import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, CreateVariable, ExceptionHandler, ForStatement, IfElseStatement, IterateStatement, LeaveStatement, LoopStatement, Project, RepeatStatement, SearchedCaseStatement, SetVariable, SimpleCaseStatement, SingleStatement, WhileStatement}
import org.apache.spark.sql.errors.DataTypeErrors.toSQLId
import org.apache.spark.sql.exceptions.SqlScriptingException
import org.apache.spark.sql.internal.SQLConf

class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper {
import CatalystSqlParser._

// Tests setup
protected override def beforeAll(): Unit = {
super.beforeAll()
conf.setConfString(SQLConf.SQL_SCRIPTING_ENABLED.key, "true")
}

protected override def afterAll(): Unit = {
conf.unsetConf(SQLConf.SQL_SCRIPTING_ENABLED.key)
super.afterAll()
}

// Tests
test("single select") {
val sqlScriptText = "SELECT 1;"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@
package org.apache.spark.sql.execution

import org.apache.spark.sql.{AnalysisException, QueryTest}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSparkSession

class ExecuteImmediateEndToEndSuite extends QueryTest with SharedSparkSession {
Expand All @@ -39,14 +38,12 @@ class ExecuteImmediateEndToEndSuite extends QueryTest with SharedSparkSession {
}

test("SQL Scripting not supported inside EXECUTE IMMEDIATE") {
withSQLConf(SQLConf.SQL_SCRIPTING_ENABLED.key -> "true") {
val executeImmediateText = "EXECUTE IMMEDIATE 'BEGIN SELECT 1; END'"
checkError(
exception = intercept[AnalysisException ] {
spark.sql(executeImmediateText)
},
condition = "SQL_SCRIPT_IN_EXECUTE_IMMEDIATE",
parameters = Map("sqlString" -> "BEGIN SELECT 1; END"))
}
val executeImmediateText = "EXECUTE IMMEDIATE 'BEGIN SELECT 1; END'"
checkError(
exception = intercept[AnalysisException ] {
spark.sql(executeImmediateText)
},
condition = "SQL_SCRIPT_IN_EXECUTE_IMMEDIATE",
parameters = Map("sqlString" -> "BEGIN SELECT 1; END"))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,6 @@ class SqlScriptingE2eSuite extends QueryTest with SharedSparkSession {
override protected def sparkConf: SparkConf = {
super.sparkConf
.set(SQLConf.ANSI_ENABLED.key, "true")
.set(SQLConf.SQL_SCRIPTING_ENABLED.key, "true")
}

// Tests
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,6 @@ class SqlScriptingExecutionSuite extends QueryTest with SharedSparkSession {
override protected def sparkConf: SparkConf = {
super.sparkConf
.set(SQLConf.ANSI_ENABLED.key, "true")
.set(SQLConf.SQL_SCRIPTING_ENABLED.key, "true")
}

// Helpers
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.sql.scripting

import org.apache.spark.{SparkConf, SparkException, SparkNumberFormatException}
import org.apache.spark.{SparkException, SparkNumberFormatException}
import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
import org.apache.spark.sql.catalyst.QueryPlanningTracker
import org.apache.spark.sql.catalyst.expressions.Expression
Expand All @@ -38,11 +38,6 @@ class SqlScriptingInterpreterSuite
with SharedSparkSession
with SqlScriptingTestUtils {

// Tests setup
override protected def sparkConf: SparkConf = {
super.sparkConf.set(SQLConf.SQL_SCRIPTING_ENABLED.key, "true")
}

// Helpers
private def runSqlScript(
sqlText: String,
Expand Down