final class DataStreamReader extends Logging
Interface used to load a streaming Dataset
from external storage systems (e.g. file systems,
key-value stores, etc). Use SparkSession.readStream
to access this.
- Annotations
- @Evolving()
- Source
- DataStreamReader.scala
- Since
2.0.0
- Alphabetic
- By Inheritance
- DataStreamReader
- Logging
- AnyRef
- Any
- Hide All
- Show All
- Public
- All
Value Members
-
final
def
!=(arg0: Any): Boolean
- Definition Classes
- AnyRef → Any
-
final
def
##(): Int
- Definition Classes
- AnyRef → Any
-
final
def
==(arg0: Any): Boolean
- Definition Classes
- AnyRef → Any
-
final
def
asInstanceOf[T0]: T0
- Definition Classes
- Any
-
def
clone(): AnyRef
- Attributes
- protected[lang]
- Definition Classes
- AnyRef
- Annotations
- @throws( ... ) @native()
-
def
csv(path: String): DataFrame
Loads a CSV file stream and returns the result as a
DataFrame
.Loads a CSV file stream and returns the result as a
DataFrame
.This function will go through the input once to determine the input schema if
inferSchema
is enabled. To avoid going through the entire data once, disableinferSchema
option or specify the schema explicitly usingschema
.You can set the following CSV-specific options to deal with CSV files:
maxFilesPerTrigger
(default: no max limit): sets the maximum number of new files to be considered in every trigger.sep
(default,
): sets a single character as a separator for each field and value.encoding
(defaultUTF-8
): decodes the CSV files by the given encoding type.quote
(default"
): sets a single character used for escaping quoted values where the separator can be part of the value. If you would like to turn off quotations, you need to set notnull
but an empty string. This behaviour is different formcom.databricks.spark.csv
.escape
(default\
): sets a single character used for escaping quotes inside an already quoted value.charToEscapeQuoteEscaping
(defaultescape
or\0
): sets a single character used for escaping the escape for the quote character. The default value is escape character when escape and quote characters are different,\0
otherwise.comment
(default empty string): sets a single character used for skipping lines beginning with this character. By default, it is disabled.header
(defaultfalse
): uses the first line as names of columns.inferSchema
(defaultfalse
): infers the input schema automatically from data. It requires one extra pass over the data.ignoreLeadingWhiteSpace
(defaultfalse
): a flag indicating whether or not leading whitespaces from values being read should be skipped.ignoreTrailingWhiteSpace
(defaultfalse
): a flag indicating whether or not trailing whitespaces from values being read should be skipped.nullValue
(default empty string): sets the string representation of a null value. Since 2.0.1, this applies to all supported types including the string type.emptyValue
(default empty string): sets the string representation of an empty value.nanValue
(defaultNaN
): sets the string representation of a non-number" value.positiveInf
(defaultInf
): sets the string representation of a positive infinity value.negativeInf
(default-Inf
): sets the string representation of a negative infinity value.dateFormat
(defaultyyyy-MM-dd
): sets the string that indicates a date format. Custom date formats follow the formats at Datetime Patterns. This applies to date type.timestampFormat
(defaultyyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]
): sets the string that indicates a timestamp format. Custom date formats follow the formats at Datetime Patterns. This applies to timestamp type.maxColumns
(default20480
): defines a hard limit of how many columns a record can have.maxCharsPerColumn
(default-1
): defines the maximum number of characters allowed for any given value being read. By default, it is -1 meaning unlimited lengthmode
(defaultPERMISSIVE
): allows a mode for dealing with corrupt records during parsing. It supports the following case-insensitive modes.PERMISSIVE
: when it meets a corrupted record, puts the malformed string into a field configured bycolumnNameOfCorruptRecord
, and sets malformed fields tonull
. To keep corrupt records, an user can set a string type field namedcolumnNameOfCorruptRecord
in an user-defined schema. If a schema does not have the field, it drops corrupt records during parsing. A record with less/more tokens than schema is not a corrupted record to CSV. When it meets a record having fewer tokens than the length of the schema, setsnull
to extra fields. When the record has more tokens than the length of the schema, it drops extra tokens.DROPMALFORMED
: ignores the whole corrupted records.FAILFAST
: throws an exception when it meets corrupted records.columnNameOfCorruptRecord
(default is the value specified inspark.sql.columnNameOfCorruptRecord
): allows renaming the new field having malformed string created byPERMISSIVE
mode. This overridesspark.sql.columnNameOfCorruptRecord
.multiLine
(defaultfalse
): parse one record, which may span multiple lines.locale
(default isen-US
): sets a locale as language tag in IETF BCP 47 format. For instance, this is used while parsing dates and timestamps.lineSep
(default covers all\r
,\r\n
and\n
): defines the line separator that should be used for parsing. Maximum length is 1 character.pathGlobFilter
: an optional glob pattern to only include files with paths matching the pattern. The syntax followsorg.apache.hadoop.fs.GlobFilter
. It does not change the behavior of partition discovery.recursiveFileLookup
: recursively scan a directory for files. Using this option disables partition discovery
- Since
2.0.0
-
final
def
eq(arg0: AnyRef): Boolean
- Definition Classes
- AnyRef
-
def
equals(arg0: Any): Boolean
- Definition Classes
- AnyRef → Any
-
def
finalize(): Unit
- Attributes
- protected[lang]
- Definition Classes
- AnyRef
- Annotations
- @throws( classOf[java.lang.Throwable] )
-
def
format(source: String): DataStreamReader
Specifies the input data source format.
Specifies the input data source format.
- Since
2.0.0
-
final
def
getClass(): Class[_]
- Definition Classes
- AnyRef → Any
- Annotations
- @native()
-
def
hashCode(): Int
- Definition Classes
- AnyRef → Any
- Annotations
- @native()
-
def
initializeLogIfNecessary(isInterpreter: Boolean, silent: Boolean): Boolean
- Attributes
- protected
- Definition Classes
- Logging
-
def
initializeLogIfNecessary(isInterpreter: Boolean): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
final
def
isInstanceOf[T0]: Boolean
- Definition Classes
- Any
-
def
isTraceEnabled(): Boolean
- Attributes
- protected
- Definition Classes
- Logging
-
def
json(path: String): DataFrame
Loads a JSON file stream and returns the results as a
DataFrame
.Loads a JSON file stream and returns the results as a
DataFrame
.JSON Lines (newline-delimited JSON) is supported by default. For JSON (one record per file), set the
multiLine
option to true.This function goes through the input once to determine the input schema. If you know the schema in advance, use the version that specifies the schema to avoid the extra scan.
You can set the following JSON-specific options to deal with non-standard JSON files:
maxFilesPerTrigger
(default: no max limit): sets the maximum number of new files to be considered in every trigger.primitivesAsString
(defaultfalse
): infers all primitive values as a string typeprefersDecimal
(defaultfalse
): infers all floating-point values as a decimal type. If the values do not fit in decimal, then it infers them as doubles.allowComments
(defaultfalse
): ignores Java/C++ style comment in JSON recordsallowUnquotedFieldNames
(defaultfalse
): allows unquoted JSON field namesallowSingleQuotes
(defaulttrue
): allows single quotes in addition to double quotesallowNumericLeadingZeros
(defaultfalse
): allows leading zeros in numbers (e.g. 00012)allowBackslashEscapingAnyCharacter
(defaultfalse
): allows accepting quoting of all character using backslash quoting mechanismallowUnquotedControlChars
(defaultfalse
): allows JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters) or not.mode
(defaultPERMISSIVE
): allows a mode for dealing with corrupt records during parsing.PERMISSIVE
: when it meets a corrupted record, puts the malformed string into a field configured bycolumnNameOfCorruptRecord
, and sets malformed fields tonull
. To keep corrupt records, an user can set a string type field namedcolumnNameOfCorruptRecord
in an user-defined schema. If a schema does not have the field, it drops corrupt records during parsing. When inferring a schema, it implicitly adds acolumnNameOfCorruptRecord
field in an output schema.DROPMALFORMED
: ignores the whole corrupted records.FAILFAST
: throws an exception when it meets corrupted records.columnNameOfCorruptRecord
(default is the value specified inspark.sql.columnNameOfCorruptRecord
): allows renaming the new field having malformed string created byPERMISSIVE
mode. This overridesspark.sql.columnNameOfCorruptRecord
.dateFormat
(defaultyyyy-MM-dd
): sets the string that indicates a date format. Custom date formats follow the formats at Datetime Patterns. This applies to date type.timestampFormat
(defaultyyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]
): sets the string that indicates a timestamp format. Custom date formats follow the formats at Datetime Patterns. This applies to timestamp type.multiLine
(defaultfalse
): parse one record, which may span multiple lines, per filelineSep
(default covers all\r
,\r\n
and\n
): defines the line separator that should be used for parsing.dropFieldIfAllNull
(defaultfalse
): whether to ignore column of all null values or empty array/struct during schema inference.locale
(default isen-US
): sets a locale as language tag in IETF BCP 47 format. For instance, this is used while parsing dates and timestamps.pathGlobFilter
: an optional glob pattern to only include files with paths matching the pattern. The syntax followsorg.apache.hadoop.fs.GlobFilter
. It does not change the behavior of partition discovery.recursiveFileLookup
: recursively scan a directory for files. Using this option disables partition discovery
- Since
2.0.0
-
def
load(path: String): DataFrame
Loads input in as a
DataFrame
, for data streams that read from some path.Loads input in as a
DataFrame
, for data streams that read from some path.- Since
2.0.0
-
def
load(): DataFrame
Loads input data stream in as a
DataFrame
, for data streams that don't require a path (e.g.Loads input data stream in as a
DataFrame
, for data streams that don't require a path (e.g. external key-value stores).- Since
2.0.0
-
def
log: Logger
- Attributes
- protected
- Definition Classes
- Logging
-
def
logDebug(msg: ⇒ String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logDebug(msg: ⇒ String): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logError(msg: ⇒ String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logError(msg: ⇒ String): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logInfo(msg: ⇒ String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logInfo(msg: ⇒ String): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logName: String
- Attributes
- protected
- Definition Classes
- Logging
-
def
logTrace(msg: ⇒ String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logTrace(msg: ⇒ String): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logWarning(msg: ⇒ String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logWarning(msg: ⇒ String): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
final
def
ne(arg0: AnyRef): Boolean
- Definition Classes
- AnyRef
-
final
def
notify(): Unit
- Definition Classes
- AnyRef
- Annotations
- @native()
-
final
def
notifyAll(): Unit
- Definition Classes
- AnyRef
- Annotations
- @native()
-
def
option(key: String, value: Double): DataStreamReader
Adds an input option for the underlying data source.
Adds an input option for the underlying data source.
- Since
2.0.0
-
def
option(key: String, value: Long): DataStreamReader
Adds an input option for the underlying data source.
Adds an input option for the underlying data source.
- Since
2.0.0
-
def
option(key: String, value: Boolean): DataStreamReader
Adds an input option for the underlying data source.
Adds an input option for the underlying data source.
- Since
2.0.0
-
def
option(key: String, value: String): DataStreamReader
Adds an input option for the underlying data source.
Adds an input option for the underlying data source.
You can set the following option(s):
timeZone
(default session local timezone): sets the string that indicates a time zone ID to be used to parse timestamps in the JSON/CSV datasources or partition values. The following formats oftimeZone
are supported:- Region-based zone ID: It should have the form 'area/city', such as 'America/Los_Angeles'.
- Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
Other short names like 'CST' are not recommended to use because they can be ambiguous. If it isn't set, the current value of the SQL config
spark.sql.session.timeZone
is used by default.- Since
2.0.0
-
def
options(options: Map[String, String]): DataStreamReader
(Java-specific) Adds input options for the underlying data source.
(Java-specific) Adds input options for the underlying data source.
You can set the following option(s):
timeZone
(default session local timezone): sets the string that indicates a time zone ID to be used to parse timestamps in the JSON/CSV datasources or partition values. The following formats oftimeZone
are supported:- Region-based zone ID: It should have the form 'area/city', such as 'America/Los_Angeles'.
- Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
Other short names like 'CST' are not recommended to use because they can be ambiguous. If it isn't set, the current value of the SQL config
spark.sql.session.timeZone
is used by default.- Since
2.0.0
-
def
options(options: Map[String, String]): DataStreamReader
(Scala-specific) Adds input options for the underlying data source.
(Scala-specific) Adds input options for the underlying data source.
You can set the following option(s):
timeZone
(default session local timezone): sets the string that indicates a time zone ID to be used to parse timestamps in the JSON/CSV datasources or partition values. The following formats oftimeZone
are supported:- Region-based zone ID: It should have the form 'area/city', such as 'America/Los_Angeles'.
- Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
Other short names like 'CST' are not recommended to use because they can be ambiguous. If it isn't set, the current value of the SQL config
spark.sql.session.timeZone
is used by default.- Since
2.0.0
-
def
orc(path: String): DataFrame
Loads a ORC file stream, returning the result as a
DataFrame
.Loads a ORC file stream, returning the result as a
DataFrame
.You can set the following ORC-specific option(s) for reading ORC files:
maxFilesPerTrigger
(default: no max limit): sets the maximum number of new files to be considered in every trigger.mergeSchema
(default is the value specified inspark.sql.orc.mergeSchema
): sets whether we should merge schemas collected from all ORC part-files. This will overridespark.sql.orc.mergeSchema
.pathGlobFilter
: an optional glob pattern to only include files with paths matching the pattern. The syntax followsorg.apache.hadoop.fs.GlobFilter
. It does not change the behavior of partition discovery.recursiveFileLookup
: recursively scan a directory for files. Using this option disables partition discovery
- Since
2.3.0
-
def
parquet(path: String): DataFrame
Loads a Parquet file stream, returning the result as a
DataFrame
.Loads a Parquet file stream, returning the result as a
DataFrame
.You can set the following Parquet-specific option(s) for reading Parquet files:
maxFilesPerTrigger
(default: no max limit): sets the maximum number of new files to be considered in every trigger.mergeSchema
(default is the value specified inspark.sql.parquet.mergeSchema
): sets whether we should merge schemas collected from all Parquet part-files. This will overridespark.sql.parquet.mergeSchema
.pathGlobFilter
: an optional glob pattern to only include files with paths matching the pattern. The syntax followsorg.apache.hadoop.fs.GlobFilter
. It does not change the behavior of partition discovery.recursiveFileLookup
: recursively scan a directory for files. Using this option disables partition discovery
- Since
2.0.0
-
def
schema(schemaString: String): DataStreamReader
Specifies the schema by using the input DDL-formatted string.
Specifies the schema by using the input DDL-formatted string. Some data sources (e.g. JSON) can infer the input schema automatically from data. By specifying the schema here, the underlying data source can skip the schema inference step, and thus speed up data loading.
- Since
2.3.0
-
def
schema(schema: StructType): DataStreamReader
Specifies the input schema.
Specifies the input schema. Some data sources (e.g. JSON) can infer the input schema automatically from data. By specifying the schema here, the underlying data source can skip the schema inference step, and thus speed up data loading.
- Since
2.0.0
-
final
def
synchronized[T0](arg0: ⇒ T0): T0
- Definition Classes
- AnyRef
-
def
text(path: String): DataFrame
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.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. The text files must be encoded as UTF-8.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/") // Java: spark.readStream().text("/path/to/directory/")
You can set the following text-specific options to deal with text files:
maxFilesPerTrigger
(default: no max limit): sets the maximum number of new files to be considered in every trigger.wholetext
(defaultfalse
): If true, read a file as a single row and not split by "\n".lineSep
(default covers all\r
,\r\n
and\n
): defines the line separator that should be used for parsing.pathGlobFilter
: an optional glob pattern to only include files with paths matching the pattern. The syntax followsorg.apache.hadoop.fs.GlobFilter
. It does not change the behavior of partition discovery.recursiveFileLookup
: recursively scan a directory for files. Using this option disables partition discovery
- Since
2.0.0
-
def
textFile(path: String): Dataset[String]
Loads text file(s) and returns a
Dataset
of String.Loads text file(s) and returns a
Dataset
of String. The underlying schema of the Dataset contains a single string column named "value". The text files must be encoded as UTF-8.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
.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") // Java: spark.readStream().textFile("/path/to/spark/README.md")
You can set the text-specific options as specified in
DataStreamReader.text
.- path
input path
- Since
2.1.0
-
def
toString(): String
- Definition Classes
- AnyRef → Any
-
final
def
wait(): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws( ... )
-
final
def
wait(arg0: Long, arg1: Int): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws( ... )
-
final
def
wait(arg0: Long): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws( ... ) @native()