-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-16496][SQL] Add wholetext as option for reading text in SQL. #14151
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
4 commits
Select commit
Hold shift + click to select a range
dd2ed3d
[SPARK-16496][SQL] Add wholetext as option for reading text in SQL.
ScrapCodes 7e91020
Try out escaping slash.
ScrapCodes 66d5b45
Added a WholeTextFileSuite, covering more cases from the correspondin…
ScrapCodes 021039b
fixed tests
ScrapCodes 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
57 changes: 57 additions & 0 deletions
57
...src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileWholeTextReader.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,57 @@ | ||
| /* | ||
| * 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 | ||
|
|
||
| import java.io.Closeable | ||
| import java.net.URI | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.io.Text | ||
| import org.apache.hadoop.mapreduce._ | ||
| import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit | ||
| import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl | ||
|
|
||
| import org.apache.spark.input.WholeTextFileRecordReader | ||
|
|
||
| /** | ||
| * An adaptor from a [[PartitionedFile]] to an [[Iterator]] of [[Text]], which is all of the lines | ||
| * in that file. | ||
| */ | ||
| class HadoopFileWholeTextReader(file: PartitionedFile, conf: Configuration) | ||
| extends Iterator[Text] with Closeable { | ||
| private val iterator = { | ||
| val fileSplit = new CombineFileSplit( | ||
| Array(new Path(new URI(file.filePath))), | ||
| Array(file.start), | ||
| Array(file.length), | ||
| // TODO: Implement Locality | ||
| Array.empty[String]) | ||
| val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) | ||
| val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) | ||
| val reader = new WholeTextFileRecordReader(fileSplit, hadoopAttemptContext, 0) | ||
| reader.initialize(fileSplit, hadoopAttemptContext) | ||
| new RecordReaderIterator(reader) | ||
| } | ||
|
|
||
| override def hasNext: Boolean = iterator.hasNext | ||
|
|
||
| override def next(): Text = iterator.next() | ||
|
|
||
| override def close(): Unit = iterator.close() | ||
| } | ||
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
108 changes: 108 additions & 0 deletions
108
...e/src/test/scala/org/apache/spark/sql/execution/datasources/text/WholeTextFileSuite.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,108 @@ | ||
| /* | ||
| * 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.File | ||
|
|
||
| import org.apache.spark.sql.{QueryTest, Row} | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.test.SharedSQLContext | ||
| import org.apache.spark.sql.types.{StringType, StructType} | ||
|
|
||
| class WholeTextFileSuite extends QueryTest with SharedSQLContext { | ||
|
|
||
| // Hadoop's FileSystem caching does not use the Configuration as part of its cache key, which | ||
| // can cause Filesystem.get(Configuration) to return a cached instance created with a different | ||
| // configuration than the one passed to get() (see HADOOP-8490 for more details). This caused | ||
| // hard-to-reproduce test failures, since any suites that were run after this one would inherit | ||
| // the new value of "fs.local.block.size" (see SPARK-5227 and SPARK-5679). To work around this, | ||
| // we disable FileSystem caching in this suite. | ||
| protected override def sparkConf = | ||
| super.sparkConf.set("spark.hadoop.fs.file.impl.disable.cache", "true") | ||
|
|
||
| private def testFile: String = { | ||
| Thread.currentThread().getContextClassLoader.getResource("test-data/text-suite.txt").toString | ||
| } | ||
|
|
||
| test("reading text file with option wholetext=true") { | ||
| val df = spark.read.option("wholetext", "true") | ||
| .format("text").load(testFile) | ||
| // schema | ||
| assert(df.schema == new StructType().add("value", StringType)) | ||
|
|
||
| // verify content | ||
| val data = df.collect() | ||
| assert(data(0) == | ||
| Row( | ||
| // scalastyle:off nonascii | ||
| """This is a test file for the text data source | ||
| |1+1 | ||
| |数据砖头 | ||
| |"doh" | ||
| |""".stripMargin)) | ||
| // scalastyle:on nonascii | ||
| assert(data.length == 1) | ||
| } | ||
|
|
||
| test("correctness of wholetext option") { | ||
| import org.apache.spark.sql.catalyst.util._ | ||
| withTempDir { dir => | ||
| val file1 = new File(dir, "text1.txt") | ||
| stringToFile(file1, | ||
| """text file 1 contents. | ||
| |From: None to: ?? | ||
| """.stripMargin) | ||
| val file2 = new File(dir, "text2.txt") | ||
| stringToFile(file2, "text file 2 contents.") | ||
| val file3 = new File(dir, "text3.txt") | ||
| stringToFile(file3, "text file 3 contents.") | ||
| val df = spark.read.option("wholetext", "true").text(dir.getAbsolutePath) | ||
| // Since wholetext option reads each file into a single row, df.length should be no. of files. | ||
| val data = df.sort("value").collect() | ||
| assert(data.length == 3) | ||
| // Each files should represent a single Row/element in Dataframe/Dataset | ||
| assert(data(0) == Row( | ||
| """text file 1 contents. | ||
| |From: None to: ?? | ||
| """.stripMargin)) | ||
| assert(data(1) == Row( | ||
| """text file 2 contents.""".stripMargin)) | ||
| assert(data(2) == Row( | ||
| """text file 3 contents.""".stripMargin)) | ||
| } | ||
| } | ||
|
|
||
|
|
||
| test("Correctness of wholetext option with gzip compression mode.") { | ||
| withTempDir { dir => | ||
| val path = dir.getCanonicalPath | ||
| val df1 = spark.range(0, 1000).selectExpr("CAST(id AS STRING) AS s").repartition(1) | ||
| df1.write.option("compression", "gzip").mode("overwrite").text(path) | ||
| // On reading through wholetext mode, one file will be read as a single row, i.e. not | ||
| // delimited by "next line" character. | ||
| val expected = Row(Range(0, 1000).mkString("", "\n", "\n")) | ||
| Seq(10, 100, 1000).foreach { bytes => | ||
| withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> bytes.toString) { | ||
| val df2 = spark.read.option("wholetext", "true").format("text").load(path) | ||
| val result = df2.collect().head | ||
| assert(result === expected) | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } |
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'd like to remind that a
PartitionedFilecan be just a part of a input file, instead of a whole file. So you cannot guarantee that in this case the reader reads all content of a file.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We may want to override
isSplitableofTextFileFormatand return false whenwholetextoption is enabled.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thank you, for catching this.