Skip to content

Commit c9a7ca1

Browse files
unescapePathName on the columnName parsed from partition path.
1 parent 55bea56 commit c9a7ca1

File tree

2 files changed

+13
-1
lines changed

2 files changed

+13
-1
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -243,7 +243,7 @@ object PartitioningUtils {
243243
if (equalSignIndex == -1) {
244244
None
245245
} else {
246-
val columnName = columnSpec.take(equalSignIndex)
246+
val columnName = unescapePathName(columnSpec.take(equalSignIndex))
247247
assert(columnName.nonEmpty, s"Empty partition column name in '$columnSpec'")
248248

249249
val rawColumnValue = columnSpec.drop(equalSignIndex + 1)

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem}
2727

2828
import org.apache.spark.metrics.source.HiveCatalogMetrics
2929
import org.apache.spark.sql.catalyst.util._
30+
import org.apache.spark.sql.functions.col
3031
import org.apache.spark.sql.internal.SQLConf
3132
import org.apache.spark.sql.test.SharedSQLContext
3233
import org.apache.spark.util.{KnownSizeEstimation, SizeEstimator}
@@ -236,6 +237,17 @@ class FileIndexSuite extends SharedSQLContext {
236237
val fileStatusCache = FileStatusCache.getOrCreate(spark)
237238
fileStatusCache.putLeafFiles(new Path("/tmp", "abc"), files.toArray)
238239
}
240+
241+
test("SPARK-20367 - properly unescape column names in inferPartitioning") {
242+
withTempPath { path =>
243+
val colToUnescape = "Column/#%'?"
244+
spark
245+
.range(1)
246+
.select(col("id").as(colToUnescape), col("id"))
247+
.write.partitionBy(colToUnescape).parquet(path.getAbsolutePath)
248+
assert(spark.read.parquet(path.getAbsolutePath).schema.exists(_.name == colToUnescape))
249+
}
250+
}
239251
}
240252

241253
class FakeParentPathFileSystem extends RawLocalFileSystem {

0 commit comments

Comments
 (0)