aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala5
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala24
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala9
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala17
4 files changed, 42 insertions, 13 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 9e7c1ec211..192083e2ea 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -915,4 +915,9 @@ object StaticSQLConf {
.internal()
.intConf
.createWithDefault(4000)
+
+ val DEBUG_MODE = buildConf("spark.sql.debug")
+ .internal()
+ .booleanConf
+ .createWithDefault(false)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
index f545de0e10..df640ffab9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.internal
import org.apache.hadoop.fs.Path
+import org.apache.spark.SparkContext
import org.apache.spark.sql._
import org.apache.spark.sql.execution.WholeStageCodegenExec
import org.apache.spark.sql.internal.StaticSQLConf._
@@ -254,18 +255,21 @@ class SQLConfSuite extends QueryTest with SharedSQLContext {
}
}
- test("global SQL conf comes from SparkConf") {
- val newSession = SparkSession.builder()
- .config(SCHEMA_STRING_LENGTH_THRESHOLD.key, "2000")
- .getOrCreate()
-
- assert(newSession.conf.get(SCHEMA_STRING_LENGTH_THRESHOLD.key) == "2000")
- checkAnswer(
- newSession.sql(s"SET ${SCHEMA_STRING_LENGTH_THRESHOLD.key}"),
- Row(SCHEMA_STRING_LENGTH_THRESHOLD.key, "2000"))
+ test("static SQL conf comes from SparkConf") {
+ val previousValue = sparkContext.conf.get(SCHEMA_STRING_LENGTH_THRESHOLD)
+ try {
+ sparkContext.conf.set(SCHEMA_STRING_LENGTH_THRESHOLD, 2000)
+ val newSession = new SparkSession(sparkContext)
+ assert(newSession.conf.get(SCHEMA_STRING_LENGTH_THRESHOLD) == 2000)
+ checkAnswer(
+ newSession.sql(s"SET ${SCHEMA_STRING_LENGTH_THRESHOLD.key}"),
+ Row(SCHEMA_STRING_LENGTH_THRESHOLD.key, "2000"))
+ } finally {
+ sparkContext.conf.set(SCHEMA_STRING_LENGTH_THRESHOLD, previousValue)
+ }
}
- test("cannot set/unset global SQL conf") {
+ test("cannot set/unset static SQL conf") {
val e1 = intercept[AnalysisException](sql(s"SET ${SCHEMA_STRING_LENGTH_THRESHOLD.key}=10"))
assert(e1.message.contains("Cannot modify the value of a static config"))
val e2 = intercept[AnalysisException](spark.conf.unset(SCHEMA_STRING_LENGTH_THRESHOLD.key))
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
index e1c0cad907..ed189724a2 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
@@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.command.{ColumnStatStruct, DDLUtils}
import org.apache.spark.sql.execution.datasources.CaseInsensitiveMap
import org.apache.spark.sql.hive.client.HiveClient
import org.apache.spark.sql.internal.HiveSerDe
-import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
+import org.apache.spark.sql.internal.StaticSQLConf._
import org.apache.spark.sql.types.{DataType, StructType}
@@ -461,13 +461,18 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
} else {
table.storage
}
+ val tableProps = if (conf.get(DEBUG_MODE)) {
+ table.properties
+ } else {
+ getOriginalTableProperties(table)
+ }
table.copy(
storage = storage,
schema = getSchemaFromTableProperties(table),
provider = Some(provider),
partitionColumnNames = getPartitionColumnsFromTableProperties(table),
bucketSpec = getBucketSpecFromTableProperties(table),
- properties = getOriginalTableProperties(table))
+ properties = tableProps)
} getOrElse {
table.copy(provider = Some("hive"))
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index 0477122fc6..7cc6179d44 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer
import org.apache.hadoop.fs.Path
+import org.apache.spark.SparkContext
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType}
@@ -31,7 +32,7 @@ import org.apache.spark.sql.hive.HiveExternalCatalog._
import org.apache.spark.sql.hive.client.HiveClient
import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
+import org.apache.spark.sql.internal.StaticSQLConf._
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
@@ -1324,4 +1325,18 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
hiveClient.dropTable("default", "t", ignoreIfNotExists = true, purge = true)
}
}
+
+ test("should keep data source entries in table properties when debug mode is on") {
+ val previousValue = sparkSession.sparkContext.conf.get(DEBUG_MODE)
+ try {
+ sparkSession.sparkContext.conf.set(DEBUG_MODE, true)
+ val newSession = sparkSession.newSession()
+ newSession.sql("CREATE TABLE abc(i int) USING json")
+ val tableMeta = newSession.sessionState.catalog.getTableMetadata(TableIdentifier("abc"))
+ assert(tableMeta.properties(DATASOURCE_SCHEMA_NUMPARTS).toInt == 1)
+ assert(tableMeta.properties(DATASOURCE_PROVIDER) == "json")
+ } finally {
+ sparkSession.sparkContext.conf.set(DEBUG_MODE, previousValue)
+ }
+ }
}