-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-23577][SQL] Supports custom line separator for text datasource #20727
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
Changes from all commits
a8eeef8
84d2cef
b5c8246
6f1d317
4835632
d6e9160
f1c951f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
|
|
@@ -17,6 +17,8 @@ | |||
|
|
||||
| package org.apache.spark.sql.execution.datasources.text | ||||
|
|
||||
| import java.nio.charset.StandardCharsets | ||||
|
|
||||
| import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CompressionCodecs} | ||||
|
|
||||
| /** | ||||
|
|
@@ -39,9 +41,19 @@ private[text] class TextOptions(@transient private val parameters: CaseInsensiti | |||
| */ | ||||
| val wholeText = parameters.getOrElse(WHOLETEXT, "false").toBoolean | ||||
|
|
||||
| private val lineSeparator: Option[String] = parameters.get(LINE_SEPARATOR).map { sep => | ||||
| require(sep.nonEmpty, s"'$LINE_SEPARATOR' cannot be an empty string.") | ||||
| sep | ||||
| } | ||||
| // Note that the option 'lineSep' uses a different default value in read and write. | ||||
| val lineSeparatorInRead: Option[Array[Byte]] = | ||||
| lineSeparator.map(_.getBytes(StandardCharsets.UTF_8)) | ||||
| val lineSeparatorInWrite: Array[Byte] = | ||||
| lineSeparatorInRead.getOrElse("\n".getBytes(StandardCharsets.UTF_8)) | ||||
| } | ||||
|
|
||||
| private[text] object TextOptions { | ||||
| val COMPRESSION = "compression" | ||||
| val WHOLETEXT = "wholetext" | ||||
| val LINE_SEPARATOR = "lineSep" | ||||
|
||||
| |LINES TERMINATED BY '\n' |
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 already used the term "line" everywhere in the doc. We could just say lines are separated by a character and minimise the doc fix and etc.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -387,7 +387,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo | |
| * Loads text files and returns a `DataFrame` whose schema starts with a string column named | ||
| * "value", and followed by partitioned columns if there are any. | ||
| * | ||
| * Each line in the text files is a new row in the resulting DataFrame. For example: | ||
| * By default, each line in the text files is a new row in the resulting DataFrame. For example: | ||
| * {{{ | ||
| * // Scala: | ||
| * spark.readStream.text("/path/to/directory/") | ||
|
|
@@ -400,6 +400,10 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo | |
| * <ul> | ||
| * <li>`maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be | ||
| * considered in every trigger.</li> | ||
| * <li>`wholetext` (default `false`): If true, read a file as a single row and not split by "\n". | ||
|
||
| * </li> | ||
| * <li>`lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator | ||
| * that should be used for parsing.</li> | ||
| * </ul> | ||
| * | ||
| * @since 2.0.0 | ||
|
|
@@ -413,7 +417,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo | |
| * If the directory structure of the text files contains partitioning information, those are | ||
| * ignored in the resulting Dataset. To include partitioning information as columns, use `text`. | ||
| * | ||
| * Each line in the text file is a new element in the resulting Dataset. For example: | ||
| * By default, each line in the text file is a new element in the resulting Dataset. For example: | ||
| * {{{ | ||
| * // Scala: | ||
| * spark.readStream.textFile("/path/to/spark/README.md") | ||
|
|
@@ -426,6 +430,10 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo | |
| * <ul> | ||
| * <li>`maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be | ||
| * considered in every trigger.</li> | ||
| * <li>`wholetext` (default `false`): If true, read a file as a single row and not split by "\n". | ||
| * </li> | ||
| * <li>`lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator | ||
| * that should be used for parsing.</li> | ||
| * </ul> | ||
| * | ||
| * @param path input path | ||
|
|
||
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 should mention that this default rule is not defined by us, but by hadoop.
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.
Sure.