Packages

t

org.apache.spark.sql.connector.read.streaming

SupportsTriggerAvailableNow

trait SupportsTriggerAvailableNow extends SupportsAdmissionControl

An interface for streaming sources that supports running in Trigger.AvailableNow mode, which will process all the available data at the beginning of the query in (possibly) multiple batches.

This mode will have better scalability comparing to Trigger.Once mode.

Annotations
@Evolving()
Source
SupportsTriggerAvailableNow.java
Since

3.3.0

Linear Supertypes
Ordering
  1. Alphabetic
  2. By Inheritance
Inherited
  1. SupportsTriggerAvailableNow
  2. SupportsAdmissionControl
  3. SparkDataStream
  4. AnyRef
  5. Any
  1. Hide All
  2. Show All
Visibility
  1. Public
  2. Protected

Abstract Value Members

  1. abstract def commit(end: Offset): Unit

    Informs the source that Spark has completed processing all data for offsets less than or equal to end and will only request offsets greater than end in the future.

    Informs the source that Spark has completed processing all data for offsets less than or equal to end and will only request offsets greater than end in the future.

    Definition Classes
    SparkDataStream
  2. abstract def deserializeOffset(json: String): Offset

    Deserialize a JSON string into an Offset of the implementation-defined offset type.

    Deserialize a JSON string into an Offset of the implementation-defined offset type.

    Definition Classes
    SparkDataStream
    Exceptions thrown

    IllegalArgumentException if the JSON does not encode a valid offset for this reader

  3. abstract def initialOffset(): Offset

    Returns the initial offset for a streaming query to start reading from.

    Returns the initial offset for a streaming query to start reading from. Note that the streaming data source should not assume that it will start reading from its initial offset: if Spark is restarting an existing query, it will restart from the check-pointed offset rather than the initial one.

    Definition Classes
    SparkDataStream
  4. abstract def latestOffset(startOffset: Offset, limit: ReadLimit): Offset

    Returns the most recent offset available given a read limit.

    Returns the most recent offset available given a read limit. The start offset can be used to figure out how much new data should be read given the limit. Users should implement this method instead of latestOffset for a MicroBatchStream or getOffset for Source.

    When this method is called on a Source, the source can return null if there is no data to process. In addition, for the very first micro-batch, the startOffset will be null as well.

    When this method is called on a MicroBatchStream, the startOffset will be initialOffset for the very first micro-batch. The source can return null if there is no data to process.

    Definition Classes
    SupportsAdmissionControl
  5. abstract def prepareForTriggerAvailableNow(): Unit

    This will be called at the beginning of streaming queries with Trigger.AvailableNow, to let the source record the offset for the current latest data at the time (a.k.a the target offset for the query).

    This will be called at the beginning of streaming queries with Trigger.AvailableNow, to let the source record the offset for the current latest data at the time (a.k.a the target offset for the query). The source will behave as if there is no new data coming in after the target offset, i.e., the source will not return an offset higher than the target offset when ReadLimit) latestOffset is called.

  6. abstract def stop(): Unit

    Stop this source and free any resources it has allocated.

    Stop this source and free any resources it has allocated.

    Definition Classes
    SparkDataStream

Concrete Value Members

  1. final def !=(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  2. final def ##: Int
    Definition Classes
    AnyRef → Any
  3. final def ==(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  4. final def asInstanceOf[T0]: T0
    Definition Classes
    Any
  5. def clone(): AnyRef
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.CloneNotSupportedException]) @IntrinsicCandidate() @native()
  6. final def eq(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  7. def equals(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef → Any
  8. final def getClass(): Class[_ <: AnyRef]
    Definition Classes
    AnyRef → Any
    Annotations
    @IntrinsicCandidate() @native()
  9. def getDefaultReadLimit(): ReadLimit

    Returns the read limits potentially passed to the data source through options when creating the data source.

    Returns the read limits potentially passed to the data source through options when creating the data source.

    Definition Classes
    SupportsAdmissionControl
  10. def hashCode(): Int
    Definition Classes
    AnyRef → Any
    Annotations
    @IntrinsicCandidate() @native()
  11. final def isInstanceOf[T0]: Boolean
    Definition Classes
    Any
  12. final def ne(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  13. final def notify(): Unit
    Definition Classes
    AnyRef
    Annotations
    @IntrinsicCandidate() @native()
  14. final def notifyAll(): Unit
    Definition Classes
    AnyRef
    Annotations
    @IntrinsicCandidate() @native()
  15. def reportLatestOffset(): Offset

    Returns the most recent offset available.

    Returns the most recent offset available.

    The source can return null, if there is no data to process or the source does not support to this method.

    Definition Classes
    SupportsAdmissionControl
  16. final def synchronized[T0](arg0: => T0): T0
    Definition Classes
    AnyRef
  17. def toString(): String
    Definition Classes
    AnyRef → Any
  18. final def wait(arg0: Long, arg1: Int): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.InterruptedException])
  19. final def wait(arg0: Long): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.InterruptedException]) @native()
  20. final def wait(): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.InterruptedException])

Deprecated Value Members

  1. def finalize(): Unit
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.Throwable]) @Deprecated
    Deprecated

    (Since version 9)

Inherited from SparkDataStream

Inherited from AnyRef

Inherited from Any

Ungrouped