@@ -40,7 +40,7 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils {
40
40
private val tmpDataDir = Utils .createTempDir(namePrefix = " test-data" )
41
41
// For local test, you can set `sparkTestingDir` to a static value like `/tmp/test-spark`, to
42
42
// avoid downloading Spark of different versions in each run.
43
- private val sparkTestingDir = Utils .createTempDir(namePrefix = " test-spark" )
43
+ private val sparkTestingDir = new File ( " /tmp/ test-spark" )
44
44
private val unusedJar = TestUtils .createJarWithClasses(Seq .empty)
45
45
46
46
override def afterAll (): Unit = {
@@ -77,35 +77,38 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils {
77
77
super .beforeAll()
78
78
79
79
val tempPyFile = File .createTempFile(" test" , " .py" )
80
+ // scalastyle:off line.size.limit
80
81
Files .write(tempPyFile.toPath,
81
82
s """
82
83
|from pyspark.sql import SparkSession
84
+ |import os
83
85
|
84
86
|spark = SparkSession.builder.enableHiveSupport().getOrCreate()
85
87
|version_index = spark.conf.get("spark.sql.test.version.index", None)
86
88
|
87
89
|spark.sql("create table data_source_tbl_{} using json as select 1 i".format(version_index))
88
90
|
89
- |spark.sql("create table hive_compatible_data_source_tbl_" + version_index + \\
90
- | " using parquet as select 1 i")
91
+ |spark.sql("create table hive_compatible_data_source_tbl_{} using parquet as select 1 i".format(version_index))
91
92
|
92
93
|json_file = " ${genDataDir(" json_" )}" + str(version_index)
93
94
|spark.range(1, 2).selectExpr("cast(id as int) as i").write.json(json_file)
94
- |spark.sql("create table external_data_source_tbl_" + version_index + \\
95
- | "(i int) using json options (path '{}')".format(json_file))
95
+ |spark.sql("create table external_data_source_tbl_{}(i int) using json options (path '{}')".format(version_index, json_file))
96
96
|
97
97
|parquet_file = " ${genDataDir(" parquet_" )}" + str(version_index)
98
98
|spark.range(1, 2).selectExpr("cast(id as int) as i").write.parquet(parquet_file)
99
- |spark.sql("create table hive_compatible_external_data_source_tbl_" + version_index + \\
100
- | "(i int) using parquet options (path '{}')".format(parquet_file))
99
+ |spark.sql("create table hive_compatible_external_data_source_tbl_{}(i int) using parquet options (path '{}')".format(version_index, parquet_file))
101
100
|
102
101
|json_file2 = " ${genDataDir(" json2_" )}" + str(version_index)
103
102
|spark.range(1, 2).selectExpr("cast(id as int) as i").write.json(json_file2)
104
- |spark.sql("create table external_table_without_schema_" + version_index + \\
105
- | " using json options (path '{}')".format(json_file2))
103
+ |spark.sql("create table external_table_without_schema_{} using json options (path '{}')".format(version_index, json_file2))
104
+ |
105
+ |parquet_file2 = " ${genDataDir(" parquet2_" )}" + str(version_index)
106
+ |spark.range(1, 3).selectExpr("1 as i", "cast(id as int) as p", "1 as j").write.parquet(os.path.join(parquet_file2, "p=1"))
107
+ |spark.sql("create table tbl_with_col_overlap_{} using parquet options(path '{}')".format(version_index, parquet_file2))
106
108
|
107
109
|spark.sql("create view v_{} as select 1 i".format(version_index))
108
110
""" .stripMargin.getBytes(" utf8" ))
111
+ // scalastyle:on line.size.limit
109
112
110
113
PROCESS_TABLES .testingVersions.zipWithIndex.foreach { case (version, index) =>
111
114
val sparkHome = new File (sparkTestingDir, s " spark- $version" )
@@ -153,6 +156,7 @@ object PROCESS_TABLES extends QueryTest with SQLTestUtils {
153
156
.enableHiveSupport()
154
157
.getOrCreate()
155
158
spark = session
159
+ import session .implicits ._
156
160
157
161
testingVersions.indices.foreach { index =>
158
162
Seq (
@@ -194,6 +198,22 @@ object PROCESS_TABLES extends QueryTest with SQLTestUtils {
194
198
195
199
// test permanent view
196
200
checkAnswer(sql(s " select i from v_ $index" ), Row (1 ))
201
+
202
+ // SPARK-22356: overlapped columns between data and partition schema in data source tables
203
+ val tbl_with_col_overlap = s " tbl_with_col_overlap_ $index"
204
+ // For Spark 2.2.0 and 2.1.x, the behavior is different from Spark 2.0.
205
+ if (testingVersions(index).startsWith(" 2.1" ) || testingVersions(index) == " 2.2.0" ) {
206
+ spark.sql(" msck repair table " + tbl_with_col_overlap)
207
+ assert(spark.table(tbl_with_col_overlap).columns === Array (" i" , " j" , " p" ))
208
+ checkAnswer(spark.table(tbl_with_col_overlap), Row (1 , 1 , 1 ) :: Row (1 , 1 , 1 ) :: Nil )
209
+ assert(sql(" desc " + tbl_with_col_overlap).select(" col_name" )
210
+ .as[String ].collect().mkString(" ," ).contains(" i,j,p" ))
211
+ } else {
212
+ assert(spark.table(tbl_with_col_overlap).columns === Array (" i" , " p" , " j" ))
213
+ checkAnswer(spark.table(tbl_with_col_overlap), Row (1 , 1 , 1 ) :: Row (1 , 1 , 1 ) :: Nil )
214
+ assert(sql(" desc " + tbl_with_col_overlap).select(" col_name" )
215
+ .as[String ].collect().mkString(" ," ).contains(" i,p,j" ))
216
+ }
197
217
}
198
218
}
199
219
}
0 commit comments