@@ -23,7 +23,6 @@ import scala.collection.JavaConverters._
23
23
import scala .language .{existentials , implicitConversions }
24
24
import scala .util .{Failure , Success , Try }
25
25
26
- import org .apache .hadoop .conf .Configuration
27
26
import org .apache .hadoop .fs .Path
28
27
29
28
import org .apache .spark .deploy .SparkHadoopUtil
@@ -103,24 +102,6 @@ case class DataSource(
103
102
bucket.sortColumnNames, " in the sort definition" , equality)
104
103
}
105
104
106
- /**
107
- * In the read path, only managed tables by Hive provide the partition columns properly when
108
- * initializing this class. All other file based data sources will try to infer the partitioning,
109
- * and then cast the inferred types to user specified dataTypes if the partition columns exist
110
- * inside `userSpecifiedSchema`, otherwise we can hit data corruption bugs like SPARK-18510, or
111
- * inconsistent data types as reported in SPARK-21463.
112
- * @param fileIndex A FileIndex that will perform partition inference
113
- * @return The PartitionSchema resolved from inference and cast according to `userSpecifiedSchema`
114
- */
115
- private def combineInferredAndUserSpecifiedPartitionSchema (fileIndex : FileIndex ): StructType = {
116
- val resolved = fileIndex.partitionSchema.map { partitionField =>
117
- // SPARK-18510: try to get schema from userSpecifiedSchema, otherwise fallback to inferred
118
- userSpecifiedSchema.flatMap(_.find(f => equality(f.name, partitionField.name))).getOrElse(
119
- partitionField)
120
- }
121
- StructType (resolved)
122
- }
123
-
124
105
/**
125
106
* Get the schema of the given FileFormat, if provided by `userSpecifiedSchema`, or try to infer
126
107
* it. In the read path, only managed tables by Hive provide the partition columns properly when
@@ -140,31 +121,26 @@ case class DataSource(
140
121
* be any further inference in any triggers.
141
122
*
142
123
* @param format the file format object for this DataSource
143
- * @param fileStatusCache the shared cache for file statuses to speed up listing
124
+ * @param fileIndex optional [[ InMemoryFileIndex ]] for getting partition schema and file list
144
125
* @return A pair of the data schema (excluding partition columns) and the schema of the partition
145
126
* columns.
146
127
*/
147
128
private def getOrInferFileFormatSchema (
148
129
format : FileFormat ,
149
- fileStatusCache : FileStatusCache = NoopCache ): (StructType , StructType ) = {
150
- // the operations below are expensive therefore try not to do them if we don't need to, e.g.,
130
+ fileIndex : Option [ InMemoryFileIndex ] = None ): (StructType , StructType ) = {
131
+ // The operations below are expensive therefore try not to do them if we don't need to, e.g.,
151
132
// in streaming mode, we have already inferred and registered partition columns, we will
152
133
// never have to materialize the lazy val below
153
- lazy val tempFileIndex = {
154
- val allPaths = caseInsensitiveOptions.get(" path" ) ++ paths
155
- val hadoopConf = sparkSession.sessionState.newHadoopConf()
156
- val globbedPaths = allPaths.toSeq.flatMap { path =>
157
- val hdfsPath = new Path (path)
158
- val fs = hdfsPath.getFileSystem(hadoopConf)
159
- val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
160
- SparkHadoopUtil .get.globPathIfNecessary(fs, qualified)
161
- }.toArray
162
- new InMemoryFileIndex (sparkSession, globbedPaths, options, None , fileStatusCache)
134
+ lazy val tempFileIndex = fileIndex.getOrElse {
135
+ val globbedPaths =
136
+ checkAndGlobPathIfNecessary(checkEmptyGlobPath = false , checkFilesExist = false )
137
+ createInMemoryFileIndex(globbedPaths)
163
138
}
139
+
164
140
val partitionSchema = if (partitionColumns.isEmpty) {
165
141
// Try to infer partitioning, because no DataSource in the read path provides the partitioning
166
142
// columns properly unless it is a Hive DataSource
167
- combineInferredAndUserSpecifiedPartitionSchema( tempFileIndex)
143
+ tempFileIndex.partitionSchema
168
144
} else {
169
145
// maintain old behavior before SPARK-18510. If userSpecifiedSchema is empty used inferred
170
146
// partitioning
@@ -356,13 +332,7 @@ case class DataSource(
356
332
caseInsensitiveOptions.get(" path" ).toSeq ++ paths,
357
333
sparkSession.sessionState.newHadoopConf()) =>
358
334
val basePath = new Path ((caseInsensitiveOptions.get(" path" ).toSeq ++ paths).head)
359
- val tempFileCatalog = new MetadataLogFileIndex (sparkSession, basePath, None )
360
- val fileCatalog = if (userSpecifiedSchema.nonEmpty) {
361
- val partitionSchema = combineInferredAndUserSpecifiedPartitionSchema(tempFileCatalog)
362
- new MetadataLogFileIndex (sparkSession, basePath, Option (partitionSchema))
363
- } else {
364
- tempFileCatalog
365
- }
335
+ val fileCatalog = new MetadataLogFileIndex (sparkSession, basePath, userSpecifiedSchema)
366
336
val dataSchema = userSpecifiedSchema.orElse {
367
337
format.inferSchema(
368
338
sparkSession,
@@ -384,24 +354,23 @@ case class DataSource(
384
354
385
355
// This is a non-streaming file based datasource.
386
356
case (format : FileFormat , _) =>
387
- val allPaths = caseInsensitiveOptions.get(" path" ) ++ paths
388
- val hadoopConf = sparkSession.sessionState.newHadoopConf()
389
- val globbedPaths = allPaths.flatMap(
390
- DataSource .checkAndGlobPathIfNecessary(hadoopConf, _, checkFilesExist)).toArray
391
-
392
- val fileStatusCache = FileStatusCache .getOrCreate(sparkSession)
393
- val (dataSchema, partitionSchema) = getOrInferFileFormatSchema(format, fileStatusCache)
394
-
395
- val fileCatalog = if (sparkSession.sqlContext.conf.manageFilesourcePartitions &&
396
- catalogTable.isDefined && catalogTable.get.tracksPartitionsInCatalog) {
357
+ val globbedPaths =
358
+ checkAndGlobPathIfNecessary(checkEmptyGlobPath = true , checkFilesExist = checkFilesExist)
359
+ val useCatalogFileIndex = sparkSession.sqlContext.conf.manageFilesourcePartitions &&
360
+ catalogTable.isDefined && catalogTable.get.tracksPartitionsInCatalog &&
361
+ catalogTable.get.partitionColumnNames.nonEmpty
362
+ val (fileCatalog, dataSchema, partitionSchema) = if (useCatalogFileIndex) {
397
363
val defaultTableSize = sparkSession.sessionState.conf.defaultSizeInBytes
398
- new CatalogFileIndex (
364
+ val index = new CatalogFileIndex (
399
365
sparkSession,
400
366
catalogTable.get,
401
367
catalogTable.get.stats.map(_.sizeInBytes.toLong).getOrElse(defaultTableSize))
368
+ (index, catalogTable.get.dataSchema, catalogTable.get.partitionSchema)
402
369
} else {
403
- new InMemoryFileIndex (
404
- sparkSession, globbedPaths, options, Some (partitionSchema), fileStatusCache)
370
+ val index = createInMemoryFileIndex(globbedPaths)
371
+ val (resultDataSchema, resultPartitionSchema) =
372
+ getOrInferFileFormatSchema(format, Some (index))
373
+ (index, resultDataSchema, resultPartitionSchema)
405
374
}
406
375
407
376
HadoopFsRelation (
@@ -552,6 +521,40 @@ case class DataSource(
552
521
sys.error(s " ${providingClass.getCanonicalName} does not allow create table as select. " )
553
522
}
554
523
}
524
+
525
+ /** Returns an [[InMemoryFileIndex ]] that can be used to get partition schema and file list. */
526
+ private def createInMemoryFileIndex (globbedPaths : Seq [Path ]): InMemoryFileIndex = {
527
+ val fileStatusCache = FileStatusCache .getOrCreate(sparkSession)
528
+ new InMemoryFileIndex (
529
+ sparkSession, globbedPaths, options, userSpecifiedSchema, fileStatusCache)
530
+ }
531
+
532
+ /**
533
+ * Checks and returns files in all the paths.
534
+ */
535
+ private def checkAndGlobPathIfNecessary (
536
+ checkEmptyGlobPath : Boolean ,
537
+ checkFilesExist : Boolean ): Seq [Path ] = {
538
+ val allPaths = caseInsensitiveOptions.get(" path" ) ++ paths
539
+ val hadoopConf = sparkSession.sessionState.newHadoopConf()
540
+ allPaths.flatMap { path =>
541
+ val hdfsPath = new Path (path)
542
+ val fs = hdfsPath.getFileSystem(hadoopConf)
543
+ val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
544
+ val globPath = SparkHadoopUtil .get.globPathIfNecessary(fs, qualified)
545
+
546
+ if (checkEmptyGlobPath && globPath.isEmpty) {
547
+ throw new AnalysisException (s " Path does not exist: $qualified" )
548
+ }
549
+
550
+ // Sufficient to check head of the globPath seq for non-glob scenario
551
+ // Don't need to check once again if files exist in streaming mode
552
+ if (checkFilesExist && ! fs.exists(globPath.head)) {
553
+ throw new AnalysisException (s " Path does not exist: ${globPath.head}" )
554
+ }
555
+ globPath
556
+ }.toSeq
557
+ }
555
558
}
556
559
557
560
object DataSource extends Logging {
@@ -699,30 +702,6 @@ object DataSource extends Logging {
699
702
locationUri = path.map(CatalogUtils .stringToURI), properties = optionsWithoutPath)
700
703
}
701
704
702
- /**
703
- * If `path` is a file pattern, return all the files that match it. Otherwise, return itself.
704
- * If `checkFilesExist` is `true`, also check the file existence.
705
- */
706
- private def checkAndGlobPathIfNecessary (
707
- hadoopConf : Configuration ,
708
- path : String ,
709
- checkFilesExist : Boolean ): Seq [Path ] = {
710
- val hdfsPath = new Path (path)
711
- val fs = hdfsPath.getFileSystem(hadoopConf)
712
- val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
713
- val globPath = SparkHadoopUtil .get.globPathIfNecessary(fs, qualified)
714
-
715
- if (globPath.isEmpty) {
716
- throw new AnalysisException (s " Path does not exist: $qualified" )
717
- }
718
- // Sufficient to check head of the globPath seq for non-glob scenario
719
- // Don't need to check once again if files exist in streaming mode
720
- if (checkFilesExist && ! fs.exists(globPath.head)) {
721
- throw new AnalysisException (s " Path does not exist: ${globPath.head}" )
722
- }
723
- globPath
724
- }
725
-
726
705
/**
727
706
* Called before writing into a FileFormat based data source to make sure the
728
707
* supplied schema is not empty.
0 commit comments