-
Notifications
You must be signed in to change notification settings - Fork 28k
/
FileScan.scala
202 lines (172 loc) · 7.3 KB
/
FileScan.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.execution.datasources.v2
import java.util.{Locale, OptionalLong}
import org.apache.commons.lang3.StringUtils
import org.apache.hadoop.fs.Path
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.IO_WARNING_LARGEFILETHRESHOLD
import org.apache.spark.sql.{AnalysisException, SparkSession}
import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionSet}
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.connector.read.{Batch, InputPartition, Scan, Statistics, SupportsReportStatistics}
import org.apache.spark.sql.execution.PartitionedFileUtil
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.internal.connector.SupportsMetadata
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.Utils
trait FileScan extends Scan
with Batch with SupportsReportStatistics with SupportsMetadata with Logging {
/**
* Returns whether a file with `path` could be split or not.
*/
def isSplitable(path: Path): Boolean = {
false
}
def sparkSession: SparkSession
def fileIndex: PartitioningAwareFileIndex
/**
* Returns the required data schema
*/
def readDataSchema: StructType
/**
* Returns the required partition schema
*/
def readPartitionSchema: StructType
/**
* Returns the filters that can be use for partition pruning
*/
def partitionFilters: Seq[Expression]
/**
* Returns the data filters that can be use for file listing
*/
def dataFilters: Seq[Expression]
/**
* Create a new `FileScan` instance from the current one
* with different `partitionFilters` and `dataFilters`
*/
def withFilters(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan
/**
* If a file with `path` is unsplittable, return the unsplittable reason,
* otherwise return `None`.
*/
def getFileUnSplittableReason(path: Path): String = {
assert(!isSplitable(path))
"undefined"
}
protected def seqToString(seq: Seq[Any]): String = seq.mkString("[", ", ", "]")
override def equals(obj: Any): Boolean = obj match {
case f: FileScan =>
fileIndex == f.fileIndex && readSchema == f.readSchema
ExpressionSet(partitionFilters) == ExpressionSet(f.partitionFilters) &&
ExpressionSet(dataFilters) == ExpressionSet(f.dataFilters)
case _ => false
}
override def hashCode(): Int = getClass.hashCode()
val maxMetadataValueLength = sparkSession.sessionState.conf.maxMetadataStringLength
override def description(): String = {
val metadataStr = getMetaData().toSeq.sorted.map {
case (key, value) =>
val redactedValue =
Utils.redact(sparkSession.sessionState.conf.stringRedactionPattern, value)
key + ": " + StringUtils.abbreviate(redactedValue, maxMetadataValueLength)
}.mkString(", ")
s"${this.getClass.getSimpleName} $metadataStr"
}
override def getMetaData(): Map[String, String] = {
val locationDesc =
fileIndex.getClass.getSimpleName +
Utils.buildLocationMetadata(fileIndex.rootPaths, maxMetadataValueLength)
Map(
"Format" -> s"${this.getClass.getSimpleName.replace("Scan", "").toLowerCase(Locale.ROOT)}",
"ReadSchema" -> readDataSchema.catalogString,
"PartitionFilters" -> seqToString(partitionFilters),
"DataFilters" -> seqToString(dataFilters),
"Location" -> locationDesc)
}
protected def partitions: Seq[FilePartition] = {
val selectedPartitions = fileIndex.listFiles(partitionFilters, dataFilters)
val maxSplitBytes = FilePartition.maxSplitBytes(sparkSession, selectedPartitions)
val partitionAttributes = fileIndex.partitionSchema.toAttributes
val attributeMap = partitionAttributes.map(a => normalizeName(a.name) -> a).toMap
val readPartitionAttributes = readPartitionSchema.map { readField =>
attributeMap.get(normalizeName(readField.name)).getOrElse {
throw new AnalysisException(s"Can't find required partition column ${readField.name} " +
s"in partition schema ${fileIndex.partitionSchema}")
}
}
lazy val partitionValueProject =
GenerateUnsafeProjection.generate(readPartitionAttributes, partitionAttributes)
val splitFiles = selectedPartitions.flatMap { partition =>
// Prune partition values if part of the partition columns are not required.
val partitionValues = if (readPartitionAttributes != partitionAttributes) {
partitionValueProject(partition.values).copy()
} else {
partition.values
}
partition.files.flatMap { file =>
val filePath = file.getPath
PartitionedFileUtil.splitFiles(
sparkSession = sparkSession,
file = file,
filePath = filePath,
isSplitable = isSplitable(filePath),
maxSplitBytes = maxSplitBytes,
partitionValues = partitionValues
)
}.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse)
}
if (splitFiles.length == 1) {
val path = new Path(splitFiles(0).filePath)
if (!isSplitable(path) && splitFiles(0).length >
sparkSession.sparkContext.getConf.get(IO_WARNING_LARGEFILETHRESHOLD)) {
logWarning(s"Loading one large unsplittable file ${path.toString} with only one " +
s"partition, the reason is: ${getFileUnSplittableReason(path)}")
}
}
FilePartition.getFilePartitions(sparkSession, splitFiles, maxSplitBytes)
}
override def planInputPartitions(): Array[InputPartition] = {
partitions.toArray
}
override def estimateStatistics(): Statistics = {
new Statistics {
override def sizeInBytes(): OptionalLong = {
val compressionFactor = sparkSession.sessionState.conf.fileCompressionFactor
val size = (compressionFactor * fileIndex.sizeInBytes).toLong
OptionalLong.of(size)
}
override def numRows(): OptionalLong = OptionalLong.empty()
}
}
override def toBatch: Batch = this
override def readSchema(): StructType =
StructType(readDataSchema.fields ++ readPartitionSchema.fields)
// Returns whether the two given arrays of [[Filter]]s are equivalent.
protected def equivalentFilters(a: Array[Filter], b: Array[Filter]): Boolean = {
a.sortBy(_.hashCode()).sameElements(b.sortBy(_.hashCode()))
}
private val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis
private def normalizeName(name: String): String = {
if (isCaseSensitive) {
name
} else {
name.toLowerCase(Locale.ROOT)
}
}
}