-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-27271][SQL] Migrate Text to File Data Source V2 #24207
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
54 changes: 54 additions & 0 deletions
54
...ore/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOutputWriter.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,54 @@ | ||
| /* | ||
| * 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.text | ||
|
|
||
| import java.io.OutputStream | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.mapreduce.TaskAttemptContext | ||
|
|
||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.execution.datasources.{CodecStreams, OutputWriter} | ||
| import org.apache.spark.sql.types.StructType | ||
|
|
||
| class TextOutputWriter( | ||
| path: String, | ||
| dataSchema: StructType, | ||
| lineSeparator: Array[Byte], | ||
| context: TaskAttemptContext) | ||
| extends OutputWriter { | ||
|
|
||
| private var outputStream: Option[OutputStream] = None | ||
|
|
||
| override def write(row: InternalRow): Unit = { | ||
| val os = outputStream.getOrElse { | ||
| val newStream = CodecStreams.createOutputStream(context, new Path(path)) | ||
| outputStream = Some(newStream) | ||
| newStream | ||
| } | ||
|
|
||
| if (!row.isNullAt(0)) { | ||
| val utf8string = row.getUTF8String(0) | ||
| utf8string.writeTo(os) | ||
| } | ||
| os.write(lineSeparator) | ||
| } | ||
|
|
||
| override def close(): Unit = { | ||
| outputStream.foreach(_.close()) | ||
| } | ||
| } | ||
44 changes: 44 additions & 0 deletions
44
.../src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,44 @@ | ||
| /* | ||
| * 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.text | ||
|
|
||
| import org.apache.spark.sql.execution.datasources.FileFormat | ||
| import org.apache.spark.sql.execution.datasources.text.TextFileFormat | ||
| import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 | ||
| import org.apache.spark.sql.sources.v2.Table | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
|
||
| class TextDataSourceV2 extends FileDataSourceV2 { | ||
|
|
||
| override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[TextFileFormat] | ||
|
|
||
| override def shortName(): String = "text" | ||
|
|
||
| override def getTable(options: CaseInsensitiveStringMap): Table = { | ||
| val paths = getPaths(options) | ||
| val tableName = getTableName(paths) | ||
| TextTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) | ||
| } | ||
|
|
||
| override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { | ||
| val paths = getPaths(options) | ||
| val tableName = getTableName(paths) | ||
| TextTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) | ||
| } | ||
| } | ||
|
|
73 changes: 73 additions & 0 deletions
73
...scala/org/apache/spark/sql/execution/datasources/v2/text/TextPartitionReaderFactory.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,73 @@ | ||
| /* | ||
| * 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.text | ||
|
|
||
| import org.apache.spark.TaskContext | ||
| import org.apache.spark.broadcast.Broadcast | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions.UnsafeRow | ||
| import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter | ||
| import org.apache.spark.sql.execution.datasources.{HadoopFileLinesReader, HadoopFileWholeTextReader, PartitionedFile} | ||
| import org.apache.spark.sql.execution.datasources.text.TextOptions | ||
| import org.apache.spark.sql.execution.datasources.v2._ | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.sources.v2.reader.PartitionReader | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.util.SerializableConfiguration | ||
|
|
||
| /** | ||
| * A factory used to create Text readers. | ||
| * | ||
| * @param sqlConf SQL configuration. | ||
| * @param broadcastedConf Broadcasted serializable Hadoop Configuration. | ||
| * @param readDataSchema Required schema in the batch scan. | ||
| * @param partitionSchema Schema of partitions. | ||
| * @param textOptions Options for reading a text file. | ||
| * */ | ||
| case class TextPartitionReaderFactory( | ||
| sqlConf: SQLConf, | ||
| broadcastedConf: Broadcast[SerializableConfiguration], | ||
| readDataSchema: StructType, | ||
| partitionSchema: StructType, | ||
| textOptions: TextOptions) extends FilePartitionReaderFactory { | ||
|
|
||
| override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { | ||
| val confValue = broadcastedConf.value.value | ||
| val reader = if (!textOptions.wholeText) { | ||
| new HadoopFileLinesReader(file, textOptions.lineSeparatorInRead, confValue) | ||
| } else { | ||
| new HadoopFileWholeTextReader(file, confValue) | ||
| } | ||
| Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => reader.close())) | ||
| val iter = if (readDataSchema.isEmpty) { | ||
| val emptyUnsafeRow = new UnsafeRow(0) | ||
| reader.map(_ => emptyUnsafeRow) | ||
| } else { | ||
| val unsafeRowWriter = new UnsafeRowWriter(1) | ||
|
|
||
| reader.map { line => | ||
| // Writes to an UnsafeRow directly | ||
| unsafeRowWriter.reset() | ||
| unsafeRowWriter.write(0, line.getBytes, 0, line.getLength) | ||
| unsafeRowWriter.getRow() | ||
| } | ||
| } | ||
| val fileReader = new PartitionReaderFromIterator[InternalRow](iter) | ||
| new PartitionReaderWithPartitionValues(fileReader, readDataSchema, | ||
| partitionSchema, file.partitionValues) | ||
| } | ||
| } |
61 changes: 61 additions & 0 deletions
61
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,61 @@ | ||
| /* | ||
| * 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.text | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
|
|
||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex | ||
| import org.apache.spark.sql.execution.datasources.text.TextOptions | ||
| import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan | ||
| import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
| import org.apache.spark.util.SerializableConfiguration | ||
|
|
||
| case class TextScan( | ||
| sparkSession: SparkSession, | ||
| fileIndex: PartitioningAwareFileIndex, | ||
| readDataSchema: StructType, | ||
| readPartitionSchema: StructType, | ||
| options: CaseInsensitiveStringMap) | ||
| extends TextBasedFileScan(sparkSession, fileIndex, readDataSchema, readPartitionSchema, options) { | ||
|
|
||
| private val optionsAsScala = options.asScala.toMap | ||
| private lazy val textOptions: TextOptions = new TextOptions(optionsAsScala) | ||
|
|
||
| override def isSplitable(path: Path): Boolean = { | ||
| super.isSplitable(path) && !textOptions.wholeText | ||
| } | ||
|
|
||
| override def createReaderFactory(): PartitionReaderFactory = { | ||
| assert( | ||
| readDataSchema.length <= 1, | ||
| "Text data source only produces a single data column named \"value\".") | ||
| val hadoopConf = { | ||
| val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap | ||
| // Hadoop Configurations are case sensitive. | ||
| sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) | ||
| } | ||
| val broadcastedConf = sparkSession.sparkContext.broadcast( | ||
| new SerializableConfiguration(hadoopConf)) | ||
| TextPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, readDataSchema, | ||
| readPartitionSchema, textOptions) | ||
| } | ||
| } |
38 changes: 38 additions & 0 deletions
38
...e/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScanBuilder.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,38 @@ | ||
| /* | ||
| * 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.text | ||
|
|
||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex | ||
| import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder | ||
| import org.apache.spark.sql.sources.v2.reader.Scan | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
|
||
| case class TextScanBuilder( | ||
| sparkSession: SparkSession, | ||
| fileIndex: PartitioningAwareFileIndex, | ||
| schema: StructType, | ||
| dataSchema: StructType, | ||
| options: CaseInsensitiveStringMap) | ||
| extends FileScanBuilder(sparkSession, fileIndex, dataSchema) { | ||
|
|
||
| override def build(): Scan = { | ||
| TextScan(sparkSession, fileIndex, readDataSchema(), readPartitionSchema(), options) | ||
| } | ||
| } |
48 changes: 48 additions & 0 deletions
48
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,48 @@ | ||
| /* | ||
| * 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.text | ||
|
|
||
| import org.apache.hadoop.fs.FileStatus | ||
|
|
||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.execution.datasources.FileFormat | ||
| import org.apache.spark.sql.execution.datasources.v2.FileTable | ||
| import org.apache.spark.sql.sources.v2.writer.WriteBuilder | ||
| import org.apache.spark.sql.types.{DataType, StringType, StructField, StructType} | ||
| import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
|
||
| case class TextTable( | ||
| name: String, | ||
| sparkSession: SparkSession, | ||
| options: CaseInsensitiveStringMap, | ||
| paths: Seq[String], | ||
| userSpecifiedSchema: Option[StructType], | ||
| fallbackFileFormat: Class[_ <: FileFormat]) | ||
| extends FileTable(sparkSession, options, paths, userSpecifiedSchema) { | ||
| override def newScanBuilder(options: CaseInsensitiveStringMap): TextScanBuilder = | ||
| TextScanBuilder(sparkSession, fileIndex, schema, dataSchema, options) | ||
|
|
||
| override def inferSchema(files: Seq[FileStatus]): Option[StructType] = | ||
| Some(StructType(Seq(StructField("value", StringType)))) | ||
|
|
||
| override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = | ||
| new TextWriteBuilder(options, paths, formatName, supportsDataType) | ||
|
|
||
| override def supportsDataType(dataType: DataType): Boolean = dataType == StringType | ||
|
|
||
| override def formatName: String = "Text" | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.