Packages

t

org.apache.spark.sql.connector.read

PartitionReaderFactory

trait PartitionReaderFactory extends Serializable

A factory used to create PartitionReader instances.

If Spark fails to execute any methods in the implementations of this interface or in the returned PartitionReader (by throwing an exception), corresponding Spark task would fail and get retried until hitting the maximum retry times.

Annotations
@Evolving()
Source
PartitionReaderFactory.java
Since

3.0.0

Linear Supertypes
Serializable, AnyRef, Any
Ordering
  1. Alphabetic
  2. By Inheritance
Inherited
  1. PartitionReaderFactory
  2. Serializable
  3. AnyRef
  4. Any
  1. Hide All
  2. Show All
Visibility
  1. Public
  2. Protected

Abstract Value Members

  1. abstract def createReader(partition: InputPartition): PartitionReader[InternalRow]

    Returns a row-based partition reader to read data from the given InputPartition.

    Returns a row-based partition reader to read data from the given InputPartition.

    Implementations probably need to cast the input partition to the concrete InputPartition class defined for the data source.

Concrete Value Members

  1. def createColumnarReader(partition: InputPartition): PartitionReader[ColumnarBatch]

    Returns a columnar partition reader to read data from the given InputPartition.

    Returns a columnar partition reader to read data from the given InputPartition.

    Implementations probably need to cast the input partition to the concrete InputPartition class defined for the data source.

  2. def supportColumnarReads(partition: InputPartition): Boolean

    Returns true if the given InputPartition should be read by Spark in a columnar way.

    Returns true if the given InputPartition should be read by Spark in a columnar way. This means, implementations must also implement #createColumnarReader(InputPartition) for the input partitions that this method returns true.

    As of Spark 2.4, Spark can only read all input partition in a columnar way, or none of them. Data source can't mix columnar and row-based partitions. This may be relaxed in future versions.