All Known Subinterfaces:
OneInputWindowContext<IN>, TwoInputWindowContext<IN1,IN2>

@Experimental public interface WindowContext
This interface represents a context for window operations and provides methods to interact with state that is scoped to the window.
  • Method Summary

    Modifier and Type
    Method
    Description
    long
    Gets the end timestamp of this window.
    long
    Gets the starting timestamp of the window.
    <T, ACC, OUT>
    Optional<org.apache.flink.api.common.state.v2.AggregatingState<T,OUT>>
    getWindowState(org.apache.flink.api.common.state.AggregatingStateDeclaration<T,ACC,OUT> stateDeclaration)
    Retrieves a AggregatingState object that can be used to interact with fault-tolerant state that is scoped to the window.
    <T> Optional<org.apache.flink.api.common.state.v2.ListState<T>>
    getWindowState(org.apache.flink.api.common.state.ListStateDeclaration<T> stateDeclaration)
    Retrieves a ListState object that can be used to interact with fault-tolerant state that is scoped to the window.
    <KEY, V> Optional<org.apache.flink.api.common.state.v2.MapState<KEY,V>>
    getWindowState(org.apache.flink.api.common.state.MapStateDeclaration<KEY,V> stateDeclaration)
    Retrieves a MapState object that can be used to interact with fault-tolerant state that is scoped to the window.
    <T> Optional<org.apache.flink.api.common.state.v2.ReducingState<T>>
    getWindowState(org.apache.flink.api.common.state.ReducingStateDeclaration<T> stateDeclaration)
    Retrieves a ReducingState object that can be used to interact with fault-tolerant state that is scoped to the window.
    <T> Optional<org.apache.flink.api.common.state.v2.ValueState<T>>
    getWindowState(org.apache.flink.api.common.state.ValueStateDeclaration<T> stateDeclaration)
    Retrieves a ValueState object that can be used to interact with fault-tolerant state that is scoped to the window.
  • Method Details

    • getStartTime

      long getStartTime()
      Gets the starting timestamp of the window. This is the first timestamp that belongs to this window.
      Returns:
      The starting timestamp of this window, or -1 if the window is not a time window or a session window.
    • getEndTime

      long getEndTime()
      Gets the end timestamp of this window. The end timestamp is exclusive, meaning it is the first timestamp that does not belong to this window anymore.
      Returns:
      The exclusive end timestamp of this window, or -1 if the window is a session window or not a time window.
    • getWindowState

      <T> Optional<org.apache.flink.api.common.state.v2.ListState<T>> getWindowState(org.apache.flink.api.common.state.ListStateDeclaration<T> stateDeclaration) throws Exception
      Retrieves a ListState object that can be used to interact with fault-tolerant state that is scoped to the window.
      Throws:
      Exception
    • getWindowState

      <KEY, V> Optional<org.apache.flink.api.common.state.v2.MapState<KEY,V>> getWindowState(org.apache.flink.api.common.state.MapStateDeclaration<KEY,V> stateDeclaration) throws Exception
      Retrieves a MapState object that can be used to interact with fault-tolerant state that is scoped to the window.
      Throws:
      Exception
    • getWindowState

      <T> Optional<org.apache.flink.api.common.state.v2.ValueState<T>> getWindowState(org.apache.flink.api.common.state.ValueStateDeclaration<T> stateDeclaration) throws Exception
      Retrieves a ValueState object that can be used to interact with fault-tolerant state that is scoped to the window.
      Throws:
      Exception
    • getWindowState

      <T> Optional<org.apache.flink.api.common.state.v2.ReducingState<T>> getWindowState(org.apache.flink.api.common.state.ReducingStateDeclaration<T> stateDeclaration) throws Exception
      Retrieves a ReducingState object that can be used to interact with fault-tolerant state that is scoped to the window.
      Throws:
      Exception
    • getWindowState

      <T, ACC, OUT> Optional<org.apache.flink.api.common.state.v2.AggregatingState<T,OUT>> getWindowState(org.apache.flink.api.common.state.AggregatingStateDeclaration<T,ACC,OUT> stateDeclaration) throws Exception
      Retrieves a AggregatingState object that can be used to interact with fault-tolerant state that is scoped to the window.
      Throws:
      Exception