@@ -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
@@ -95,14 +94,6 @@ case class DataSource(
95
94
lazy val sourceInfo : SourceInfo = sourceSchema()
96
95
private val caseInsensitiveOptions = CaseInsensitiveMap (options)
97
96
private val equality = sparkSession.sessionState.conf.resolver
98
- // The operations below are expensive therefore try not to do them if we don't need to, e.g.,
99
- // in streaming mode, we have already inferred and registered partition columns, we will
100
- // never have to materialize the lazy val below
101
- private lazy val tempFileIndex = {
102
- val globbedPaths =
103
- checkAndGlobPathIfNecessary(checkEmptyGlobPath = false , checkFilesExist = false )
104
- createInMemoryFileIndex(globbedPaths)
105
- }
106
97
107
98
bucketSpec.map { bucket =>
108
99
SchemaUtils .checkColumnNameDuplication(
@@ -137,22 +128,29 @@ case class DataSource(
137
128
private def getOrInferFileFormatSchema (
138
129
format : FileFormat ,
139
130
fileIndex : Option [InMemoryFileIndex ] = None ): (StructType , StructType ) = {
140
- def inMemoryFileIndex = fileIndex.getOrElse(tempFileIndex)
131
+ // The operations below are expensive therefore try not to do them if we don't need to, e.g.,
132
+ // in streaming mode, we have already inferred and registered partition columns, we will
133
+ // never have to materialize the lazy val below
134
+ lazy val tempFileIndex = fileIndex.getOrElse {
135
+ val globbedPaths =
136
+ checkAndGlobPathIfNecessary(checkEmptyGlobPath = false , checkFilesExist = false )
137
+ createInMemoryFileIndex(globbedPaths)
138
+ }
141
139
142
140
val partitionSchema = if (partitionColumns.isEmpty) {
143
141
// Try to infer partitioning, because no DataSource in the read path provides the partitioning
144
142
// columns properly unless it is a Hive DataSource
145
- inMemoryFileIndex .partitionSchema
143
+ tempFileIndex .partitionSchema
146
144
} else {
147
145
// maintain old behavior before SPARK-18510. If userSpecifiedSchema is empty used inferred
148
146
// partitioning
149
147
if (userSpecifiedSchema.isEmpty) {
150
- val inferredPartitions = inMemoryFileIndex .partitionSchema
148
+ val inferredPartitions = tempFileIndex .partitionSchema
151
149
inferredPartitions
152
150
} else {
153
151
val partitionFields = partitionColumns.map { partitionColumn =>
154
152
userSpecifiedSchema.flatMap(_.find(c => equality(c.name, partitionColumn))).orElse {
155
- val inferredPartitions = inMemoryFileIndex .partitionSchema
153
+ val inferredPartitions = tempFileIndex .partitionSchema
156
154
val inferredOpt = inferredPartitions.find(p => equality(p.name, partitionColumn))
157
155
if (inferredOpt.isDefined) {
158
156
logDebug(
@@ -181,7 +179,7 @@ case class DataSource(
181
179
format.inferSchema(
182
180
sparkSession,
183
181
caseInsensitiveOptions,
184
- inMemoryFileIndex .allFiles())
182
+ tempFileIndex .allFiles())
185
183
}.getOrElse {
186
184
throw new AnalysisException (
187
185
s " Unable to infer schema for $format. It must be specified manually. " )
@@ -360,7 +358,7 @@ case class DataSource(
360
358
checkAndGlobPathIfNecessary(checkEmptyGlobPath = true , checkFilesExist = checkFilesExist)
361
359
val useCatalogFileIndex = sparkSession.sqlContext.conf.manageFilesourcePartitions &&
362
360
catalogTable.isDefined && catalogTable.get.tracksPartitionsInCatalog &&
363
- catalogTable.get.partitionSchema .nonEmpty
361
+ catalogTable.get.partitionColumnNames .nonEmpty
364
362
val (fileCatalog, dataSchema, partitionSchema) = if (useCatalogFileIndex) {
365
363
val defaultTableSize = sparkSession.sessionState.conf.defaultSizeInBytes
366
364
val index = new CatalogFileIndex (
0 commit comments