pyflink.datastream package¶
Module contents¶
Entry point classes of Flink DataStream API:
StreamExecutionEnvironment
: The context in which a streaming program is executed.
DataStream
: Represents a stream of elements of the same type. A DataStream can be transformed into another DataStream by applying a transformation.
KeyedStream
: Represents aDataStream
where elements are partitioned by key using a provided KeySelector.
WindowedStream
: Represents a data stream where elements are grouped by key, and for each key, the stream of elements is split into windows based on a WindowAssigner. Window emission is triggered based on a Trigger.
ConnectedStreams
: Represent two connected streams of (possibly) different data types. Connected streams are useful for cases where operations on one stream directly affect the operations on the other stream, usually via shared state between the streams.
BroadcastStream
: Represent a stream withstate.BroadcastState
(s).
BroadcastConnectedStream
: Represents the result of connecting a keyed or non-keyed stream, with aBroadcastStream
withstate.BroadcastState
(s)
Functions used to transform a DataStream
into another DataStream
:
MapFunction
: Performs a map transformation of aDataStream
at element wise.
CoMapFunction
: Performs a map transformation over two connected streams.
FlatMapFunction
: Performs a flatmap transformation of aDataStream
which produces zero, one, or more elements for each input element.
CoFlatMapFunction
: Performs a flatmap transformation over two connected streams.
FilterFunction
: A filter function is a predicate applied individually to each record.
ReduceFunction
: Combines groups of elements to a single value.
ProcessFunction
: Similar toFlatMapFunction
, except that it could access the current timestamp and watermark inProcessFunction
.
KeyedProcessFunction
: Similar toProcessFunction
, except that it was applied to aKeyedStream
and could register event-time and processing-time timers.
CoProcessFunction
: Similar toCoFlatMapFunction
, except that it could access the current timestamp and watermark inCoProcessFunction
.
KeyedCoProcessFunction
: Similar toCoProcessFunction
, except that it was applied to a keyedConnectedStreams
and could register event-time and processing-time timers.
WindowFunction
: Base interface for functions that are evaluated over keyed (grouped) windows.
ProcessWindowFunction
: Similar toWindowFunction
, except that it could access a context for retrieving extra information such as the current timestamp, the watermark, etc.
AggregateFunction
: Base class for a user-defined aggregate function.
BroadcastProcessFunction
: A function to be applied to aBroadcastConnectedStream
that connectsBroadcastStream
, i.e. a stream with broadcast state, with a non-keyedDataStream
.
KeyedBroadcastProcessFunction
: A function to be applied to aBroadcastConnectedStream
that connectsBroadcastStream
, i.e. a stream with broadcast state, with aKeyedStream
.
RuntimeContext
: Contains information about the context in which functions are executed. Each parallel instance of the function will have a context through which it can access static contextual information (such as the current parallelism), etc.
Classes to define window:
Window
: A grouping of elements into finite buckets.
TimeWindow
: A grouping of elements according to a time interval from start (inclusive) to end (exclusive).
CountWindow
: A grouping of elements according to element count from start (inclusive) to end (exclusive).
GlobalWindow
: The window into which all data is placed.
WindowAssigner
: Assigns zero or moreWindow
to an element.
MergingWindowAssigner
: AWindowAssigner
that can merge windows.
TriggerResult
: Result type for trigger methods. This determines what happens with the window, for example whether the window function should be called, or the window should be discarded.
Trigger
: Determines when a pane of a window should be evaluated to emit the results for that part of the window.
Classes to define the behavior of checkpoint and state backend:
CheckpointingMode
: Defines what consistency guarantees the system gives in the presence of failures.
CheckpointConfig
: Configuration that captures all checkpointing related settings.
StateBackend
: Base class of the state backends which define how the state of a streaming application is stored locally within the cluster. Different state backends store their state in different fashions, and use different data structures to hold the state of a running application.
HashMapStateBackend
: Holds the working state in the memory (JVM heap) of the TaskManagers and checkpoints based on the configuredCheckpointStorage
.
EmbeddedRocksDBStateBackend
: Stores its state in an embedded RocksDB instance. This state backend can store very large state that exceeds memory and spills to local disk.
CustomStateBackend
: A wrapper of customized java state backend.
JobManagerCheckpointStorage
: Checkpoints state directly to the JobManager’s memory (hence the name), but savepoints will be persisted to a file system.
FileSystemCheckpointStorage
: Checkpoints state as files to a file system. Each checkpoint individually will store all its files in a subdirectory that includes the checkpoint number, such as hdfs://namenode:port/flink-checkpoints/chk-17/.
CustomCheckpointStorage
: A wrapper of customized java checkpoint storage.
Classes for state operations:
state.ValueState
: Interface for partitioned single-value state. The value can be retrieved or updated.
state.ListState
: Interface for partitioned list state in Operations. The state is accessed and modified by user functions, and checkpointed consistently by the system as part of the distributed snapshots.
state.MapState
: Interface for partitioned key-value state. The key-value pair can be added, updated and retrieved.
state.ReducingState
: Interface for reducing state. Elements can be added to the state, they will be combined using aReduceFunction
. The current state can be inspected.
state.AggregatingState
: Interface for aggregating state, based on anAggregateFunction
. Elements that are added to this type of state will be eagerly pre-aggregated using a given AggregateFunction.
state.BroadcastState
: A type of state that can be created to store the state of aBroadcastStream
. This state assumes that the same elements are sent to all instances of an operator.
state.ReadOnlyBroadcastState
: A read-only view of thestate.BroadcastState
.
state.StateTtlConfig
: Configuration of state TTL logic.
Classes to define source & sink:
connectors.elasticsearch.ElasticsearchSink
: A sink for publishing data into Elasticsearch 6 or Elasticsearch 7.
connectors.kafka.FlinkKafkaConsumer
: A streaming data source that pulls a parallel data stream from Apache Kafka.
connectors.kafka.FlinkKafkaProducer
: A streaming data sink to produce data into a Kafka topic.
connectors.kafka.KafkaSource
: The new API to read data in parallel from Apache Kafka.
connectors.kafka.KafkaSink
: The new API to write data into to Apache Kafka topics.
connectors.file_system.FileSource
: A unified data source that reads files - both in batch and in streaming mode. This source supports all (distributed) file systems and object stores that can be accessed via the Flink’s FileSystem class.
connectors.file_system.FileSink
: A unified sink that emits its input elements to FileSystem files within buckets. This sink achieves exactly-once semantics for both BATCH and STREAMING.
connectors.file_system.StreamingFileSink
: Sink that emits its input elements to files within buckets. This is integrated with the checkpointing mechanism to provide exactly once semantics.
connectors.number_seq.NumberSequenceSource
: A data source that produces a sequence of numbers (longs). This source is useful for testing and for cases that just need a stream of N events of any kind.
connectors.jdbc.JdbcSink
: A data sink to produce data into an external storage using JDBC.
connectors.pulsar.PulsarSource
: A streaming data source that pulls a parallel data stream from Pulsar.
connectors.pulsar.PulsarSink
: A streaming data sink to produce data into Pulsar.
connectors.rabbitmq.RMQSource
: A streaming data source that pulls a parallel data stream from RabbitMQ.
connectors.rabbitmq.RMQSink
: A Sink for publishing data into RabbitMQ.
connectors.cassandra.CassandraSink
: A Sink for publishing data into Cassandra.
connectors.kinesis.FlinkKinesisConsumer
: A streaming data source that pulls a parallel data stream from Kinesis.
connectors.kinesis.KinesisStreamsSink
: A Kinesis Data Streams (KDS) Sink that performs async requests against a destination stream using the buffering protocol.
connectors.kinesis.KinesisFirehoseSink
: A Kinesis Data Firehose (KDF) Sink that performs async requests against a destination delivery stream using the buffering protocol.
connectors.hybrid_source.HybridSource
: A Hybrid source that switches underlying sources based on configured source chain.
Classes to define formats used together with source & sink:
formats.csv.CsvReaderFormat
: AStreamFormat
to read CSV files into Row data.
formats.csv.CsvBulkWriter
: CreatesBulkWriterFactory
to write Row data into CSV files.
formats.avro.GenericRecordAvroTypeInfo
: ATypeInformation
to indicate vanilla Python records will be translated to GenericRecordAvroTypeInfo on the Java side.
formats.avro.AvroInputFormat
: An InputFormat to read avro files in a streaming fashion.
formats.avro.AvroWriters
: A class to provideBulkWriterFactory
to write vanilla Python objects into avro files in a batch fashion.
formats.parquet.ParquetColumnarRowInputFormat
: ABulkFormat
to read columnar parquet files into Row data in a batch-processing fashion.
formats.parquet.ParquetBulkWriters
: Convenient builder to create aBulkWriterFactory
that writes Rows with a defined RowType into Parquet files in a batch fashion.
formats.parquet.AvroParquetReaders
: A convenience builder to create reader format that reads individual Avro records from a Parquet stream. Only GenericRecord is supported in PyFlink.
formats.parquet.AvroParquetWriters
: Convenience builder to create ParquetWriterFactory instances for Avro types. Only GenericRecord is supported in PyFlink.
formats.orc.OrcBulkWriters
: Convenient builder to create aBulkWriterFactory
that writes Row records with a definedRowType
into Orc files.
Other important classes:
TimeCharacteristic
: Defines how the system determines time for time-dependent order and operations that depend on time (such as time windows).
TimeDomain
: Specifies whether a firing timer is based on event time or processing time.
KeySelector
: The extractor takes an object and returns the deterministic key for that object.
Partitioner
: Function to implement a custom partition assignment for keys.
SinkFunction
: Interface for implementing user defined sink functionality.
SourceFunction
: Interface for implementing user defined source functionality.
OutputTag
: Tag with a name and type for identifying side output of an operator
-
class
pyflink.datastream.
AggregateFunction
[source]¶ Bases:
pyflink.datastream.functions.Function
The AggregateFunction is a flexible aggregation function, characterized by the following features:
The aggregates may use different types for input values, intermediate aggregates, and result type, to support a wide range of aggregation types.
Support for distributive aggregations: Different intermediate aggregates can be merged together, to allow for pre-aggregation/final-aggregation optimizations.
The AggregateFunction’s intermediate aggregate (in-progress aggregation state) is called the accumulator. Values are added to the accumulator, and final aggregates are obtained by finalizing the accumulator state. This supports aggregation functions where the intermediate state needs to be different than the aggregated values and the final result type, such as for example average (which typically keeps a count and sum). Merging intermediate aggregates (partial aggregates) means merging the accumulators.
The AggregationFunction itself is stateless. To allow a single AggregationFunction instance to maintain multiple aggregates (such as one aggregate per key), the AggregationFunction creates a new accumulator whenever a new aggregation is started.
-
abstract
add
(value, accumulator)[source]¶ Adds the given input value to the given accumulator, returning the new accumulator value.
For efficiency, the input accumulator may be modified and returned.
- Parameters
value – The value to add.
accumulator – The accumulator to add the value to.
- Returns
The accumulator with the updated state.
-
close
()¶
-
abstract
create_accumulator
()[source]¶ Creates a new accumulator, starting a new aggregate.
The new accumulator is typically meaningless unless a value is added via
add()
.The accumulator is the state of a running aggregation. When a program has multiple aggregates in progress (such as per key and window), the state (per key and window) is the size of the accumulator.
- Returns
A new accumulator, corresponding to an empty aggregate.
-
abstract
get_result
(accumulator)[source]¶ Gets the result of the aggregation from the accumulator.
- Parameters
accumulator – The accumulator of the aggregation.
- Returns
The final aggregation result.
-
abstract
merge
(acc_a, acc_b)[source]¶ Merges two accumulators, returning an accumulator with the merged state.
This function may reuse any of the given accumulators as the target for the merge and return that. The assumption is that the given accumulators will not be used any more after having been passed to this function.
- Parameters
acc_a – An accumulator to merge.
acc_b – Another accumulator to merge.
- Returns
The accumulator with the merged state.
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
class
pyflink.datastream.
BroadcastConnectedStream
(non_broadcast_stream: Union[DataStream, KeyedStream], broadcast_stream: pyflink.datastream.data_stream.BroadcastStream, broadcast_state_descriptors: List[pyflink.datastream.state.MapStateDescriptor])[source]¶ Bases:
object
A BroadcastConnectedStream represents the result of connecting a keyed or non-keyed stream, with a
BroadcastStream
withBroadcastState
(s). As in the case ofConnectedStreams
these streams are useful for cases where operations on one stream directly affect the operations on the other stream, usually via shared state between the streams.An example for the use of such connected streams would be to apply rules that change over time onto another, possibly keyed stream. The stream with the broadcast state has the rules, and will store them in the broadcast state, while the other stream will contain the elements to apply the rules to. By broadcasting the rules, these will be available in all parallel instances, and can be applied to all partitions of the other stream.
New in version 1.16.0.
-
process
(func: BroadcastProcessFunction, output_type: TypeInformation = 'None') → ’DataStream’[source]¶ -
process
(func: KeyedBroadcastProcessFunction, output_type: TypeInformation = 'None') → ’DataStream’ Assumes as inputs a
BroadcastStream
and aDataStream
orKeyedStream
and applies the givenBroadcastProcessFunction
orKeyedBroadcastProcessFunction
on them, thereby creating a transformed output stream.- Parameters
func – The
BroadcastProcessFunction
that is called for each element in the non-broadcastedDataStream
, or theKeyedBroadcastProcessFunction
that is called for each element in the non-broadcastedKeyedStream
.output_type – The type of the output elements, should be
common.TypeInformation
or list (implicitRowTypeInfo
) or None ( implicitTypes.PICKLED_BYTE_ARRAY()
).
- Returns
The transformed
DataStream
.
-
-
class
pyflink.datastream.
BroadcastProcessFunction
(*args, **kwds)[source]¶ Bases:
pyflink.datastream.functions.BaseBroadcastProcessFunction
,typing.Generic
A function to be applied to a
BroadcastConnectedStream
that connectsBroadcastStream
, i.e. a stream with broadcast state, with a non-keyedDataStream
.The stream with the broadcast state can be created using the
DataStream.broadcast()
method.The user has to implement two methods:
the
process_broadcast_element()
which will be applied to each element in the broadcast sidethe
process_element()
which will be applied to the non-broadcasted side.
The
process_broadcast_element()
takes a context as an argument (among others), which allows it to read/write to the broadcast state, while theprocess_element()
has read-only access to the broadcast state.New in version 1.16.0.
-
class
BaseContext
¶ Bases:
abc.ABC
The base context available to all methods in a broadcast process function. This includes
BroadcastProcessFunction
andKeyedBroadcastProcessFunction
.-
abstract
current_processing_time
() → int¶ Returns the current processing time.
-
abstract
current_watermark
() → int¶ Returns the current watermark.
-
abstract
timestamp
() → int¶ Timestamp of the element currently being processed or timestamp of a firing timer. This might be None, for example if the time characteristic of your program is set to
TimeCharacteristic.ProcessingTime
.
-
abstract
-
class
Context
[source]¶ Bases:
pyflink.datastream.functions.BaseBroadcastProcessFunction.Context
,abc.ABC
A
BaseBroadcastProcessFunction.Context
available to the broadcast side of aBroadcastConnectedStream
.-
abstract
current_processing_time
() → int¶ Returns the current processing time.
-
abstract
current_watermark
() → int¶ Returns the current watermark.
-
abstract
get_broadcast_state
(state_descriptor: pyflink.datastream.state.MapStateDescriptor) → pyflink.datastream.state.BroadcastState¶ Fetches the
BroadcastState
with the specified name.- Parameters
state_descriptor – the
MapStateDescriptor
of the state to be fetched.- Returns
The required
BroadcastState
.
-
abstract
timestamp
() → int¶ Timestamp of the element currently being processed or timestamp of a firing timer. This might be None, for example if the time characteristic of your program is set to
TimeCharacteristic.ProcessingTime
.
-
abstract
-
class
ReadOnlyContext
[source]¶ Bases:
pyflink.datastream.functions.BaseBroadcastProcessFunction.ReadOnlyContext
,abc.ABC
A
BaseBroadcastProcessFunction.ReadOnlyContext
available to the non-keyed side of aBroadcastConnectedStream
(if any).-
abstract
current_processing_time
() → int¶ Returns the current processing time.
-
abstract
current_watermark
() → int¶ Returns the current watermark.
-
abstract
get_broadcast_state
(state_descriptor: pyflink.datastream.state.MapStateDescriptor) → pyflink.datastream.state.ReadOnlyBroadcastState¶ Fetches a read-only view of the broadcast state with the specified name.
- Parameters
state_descriptor – the
MapStateDescriptor
of the state to be fetched.- Returns
The required read-only view of the broadcast state.
-
abstract
timestamp
() → int¶ Timestamp of the element currently being processed or timestamp of a firing timer. This might be None, for example if the time characteristic of your program is set to
TimeCharacteristic.ProcessingTime
.
-
abstract
-
close
()¶
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
abstract
process_broadcast_element
(value: IN2, ctx: pyflink.datastream.functions.BroadcastProcessFunction.Context)[source]¶ This method is called for each element in the
BroadcastStream
.This function can output zero or more elements via
yield
statement, query the current processing/event time, and also query and update the internalstate.BroadcastState
. These can be done through the providedBroadcastProcessFunction.Context
. The context is only valid during the invocation of this method, do not store it.- Parameters
value – The stream element.
ctx – A
BroadcastProcessFunction.Context
that allows querying the timestamp of the element, querying the current processing/event time and updating the broadcast state. The context is only valid during the invocation of this method, do not store it.
-
abstract
process_element
(value: IN1, ctx: pyflink.datastream.functions.BroadcastProcessFunction.ReadOnlyContext)[source]¶ This method is called for each element in the (non-broadcast)
DataStream
.This function can output zero or more elements via
yield
statement, and query the current processing/event time. Finally, it has read-only access to the broadcast state. The context is only valid during the invocation of this method, do not store it.- Parameters
value – The stream element.
ctx – A
BroadcastProcessFunction.ReadOnlyContext
that allows querying the timestamp of the element, querying the current processing/event time and reading the broadcast state. The context is only valid during the invocation of this method, do not store it.
-
class
pyflink.datastream.
BroadcastStream
(input_stream: Union[DataStream, KeyedStream], broadcast_state_descriptors: List[pyflink.datastream.state.MapStateDescriptor])[source]¶ Bases:
object
A BroadcastStream is a stream with
state.BroadcastState
(s). This can be created by any stream using theDataStream.broadcast()
method and implicitly creates states where the user can store elements of the createdBroadcastStream
. (seeBroadcastConnectedStream
).Note that no further operation can be applied to these streams. The only available option is to connect them with a keyed or non-keyed stream, using the
KeyedStream.connect()
and theDataStream.connect()
respectively. Applying these methods will result it aBroadcastConnectedStream
for further processing.New in version 1.16.0.
-
class
pyflink.datastream.
CheckpointConfig
(j_checkpoint_config)[source]¶ Bases:
object
Configuration that captures all checkpointing related settings.
The default checkpoint mode: exactly once.
The default timeout of a checkpoint attempt: 10 minutes.
DEFAULT_MIN_PAUSE_BETWEEN_CHECKPOINTS
:The default minimum pause to be made between checkpoints: none.
DEFAULT_MAX_CONCURRENT_CHECKPOINTS
:The default limit of concurrently happening checkpoints: one.
-
DEFAULT_MAX_CONCURRENT_CHECKPOINTS
= 1¶
-
DEFAULT_MIN_PAUSE_BETWEEN_CHECKPOINTS
= 0¶
-
DEFAULT_MODE
= 0¶
-
DEFAULT_TIMEOUT
= 600000¶
-
disable_unaligned_checkpoints
() → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Enables unaligned checkpoints, which greatly reduce checkpointing times under backpressure (experimental).
Unaligned checkpoints contain data stored in buffers as part of the checkpoint state, which allows checkpoint barriers to overtake these buffers. Thus, the checkpoint duration becomes independent of the current throughput as checkpoint barriers are effectively not embedded into the stream of data anymore.
Unaligned checkpoints can only be enabled if
get_checkpointing_mode()
isCheckpointingMode.EXACTLY_ONCE
.
-
enable_externalized_checkpoints
(cleanup_mode: pyflink.datastream.checkpoint_config.ExternalizedCheckpointCleanup) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Sets the mode for externalized checkpoint clean-up. Externalized checkpoints will be enabled automatically unless the mode is set to
ExternalizedCheckpointCleanup.NO_EXTERNALIZED_CHECKPOINTS
.Externalized checkpoints write their meta data out to persistent storage and are not automatically cleaned up when the owning job fails or is suspended (terminating with job status
FAILED
orSUSPENDED
). In this case, you have to manually clean up the checkpoint state, both the meta data and actual program state.The
ExternalizedCheckpointCleanup
mode defines how an externalized checkpoint should be cleaned up on job cancellation. If you choose to retain externalized checkpoints on cancellation you have to handle checkpoint clean-up manually when you cancel the job as well (terminating with job statusCANCELED
).The target directory for externalized checkpoints is configured via
org.apache.flink.configuration.CheckpointingOptions#CHECKPOINTS_DIRECTORY
.Example:
>>> config.enable_externalized_checkpoints( ... ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION)
- Parameters
cleanup_mode – Externalized checkpoint clean-up behaviour, the mode could be
ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION
,ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION
orExternalizedCheckpointCleanup.NO_EXTERNALIZED_CHECKPOINTS
Note
Deprecated in 1.15. Use
set_externalized_checkpoint_cleanup()
instead.
-
enable_unaligned_checkpoints
(enabled: bool = True) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Enables unaligned checkpoints, which greatly reduce checkpointing times under backpressure.
Unaligned checkpoints contain data stored in buffers as part of the checkpoint state, which allows checkpoint barriers to overtake these buffers. Thus, the checkpoint duration becomes independent of the current throughput as checkpoint barriers are effectively not embedded into the stream of data anymore.
Unaligned checkpoints can only be enabled if
get_checkpointing_mode()
isCheckpointingMode.EXACTLY_ONCE
.- Parameters
enabled –
True
if a checkpoints should be taken in unaligned mode.
-
get_alignment_timeout
() → pyflink.common.time.Duration[source]¶ Returns the alignment timeout, as configured via
set_alignment_timeout()
ororg.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions#ALIGNMENT_TIMEOUT
.- Returns
the alignment timeout.
-
get_checkpoint_interval
() → int[source]¶ Gets the interval in which checkpoints are periodically scheduled.
This setting defines the base interval. Checkpoint triggering may be delayed by the settings
get_max_concurrent_checkpoints()
andget_min_pause_between_checkpoints()
.- Returns
The checkpoint interval, in milliseconds.
-
get_checkpoint_storage
() → Optional[pyflink.datastream.checkpoint_storage.CheckpointStorage][source]¶ The checkpoint storage that has been configured for the Job, or None if none has been set.
-
get_checkpoint_timeout
() → int[source]¶ Gets the maximum time that a checkpoint may take before being discarded.
- Returns
The checkpoint timeout, in milliseconds.
-
get_checkpointing_mode
() → pyflink.datastream.checkpointing_mode.CheckpointingMode[source]¶ Gets the checkpointing mode (exactly-once vs. at-least-once).
See also
- Returns
The
CheckpointingMode
.
-
get_externalized_checkpoint_cleanup
() → Optional[pyflink.datastream.checkpoint_config.ExternalizedCheckpointCleanup][source]¶ Returns the cleanup behaviour for externalized checkpoints.
- Returns
The cleanup behaviour for externalized checkpoints or
None
if none is configured.
-
get_max_concurrent_checkpoints
() → int[source]¶ Gets the maximum number of checkpoint attempts that may be in progress at the same time. If this value is n, then no checkpoints will be triggered while n checkpoint attempts are currently in flight. For the next checkpoint to be triggered, one checkpoint attempt would need to finish or expire.
- Returns
The maximum number of concurrent checkpoint attempts.
-
get_min_pause_between_checkpoints
() → int[source]¶ Gets the minimal pause between checkpointing attempts. This setting defines how soon the checkpoint coordinator may trigger another checkpoint after it becomes possible to trigger another checkpoint with respect to the maximum number of concurrent checkpoints (see
get_max_concurrent_checkpoints()
).- Returns
The minimal pause before the next checkpoint is triggered.
-
get_tolerable_checkpoint_failure_number
() → int[source]¶ Get the defined number of consecutive checkpoint failures that will be tolerated, before the whole job is failed over.
- Returns
The maximum number of tolerated checkpoint failures.
-
is_checkpointing_enabled
() → bool[source]¶ Checks whether checkpointing is enabled.
- Returns
True if checkpointing is enables, false otherwise.
-
is_externalized_checkpoints_enabled
() → bool[source]¶ Returns whether checkpoints should be persisted externally.
- Returns
True
if checkpoints should be externalized, false otherwise.
-
is_fail_on_checkpointing_errors
() → bool[source]¶ This determines the behaviour of tasks if there is an error in their local checkpointing. If this returns true, tasks will fail as a reaction. If this returns false, task will only decline the failed checkpoint.
- Returns
True
if failing on checkpointing errors, false otherwise.
-
is_force_unaligned_checkpoints
() → bool[source]¶ Checks whether unaligned checkpoints are forced, despite iteration feedback or custom partitioners.
- Returns
True, if unaligned checkpoints are forced, false otherwise.
-
is_unaligned_checkpoints_enabled
() → bool[source]¶ Returns whether unaligned checkpoints are enabled.
- Returns
True
if unaligned checkpoints are enabled.
-
set_alignment_timeout
(alignment_timeout: pyflink.common.time.Duration) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Only relevant if
enable_unaligned_checkpoints()
is enabled.If
alignment_timeout
has value equal to0
, checkpoints will always start unaligned. Ifalignment_timeout
has value greater then0
, checkpoints will start aligned. If during checkpointing, checkpoint start delay exceeds thisalignment_timeout
, alignment will timeout and checkpoint will start working as unaligned checkpoint.- Parameters
alignment_timeout – The duration until the aligned checkpoint will be converted into an unaligned checkpoint.
-
set_checkpoint_interval
(checkpoint_interval: int) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Sets the interval in which checkpoints are periodically scheduled.
This setting defines the base interval. Checkpoint triggering may be delayed by the settings
set_max_concurrent_checkpoints()
andset_min_pause_between_checkpoints()
.- Parameters
checkpoint_interval – The checkpoint interval, in milliseconds.
-
set_checkpoint_storage
(storage: pyflink.datastream.checkpoint_storage.CheckpointStorage) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Checkpoint storage defines how stat backends checkpoint their state for fault tolerance in streaming applications. Various implementations store their checkpoints in different fashions and have different requirements and availability guarantees.
For example, JobManagerCheckpointStorage stores checkpoints in the memory of the JobManager. It is lightweight and without additional dependencies but is not highly available and only supports small state sizes. This checkpoint storage policy is convenient for local testing and development.
The FileSystemCheckpointStorage stores checkpoints in a filesystem. For systems like HDFS, NFS Drivs, S3, and GCS, this storage policy supports large state size, in the magnitude of many terabytes while providing a highly available foundation for stateful applications. This checkpoint storage policy is recommended for most production deployments.
-
set_checkpoint_storage_dir
(checkpoint_path: str) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Configures the application to write out checkpoint snapshots to the configured directory. See FileSystemCheckpointStorage for more details on checkpointing to a file system.
-
set_checkpoint_timeout
(checkpoint_timeout: int) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Sets the maximum time that a checkpoint may take before being discarded.
- Parameters
checkpoint_timeout – The checkpoint timeout, in milliseconds.
-
set_checkpointing_mode
(checkpointing_mode: pyflink.datastream.checkpointing_mode.CheckpointingMode) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Sets the checkpointing mode (
CheckpointingMode.EXACTLY_ONCE
vs.CheckpointingMode.AT_LEAST_ONCE
).Example:
>>> config.set_checkpointing_mode(CheckpointingMode.AT_LEAST_ONCE)
- Parameters
checkpointing_mode – The
CheckpointingMode
.
-
set_externalized_checkpoint_cleanup
(cleanup_mode: pyflink.datastream.checkpoint_config.ExternalizedCheckpointCleanup) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Sets the mode for externalized checkpoint clean-up. Externalized checkpoints will be enabled automatically unless the mode is set to
ExternalizedCheckpointCleanup.NO_EXTERNALIZED_CHECKPOINTS
.Externalized checkpoints write their meta data out to persistent storage and are not automatically cleaned up when the owning job fails or is suspended (terminating with job status
FAILED
orSUSPENDED
). In this case, you have to manually clean up the checkpoint state, both the meta data and actual program state.The
ExternalizedCheckpointCleanup
mode defines how an externalized checkpoint should be cleaned up on job cancellation. If you choose to retain externalized checkpoints on cancellation you have to handle checkpoint clean-up manually when you cancel the job as well (terminating with job statusCANCELED
).The target directory for externalized checkpoints is configured via
org.apache.flink.configuration.CheckpointingOptions#CHECKPOINTS_DIRECTORY
.Example:
>>> config.set_externalized_checkpoint_cleanup( ... ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION)
- Parameters
cleanup_mode – Externalized checkpoint clean-up behaviour, the mode could be
ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION
,ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION
orExternalizedCheckpointCleanup.NO_EXTERNALIZED_CHECKPOINTS
-
set_fail_on_checkpointing_errors
(fail_on_checkpointing_errors: bool) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Sets the expected behaviour for tasks in case that they encounter an error in their checkpointing procedure. If this is set to true, the task will fail on checkpointing error. If this is set to false, the task will only decline a the checkpoint and continue running. The default is true.
Example:
>>> config.set_fail_on_checkpointing_errors(False)
- Parameters
fail_on_checkpointing_errors –
True
if failing on checkpointing errors, false otherwise.
-
set_force_unaligned_checkpoints
(force_unaligned_checkpoints: bool = True) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Checks whether unaligned checkpoints are forced, despite currently non-checkpointable iteration feedback or custom partitioners.
- Parameters
force_unaligned_checkpoints – The flag to force unaligned checkpoints.
-
set_max_concurrent_checkpoints
(max_concurrent_checkpoints: int) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Sets the maximum number of checkpoint attempts that may be in progress at the same time. If this value is n, then no checkpoints will be triggered while n checkpoint attempts are currently in flight. For the next checkpoint to be triggered, one checkpoint attempt would need to finish or expire.
- Parameters
max_concurrent_checkpoints – The maximum number of concurrent checkpoint attempts.
-
set_min_pause_between_checkpoints
(min_pause_between_checkpoints: int) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Sets the minimal pause between checkpointing attempts. This setting defines how soon the checkpoint coordinator may trigger another checkpoint after it becomes possible to trigger another checkpoint with respect to the maximum number of concurrent checkpoints (see
set_max_concurrent_checkpoints()
).If the maximum number of concurrent checkpoints is set to one, this setting makes effectively sure that a minimum amount of time passes where no checkpoint is in progress at all.
- Parameters
min_pause_between_checkpoints – The minimal pause before the next checkpoint is triggered.
-
set_tolerable_checkpoint_failure_number
(tolerable_checkpoint_failure_number: int) → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ This defines how many consecutive checkpoint failures will be tolerated, before the whole job is failed over. The default value is 0, which means no checkpoint failures will be tolerated, and the job will fail on first reported checkpoint failure.
Example:
>>> config.set_tolerable_checkpoint_failure_number(2)
- Parameters
tolerable_checkpoint_failure_number – The maximum number of tolerated checkpoint failures.
-
-
class
pyflink.datastream.
CheckpointStorage
(j_checkpoint_storage)[source]¶ Bases:
object
Checkpoint storage defines how
StateBackend
’s store their state for fault-tolerance in streaming applications. Various implementations store their checkpoints in different fashions and have different requirements and availability guarantees.For example,
JobManagerCheckpointStorage
stores checkpoints in the memory of the JobManager. It is lightweight and without additional dependencies but is not scalable and only supports small state sizes. This checkpoints storage policy is convenient for local testing and development.FileSystemCheckpointStorage
stores checkpoints in a filesystem. For systems like HDFS NFS drives, S3, and GCS, this storage policy supports large state size, in the magnitude of many terabytes while providing a highly available foundation for streaming applications. This checkpoint storage policy is recommended for most production deployments.Raw Bytes Storage
The CheckpointStorage creates services for raw bytes storage.
The raw bytes storage (through the CheckpointStreamFactory) is the fundamental service that simply stores bytes in a fault tolerant fashion. This service is used by the JobManager to store checkpoint and recovery metadata and is typically also used by the keyed- and operator- state backends to store checkpoint state.
Serializability
Implementations need to be serializable(java.io.Serializable), because they are distributed across parallel processes (for distributed execution) together with the streaming application code.
Because of that CheckpointStorage implementations are meant to be like _factories_ that create the proper state stores that provide access to the persistent layer. That way, the storage policy can be very lightweight (contain only configurations) which makes it easier to be serializable.
Thread Safety
Checkpoint storage implementations have to be thread-safe. Multiple threads may be creating streams concurrently.
-
class
pyflink.datastream.
CheckpointingMode
(value)[source]¶ Bases:
enum.Enum
The checkpointing mode defines what consistency guarantees the system gives in the presence of failures.
When checkpointing is activated, the data streams are replayed such that lost parts of the processing are repeated. For stateful operations and functions, the checkpointing mode defines whether the system draws checkpoints such that a recovery behaves as if the operators/functions see each record “exactly once” (
CheckpointingMode.EXACTLY_ONCE
), or whether the checkpoints are drawn in a simpler fashion that typically encounters some duplicates upon recovery (CheckpointingMode.AT_LEAST_ONCE
)Sets the checkpointing mode to “exactly once”. This mode means that the system will checkpoint the operator and user function state in such a way that, upon recovery, every record will be reflected exactly once in the operator state.
For example, if a user function counts the number of elements in a stream, this number will consistently be equal to the number of actual elements in the stream, regardless of failures and recovery.
Note that this does not mean that each record flows through the streaming data flow only once. It means that upon recovery, the state of operators/functions is restored such that the resumed data streams pick up exactly at after the last modification to the state.
Note that this mode does not guarantee exactly-once behavior in the interaction with external systems (only state in Flink’s operators and user functions). The reason for that is that a certain level of “collaboration” is required between two systems to achieve exactly-once guarantees. However, for certain systems, connectors can be written that facilitate this collaboration.
This mode sustains high throughput. Depending on the data flow graph and operations, this mode may increase the record latency, because operators need to align their input streams, in order to create a consistent snapshot point. The latency increase for simple dataflows (no repartitioning) is negligible. For simple dataflows with repartitioning, the average latency remains small, but the slowest records typically have an increased latency.
Sets the checkpointing mode to “at least once”. This mode means that the system will checkpoint the operator and user function state in a simpler way. Upon failure and recovery, some records may be reflected multiple times in the operator state.
For example, if a user function counts the number of elements in a stream, this number will equal to, or larger, than the actual number of elements in the stream, in the presence of failure and recovery.
This mode has minimal impact on latency and may be preferable in very-low latency scenarios, where a sustained very-low latency (such as few milliseconds) is needed, and where occasional duplicate messages (on recovery) do not matter.
-
AT_LEAST_ONCE
= 1¶
-
EXACTLY_ONCE
= 0¶
-
-
class
pyflink.datastream.
CoFlatMapFunction
[source]¶ Bases:
pyflink.datastream.functions.Function
A CoFlatMapFunction implements a flat-map transformation over two connected streams.
The same instance of the transformation function is used to transform both of the connected streams. That way, the stream transformations can share state.
An example for the use of connected streams would be to apply rules that change over time onto elements of a stream. One of the connected streams has the rules, the other stream the elements to apply the rules to. The operation on the connected stream maintains the current set of rules in the state. It may receive either a rule update (from the first stream) and update the state, or a data element (from the second stream) and apply the rules in the state to the element. The result of applying the rules would be emitted.
The basic syntax for using a CoFlatMapFunction is as follows:
- ::
>>> ds1 = ... >>> ds2 = ...
>>> class MyCoFlatMapFunction(CoFlatMapFunction): >>> def flat_map1(self, value): >>> for i in range(value): >>> yield i >>> def flat_map2(self, value): >>> for i in range(value): >>> yield i
>>> new_ds = ds1.connect(ds2).flat_map(MyCoFlatMapFunction())
-
close
()¶
-
abstract
flat_map1
(value)[source]¶ This method is called for each element in the first of the connected streams.
- Parameters
value – The input value.
- Returns
A generator
-
abstract
flat_map2
(value)[source]¶ This method is called for each element in the second of the connected streams.
- Parameters
value – The input value.
- Returns
A generator
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
class
pyflink.datastream.
CoMapFunction
[source]¶ Bases:
pyflink.datastream.functions.Function
A CoMapFunction implements a map() transformation over two connected streams.
The same instance of the transformation function is used to transform both of the connected streams. That way, the stream transformations can share state.
The basic syntax for using a CoMapFunction is as follows:
- ::
>>> ds1 = ... >>> ds2 = ... >>> new_ds = ds1.connect(ds2).map(MyCoMapFunction())
-
close
()¶
-
abstract
map1
(value)[source]¶ This method is called for each element in the first of the connected streams.
- Parameters
value – The stream element
- Returns
The resulting element
-
abstract
map2
(value)[source]¶ This method is called for each element in the second of the connected streams.
- Parameters
value – The stream element
- Returns
The resulting element
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
class
pyflink.datastream.
CoProcessFunction
[source]¶ Bases:
pyflink.datastream.functions.Function
A function that processes elements of two streams and produces a single output one.
The function will be called for every element in the input streams and can produce zero or more output elements. Contrary to the
CoFlatMapFunction
, this function can also query the time (both event and processing) and set timers, through the providedCoProcessFunction.Context
. When reacting to the firing of set timers the function can emit yet more elements.An example use-case for connected streams would be the application of a set of rules that change over time ({@code stream A}) to the elements contained in another stream (stream {@code B}). The rules contained in {@code stream A} can be stored in the state and wait for new elements to arrive on {@code stream B}. Upon reception of a new element on {@code stream B}, the function can now apply the previously stored rules to the element and directly emit a result, and/or register a timer that will trigger an action in the future.
-
class
Context
[source]¶ Bases:
abc.ABC
-
close
()¶
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
abstract
process_element1
(value, ctx: pyflink.datastream.functions.CoProcessFunction.Context)[source]¶ This method is called for each element in the first of the connected streams.
This function can output zero or more elements using the Collector parameter and also update internal state or set timers using the Context parameter.
- Parameters
value – The input value.
ctx – A Context that allows querying the timestamp of the element and getting a TimerService for registering timers and querying the time. The context is only valid during the invocation of this method, do not store it.
-
abstract
process_element2
(value, ctx: pyflink.datastream.functions.CoProcessFunction.Context)[source]¶ This method is called for each element in the second of the connected streams.
This function can output zero or more elements using the Collector parameter and also update internal state or set timers using the Context parameter.
- Parameters
value – The input value.
ctx – A Context that allows querying the timestamp of the element and getting a TimerService for registering timers and querying the time. The context is only valid during the invocation of this method, do not store it.
-
class
-
class
pyflink.datastream.
ConnectedStreams
(stream1: pyflink.datastream.data_stream.DataStream, stream2: pyflink.datastream.data_stream.DataStream)[source]¶ Bases:
object
ConnectedStreams represent two connected streams of (possibly) different data types. Connected streams are useful for cases where operations on one stream directly affect the operations on the other stream, usually via shared state between the streams.
An example for the use of connected streams would be to apply rules that change over time onto another stream. One of the connected streams has the rules, the other stream the elements to apply the rules to. The operation on the connected stream maintains the current set of rules in the state. It may receive either a rule update and update the state or a data element and apply the rules in the state to the element.
The connected stream can be conceptually viewed as a union stream of an Either type, that holds either the first stream’s type or the second stream’s type.
-
flat_map
(func: pyflink.datastream.functions.CoFlatMapFunction, output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Applies a CoFlatMap transformation on a ConnectedStreams and maps the output to a common type. The transformation calls a CoFlatMapFunction.flatMap1 for each element of the first input and CoFlatMapFunction.flatMap2 for each element of the second input. Each CoFlatMapFunction call returns any number of elements including none.
- Parameters
func – The CoFlatMapFunction used to jointly transform the two input DataStreams
output_type – TypeInformation for the result type of the function.
- Returns
The transformed DataStream
-
key_by
(key_selector1: Union[Callable, pyflink.datastream.functions.KeySelector], key_selector2: Union[Callable, pyflink.datastream.functions.KeySelector], key_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.ConnectedStreams[source]¶ KeyBy operation for connected data stream. Assigns keys to the elements of input1 and input2 using keySelector1 and keySelector2 with explicit type information for the common key type.
- Parameters
key_selector1 – The KeySelector used for grouping the first input.
key_selector2 – The KeySelector used for grouping the second input.
key_type – The type information of the common key type
- Returns
The partitioned ConnectedStreams
-
map
(func: pyflink.datastream.functions.CoMapFunction, output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Applies a CoMap transformation on a ConnectedStreams and maps the output to a common type. The transformation calls a CoMapFunction.map1 for each element of the first input and CoMapFunction.map2 for each element of the second input. Each CoMapFunction call returns exactly one element.
- Parameters
func – The CoMapFunction used to jointly transform the two input DataStreams
output_type – TypeInformation for the result type of the function.
- Returns
The transformed DataStream
-
-
class
pyflink.datastream.
CountWindow
(id: int)[source]¶ Bases:
pyflink.datastream.window.Window
A Window that represents a count window. For each count window, we will assign a unique id. Thus this CountWindow can act as namespace part in state. We can attach data to each different CountWindow.
-
class
pyflink.datastream.
CustomCheckpointStorage
(j_custom_checkpoint_storage)[source]¶ Bases:
pyflink.datastream.checkpoint_storage.CheckpointStorage
A wrapper of customized java checkpoint storage.
-
class
pyflink.datastream.
CustomStateBackend
(j_custom_state_backend)[source]¶ Bases:
pyflink.datastream.state_backend.StateBackend
A wrapper of customized java state backend.
-
class
pyflink.datastream.
DataStream
(j_data_stream)[source]¶ Bases:
object
A DataStream represents a stream of elements of the same type. A DataStream can be transformed into another DataStream by applying a transformation as for example:
- ::
>>> DataStream.map(MapFunctionImpl()) >>> DataStream.filter(FilterFunctionImpl())
-
add_sink
(sink_func: pyflink.datastream.functions.SinkFunction) → pyflink.datastream.data_stream.DataStreamSink[source]¶ Adds the given sink to this DataStream. Only streams with sinks added will be executed once the StreamExecutionEnvironment.execute() method is called.
- Parameters
sink_func – The SinkFunction object.
- Returns
The closed DataStream.
-
assign_timestamps_and_watermarks
(watermark_strategy: pyflink.common.watermark_strategy.WatermarkStrategy) → pyflink.datastream.data_stream.DataStream[source]¶ Assigns timestamps to the elements in the data stream and generates watermarks to signal event time progress. The given {@link WatermarkStrategy} is used to create a TimestampAssigner and WatermarkGenerator.
- Parameters
watermark_strategy – The strategy to generate watermarks based on event timestamps.
- Returns
The stream after the transformation, with assigned timestamps and watermarks.
-
broadcast
() → ’DataStream’[source]¶ -
broadcast
(broadcast_state_descriptor: MapStateDescriptor, *other_broadcast_state_descriptors: MapStateDescriptor) → ’BroadcastStream’ Sets the partitioning of the DataStream so that the output elements are broadcasted to every parallel instance of the next operation.
If
MapStateDescriptor
s are passed in, it returns aBroadcastStream
withBroadcastState
s implicitly created as the descriptors specified.Example:
>>> map_state_desc1 = MapStateDescriptor("state1", Types.INT(), Types.INT()) >>> map_state_desc2 = MapStateDescriptor("state2", Types.INT(), Types.STRING()) >>> broadcast_stream = ds1.broadcast(map_state_desc1, map_state_desc2) >>> broadcast_connected_stream = ds2.connect(broadcast_stream)
- Parameters
broadcast_state_descriptor – the first MapStateDescriptor describing BroadcastState.
other_broadcast_state_descriptors – the rest of MapStateDescriptors describing BroadcastStates, if any.
- Returns
The DataStream with broadcast partitioning set or a BroadcastStream which can be used in
connect()
to create a BroadcastConnectedStream for further processing of the elements.
Changed in version 1.16.0: Support return BroadcastStream
-
cache
() → pyflink.datastream.data_stream.CachedDataStream[source]¶ Cache the intermediate result of the transformation. Only support bounded streams and currently only block mode is supported. The cache is generated lazily at the first time the intermediate result is computed. The cache will be clear when the StreamExecutionEnvironment close.
- Returns
The cached DataStream that can use in later job to reuse the cached intermediate result.
New in version 1.16.0.
-
connect
(ds: ‘DataStream’) → ’ConnectedStreams’[source]¶ -
connect
(ds: ‘BroadcastStream’) → ’BroadcastConnectedStream’ If ds is a
DataStream
, creates a newConnectedStreams
by connecting DataStream outputs of (possible) different types with each other. The DataStreams connected using this operator can be used with CoFunctions to apply joint transformations.If ds is a
BroadcastStream
, creates a newBroadcastConnectedStream
by connecting the currentDataStream
with aBroadcastStream
. The latter can be created using thebroadcast()
method. The resulting stream can be further processed using theBroadcastConnectedStream.process()
method.- Parameters
ds – The DataStream or BroadcastStream with which this stream will be connected.
- Returns
The ConnectedStreams or BroadcastConnectedStream.
Changed in version 1.16.0: Support connect BroadcastStream
-
disable_chaining
() → pyflink.datastream.data_stream.DataStream[source]¶ Turns off chaining for this operator so thread co-location will not be used as an optimization. Chaining can be turned off for the whole job by StreamExecutionEnvironment.disableOperatorChaining() however it is not advised for performance consideration.
- Returns
The operator with chaining disabled.
-
execute_and_collect
(job_execution_name: str = None, limit: int = None) → Union[pyflink.datastream.data_stream.CloseableIterator, list][source]¶ Triggers the distributed execution of the streaming dataflow and returns an iterator over the elements of the given DataStream.
The DataStream application is executed in the regular distributed manner on the target environment, and the events from the stream are polled back to this application process and thread through Flink’s REST API.
The returned iterator must be closed to free all cluster resources.
- Parameters
job_execution_name – The name of the job execution.
limit – The limit for the collected elements.
-
filter
(func: Union[Callable, pyflink.datastream.functions.FilterFunction]) → pyflink.datastream.data_stream.DataStream[source]¶ Applies a Filter transformation on a DataStream. The transformation calls a FilterFunction for each element of the DataStream and retains only those element for which the function returns true. Elements for which the function returns false are filtered.
- Parameters
func – The FilterFunction that is called for each element of the DataStream.
- Returns
The filtered DataStream.
-
flat_map
(func: Union[Callable, pyflink.datastream.functions.FlatMapFunction], output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Applies a FlatMap transformation on a DataStream. The transformation calls a FlatMapFunction for each element of the DataStream. Each FlatMapFunction call can return any number of elements including none.
- Parameters
func – The FlatMapFunction that is called for each element of the DataStream.
output_type – The type information of output data.
- Returns
The transformed DataStream.
-
force_non_parallel
() → pyflink.datastream.data_stream.DataStream[source]¶ Sets the parallelism and maximum parallelism of this operator to one. And mark this operator cannot set a non-1 degree of parallelism.
- Returns
The operator with only one parallelism.
-
forward
() → pyflink.datastream.data_stream.DataStream[source]¶ Sets the partitioning of the DataStream so that the output elements are forwarded to the local sub-task of the next operation.
- Returns
The DataStream with forward partitioning set.
-
get_execution_environment
()[source]¶ Returns the StreamExecutionEnvironment that was used to create this DataStream.
- Returns
The Execution Environment.
-
get_name
() → str[source]¶ Gets the name of the current data stream. This name is used by the visualization and logging during runtime.
- Returns
Name of the stream.
-
get_side_output
(output_tag: pyflink.datastream.output_tag.OutputTag) → pyflink.datastream.data_stream.DataStream[source]¶ Gets the
DataStream
that contains the elements that are emitted from an operation into the side output with the givenOutputTag
.- Parameters
output_tag – output tag for the side stream
- Returns
The DataStream with specified output tag
New in version 1.16.0.
-
get_type
() → pyflink.common.typeinfo.TypeInformation[source]¶ Gets the type of the stream.
- Returns
The type of the DataStream.
-
key_by
(key_selector: Union[Callable, pyflink.datastream.functions.KeySelector], key_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.KeyedStream[source]¶ Creates a new KeyedStream that uses the provided key for partitioning its operator states.
- Parameters
key_selector – The KeySelector to be used for extracting the key for partitioning.
key_type – The type information describing the key type.
- Returns
The DataStream with partitioned state(i.e. KeyedStream).
-
map
(func: Union[Callable, pyflink.datastream.functions.MapFunction], output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Applies a Map transformation on a DataStream. The transformation calls a MapFunction for each element of the DataStream. Each MapFunction call returns exactly one element.
Note that If user does not specify the output data type, the output data will be serialized as pickle primitive byte array.
- Parameters
func – The MapFunction that is called for each element of the DataStream.
output_type – The type information of the MapFunction output data.
- Returns
The transformed DataStream.
-
name
(name: str) → pyflink.datastream.data_stream.DataStream[source]¶ Sets the name of the current data stream. This name is used by the visualization and logging during runtime.
- Parameters
name – Name of the stream.
- Returns
The named operator.
-
partition_custom
(partitioner: Union[Callable, pyflink.datastream.functions.Partitioner], key_selector: Union[Callable, pyflink.datastream.functions.KeySelector]) → pyflink.datastream.data_stream.DataStream[source]¶ Partitions a DataStream on the key returned by the selector, using a custom partitioner. This method takes the key selector to get the key to partition on, and a partitioner that accepts the key type.
Note that this method works only on single field keys, i.e. the selector cannot return tuples of fields.
- Parameters
partitioner – The partitioner to assign partitions to keys.
key_selector – The KeySelector with which the DataStream is partitioned.
- Returns
The partitioned DataStream.
-
print
(sink_identifier: str = None) → pyflink.datastream.data_stream.DataStreamSink[source]¶ Writes a DataStream to the standard output stream (stdout). For each element of the DataStream the object string is written.
NOTE: This will print to stdout on the machine where the code is executed, i.e. the Flink worker, and is not fault tolerant.
- Parameters
sink_identifier – The string to prefix the output with.
- Returns
The closed DataStream.
-
process
(func: pyflink.datastream.functions.ProcessFunction, output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Applies the given ProcessFunction on the input stream, thereby creating a transformed output stream.
The function will be called for every element in the input streams and can produce zero or more output elements.
- Parameters
func – The ProcessFunction that is called for each element in the stream.
output_type – TypeInformation for the result type of the function.
- Returns
The transformed DataStream.
-
project
(*field_indexes: int) → pyflink.datastream.data_stream.DataStream[source]¶ Initiates a Project transformation on a Tuple DataStream.
Note that only Tuple DataStreams can be projected.
- Parameters
field_indexes – The field indexes of the input tuples that are retained. The order of fields in the output tuple corresponds to the order of field indexes.
- Returns
The projected DataStream.
-
rebalance
() → pyflink.datastream.data_stream.DataStream[source]¶ Sets the partitioning of the DataStream so that the output elements are distributed evenly to instances of the next operation in a round-robin fashion.
- Returns
The DataStream with rebalance partition set.
-
rescale
() → pyflink.datastream.data_stream.DataStream[source]¶ Sets the partitioning of the DataStream so that the output elements are distributed evenly to a subset of instances of the next operation in a round-robin fashion.
The subset of downstream operations to which the upstream operation sends elements depends on the degree of parallelism of both the upstream and downstream operation. For example, if the upstream operation has parallelism 2 and the downstream operation has parallelism 4, then one upstream operation would distribute elements to two downstream operations. If, on the other hand, the downstream operation has parallelism 4 then two upstream operations will distribute to one downstream operation while the other two upstream operations will distribute to the other downstream operations.
In cases where the different parallelisms are not multiples of each one or several downstream operations will have a differing number of inputs from upstream operations.
- Returns
The DataStream with rescale partitioning set.
-
set_buffer_timeout
(timeout_millis: int) → pyflink.datastream.data_stream.DataStream[source]¶ Sets the buffering timeout for data produced by this operation. The timeout defines how long data may linger ina partially full buffer before being sent over the network.
Lower timeouts lead to lower tail latencies, but may affect throughput. Timeouts of 1 ms still sustain high throughput, even for jobs with high parallelism.
A value of ‘-1’ means that the default buffer timeout should be used. A value of ‘0’ indicates that no buffering should happen, and all records/events should be immediately sent through the network, without additional buffering.
- Parameters
timeout_millis – The maximum time between two output flushes.
- Returns
The operator with buffer timeout set.
-
set_description
(description: str) → pyflink.datastream.data_stream.DataStream[source]¶ Sets the description for this operator.
Description is used in json plan and web ui, but not in logging and metrics where only name is available. Description is expected to provide detailed information about the operator, while name is expected to be more simple, providing summary information only, so that we can have more user-friendly logging messages and metric tags without losing useful messages for debugging.
- Parameters
description – The description for this operator.
- Returns
The operator with new description.
New in version 1.15.0.
-
set_max_parallelism
(max_parallelism: int) → pyflink.datastream.data_stream.DataStream[source]¶ Sets the maximum parallelism of this operator.
The maximum parallelism specifies the upper bound for dynamic scaling. It also defines the number of key groups used for partitioned state.
- Parameters
max_parallelism – Maximum parallelism.
- Returns
The operator with set maximum parallelism.
-
set_parallelism
(parallelism: int) → pyflink.datastream.data_stream.DataStream[source]¶ Sets the parallelism for this operator.
- Parameters
parallelism – THe parallelism for this operator.
- Returns
The operator with set parallelism.
-
set_uid_hash
(uid_hash: str) → pyflink.datastream.data_stream.DataStream[source]¶ Sets an user provided hash for this operator. This will be used AS IS the create the JobVertexID. The user provided hash is an alternative to the generated hashed, that is considered when identifying an operator through the default hash mechanics fails (e.g. because of changes between Flink versions).
Important: this should be used as a workaround or for trouble shooting. The provided hash needs to be unique per transformation and job. Otherwise, job submission will fail. Furthermore, you cannot assign user-specified hash to intermediate nodes in an operator chain and trying so will let your job fail.
A use case for this is in migration between Flink versions or changing the jobs in a way that changes the automatically generated hashes. In this case, providing the previous hashes directly through this method (e.g. obtained from old logs) can help to reestablish a lost mapping from states to their target operator.
- Parameters
uid_hash – The user provided hash for this operator. This will become the jobVertexID, which is shown in the logs and web ui.
- Returns
The operator with the user provided hash.
-
shuffle
() → pyflink.datastream.data_stream.DataStream[source]¶ Sets the partitioning of the DataStream so that the output elements are shuffled uniformly randomly to the next operation.
- Returns
The DataStream with shuffle partitioning set.
-
sink_to
(sink: pyflink.datastream.connectors.base.Sink) → pyflink.datastream.data_stream.DataStreamSink[source]¶ Adds the given sink to this DataStream. Only streams with sinks added will be executed once the
execute()
method is called.- Parameters
sink – The user defined sink.
- Returns
The closed DataStream.
-
slot_sharing_group
(slot_sharing_group: Union[str, pyflink.datastream.slot_sharing_group.SlotSharingGroup]) → pyflink.datastream.data_stream.DataStream[source]¶ Sets the slot sharing group of this operation. Parallel instances of operations that are in the same slot sharing group will be co-located in the same TaskManager slot, if possible.
Operations inherit the slot sharing group of input operations if all input operations are in the same slot sharing group and no slot sharing group was explicitly specified.
Initially an operation is in the default slot sharing group. An operation can be put into the default group explicitly by setting the slot sharing group to ‘default’.
- Parameters
slot_sharing_group – The slot sharing group name or which contains name and its resource spec.
- Returns
This operator.
-
start_new_chain
() → pyflink.datastream.data_stream.DataStream[source]¶ Starts a new task chain beginning at this operator. This operator will be chained (thread co-located for increased performance) to any previous tasks even if possible.
- Returns
The operator with chaining set.
-
uid
(uid: str) → pyflink.datastream.data_stream.DataStream[source]¶ Sets an ID for this operator. The specified ID is used to assign the same operator ID across job submissions (for example when starting a job from a savepoint).
Important: this ID needs to be unique per transformation and job. Otherwise, job submission will fail.
- Parameters
uid – The unique user-specified ID of this transformation.
- Returns
The operator with the specified ID.
-
union
(*streams: pyflink.datastream.data_stream.DataStream) → pyflink.datastream.data_stream.DataStream[source]¶ Creates a new DataStream by merging DataStream outputs of the same type with each other. The DataStreams merged using this operator will be transformed simultaneously.
- Parameters
streams – The DataStream to union outputwith.
- Returns
The DataStream.
-
window_all
(window_assigner: pyflink.datastream.window.WindowAssigner) → pyflink.datastream.data_stream.AllWindowedStream[source]¶ Windows this data stream to a AllWindowedStream, which evaluates windows over a non key grouped stream. Elements are put into windows by a WindowAssigner. The grouping of elements is done by window.
A Trigger can be defined to specify when windows are evaluated. However, WindowAssigners have a default Trigger that is used if a Trigger is not specified.
- Parameters
window_assigner – The WindowAssigner that assigns elements to windows.
- Returns
The trigger windows data stream.
New in version 1.16.0.
-
class
pyflink.datastream.
DataStreamSink
(j_data_stream_sink)[source]¶ Bases:
object
A Stream Sink. This is used for emitting elements from a streaming topology.
-
disable_chaining
() → pyflink.datastream.data_stream.DataStreamSink[source]¶ Turns off chaining for this operator so thread co-location will not be used as an optimization. Chaining can be turned off for the whole job by StreamExecutionEnvironment.disableOperatorChaining() however it is not advised for performance consideration.
- Returns
The operator with chaining disabled.
-
name
(name: str) → pyflink.datastream.data_stream.DataStreamSink[source]¶ Sets the name of this sink. THis name is used by the visualization and logging during runtime.
- Parameters
name – The name of this sink.
- Returns
The named sink.
-
set_description
(description: str) → pyflink.datastream.data_stream.DataStreamSink[source]¶ Sets the description for this sink.
Description is used in json plan and web ui, but not in logging and metrics where only name is available. Description is expected to provide detailed information about the sink, while name is expected to be more simple, providing summary information only, so that we can have more user-friendly logging messages and metric tags without losing useful messages for debugging.
- Parameters
description – The description for this sink.
- Returns
The sink with new description.
New in version 1.15.0.
-
set_parallelism
(parallelism: int) → pyflink.datastream.data_stream.DataStreamSink[source]¶ Sets the parallelism for this operator.
- Parameters
parallelism – THe parallelism for this operator.
- Returns
The operator with set parallelism.
-
set_uid_hash
(uid_hash: str) → pyflink.datastream.data_stream.DataStreamSink[source]¶ Sets an user provided hash for this operator. This will be used AS IS the create the JobVertexID. The user provided hash is an alternative to the generated hashed, that is considered when identifying an operator through the default hash mechanics fails (e.g. because of changes between Flink versions).
Important: this should be used as a workaround or for trouble shooting. The provided hash needs to be unique per transformation and job. Otherwise, job submission will fail. Furthermore, you cannot assign user-specified hash to intermediate nodes in an operator chain and trying so will let your job fail.
A use case for this is in migration between Flink versions or changing the jobs in a way that changes the automatically generated hashes. In this case, providing the previous hashes directly through this method (e.g. obtained from old logs) can help to reestablish a lost mapping from states to their target operator.
- Parameters
uid_hash – The user provided hash for this operator. This will become the jobVertexID, which is shown in the logs and web ui.
- Returns
The operator with the user provided hash.
-
slot_sharing_group
(slot_sharing_group: Union[str, pyflink.datastream.slot_sharing_group.SlotSharingGroup]) → pyflink.datastream.data_stream.DataStreamSink[source]¶ Sets the slot sharing group of this operation. Parallel instances of operations that are in the same slot sharing group will be co-located in the same TaskManager slot, if possible.
Operations inherit the slot sharing group of input operations if all input operations are in the same slot sharing group and no slot sharing group was explicitly specified.
Initially an operation is in the default slot sharing group. An operation can be put into the default group explicitly by setting the slot sharing group to ‘default’.
- Parameters
slot_sharing_group – The slot sharing group name or which contains name and its resource spec.
- Returns
This operator.
-
uid
(uid: str) → pyflink.datastream.data_stream.DataStreamSink[source]¶ Sets an ID for this operator. The specified ID is used to assign the same operator ID across job submissions (for example when starting a job from a savepoint).
Important: this ID needs to be unique per transformation and job. Otherwise, job submission will fail.
- Parameters
uid – The unique user-specified ID of this transformation.
- Returns
The operator with the specified ID.
-
-
class
pyflink.datastream.
EmbeddedRocksDBStateBackend
(enable_incremental_checkpointing=None, j_embedded_rocks_db_state_backend=None)[source]¶ Bases:
pyflink.datastream.state_backend.StateBackend
A State Backend that stores its state in an embedded
RocksDB
instance. This state backend can store very large state that exceeds memory and spills to local disk.All key/value state (including windows) is stored in the key/value index of RocksDB. For persistence against loss of machines, please configure a CheckpointStorage instance for the Job.
The behavior of the RocksDB instances can be parametrized by setting RocksDB Options using the methods
set_predefined_options()
andset_options()
.-
get_db_storage_paths
() → List[str][source]¶ Gets the configured local DB storage paths, or null, if none were configured.
Under these directories on the TaskManager, RocksDB stores its SST files and metadata files. These directories do not need to be persistent, they can be ephermeral, meaning that they are lost on a machine failure, because state in RocksDB is persisted in checkpoints.
If nothing is configured, these directories default to the TaskManager’s local temporary file directories.
- Returns
The list of configured local DB storage paths.
-
get_number_of_transfer_threads
() → int[source]¶ Gets the number of threads used to transfer files while snapshotting/restoring.
- Returns
The number of threads used to transfer files while snapshotting/restoring.
-
get_options
() → Optional[str][source]¶ Gets the fully-qualified class name of the options factory in Java that lazily creates the RocksDB options.
- Returns
The fully-qualified class name of the options factory in Java.
-
get_predefined_options
() → pyflink.datastream.state_backend.PredefinedOptions[source]¶ Gets the current predefined options for RocksDB. The default options (if nothing was set via
setPredefinedOptions()
) arePredefinedOptions.DEFAULT
.If user-configured options within
RocksDBConfigurableOptions
is set (through flink-conf.yaml) or a user-defined options factory is set (viasetOptions()
), then the options from the factory are applied on top of the predefined and customized options.See also
- Returns
Current predefined options.
-
is_incremental_checkpoints_enabled
() → bool[source]¶ Gets whether incremental checkpoints are enabled for this state backend.
- Returns
True if incremental checkpoints are enabled, false otherwise.
-
set_db_storage_paths
(*paths: str)[source]¶ Sets the directories in which the local RocksDB database puts its files (like SST and metadata files). These directories do not need to be persistent, they can be ephemeral, meaning that they are lost on a machine failure, because state in RocksDB is persisted in checkpoints.
If nothing is configured, these directories default to the TaskManager’s local temporary file directories.
Each distinct state will be stored in one path, but when the state backend creates multiple states, they will store their files on different paths.
Passing
None
to this function restores the default behavior, where the configured temp directories will be used.- Parameters
paths – The paths across which the local RocksDB database files will be spread. this parameter is optional.
-
set_number_of_transfer_threads
(number_of_transfering_threads: int)[source]¶ Sets the number of threads used to transfer files while snapshotting/restoring.
- Parameters
number_of_transfering_threads – The number of threads used to transfer files while snapshotting/restoring.
-
set_options
(options_factory_class_name: str)[source]¶ Sets
org.rocksdb.Options
for the RocksDB instances. Because the options are not serializable and hold native code references, they must be specified through a factory.The options created by the factory here are applied on top of the pre-defined options profile selected via
set_predefined_options()
and user-configured options from configuration set through flink-conf.yaml with keys inRocksDBConfigurableOptions
.- Parameters
options_factory_class_name – The fully-qualified class name of the options factory in Java that lazily creates the RocksDB options. The options factory must have a default constructor.
-
set_predefined_options
(options: pyflink.datastream.state_backend.PredefinedOptions)[source]¶ Sets the predefined options for RocksDB.
If user-configured options within
RocksDBConfigurableOptions
is set (through flink-conf.yaml) or a user-defined options factory is set (viasetOptions()
), then the options from the factory are applied on top of the here specified predefined options and customized options.Example:
>>> state_backend.set_predefined_options(PredefinedOptions.SPINNING_DISK_OPTIMIZED)
- Parameters
options – The options to set (must not be null), see
PredefinedOptions
.
-
-
class
pyflink.datastream.
ExternalizedCheckpointCleanup
(value)[source]¶ Bases:
enum.Enum
Cleanup behaviour for externalized checkpoints when the job is cancelled.
Delete externalized checkpoints on job cancellation.
All checkpoint state will be deleted when you cancel the owning job, both the meta data and actual program state. Therefore, you cannot resume from externalized checkpoints after the job has been cancelled.
Note that checkpoint state is always kept if the job terminates with state
FAILED
.Retain externalized checkpoints on job cancellation.
All checkpoint state is kept when you cancel the owning job. You have to manually delete both the checkpoint meta data and actual program state after cancelling the job.
Note that checkpoint state is always kept if the job terminates with state
FAILED
.Externalized checkpoints are disabled completely.
-
DELETE_ON_CANCELLATION
= 0¶
-
NO_EXTERNALIZED_CHECKPOINTS
= 2¶
-
RETAIN_ON_CANCELLATION
= 1¶
-
-
class
pyflink.datastream.
FileSystemCheckpointStorage
(checkpoint_path=None, file_state_size_threshold=None, write_buffer_size=- 1, j_filesystem_checkpoint_storage=None)[source]¶ Bases:
pyflink.datastream.checkpoint_storage.CheckpointStorage
FileSystemCheckpointStorage checkpoints state as files to a filesystem.
Each checkpoint will store all its files in a subdirectory that includes the checkpoints number, such as hdfs://namenode:port/flink-checkpoints/chk-17/.
State Size Considerations
This checkpoint storage stores small state chunks directly with the metadata, to avoid creating many small files. The threshold for that is configurable. When increasing this threshold, the size of the checkpoint metadata increases. The checkpoint metadata of all retained completed checkpoints needs to fit into the JobManager’s heap memory. This is typically not a problem, unless the threashold get_min_file_size_threshold is increased significantly.
Persistence Guarantees
Checkpoints from this checkpoint storage are as persistent and available as the filesystem that it is written to. If the file system is a persistent distributed file system, this checkpoint storage supports highly available setups. The backend additionally supports savepoints and externalized checkpoints.
Configuration
As for all checkpoint storage policies, this backend can either be configured within the application (by creating the storage with the respective constructor parameters and setting it on the execution environment) or by specifying it in the Flink configuration.
If the checkpoint storage was specified in the application, it may pick up additional configuration parameters from the Flink configuration. For example, if the storage is configured in the application without a default savepoint directory, it will pick up a default savepoint directory specified in the Flink configuration of the running job/cluster.
-
MAX_FILE_STATE_THRESHOLD
= 1048576¶
-
get_checkpoint_path
() → str[source]¶ Gets the base directory where all the checkpoints are stored. The job-specific checkpoint directory is created inside this directory.
- Returns
The base directory for checkpoints.
-
get_min_file_size_threshold
() → int[source]¶ Gets the threshold below which state is stored as part of the metadata, rather than in file. This threshold ensures the backend does not create a large amount of small files, where potentially the file pointers are larget than the state itself.
-
-
class
pyflink.datastream.
FilterFunction
[source]¶ Bases:
pyflink.datastream.functions.Function
A filter function is a predicate applied individually to each record. The predicate decides whether to keep the element, or to discard it.
The basic syntax for using a FilterFunction is as follows:
- ::
>>> ds = ... >>> result = ds.filter(MyFilterFunction())
Note that the system assumes that the function does not modify the elements on which the predicate is applied. Violating this assumption can lead to incorrect results.
-
close
()¶
-
abstract
filter
(value)[source]¶ The filter function that evaluates the predicate.
- Parameters
value – The value to be filtered.
- Returns
True for values that should be retained, false for values to be filtered out.
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
class
pyflink.datastream.
FlatMapFunction
[source]¶ Bases:
pyflink.datastream.functions.Function
Base class for flatMap functions. FlatMap functions take elements and transform them, into zero, one, or more elements. Typical applications can be splitting elements, or unnesting lists and arrays. Operations that produce multiple strictly one result element per input element can also use the MapFunction. The basic syntax for using a MapFUnction is as follows:
- ::
>>> ds = ... >>> new_ds = ds.flat_map(MyFlatMapFunction())
-
close
()¶
-
abstract
flat_map
(value)[source]¶ The core mthod of the FlatMapFunction. Takes an element from the input data and transforms it into zero, one, or more elements. A basic implementation of flat map is as follows:
- ::
>>> class MyFlatMapFunction(FlatMapFunction): >>> def flat_map(self, value): >>> for i in range(value): >>> yield i
- Parameters
value – The input value.
- Returns
A generator
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
class
pyflink.datastream.
FsStateBackend
(checkpoint_directory_uri=None, default_savepoint_directory_uri=None, file_state_size_threshold=None, write_buffer_size=None, using_asynchronous_snapshots=None, j_fs_state_backend=None)[source]¶ Bases:
pyflink.datastream.state_backend.StateBackend
IMPORTANT FsStateBackend is deprecated in favor of `HashMapStateBackend and FileSystemCheckpointStorage. This change does not affect the runtime characteristics of your Jobs and is simply an API change to help better communicate the ways Flink separates local state storage from fault tolerance. Jobs can be upgraded without loss of state. If configuring your state backend via the StreamExecutionEnvironment please make the following changes.
>> env.set_state_backend(HashMapStateBackend()) >> env.get_checkpoint_config().set_checkpoint_storage("hdfs://checkpoints")
If you are configuring your state backend via the flink-conf.yaml please set your state backend type to hashmap.
This state backend holds the working state in the memory (JVM heap) of the TaskManagers. The state backend checkpoints state as files to a file system (hence the backend’s name).
Each checkpoint individually will store all its files in a subdirectory that includes the checkpoint number, such as
hdfs://namenode:port/flink-checkpoints/chk-17/
.State Size Considerations
Working state is kept on the TaskManager heap. If a TaskManager executes multiple tasks concurrently (if the TaskManager has multiple slots, or if slot-sharing is used) then the aggregate state of all tasks needs to fit into that TaskManager’s memory.
This state backend stores small state chunks directly with the metadata, to avoid creating many small files. The threshold for that is configurable. When increasing this threshold, the size of the checkpoint metadata increases. The checkpoint metadata of all retained completed checkpoints needs to fit into the JobManager’s heap memory. This is typically not a problem, unless the threshold
get_min_file_size_threshold()
is increased significantly.Persistence Guarantees
Checkpoints from this state backend are as persistent and available as filesystem that is written to. If the file system is a persistent distributed file system, this state backend supports highly available setups. The backend additionally supports savepoints and externalized checkpoints.
Configuration
As for all state backends, this backend can either be configured within the application (by creating the backend with the respective constructor parameters and setting it on the execution environment) or by specifying it in the Flink configuration.
If the state backend was specified in the application, it may pick up additional configuration parameters from the Flink configuration. For example, if the backend if configured in the application without a default savepoint directory, it will pick up a default savepoint directory specified in the Flink configuration of the running job/cluster. That behavior is implemented via the
configure()
method.-
get_checkpoint_path
() → str[source]¶ Gets the base directory where all the checkpoints are stored. The job-specific checkpoint directory is created inside this directory.
- Returns
The base directory for checkpoints.
-
get_min_file_size_threshold
() → int[source]¶ Gets the threshold below which state is stored as part of the metadata, rather than in files. This threshold ensures that the backend does not create a large amount of very small files, where potentially the file pointers are larger than the state itself.
If not explicitly configured, this is the default value of
org.apache.flink.configuration.CheckpointingOptions.FS_SMALL_FILE_THRESHOLD
.- Returns
The file size threshold, in bytes.
-
get_write_buffer_size
() → int[source]¶ Gets the write buffer size for created checkpoint stream.
If not explicitly configured, this is the default value of
org.apache.flink.configuration.CheckpointingOptions.FS_WRITE_BUFFER_SIZE
.- Returns
The write buffer size, in bytes.
-
is_using_asynchronous_snapshots
() → bool[source]¶ Gets whether the key/value data structures are asynchronously snapshotted.
If not explicitly configured, this is the default value of
org.apache.flink.configuration.CheckpointingOptions.ASYNC_SNAPSHOTS
.- Returns
True if the key/value data structures are asynchronously snapshotted, false otherwise.
-
-
class
pyflink.datastream.
GlobalWindow
[source]¶ Bases:
pyflink.datastream.window.Window
The default window into which all data is placed GlobalWindows.
-
static
get
() → pyflink.datastream.window.GlobalWindow[source]¶
-
static
-
class
pyflink.datastream.
HashMapStateBackend
(j_hashmap_state_backend=None)[source]¶ Bases:
pyflink.datastream.state_backend.StateBackend
This state backend holds the working state in the memory (JVM heap) of the TaskManagers and checkpoints based on the configured CheckpointStorage.
State Size Considerations
Working state is kept on the TaskManager heap. If a TaskManager executes multiple tasks concurrently (if the TaskManager has multiple slots, or if slot-sharing is used) then the aggregate state of all tasks needs to fit into that TaskManager’s memory.
Configuration
As for all state backends, this backend can either be configured within the application (by creating the backend with the respective constructor parameters and setting it on the execution environment) or by specifying it in the Flink configuration.
If the state backend was specified in the application, it may pick up additional configuration parameters from the Flink configuration. For example, if the backend if configured in the application without a default savepoint directory, it will pick up a default savepoint directory specified in the Flink configuration of the running job/cluster. That behavior is implemented via the
configure()
method.
-
class
pyflink.datastream.
JobManagerCheckpointStorage
(checkpoint_path=None, max_state_size=None, j_jobmanager_checkpoint_storage=None)[source]¶ Bases:
pyflink.datastream.checkpoint_storage.CheckpointStorage
The CheckpointStorage checkpoints state directly to the JobManager’s memory (hence the name), but savepoints will be persisted to a file system.
This checkpoint storage is primarily for experimentation, quick local setups, or for streaming applications that have very small state: Because it requires checkpoints to go through the JobManager’s memory, larger state will occupy larger portions of the JobManager’s main memory, reducing operational stability. For any other setup, the FileSystemCheckpointStorage should be used. The FileSystemCheckpointStorage but checkpoints state directly to files rather than to the JobManager’s memory, thus supporting larger state sizes and more highly available recovery.
State Size Considerations
State checkpointing with this checkpoint storage is subject to the following conditions:
Each individual state must not exceed the configured maximum state size (see
get_max_state_size()
.All state from one task (i.e., the sum of all operator states and keyed states from all chained operators of the task) must not exceed what the RPC system supports, which is be default < 10 MB. That limit can be configured up, but that is typically not advised.
The sum of all states in the application times all retained checkpoints must comfortably fit into the JobManager’s JVM heap space.
Persistence Guarantees
For the use cases where the state sizes can be handled by this storage, it does guarantee persistence for savepoints, externalized checkpoints (of configured), and checkpoints (when high-availability is configured).
Configuration
As for all checkpoint storage, this type can either be configured within the application (by creating the storage with the respective constructor parameters and setting it on the execution environment) or by specifying it in the Flink configuration.
If the storage was specified in the application, it may pick up additional configuration parameters from the Flink configuration. For example, if the backend if configured in the application without a default savepoint directory, it will pick up a default savepoint directory specified in the Flink configuration of the running job/cluster. That behavior is implemented via the
configure()
method.-
DEFAULT_MAX_STATE_SIZE
= 5242880¶
-
get_checkpoint_path
() → Optional[str][source]¶ Gets the base directory where all the checkpoints are stored. The job-specific checkpoint directory is created inside this directory.
- Returns
The base directory for checkpoints.
-
get_max_state_size
() → int[source]¶ Gets the maximum size that an individual state can have, as configured in the constructor. By default
DEFAULT_MAX_STATE_SIZE
will be used.
-
class
pyflink.datastream.
KeySelector
[source]¶ Bases:
pyflink.datastream.functions.Function
The KeySelector allows to use deterministic objects for operations such as reduce, reduceGroup, join coGroup, etc. If invoked multiple times on the same object, the returned key must be the same. The extractor takes an object an returns the deterministic key for that object.
-
close
()¶
-
abstract
get_key
(value)[source]¶ User-defined function that deterministically extracts the key from an object.
- Parameters
value – The object to get the key from.
- Returns
The extracted key.
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
-
class
pyflink.datastream.
KeyedBroadcastProcessFunction
(*args, **kwds)[source]¶ Bases:
pyflink.datastream.functions.BaseBroadcastProcessFunction
,typing.Generic
A function to be applied to a
BroadcastConnectedStream
that connectsBroadcastStream
, i.e. a stream with broadcast state, with aKeyedStream
.The stream with the broadcast state can be created using the
DataStream.broadcast()
method.The user has to implement two methods:
the
process_broadcast_element()
which will be applied to each element in the broadcast sidethe
process_element()
which will be applied to the non-broadcasted/keyed side.
The
process_broadcast_element()
takes a context as an argument (among others), which allows it to read/write to the broadcast state, while theprocess_element()
has read-only access to the broadcast state, but can read/write to the keyed state and register timers.New in version 1.16.0.
-
class
BaseContext
¶ Bases:
abc.ABC
The base context available to all methods in a broadcast process function. This includes
BroadcastProcessFunction
andKeyedBroadcastProcessFunction
.-
abstract
current_processing_time
() → int¶ Returns the current processing time.
-
abstract
current_watermark
() → int¶ Returns the current watermark.
-
abstract
timestamp
() → int¶ Timestamp of the element currently being processed or timestamp of a firing timer. This might be None, for example if the time characteristic of your program is set to
TimeCharacteristic.ProcessingTime
.
-
abstract
-
class
Context
[source]¶ Bases:
pyflink.datastream.functions.BaseBroadcastProcessFunction.Context
,abc.ABC
A
BaseBroadcastProcessFunction.Context
available to the broadcast side of aBroadcastConnectedStream
.Currently, the function
applyToKeyedState
in Java is not supported in PyFlink.-
abstract
current_processing_time
() → int¶ Returns the current processing time.
-
abstract
current_watermark
() → int¶ Returns the current watermark.
-
abstract
get_broadcast_state
(state_descriptor: pyflink.datastream.state.MapStateDescriptor) → pyflink.datastream.state.BroadcastState¶ Fetches the
BroadcastState
with the specified name.- Parameters
state_descriptor – the
MapStateDescriptor
of the state to be fetched.- Returns
The required
BroadcastState
.
-
abstract
timestamp
() → int¶ Timestamp of the element currently being processed or timestamp of a firing timer. This might be None, for example if the time characteristic of your program is set to
TimeCharacteristic.ProcessingTime
.
-
abstract
-
class
OnTimerContext
[source]¶ Bases:
pyflink.datastream.functions.KeyedBroadcastProcessFunction.ReadOnlyContext
,abc.ABC
Information available in an invocation of
KeyedBroadcastProcessFunction.on_timer()
.-
abstract
current_processing_time
() → int¶ Returns the current processing time.
-
abstract
current_watermark
() → int¶ Returns the current watermark.
-
abstract
get_broadcast_state
(state_descriptor: pyflink.datastream.state.MapStateDescriptor) → pyflink.datastream.state.ReadOnlyBroadcastState¶ Fetches a read-only view of the broadcast state with the specified name.
- Parameters
state_descriptor – the
MapStateDescriptor
of the state to be fetched.- Returns
The required read-only view of the broadcast state.
-
abstract
time_domain
() → pyflink.datastream.time_domain.TimeDomain[source]¶ The
TimeDomain
of the firing timer, i.e. if it is event or processing time timer.
-
abstract
timer_service
() → pyflink.datastream.timerservice.TimerService¶ A
TimerService
for querying time and registering timers.
-
abstract
timestamp
() → int¶ Timestamp of the element currently being processed or timestamp of a firing timer. This might be None, for example if the time characteristic of your program is set to
TimeCharacteristic.ProcessingTime
.
-
abstract
-
class
ReadOnlyContext
[source]¶ Bases:
pyflink.datastream.functions.BaseBroadcastProcessFunction.ReadOnlyContext
,abc.ABC
A
BaseBroadcastProcessFunction.ReadOnlyContext
available to the non-keyed side of aBroadcastConnectedStream
(if any).Apart from the basic functionality of a
BaseBroadcastProcessFunction.Context
, this also allows to get a read-only iterator over the elements stored in the broadcast state and aTimerService
for querying time and registering timers.-
abstract
current_processing_time
() → int¶ Returns the current processing time.
-
abstract
current_watermark
() → int¶ Returns the current watermark.
-
abstract
get_broadcast_state
(state_descriptor: pyflink.datastream.state.MapStateDescriptor) → pyflink.datastream.state.ReadOnlyBroadcastState¶ Fetches a read-only view of the broadcast state with the specified name.
- Parameters
state_descriptor – the
MapStateDescriptor
of the state to be fetched.- Returns
The required read-only view of the broadcast state.
-
abstract
timer_service
() → pyflink.datastream.timerservice.TimerService[source]¶ A
TimerService
for querying time and registering timers.
-
abstract
timestamp
() → int¶ Timestamp of the element currently being processed or timestamp of a firing timer. This might be None, for example if the time characteristic of your program is set to
TimeCharacteristic.ProcessingTime
.
-
abstract
-
close
()¶
-
on_timer
(timestamp: int, ctx: pyflink.datastream.functions.KeyedBroadcastProcessFunction.OnTimerContext)[source]¶ Called when a timer set using
TimerService
fires.- Parameters
timestamp – The timestamp of the firing timer.
ctx – An
KeyedBroadcastProcessFunction.OnTimerContext
that allows querying the timestamp of the firing timer, querying the current processing/event time, iterating the broadcast state with read-only access, querying theTimeDomain
of the firing timer and getting aTimerService
for registering timers and querying the time. The context is only valid during the invocation of this method, do not store it.
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
abstract
process_broadcast_element
(value: IN2, ctx: pyflink.datastream.functions.KeyedBroadcastProcessFunction.Context)[source]¶ This method is called for each element in the
BroadcastStream
.It can output zero or more elements via
yield
statement, query the current processing/event time, and also query and update the internalstate.BroadcastState
. Currently,applyToKeyedState
is not supported in PyFlink. The context is only valid during the invocation of this method, do not store it.- Parameters
value – The stream element.
ctx – A
KeyedBroadcastProcessFunction.Context
that allows querying the timestamp of the element, querying the current processing/event time and updating the broadcast state. The context is only valid during the invocation of this method, do not store it.
-
abstract
process_element
(value: IN1, ctx: pyflink.datastream.functions.KeyedBroadcastProcessFunction.ReadOnlyContext)[source]¶ This method is called for each element in the (non-broadcast)
KeyedStream
.It can output zero or more elements via
yield
statement, query the current processing/event time, and also query and update the local keyed state. In addition, it can get aTimerService
for registering timers and querying the time. Finally, it has read-only access to the broadcast state. The context is only valid during the invocation of this method, do not store it.- Parameters
value – The stream element.
ctx – A
KeyedBroadcastProcessFunction.ReadOnlyContext
that allows querying the timestamp of the element, querying the current processing/event time and iterating the broadcast state with read-only access. The context is only valid during the invocation of this method, do not store it.
-
class
pyflink.datastream.
KeyedCoProcessFunction
[source]¶ Bases:
pyflink.datastream.functions.Function
A function that processes elements of two keyed streams and produces a single output one.
The function will be called for every element in the input streams and can produce zero or more output elements. Contrary to the
CoFlatMapFunction
, this function can also query the time (both event and processing) and set timers, through the provided {@link Context}. When reacting to the firing of set timers the function can emit yet more elements.An example use-case for connected streams would be the application of a set of rules that change over time ({@code stream A}) to the elements contained in another stream (stream {@code B}). The rules contained in {@code stream A} can be stored in the state and wait for new elements to arrive on {@code stream B}. Upon reception of a new element on {@code stream B}, the function can now apply the previously stored rules to the element and directly emit a result, and/or register a timer that will trigger an action in the future.
-
class
Context
[source]¶ Bases:
abc.ABC
-
class
OnTimerContext
[source]¶ Bases:
pyflink.datastream.functions.KeyedCoProcessFunction.Context
-
abstract
get_current_key
()¶
-
abstract
time_domain
() → pyflink.datastream.time_domain.TimeDomain[source]¶ The TimeDomain of the firing timer. :return: The TimeDomain of current fired timer.
-
abstract
timer_service
() → pyflink.datastream.timerservice.TimerService¶ A Timer service for querying time and registering timers.
-
abstract
timestamp
() → int¶ Timestamp of the element currently being processed or timestamp of a firing timer.
This might be None, for example if the time characteristic of your program is set to TimeCharacteristic.ProcessTime.
-
abstract
-
close
()¶
-
on_timer
(timestamp: int, ctx: pyflink.datastream.functions.KeyedCoProcessFunction.OnTimerContext)[source]¶ Called when a timer set using TimerService fires.
- Parameters
timestamp – The timestamp of the firing timer.
ctx – An OnTimerContext that allows querying the timestamp of the firing timer, querying the TimeDomain of the firing timer and getting a TimerService for registering timers and querying the time. The context is only valid during the invocation of this method, do not store it.
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
abstract
process_element1
(value, ctx: pyflink.datastream.functions.KeyedCoProcessFunction.Context)[source]¶ Process one element from the input stream.
This function can output zero or more elements using the Collector parameter and also update internal state or set timers using the Context parameter.
- Parameters
value – The input value.
ctx – A Context that allows querying the timestamp of the element and getting a TimerService for registering timers and querying the time. The context is only valid during the invocation of this method, do not store it.
-
abstract
process_element2
(value, ctx: pyflink.datastream.functions.KeyedCoProcessFunction.Context)[source]¶ Process one element from the input stream.
This function can output zero or more elements using the Collector parameter and also update internal state or set timers using the Context parameter.
- Parameters
value – The input value.
ctx – A Context that allows querying the timestamp of the element and getting a TimerService for registering timers and querying the time. The context is only valid during the invocation of this method, do not store it.
-
class
-
class
pyflink.datastream.
KeyedProcessFunction
[source]¶ Bases:
pyflink.datastream.functions.Function
A keyed function processes elements of a stream.
For every element in the input stream, process_element() is invoked. This can produce zero or more elements as output. Implementations can also query the time and set timers through the provided Context. For firing timers on_timer() will be invoked. This can again produce zero or more elements as output and register further timers.
Note that access to keyed state and timers (which are also scoped to a key) is only available if the KeyedProcessFunction is applied on a KeyedStream.
-
class
Context
[source]¶ Bases:
abc.ABC
-
class
OnTimerContext
[source]¶ Bases:
pyflink.datastream.functions.KeyedProcessFunction.Context
-
abstract
get_current_key
()¶
-
abstract
time_domain
() → pyflink.datastream.time_domain.TimeDomain[source]¶ The TimeDomain of the firing timer. :return: The TimeDomain of current fired timer.
-
abstract
timer_service
() → pyflink.datastream.timerservice.TimerService¶ A Timer service for querying time and registering timers.
-
abstract
timestamp
() → int¶ Timestamp of the element currently being processed or timestamp of a firing timer.
This might be None, for example if the time characteristic of your program is set to TimeCharacteristic.ProcessTime.
-
abstract
-
close
()¶
-
on_timer
(timestamp: int, ctx: pyflink.datastream.functions.KeyedProcessFunction.OnTimerContext)[source]¶ Called when a timer set using TimerService fires.
- Parameters
timestamp – The timestamp of the firing timer.
ctx – An OnTimerContext that allows querying the timestamp of the firing timer, querying the TimeDomain of the firing timer and getting a TimerService for registering timers and querying the time. The context is only valid during the invocation of this method, do not store it.
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
abstract
process_element
(value, ctx: pyflink.datastream.functions.KeyedProcessFunction.Context)[source]¶ Process one element from the input stream.
This function can output zero or more elements and also update internal state or set timers using the Context parameter.
- Parameters
value – The input value.
ctx – A Context that allows querying the timestamp of the element and getting a TimerService for registering timers and querying the time. The context is only valid during the invocation of this method, do not store it.
-
class
-
class
pyflink.datastream.
KeyedStream
(j_keyed_stream, original_data_type_info, origin_stream: pyflink.datastream.data_stream.DataStream)[source]¶ Bases:
pyflink.datastream.data_stream.DataStream
A KeyedStream represents a DataStream on which operator state is partitioned by key using a provided KeySelector. Typical operations supported by a DataStream are also possible on a KeyedStream, with the exception of partitioning methods such as shuffle, forward and keyBy.
Reduce-style operations, such as reduce and sum work on elements that have the same key.
-
class
AccumulateType
(value)[source]¶ Bases:
enum.Enum
An enumeration.
-
MAX
= 2¶
-
MAX_BY
= 4¶
-
MIN
= 1¶
-
MIN_BY
= 3¶
-
SUM
= 5¶
-
-
add_sink
(sink_func: pyflink.datastream.functions.SinkFunction) → pyflink.datastream.data_stream.DataStreamSink[source]¶ Adds the given sink to this DataStream. Only streams with sinks added will be executed once the StreamExecutionEnvironment.execute() method is called.
- Parameters
sink_func – The SinkFunction object.
- Returns
The closed DataStream.
-
assign_timestamps_and_watermarks
(watermark_strategy: pyflink.common.watermark_strategy.WatermarkStrategy) → pyflink.datastream.data_stream.DataStream¶ Assigns timestamps to the elements in the data stream and generates watermarks to signal event time progress. The given {@link WatermarkStrategy} is used to create a TimestampAssigner and WatermarkGenerator.
- Parameters
watermark_strategy – The strategy to generate watermarks based on event timestamps.
- Returns
The stream after the transformation, with assigned timestamps and watermarks.
-
cache
() → pyflink.datastream.data_stream.CachedDataStream[source]¶ Cache the intermediate result of the transformation. Only support bounded streams and currently only block mode is supported. The cache is generated lazily at the first time the intermediate result is computed. The cache will be clear when the StreamExecutionEnvironment close.
- Returns
The cached DataStream that can use in later job to reuse the cached intermediate result.
New in version 1.16.0.
-
connect
(ds: ‘DataStream’) → ’ConnectedStreams’[source]¶ -
connect
(ds: ‘BroadcastStream’) → ’BroadcastConnectedStream’ If ds is a
DataStream
, creates a newConnectedStreams
by connecting DataStream outputs of (possible) different types with each other. The DataStreams connected using this operator can be used with CoFunctions to apply joint transformations.If ds is a
BroadcastStream
, creates a newBroadcastConnectedStream
by connecting the currentDataStream
with aBroadcastStream
. The latter can be created using thebroadcast()
method. The resulting stream can be further processed using theBroadcastConnectedStream.process()
method.- Parameters
ds – The DataStream or BroadcastStream with which this stream will be connected.
- Returns
The ConnectedStreams or BroadcastConnectedStream.
Changed in version 1.16.0: Support connect BroadcastStream
-
count_window
(size: int, slide: int = 0)[source]¶ Windows this KeyedStream into tumbling or sliding count windows.
- Parameters
size – The size of the windows in number of elements.
slide – The slide interval in number of elements.
New in version 1.16.0.
-
disable_chaining
() → pyflink.datastream.data_stream.DataStream[source]¶ Turns off chaining for this operator so thread co-location will not be used as an optimization. Chaining can be turned off for the whole job by StreamExecutionEnvironment.disableOperatorChaining() however it is not advised for performance consideration.
- Returns
The operator with chaining disabled.
-
execute_and_collect
(job_execution_name: str = None, limit: int = None) → Union[pyflink.datastream.data_stream.CloseableIterator, list]¶ Triggers the distributed execution of the streaming dataflow and returns an iterator over the elements of the given DataStream.
The DataStream application is executed in the regular distributed manner on the target environment, and the events from the stream are polled back to this application process and thread through Flink’s REST API.
The returned iterator must be closed to free all cluster resources.
- Parameters
job_execution_name – The name of the job execution.
limit – The limit for the collected elements.
-
filter
(func: Union[Callable, pyflink.datastream.functions.FilterFunction]) → pyflink.datastream.data_stream.DataStream[source]¶ Applies a Filter transformation on a DataStream. The transformation calls a FilterFunction for each element of the DataStream and retains only those element for which the function returns true. Elements for which the function returns false are filtered.
- Parameters
func – The FilterFunction that is called for each element of the DataStream.
- Returns
The filtered DataStream.
-
flat_map
(func: Union[Callable, pyflink.datastream.functions.FlatMapFunction], output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Applies a FlatMap transformation on a KeyedStream. The transformation calls a FlatMapFunction for each element of the DataStream. Each FlatMapFunction call can return any number of elements including none.
- Parameters
func – The FlatMapFunction that is called for each element of the DataStream.
output_type – The type information of output data.
- Returns
The transformed DataStream.
-
force_non_parallel
()[source]¶ Sets the parallelism and maximum parallelism of this operator to one. And mark this operator cannot set a non-1 degree of parallelism.
- Returns
The operator with only one parallelism.
-
forward
() → pyflink.datastream.data_stream.DataStream[source]¶ Sets the partitioning of the DataStream so that the output elements are forwarded to the local sub-task of the next operation.
- Returns
The DataStream with forward partitioning set.
-
get_execution_config
() → pyflink.common.execution_config.ExecutionConfig¶
-
get_execution_environment
()¶ Returns the StreamExecutionEnvironment that was used to create this DataStream.
- Returns
The Execution Environment.
-
get_name
() → str[source]¶ Gets the name of the current data stream. This name is used by the visualization and logging during runtime.
- Returns
Name of the stream.
-
get_side_output
(output_tag: pyflink.datastream.output_tag.OutputTag) → pyflink.datastream.data_stream.DataStream¶ Gets the
DataStream
that contains the elements that are emitted from an operation into the side output with the givenOutputTag
.- Parameters
output_tag – output tag for the side stream
- Returns
The DataStream with specified output tag
New in version 1.16.0.
-
get_type
() → pyflink.common.typeinfo.TypeInformation¶ Gets the type of the stream.
- Returns
The type of the DataStream.
-
key_by
(key_selector: Union[Callable, pyflink.datastream.functions.KeySelector], key_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.KeyedStream[source]¶ Creates a new KeyedStream that uses the provided key for partitioning its operator states.
- Parameters
key_selector – The KeySelector to be used for extracting the key for partitioning.
key_type – The type information describing the key type.
- Returns
The DataStream with partitioned state(i.e. KeyedStream).
-
map
(func: Union[Callable, pyflink.datastream.functions.MapFunction], output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Applies a Map transformation on a KeyedStream. The transformation calls a MapFunction for each element of the DataStream. Each MapFunction call returns exactly one element.
Note that If user does not specify the output data type, the output data will be serialized as pickle primitive byte array.
- Parameters
func – The MapFunction that is called for each element of the DataStream.
output_type – The type information of the MapFunction output data.
- Returns
The transformed DataStream.
-
max
(position_to_max: Union[int, str] = 0) → pyflink.datastream.data_stream.DataStream[source]¶ Applies an aggregation that gives the current maximize of the data stream at the given position by the given key. An independent aggregate is kept per key.
Example(Tuple data):
>>> ds = env.from_collection([('a', 1), ('a', 2), ('b', 1), ('b', 5)]) >>> ds.key_by(lambda x: x[0]).max(1)
Example(Row data):
>>> ds = env.from_collection([('a', 1), ('a', 2), ('a', 3), ('b', 1), ('b', 2)], ... type_info=Types.ROW([Types.STRING(), Types.INT()])) >>> ds.key_by(lambda x: x[0]).max(1)
Example(Row data with fields name):
>>> ds = env.from_collection( ... [('a', 1), ('a', 2), ('a', 3), ('b', 1), ('b', 2)], ... type_info=Types.ROW_NAMED(["key", "value"], [Types.STRING(), Types.INT()]) ... ) >>> ds.key_by(lambda x: x[0]).max("value")
- Parameters
position_to_max – The field position in the data points to maximize. The type can be int (field position) or str (field name). This is applicable to Tuple types, List types, Row types, and basic types (which is considered as having one field).
- Returns
The transformed DataStream.
New in version 1.16.0.
-
max_by
(position_to_max_by: Union[int, str] = 0) → pyflink.datastream.data_stream.DataStream[source]¶ Applies an aggregation that gives the current element with the maximize value at the given position by the given key. An independent aggregate is kept per key. If more elements have the maximize value at the given position, the operator returns the first one by default.
Example(Tuple data):
>>> ds = env.from_collection([('a', 1), ('a', 2), ('b', 1), ('b', 5)]) >>> ds.key_by(lambda x: x[0]).max_by(1)
Example(Row data):
>>> ds = env.from_collection([('a', 1), ('a', 2), ('a', 3), ('b', 1), ('b', 2)], ... type_info=Types.ROW([Types.STRING(), Types.INT()])) >>> ds.key_by(lambda x: x[0]).max_by(1)
Example(Row data with fields name):
>>> ds = env.from_collection( ... [('a', 1), ('a', 2), ('a', 3), ('b', 1), ('b', 2)], ... type_info=Types.ROW_NAMED(["key", "value"], [Types.STRING(), Types.INT()]) ... ) >>> ds.key_by(lambda x: x[0]).max_by("value")
- Parameters
position_to_max_by – The field position in the data points to maximize. The type can be int (field position) or str (field name). This is applicable to Tuple types, List types, Row types, and basic types (which is considered as having one field).
- Returns
The transformed DataStream.
New in version 1.16.0.
-
min
(position_to_min: Union[int, str] = 0) → pyflink.datastream.data_stream.DataStream[source]¶ Applies an aggregation that gives the current minimum of the data stream at the given position by the given key. An independent aggregate is kept per key.
Example(Tuple data):
>>> ds = env.from_collection([('a', 1), ('a', 2), ('b', 1), ('b', 5)]) >>> ds.key_by(lambda x: x[0]).min(1)
Example(Row data):
>>> ds = env.from_collection([('a', 1), ('a', 2), ('a', 3), ('b', 1), ('b', 2)], ... type_info=Types.ROW([Types.STRING(), Types.INT()])) >>> ds.key_by(lambda x: x[0]).min(1)
Example(Row data with fields name):
>>> ds = env.from_collection( ... [('a', 1), ('a', 2), ('a', 3), ('b', 1), ('b', 2)], ... type_info=Types.ROW_NAMED(["key", "value"], [Types.STRING(), Types.INT()]) ... ) >>> ds.key_by(lambda x: x[0]).min("value")
- Parameters
position_to_min – The field position in the data points to minimize. The type can be int (field position) or str (field name). This is applicable to Tuple types, List types, Row types, and basic types (which is considered as having one field).
- Returns
The transformed DataStream.
New in version 1.16.0.
-
min_by
(position_to_min_by: Union[int, str] = 0) → pyflink.datastream.data_stream.DataStream[source]¶ Applies an aggregation that gives the current element with the minimum value at the given position by the given key. An independent aggregate is kept per key. If more elements have the minimum value at the given position, the operator returns the first one by default.
Example(Tuple data):
>>> ds = env.from_collection([('a', 1), ('a', 2), ('b', 1), ('b', 5)]) >>> ds.key_by(lambda x: x[0]).min_by(1)
Example(Row data):
>>> ds = env.from_collection([('a', 1), ('a', 2), ('a', 3), ('b', 1), ('b', 2)], ... type_info=Types.ROW([Types.STRING(), Types.INT()])) >>> ds.key_by(lambda x: x[0]).min_by(1)
Example(Row data with fields name):
>>> ds = env.from_collection( ... [('a', 1), ('a', 2), ('a', 3), ('b', 1), ('b', 2)], ... type_info=Types.ROW_NAMED(["key", "value"], [Types.STRING(), Types.INT()]) ... ) >>> ds.key_by(lambda x: x[0]).min_by("value")
- Parameters
position_to_min_by – The field position in the data points to minimize. The type can be int (field position) or str (field name). This is applicable to Tuple types, List types, Row types, and basic types (which is considered as having one field).
- Returns
The transformed DataStream.
New in version 1.16.0.
-
name
(name: str)[source]¶ Sets the name of the current data stream. This name is used by the visualization and logging during runtime.
- Parameters
name – Name of the stream.
- Returns
The named operator.
-
partition_custom
(partitioner: Union[Callable, pyflink.datastream.functions.Partitioner], key_selector: Union[Callable, pyflink.datastream.functions.KeySelector]) → pyflink.datastream.data_stream.DataStream[source]¶ Partitions a DataStream on the key returned by the selector, using a custom partitioner. This method takes the key selector to get the key to partition on, and a partitioner that accepts the key type.
Note that this method works only on single field keys, i.e. the selector cannot return tuples of fields.
- Parameters
partitioner – The partitioner to assign partitions to keys.
key_selector – The KeySelector with which the DataStream is partitioned.
- Returns
The partitioned DataStream.
-
print
(sink_identifier=None)[source]¶ Writes a DataStream to the standard output stream (stdout). For each element of the DataStream the object string is written.
NOTE: This will print to stdout on the machine where the code is executed, i.e. the Flink worker, and is not fault tolerant.
- Parameters
sink_identifier – The string to prefix the output with.
- Returns
The closed DataStream.
-
process
(func: pyflink.datastream.functions.KeyedProcessFunction, output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Applies the given ProcessFunction on the input stream, thereby creating a transformed output stream.
The function will be called for every element in the input streams and can produce zero or more output elements.
- Parameters
func – The KeyedProcessFunction that is called for each element in the stream.
output_type – TypeInformation for the result type of the function.
- Returns
The transformed DataStream.
-
project
(*field_indexes) → pyflink.datastream.data_stream.DataStream[source]¶ Initiates a Project transformation on a Tuple DataStream.
Note that only Tuple DataStreams can be projected.
- Parameters
field_indexes – The field indexes of the input tuples that are retained. The order of fields in the output tuple corresponds to the order of field indexes.
- Returns
The projected DataStream.
-
rebalance
() → pyflink.datastream.data_stream.DataStream[source]¶ Sets the partitioning of the DataStream so that the output elements are distributed evenly to instances of the next operation in a round-robin fashion.
- Returns
The DataStream with rebalance partition set.
-
reduce
(func: Union[Callable, pyflink.datastream.functions.ReduceFunction]) → pyflink.datastream.data_stream.DataStream[source]¶ Applies a reduce transformation on the grouped data stream grouped on by the given key position. The ReduceFunction will receive input values based on the key value. Only input values with the same key will go to the same reducer.
Example:
>>> ds = env.from_collection([(1, 'a'), (2, 'a'), (3, 'a'), (4, 'b']) >>> ds.key_by(lambda x: x[1]).reduce(lambda a, b: a[0] + b[0], b[1])
- Parameters
func – The ReduceFunction that is called for each element of the DataStream.
- Returns
The transformed DataStream.
-
rescale
() → pyflink.datastream.data_stream.DataStream[source]¶ Sets the partitioning of the DataStream so that the output elements are distributed evenly to a subset of instances of the next operation in a round-robin fashion.
The subset of downstream operations to which the upstream operation sends elements depends on the degree of parallelism of both the upstream and downstream operation. For example, if the upstream operation has parallelism 2 and the downstream operation has parallelism 4, then one upstream operation would distribute elements to two downstream operations. If, on the other hand, the downstream operation has parallelism 4 then two upstream operations will distribute to one downstream operation while the other two upstream operations will distribute to the other downstream operations.
In cases where the different parallelisms are not multiples of each one or several downstream operations will have a differing number of inputs from upstream operations.
- Returns
The DataStream with rescale partitioning set.
-
set_buffer_timeout
(timeout_millis: int)[source]¶ Sets the buffering timeout for data produced by this operation. The timeout defines how long data may linger ina partially full buffer before being sent over the network.
Lower timeouts lead to lower tail latencies, but may affect throughput. Timeouts of 1 ms still sustain high throughput, even for jobs with high parallelism.
A value of ‘-1’ means that the default buffer timeout should be used. A value of ‘0’ indicates that no buffering should happen, and all records/events should be immediately sent through the network, without additional buffering.
- Parameters
timeout_millis – The maximum time between two output flushes.
- Returns
The operator with buffer timeout set.
-
set_description
(description: str) → pyflink.datastream.data_stream.DataStream¶ Sets the description for this operator.
Description is used in json plan and web ui, but not in logging and metrics where only name is available. Description is expected to provide detailed information about the operator, while name is expected to be more simple, providing summary information only, so that we can have more user-friendly logging messages and metric tags without losing useful messages for debugging.
- Parameters
description – The description for this operator.
- Returns
The operator with new description.
New in version 1.15.0.
-
set_max_parallelism
(max_parallelism: int)[source]¶ Sets the maximum parallelism of this operator.
The maximum parallelism specifies the upper bound for dynamic scaling. It also defines the number of key groups used for partitioned state.
- Parameters
max_parallelism – Maximum parallelism.
- Returns
The operator with set maximum parallelism.
-
set_parallelism
(parallelism: int)[source]¶ Sets the parallelism for this operator.
- Parameters
parallelism – THe parallelism for this operator.
- Returns
The operator with set parallelism.
-
set_uid_hash
(uid_hash: str)[source]¶ Sets an user provided hash for this operator. This will be used AS IS the create the JobVertexID. The user provided hash is an alternative to the generated hashed, that is considered when identifying an operator through the default hash mechanics fails (e.g. because of changes between Flink versions).
Important: this should be used as a workaround or for trouble shooting. The provided hash needs to be unique per transformation and job. Otherwise, job submission will fail. Furthermore, you cannot assign user-specified hash to intermediate nodes in an operator chain and trying so will let your job fail.
A use case for this is in migration between Flink versions or changing the jobs in a way that changes the automatically generated hashes. In this case, providing the previous hashes directly through this method (e.g. obtained from old logs) can help to reestablish a lost mapping from states to their target operator.
- Parameters
uid_hash – The user provided hash for this operator. This will become the jobVertexID, which is shown in the logs and web ui.
- Returns
The operator with the user provided hash.
-
shuffle
() → pyflink.datastream.data_stream.DataStream[source]¶ Sets the partitioning of the DataStream so that the output elements are shuffled uniformly randomly to the next operation.
- Returns
The DataStream with shuffle partitioning set.
-
sink_to
(sink: pyflink.datastream.connectors.base.Sink) → pyflink.datastream.data_stream.DataStreamSink¶ Adds the given sink to this DataStream. Only streams with sinks added will be executed once the
execute()
method is called.- Parameters
sink – The user defined sink.
- Returns
The closed DataStream.
-
slot_sharing_group
(slot_sharing_group: Union[str, pyflink.datastream.slot_sharing_group.SlotSharingGroup]) → pyflink.datastream.data_stream.DataStream[source]¶ Sets the slot sharing group of this operation. Parallel instances of operations that are in the same slot sharing group will be co-located in the same TaskManager slot, if possible.
Operations inherit the slot sharing group of input operations if all input operations are in the same slot sharing group and no slot sharing group was explicitly specified.
Initially an operation is in the default slot sharing group. An operation can be put into the default group explicitly by setting the slot sharing group to ‘default’.
- Parameters
slot_sharing_group – The slot sharing group name or which contains name and its resource spec.
- Returns
This operator.
-
start_new_chain
() → pyflink.datastream.data_stream.DataStream[source]¶ Starts a new task chain beginning at this operator. This operator will be chained (thread co-located for increased performance) to any previous tasks even if possible.
- Returns
The operator with chaining set.
-
sum
(position_to_sum: Union[int, str] = 0) → pyflink.datastream.data_stream.DataStream[source]¶ Applies an aggregation that gives a rolling sum of the data stream at the given position grouped by the given key. An independent aggregate is kept per key.
Example(Tuple data to sum):
>>> ds = env.from_collection([('a', 1), ('a', 2), ('b', 1), ('b', 5)]) >>> ds.key_by(lambda x: x[0]).sum(1)
Example(Row data to sum):
>>> ds = env.from_collection([('a', 1), ('a', 2), ('a', 3), ('b', 1), ('b', 2)], ... type_info=Types.ROW([Types.STRING(), Types.INT()])) >>> ds.key_by(lambda x: x[0]).sum(1)
Example(Row data with fields name to sum):
>>> ds = env.from_collection( ... [('a', 1), ('a', 2), ('a', 3), ('b', 1), ('b', 2)], ... type_info=Types.ROW_NAMED(["key", "value"], [Types.STRING(), Types.INT()]) ... ) >>> ds.key_by(lambda x: x[0]).sum("value")
- Parameters
position_to_sum – The field position in the data points to sum, type can be int which indicates the index of the column to operate on or str which indicates the name of the column to operate on.
- Returns
The transformed DataStream.
New in version 1.16.0.
-
uid
(uid: str)[source]¶ Sets an ID for this operator. The specified ID is used to assign the same operator ID across job submissions (for example when starting a job from a savepoint).
Important: this ID needs to be unique per transformation and job. Otherwise, job submission will fail.
- Parameters
uid – The unique user-specified ID of this transformation.
- Returns
The operator with the specified ID.
-
union
(*streams) → pyflink.datastream.data_stream.DataStream[source]¶ Creates a new DataStream by merging DataStream outputs of the same type with each other. The DataStreams merged using this operator will be transformed simultaneously.
- Parameters
streams – The DataStream to union outputwith.
- Returns
The DataStream.
-
window
(window_assigner: pyflink.datastream.window.WindowAssigner) → pyflink.datastream.data_stream.WindowedStream[source]¶ Windows this data stream to a WindowedStream, which evaluates windows over a key grouped stream. Elements are put into windows by a WindowAssigner. The grouping of elements is done both by key and by window.
A Trigger can be defined to specify when windows are evaluated. However, WindowAssigners have a default Trigger that is used if a Trigger is not specified.
- Parameters
window_assigner – The WindowAssigner that assigns elements to windows.
- Returns
The trigger windows data stream.
-
window_all
(window_assigner: pyflink.datastream.window.WindowAssigner) → pyflink.datastream.data_stream.AllWindowedStream¶ Windows this data stream to a AllWindowedStream, which evaluates windows over a non key grouped stream. Elements are put into windows by a WindowAssigner. The grouping of elements is done by window.
A Trigger can be defined to specify when windows are evaluated. However, WindowAssigners have a default Trigger that is used if a Trigger is not specified.
- Parameters
window_assigner – The WindowAssigner that assigns elements to windows.
- Returns
The trigger windows data stream.
New in version 1.16.0.
-
class
-
class
pyflink.datastream.
MapFunction
[source]¶ Bases:
pyflink.datastream.functions.Function
Base class for Map functions. Map functions take elements and transform them, element wise. A Map function always produces a single result element for each input element. Typical applications are parsing elements, converting data types, or projecting out fields. Operations that produce multiple result elements from a single input element can be implemented using the FlatMapFunction. The basic syntax for using a MapFunction is as follows:
- ::
>>> ds = ... >>> new_ds = ds.map(MyMapFunction())
-
close
()¶
-
abstract
map
(value)[source]¶ The mapping method. Takes an element from the input data and transforms it into exactly one element.
- Parameters
value – The input value.
- Returns
The transformed value.
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
class
pyflink.datastream.
MemorySize
(j_memory_size=None, bytes_size: int = None)[source]¶ Bases:
object
MemorySize is a representation of a number of bytes, viewable in different units.
-
get_gibi_bytes
() → int[source]¶ Gets the memory size in Gibibytes (= 1024 Mebibytes).
- Returns
The memory size in Gibibytes.
-
get_java_memory_size
()[source]¶ Gets the Java MemorySize object.
- Returns
The Java MemorySize object.
-
get_kibi_bytes
() → int[source]¶ Gets the memory size in Kibibytes (= 1024 bytes).
- Returns
The memory size in Kibibytes.
-
get_mebi_bytes
() → int[source]¶ Gets the memory size in Mebibytes (= 1024 Kibibytes).
- Returns
The memory size in Mebibytes.
-
-
class
pyflink.datastream.
MemoryStateBackend
(checkpoint_path=None, savepoint_path=None, max_state_size=None, using_asynchronous_snapshots=None, j_memory_state_backend=None)[source]¶ Bases:
pyflink.datastream.state_backend.StateBackend
IMPORTANT MemoryStateBackend is deprecated in favor of HashMapStateBackend and JobManagerCheckpointStorage. This change does not affect the runtime characteristics of your Jobs and is simply an API change to help better communicate the ways Flink separates local state storage from fault tolerance. Jobs can be upgraded without loss of state. If configuring your state backend via the StreamExecutionEnvironment please make the following changes.
>> env.set_state_backend(HashMapStateBackend()) >> env.get_checkpoint_config().set_checkpoint_storage(JobManagerCheckpointStorage())
If you are configuring your state backend via the flink-conf.yaml please make the following changes.
` state.backend: hashmap state.checkpoint-storage: jobmanager `
This state backend holds the working state in the memory (JVM heap) of the TaskManagers. The state backend checkpoints state directly to the JobManager’s memory (hence the backend’s name), but the checkpoints will be persisted to a file system for high-availability setups and savepoints. The MemoryStateBackend is consequently a FileSystem-based backend that can work without a file system dependency in simple setups.
This state backend should be used only for experimentation, quick local setups, or for streaming applications that have very small state: Because it requires checkpoints to go through the JobManager’s memory, larger state will occupy larger portions of the JobManager’s main memory, reducing operational stability. For any other setup, the
FsStateBackend
should be used. TheFsStateBackend
holds the working state on the TaskManagers in the same way, but checkpoints state directly to files rather then to the JobManager’s memory, thus supporting large state sizes.State Size Considerations
State checkpointing with this state backend is subject to the following conditions:
Each individual state must not exceed the configured maximum state size (see
get_max_state_size()
.All state from one task (i.e., the sum of all operator states and keyed states from all chained operators of the task) must not exceed what the RPC system supports, which is be default < 10 MB. That limit can be configured up, but that is typically not advised.
The sum of all states in the application times all retained checkpoints must comfortably fit into the JobManager’s JVM heap space.
Persistence Guarantees
For the use cases where the state sizes can be handled by this backend, the backend does guarantee persistence for savepoints, externalized checkpoints (of configured), and checkpoints (when high-availability is configured).
Configuration
As for all state backends, this backend can either be configured within the application (by creating the backend with the respective constructor parameters and setting it on the execution environment) or by specifying it in the Flink configuration.
If the state backend was specified in the application, it may pick up additional configuration parameters from the Flink configuration. For example, if the backend if configured in the application without a default savepoint directory, it will pick up a default savepoint directory specified in the Flink configuration of the running job/cluster. That behavior is implemented via the
configure()
method.-
DEFAULT_MAX_STATE_SIZE
= 5242880¶
-
get_max_state_size
() → int[source]¶ Gets the maximum size that an individual state can have, as configured in the constructor (by default
DEFAULT_MAX_STATE_SIZE
).- Returns
The maximum size that an individual state can have.
-
is_using_asynchronous_snapshots
() → bool[source]¶ Gets whether the key/value data structures are asynchronously snapshotted.
If not explicitly configured, this is the default value of
org.apache.flink.configuration.CheckpointingOptions.ASYNC_SNAPSHOTS
.- Returns
True if the key/value data structures are asynchronously snapshotted, false otherwise.
-
class
pyflink.datastream.
MergingWindowAssigner
(*args, **kwds)[source]¶ Bases:
pyflink.datastream.window.WindowAssigner
A WindowAssigner that can merge windows.
-
class
MergeCallback
(*args, **kwds)[source]¶ Bases:
abc.ABC
,typing.Generic
Callback to be used in
merge_windows()
for specifying which windows should be merged.
-
class
WindowAssignerContext
¶ Bases:
abc.ABC
A context provided to the
WindowAssigner
that allows it to query the current processing time.-
abstract
get_current_processing_time
() → int¶ - Returns
The current processing time.
-
abstract
get_runtime_context
() → pyflink.datastream.functions.RuntimeContext¶ - Returns
The current runtime context.
-
abstract
-
abstract
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[W]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
abstract
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, W]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
abstract
get_window_serializer
() → pyflink.common.serializer.TypeSerializer[W]¶ - Returns
A TypeSerializer for serializing windows that are assigned by this WindowAssigner.
-
abstract
is_event_time
() → bool¶ - Returns
True if elements are assigned to windows based on event time, false otherwise.
-
abstract
merge_windows
(windows: Iterable[W], callback: pyflink.datastream.window.MergingWindowAssigner.MergeCallback[W]) → None[source]¶ Determines which windows (if any) should be merged.
- Parameters
windows – The window candidates.
callback – A callback that can be invoked to signal which windows should be merged.
-
class
-
class
pyflink.datastream.
OutputTag
(tag_id: str, type_info: Optional[Union[pyflink.common.typeinfo.TypeInformation, list]] = None)[source]¶ Bases:
object
An
OutputTag
is a typed and named tag to use for tagging side outputs of an operator.Example:
# Explicitly specify output type >>> info = OutputTag("late-data", Types.TUPLE([Types.STRING(), Types.LONG()])) # Implicitly wrap list to Types.ROW >>> info_row = OutputTag("row", [Types.STRING(), Types.LONG()]) # Implicitly use pickle serialization >>> info_side = OutputTag("side") # ERROR: tag id cannot be empty string (extra requirement for Python API) >>> info_error = OutputTag("")
-
class
pyflink.datastream.
Partitioner
[source]¶ Bases:
pyflink.datastream.functions.Function
Function to implement a custom partition assignment for keys.
-
close
()¶
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
-
class
pyflink.datastream.
PredefinedOptions
(value)[source]¶ Bases:
enum.Enum
The
PredefinedOptions
are configuration settings for theRocksDBStateBackend
. The various pre-defined choices are configurations that have been empirically determined to be beneficial for performance under different settings.Some of these settings are based on experiments by the Flink community, some follow guides from the RocksDB project.
Default options for all settings, except that writes are not forced to the disk.
Note
Because Flink does not rely on RocksDB data on disk for recovery, there is no need to sync data to stable storage.
Pre-defined options for regular spinning hard disks.
This constant configures RocksDB with some options that lead empirically to better performance when the machines executing the system use regular spinning hard disks.
The following options are set:
setCompactionStyle(CompactionStyle.LEVEL)
setLevelCompactionDynamicLevelBytes(true)
setIncreaseParallelism(4)
setUseFsync(false)
setDisableDataSync(true)
setMaxOpenFiles(-1)
Note
Because Flink does not rely on RocksDB data on disk for recovery, there is no need to sync data to stable storage.
SPINNING_DISK_OPTIMIZED_HIGH_MEM
:Pre-defined options for better performance on regular spinning hard disks, at the cost of a higher memory consumption.
Note
These settings will cause RocksDB to consume a lot of memory for block caching and compactions. If you experience out-of-memory problems related to, RocksDB, consider switching back to
SPINNING_DISK_OPTIMIZED
.The following options are set:
setLevelCompactionDynamicLevelBytes(true)
setTargetFileSizeBase(256 MBytes)
setMaxBytesForLevelBase(1 GByte)
setWriteBufferSize(64 MBytes)
setIncreaseParallelism(4)
setMinWriteBufferNumberToMerge(3)
setMaxWriteBufferNumber(4)
setUseFsync(false)
setMaxOpenFiles(-1)
BlockBasedTableConfig.setBlockCacheSize(256 MBytes)
BlockBasedTableConfigsetBlockSize(128 KBytes)
Note
Because Flink does not rely on RocksDB data on disk for recovery, there is no need to sync data to stable storage.
Pre-defined options for Flash SSDs.
This constant configures RocksDB with some options that lead empirically to better performance when the machines executing the system use SSDs.
The following options are set:
setIncreaseParallelism(4)
setUseFsync(false)
setDisableDataSync(true)
setMaxOpenFiles(-1)
Note
Because Flink does not rely on RocksDB data on disk for recovery, there is no need to sync data to stable storage.
-
DEFAULT
= 0¶
-
FLASH_SSD_OPTIMIZED
= 3¶
-
SPINNING_DISK_OPTIMIZED
= 1¶
-
SPINNING_DISK_OPTIMIZED_HIGH_MEM
= 2¶
-
class
pyflink.datastream.
ProcessFunction
[source]¶ Bases:
pyflink.datastream.functions.Function
A function that process elements of a stream.
For every element in the input stream process_element(value, ctx, out) is invoked. This can produce zero or more elements as output. Implementations can also query the time and set timers through the provided Context. For firing timers on_timer(long, ctx, out) will be invoked. This can again produce zero or more elements as output and register further timers.
Note that access to keyed state and timers (which are also scoped to a key) is only available if the ProcessFunction is applied on a KeyedStream.
-
class
Context
[source]¶ Bases:
abc.ABC
Information available in an invocation of process_element(value, ctx, out) or on_timer(value, ctx, out).
-
close
()¶
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
abstract
process_element
(value, ctx: pyflink.datastream.functions.ProcessFunction.Context)[source]¶ Process one element from the input stream.
This function can output zero or more elements using the Collector parameter and also update internal state or set timers using the Context parameter.
- Parameters
value – The input value.
ctx – A Context that allows querying the timestamp of the element and getting a TimerService for registering timers and querying the time. The context is only valid during the invocation of this method, do not store it.
-
class
-
class
pyflink.datastream.
ProcessWindowFunction
(*args, **kwds)[source]¶ Bases:
pyflink.datastream.functions.Function
,typing.Generic
Base interface for functions that are evaluated over keyed (grouped) windows using a context for retrieving extra information.
-
class
Context
(*args, **kwds)[source]¶ Bases:
abc.ABC
,typing.Generic
The context holding window metadata.
-
abstract
global_state
() → pyflink.datastream.functions.KeyedStateStore[source]¶ State accessor for per-key global state.
-
abstract
window_state
() → pyflink.datastream.functions.KeyedStateStore[source]¶ State accessor for per-key and per-window state.
Note
If you use per-window state you have to ensure that you clean it up by implementing
clear()
.- Returns
The
KeyedStateStore
used to access per-key and per-window states.
-
abstract
-
clear
(context: pyflink.datastream.functions.ProcessWindowFunction.Context) → None[source]¶ Deletes any state in the
Context
when the Window expires (the watermark passes its max_timestamp + allowed_lateness).- Parameters
context – The context to which the window is being evaluated.
-
close
()¶
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
abstract
process
(key: KEY, context: pyflink.datastream.functions.ProcessWindowFunction.Context, elements: Iterable[IN]) → Iterable[OUT][source]¶ Evaluates the window and outputs none or several elements.
- Parameters
key – The key for which this window is evaluated.
context – The context in which the window is being evaluated.
elements – The elements in the window being evaluated.
- Returns
The iterable object which produces the elements to emit.
-
class
-
class
pyflink.datastream.
ReduceFunction
[source]¶ Bases:
pyflink.datastream.functions.Function
Base interface for Reduce functions. Reduce functions combine groups of elements to a single value, by taking always two elements and combining them into one. Reduce functions may be used on entire data sets, or on grouped data sets. In the latter case, each group is reduced individually.
The basic syntax for using a ReduceFunction is as follows:
- ::
>>> ds = ... >>> new_ds = ds.key_by(lambda x: x[1]).reduce(MyReduceFunction())
-
close
()¶
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
abstract
reduce
(value1, value2)[source]¶ The core method of ReduceFunction, combining two values into one value of the same type. The reduce function is consecutively applied to all values of a group until only a single value remains.
- Parameters
value1 – The first value to combine.
value2 – The second value to combine.
- Returns
The combined value of both input values.
-
class
pyflink.datastream.
RocksDBStateBackend
(checkpoint_data_uri=None, enable_incremental_checkpointing=None, checkpoint_stream_backend=None, j_rocks_db_state_backend=None)[source]¶ Bases:
pyflink.datastream.state_backend.StateBackend
IMPORTANT RocksDBStateBackend is deprecated in favor of EmbeddedRocksDBStateBackend and FileSystemCheckpointStorage. This change does not affect the runtime characteristics of your Jobs and is simply an API change to help better communicate the ways Flink separates local state storage from fault tolerance. Jobs can be upgraded without loss of state. If configuring your state backend via the StreamExecutionEnvironment please make the following changes.
>> env.set_state_backend(EmbeddedRocksDBStateBackend()) >> env.get_checkpoint_config().set_checkpoint_storage("hdfs://checkpoints")
If you are configuring your state backend via the flink-conf.yaml no changes are required.
A State Backend that stores its state in
RocksDB
. This state backend can store very large state that exceeds memory and spills to disk.All key/value state (including windows) is stored in the key/value index of RocksDB. For persistence against loss of machines, checkpoints take a snapshot of the RocksDB database, and persist that snapshot in a file system (by default) or another configurable state backend.
The behavior of the RocksDB instances can be parametrized by setting RocksDB Options using the methods
set_predefined_options()
andset_options()
.-
get_checkpoint_backend
()[source]¶ Gets the state backend that this RocksDB state backend uses to persist its bytes to.
This RocksDB state backend only implements the RocksDB specific parts, it relies on the ‘CheckpointBackend’ to persist the checkpoint and savepoint bytes streams.
- Returns
The state backend to persist the checkpoint and savepoint bytes streams.
-
get_db_storage_paths
() → List[str][source]¶ Gets the configured local DB storage paths, or null, if none were configured.
Under these directories on the TaskManager, RocksDB stores its SST files and metadata files. These directories do not need to be persistent, they can be ephermeral, meaning that they are lost on a machine failure, because state in RocksDB is persisted in checkpoints.
If nothing is configured, these directories default to the TaskManager’s local temporary file directories.
- Returns
The list of configured local DB storage paths.
-
get_number_of_transfering_threads
() → int[source]¶ Gets the number of threads used to transfer files while snapshotting/restoring.
- Returns
The number of threads used to transfer files while snapshotting/restoring.
-
get_options
() → Optional[str][source]¶ Gets the fully-qualified class name of the options factory in Java that lazily creates the RocksDB options.
- Returns
The fully-qualified class name of the options factory in Java.
-
get_predefined_options
() → pyflink.datastream.state_backend.PredefinedOptions[source]¶ Gets the current predefined options for RocksDB. The default options (if nothing was set via
setPredefinedOptions()
) arePredefinedOptions.DEFAULT
.If user-configured options within
RocksDBConfigurableOptions
is set (through flink-conf.yaml) or a user-defined options factory is set (viasetOptions()
), then the options from the factory are applied on top of the predefined and customized options.See also
- Returns
Current predefined options.
-
is_incremental_checkpoints_enabled
() → bool[source]¶ Gets whether incremental checkpoints are enabled for this state backend.
- Returns
True if incremental checkpoints are enabled, false otherwise.
-
set_db_storage_paths
(*paths: str)[source]¶ Sets the directories in which the local RocksDB database puts its files (like SST and metadata files). These directories do not need to be persistent, they can be ephemeral, meaning that they are lost on a machine failure, because state in RocksDB is persisted in checkpoints.
If nothing is configured, these directories default to the TaskManager’s local temporary file directories.
Each distinct state will be stored in one path, but when the state backend creates multiple states, they will store their files on different paths.
Passing
None
to this function restores the default behavior, where the configured temp directories will be used.- Parameters
paths – The paths across which the local RocksDB database files will be spread. this parameter is optional.
-
set_number_of_transfering_threads
(number_of_transfering_threads: int)[source]¶ Sets the number of threads used to transfer files while snapshotting/restoring.
- Parameters
number_of_transfering_threads – The number of threads used to transfer files while snapshotting/restoring.
-
set_options
(options_factory_class_name: str)[source]¶ Sets
org.rocksdb.Options
for the RocksDB instances. Because the options are not serializable and hold native code references, they must be specified through a factory.The options created by the factory here are applied on top of the pre-defined options profile selected via
set_predefined_options()
. If the pre-defined options profile is the default (PredefinedOptions.DEFAULT
), then the factory fully controls the RocksDB options.- Parameters
options_factory_class_name – The fully-qualified class name of the options factory in Java that lazily creates the RocksDB options. The options factory must have a default constructor.
-
set_predefined_options
(options: pyflink.datastream.state_backend.PredefinedOptions)[source]¶ Sets the predefined options for RocksDB.
If user-configured options within
RocksDBConfigurableOptions
is set (through flink-conf.yaml) or a user-defined options factory is set (viasetOptions()
), then the options from the factory are applied on top of the here specified predefined options and customized options.Example:
>>> state_backend.set_predefined_options(PredefinedOptions.SPINNING_DISK_OPTIMIZED)
- Parameters
options – The options to set (must not be null), see
PredefinedOptions
.
-
-
class
pyflink.datastream.
RuntimeContext
[source]¶ Bases:
pyflink.datastream.functions.KeyedStateStore
A RuntimeContext contains information about the context in which functions are executed. Each parallel instance of the function will have a context through which it can access static contextual information (such as the current parallelism).
-
abstract
get_aggregating_state
(state_descriptor: pyflink.datastream.state.AggregatingStateDescriptor) → pyflink.datastream.state.AggregatingState¶ Gets a handle to the system’s key/value aggregating state. This state is similar to the state accessed via get_state(ValueStateDescriptor), but is optimized for state that aggregates values with different types.
This state is only accessible if the function is executed on a KeyedStream.
-
abstract
get_attempt_number
() → int[source]¶ Gets the attempt number of this parallel subtask. First attempt is numbered 0.
-
abstract
get_index_of_this_subtask
() → int[source]¶ Gets the number of this parallel subtask. The numbering starts from 0 and goes up to parallelism-1 (parallelism as returned by
get_number_of_parallel_subtasks()
).
-
abstract
get_job_parameter
(key: str, default_value: str)[source]¶ Gets the global job parameter value associated with the given key as a string.
-
abstract
get_list_state
(state_descriptor: pyflink.datastream.state.ListStateDescriptor) → pyflink.datastream.state.ListState¶ Gets a handle to the system’s key/value list state. This state is similar to the value state access, but is optimized for state that holds lists. One can add elements to the list, or retrieve the list as a whle.
This state is only accessible if the function is executed on a KeyedStream.
-
abstract
get_map_state
(state_descriptor: pyflink.datastream.state.MapStateDescriptor) → pyflink.datastream.state.MapState¶ Gets a handle to the system’s key/value map state. This state is similar to the value state access, but is optimized for state that is composed of user-defined key-value pairs.
This state is only accessible if the function is executed on a KeyedStream.
-
abstract
get_max_number_of_parallel_subtasks
() → int[source]¶ Gets the number of max-parallelism with which the parallel task runs.
-
abstract
get_metrics_group
() → pyflink.metrics.metricbase.MetricGroup[source]¶ Gets the metric group.
-
abstract
get_number_of_parallel_subtasks
() → int[source]¶ Gets the parallelism with which the parallel task runs.
-
abstract
get_reducing_state
(state_descriptor: pyflink.datastream.state.ReducingStateDescriptor) → pyflink.datastream.state.ReducingState¶ Gets a handle to the system’s key/value reducing state. This state is similar to the state accessed via get_state(ValueStateDescriptor), but is optimized for state that aggregates values.
This state is only accessible if the function is executed on a KeyedStream.
-
abstract
get_state
(state_descriptor: pyflink.datastream.state.ValueStateDescriptor) → pyflink.datastream.state.ValueState¶ Gets a handle to the system’s key/value state. THe key/value state is only accessible if the function is executed on a KeyedStream. On each access, the state exposes the value for the key of the element currently processed by the function. Each function may have multiple partitioned states, addressed with different names.
Because the scope of each value is the key of the currently processed element, and the elements are distributed by the Flink runtime, the system can transparently scale out and redistribute the state and KeyedStream.
-
abstract
get_task_name
() → str[source]¶ Returns the name of the task in which the UDF runs, as assigned during plan construction.
-
abstract
get_task_name_with_subtasks
() → str[source]¶ Returns the name of the task, appended with the subtask indicator, such as “MyTask (3/6)”, where 3 would be (
get_index_of_this_subtask()
+ 1), and 6 would beget_number_of_parallel_subtasks()
.
-
abstract
-
class
pyflink.datastream.
RuntimeExecutionMode
(value)[source]¶ Bases:
enum.Enum
Runtime execution mode of DataStream programs. Among other things, this controls task scheduling, network shuffle behavior, and time semantics. Some operations will also change their record emission behaviour based on the configured execution mode.
The Pipeline will be executed with Streaming Semantics. All tasks will be deployed before execution starts, checkpoints will be enabled, and both processing and event time will be fully supported.
The Pipeline will be executed with Batch Semantics. Tasks will be scheduled gradually based on the scheduling region they belong, shuffles between regions will be blocking, watermarks are assumed to be “perfect” i.e. no late data, and processing time is assumed to not advance during execution.
Flink will set the execution mode to BATCH if all sources are bounded, or STREAMING if there is at least one source which is unbounded.
-
AUTOMATIC
= 2¶
-
BATCH
= 1¶
-
STREAMING
= 0¶
-
-
class
pyflink.datastream.
SinkFunction
(sink_func: Union[str, py4j.java_gateway.JavaObject])[source]¶ Bases:
pyflink.datastream.functions.JavaFunctionWrapper
The base class for SinkFunctions.
-
get_java_function
()¶
-
-
class
pyflink.datastream.
SlotSharingGroup
(j_slot_sharing_group)[source]¶ Bases:
object
Describe the name and the different resource components of a slot sharing group.
-
class
Builder
(j_builder)[source]¶ Bases:
object
Builder for the SlotSharingGroup.
-
build
() → pyflink.datastream.slot_sharing_group.SlotSharingGroup[source]¶ Builds the SlotSharingGroup.
- Returns
The SlotSharingGroup object.
-
set_cpu_cores
(cpu_cores: float) → pyflink.datastream.slot_sharing_group.SlotSharingGroup.Builder[source]¶ Sets the CPU cores for this SlotSharingGroup.
- Parameters
cpu_cores – The CPU cores of the SlotSharingGroup.
- Returns
This object.
-
set_external_resource
(name: str, value: float) → pyflink.datastream.slot_sharing_group.SlotSharingGroup.Builder[source]¶ Adds the given external resource. The old value with the same resource name will be replaced if present.
- Parameters
name – The resource name of the given external resource.
value – The value of the given external resource.
- Returns
This object.
-
set_managed_memory
(managed_memory: pyflink.datastream.slot_sharing_group.MemorySize) → pyflink.datastream.slot_sharing_group.SlotSharingGroup.Builder[source]¶ Sets the task managed memory for this SlotSharingGroup.
- Parameters
managed_memory – The task managed memory of the SlotSharingGroup.
- Returns
This object.
-
set_managed_memory_mb
(managed_memory_mb: int) → pyflink.datastream.slot_sharing_group.SlotSharingGroup.Builder[source]¶ Sets the task managed memory for this SlotSharingGroup in MB.
- Parameters
managed_memory_mb – The task managed memory of the SlotSharingGroup in MB.
- Returns
This object.
-
set_task_heap_memory
(task_heap_memory: pyflink.datastream.slot_sharing_group.MemorySize) → pyflink.datastream.slot_sharing_group.SlotSharingGroup.Builder[source]¶ Sets the task heap memory for this SlotSharingGroup.
- Parameters
task_heap_memory – The task heap memory of the SlotSharingGroup.
- Returns
This object.
-
set_task_heap_memory_mb
(task_heap_memory_mb: int) → pyflink.datastream.slot_sharing_group.SlotSharingGroup.Builder[source]¶ Sets the task heap memory for this SlotSharingGroup in MB.
- Parameters
task_heap_memory_mb – The task heap memory of the SlotSharingGroup in MB.
- Returns
This object.
-
set_task_off_heap_memory
(task_off_heap_memory: pyflink.datastream.slot_sharing_group.MemorySize) → pyflink.datastream.slot_sharing_group.SlotSharingGroup.Builder[source]¶ Sets the task off-heap memory for this SlotSharingGroup.
- Parameters
task_off_heap_memory – The task off-heap memory of the SlotSharingGroup.
- Returns
This object.
-
set_task_off_heap_memory_mb
(task_off_heap_memory_mb: int) → pyflink.datastream.slot_sharing_group.SlotSharingGroup.Builder[source]¶ Sets the task off-heap memory for this SlotSharingGroup in MB.
- Parameters
task_off_heap_memory_mb – The task off-heap memory of the SlotSharingGroup in MB.
- Returns
This object.
-
-
static
builder
(name: str) → Builder[source]¶ Gets the Builder with the given name for this SlotSharingGroup.
- Parameters
name – The name of the SlotSharingGroup.
- Returns
The builder for the SlotSharingGroup.
-
get_cpu_cores
() → Optional[float][source]¶ Gets the CPU cores for this SlotSharingGroup.
- return
The CPU cores of the SlotSharingGroup.
-
get_external_resources
() → dict[source]¶ Gets the external resource from this SlotSharingGroup.
- Returns
User specified resources of the SlotSharingGroup.
-
get_java_slot_sharing_group
()[source]¶ Gets the Java SlotSharingGroup object.
- Returns
The Java SlotSharingGroup object.
-
get_managed_memory
() → Optional[pyflink.datastream.slot_sharing_group.MemorySize][source]¶ Gets the task managed memory for this SlotSharingGroup.
- Returns
The task managed memory of the SlotSharingGroup.
-
get_name
() → str[source]¶ Gets the name of this SlotSharingGroup.
- Returns
The name of the SlotSharingGroup.
-
class
-
class
pyflink.datastream.
SourceFunction
(source_func: Union[str, py4j.java_gateway.JavaObject])[source]¶ Bases:
pyflink.datastream.functions.JavaFunctionWrapper
Base class for all stream data source in Flink.
-
get_java_function
()¶
-
-
class
pyflink.datastream.
StateBackend
(j_state_backend)[source]¶ Bases:
object
A State Backend defines how the state of a streaming application is stored locally within the cluster. Different state backends store their state in different fashions, and use different data structures to hold the state of running applications.
For example, the
HashMapStateBackend
keeps working state in the memory of the TaskManager. The backend is lightweight and without additional dependencies.The
EmbeddedRocksDBStateBackend
keeps working state in the memory of the TaskManager and stores state checkpoints in a filesystem(typically a replicated highly-available filesystem, like HDFS, Ceph, S3, GCS, etc).The
EmbeddedRocksDBStateBackend
stores working state in an embedded RocksDB, instance and is able to scale working state to many terrabytes in size, only limited by available disk space across all task managers.Raw Bytes Storage and Backends
The
StateBackend
creates services for raw bytes storage and for keyed state and operator state.The org.apache.flink.runtime.state.AbstractKeyedStateBackend and `org.apache.flink.runtime.state.OperatorStateBackend created by this state backend define how to hold the working state for keys and operators. They also define how to checkpoint that state, frequently using the raw bytes storage (via the org.apache.flink.runtime.state.CheckpointStreamFactory). However, it is also possible that for example a keyed state backend simply implements the bridge to a key/value store, and that it does not need to store anything in the raw byte storage upon a checkpoint.
Serializability
State Backends need to be serializable(java.io.Serializable), because they distributed across parallel processes (for distributed execution) together with the streaming application code.
Because of that,
StateBackend
implementations are meant to be like factories that create the proper states stores that provide access to the persistent storage and hold the keyed- and operator state data structures. That way, the State Backend can be very lightweight (contain only configurations) which makes it easier to be serializable.Thread Safety
State backend implementations have to be thread-safe. Multiple threads may be creating streams and keyed-/operator state backends concurrently.
-
class
pyflink.datastream.
StreamExecutionEnvironment
(j_stream_execution_environment, serializer=PickleSerializer())[source]¶ Bases:
object
The StreamExecutionEnvironment is the context in which a streaming program is executed. A LocalStreamEnvironment will cause execution in the attached JVM, a RemoteStreamEnvironment will cause execution on a remote setup.
The environment provides methods to control the job execution (such as setting the parallelism or the fault tolerance/checkpointing parameters) and to interact with the outside world (data access).
-
add_classpaths
(*classpaths: str)[source]¶ Adds a list of URLs that are added to the classpath of each user code classloader of the program. Paths must specify a protocol (e.g. file://) and be accessible on all nodes
- Parameters
classpaths – Classpaths that will be added.
-
add_default_kryo_serializer
(type_class_name: str, serializer_class_name: str)[source]¶ Adds a new Kryo default serializer to the Runtime.
Example:
>>> env.add_default_kryo_serializer("com.aaa.bbb.TypeClass", "com.aaa.bbb.Serializer")
- Parameters
type_class_name – The full-qualified java class name of the types serialized with the given serializer.
serializer_class_name – The full-qualified java class name of the serializer to use.
-
add_jars
(*jars_path: str)[source]¶ Adds a list of jar files that will be uploaded to the cluster and referenced by the job.
- Parameters
jars_path – Path of jars.
-
add_python_archive
(archive_path: str, target_dir: str = None)[source]¶ Adds a python archive file. The file will be extracted to the working directory of python UDF worker.
If the parameter “target_dir” is specified, the archive file will be extracted to a directory named ${target_dir}. Otherwise, the archive file will be extracted to a directory with the same name of the archive file.
If python UDF depends on a specific python version which does not exist in the cluster, this method can be used to upload the virtual environment. Note that the path of the python interpreter contained in the uploaded environment should be specified via the method
pyflink.table.TableConfig.set_python_executable()
.The files uploaded via this method are also accessible in UDFs via relative path.
Example:
# command executed in shell # assert the relative path of python interpreter is py_env/bin/python $ zip -r py_env.zip py_env # python code >>> stream_env.add_python_archive("py_env.zip") >>> stream_env.set_python_executable("py_env.zip/py_env/bin/python") # or >>> stream_env.add_python_archive("py_env.zip", "myenv") >>> stream_env.set_python_executable("myenv/py_env/bin/python") # the files contained in the archive file can be accessed in UDF >>> def my_udf(): ... with open("myenv/py_env/data/data.txt") as f: ... ...
Note
Please make sure the uploaded python environment matches the platform that the cluster is running on and that the python version must be 3.6 or higher.
Note
Currently only zip-format is supported. i.e. zip, jar, whl, egg, etc. The other archive formats such as tar, tar.gz, 7z, rar, etc are not supported.
- Parameters
archive_path – The archive file path.
target_dir – Optional, the target dir name that the archive file extracted to.
-
add_python_file
(file_path: str)[source]¶ Adds a python dependency which could be python files, python packages or local directories. They will be added to the PYTHONPATH of the python UDF worker. Please make sure that these dependencies can be imported.
- Parameters
file_path – The path of the python dependency.
-
add_source
(source_func: pyflink.datastream.functions.SourceFunction, source_name: str = 'Custom Source', type_info: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Adds a data source to the streaming topology.
- Parameters
source_func – the user defined function.
source_name – name of the data source. Optional.
type_info – type of the returned stream. Optional.
- Returns
the data stream constructed.
-
close
()[source]¶ Close and clean up the execution environment. All the cached intermediate results will be released physically.
New in version 1.16.0.
-
configure
(configuration: pyflink.common.configuration.Configuration)[source]¶ Sets all relevant options contained in the
Configuration
. such as e.g. pipeline.time-characteristic. It will reconfigureStreamExecutionEnvironment
,ExecutionConfig
andCheckpointConfig
.It will change the value of a setting only if a corresponding option was set in the configuration. If a key is not present, the current value of a field will remain untouched.
- Parameters
configuration – a configuration to read the values from.
New in version 1.15.0.
-
create_input
(input_format: pyflink.common.io.InputFormat, type_info: Optional[pyflink.common.typeinfo.TypeInformation] = None)[source]¶ Create an input data stream with InputFormat.
If the input_format needs a well-defined type information (e.g. Avro’s generic record), you can either explicitly use type_info argument or use InputFormats implementing ResultTypeQueryable.
- Parameters
input_format – The input format to read from.
type_info – Optional type information to explicitly declare output type.
New in version 1.16.0.
-
disable_operator_chaining
() → pyflink.datastream.stream_execution_environment.StreamExecutionEnvironment[source]¶ Disables operator chaining for streaming operators. Operator chaining allows non-shuffle operations to be co-located in the same thread fully avoiding serialization and de-serialization.
- Returns
This object.
-
enable_changelog_state_backend
(enabled: bool) → pyflink.datastream.stream_execution_environment.StreamExecutionEnvironment[source]¶ Enable the change log for current state backend. This change log allows operators to persist state changes in a very fine-grained manner. Currently, the change log only applies to keyed state, so non-keyed operator state and channel state are persisted as usual. The ‘state’ here refers to ‘keyed state’. Details are as follows:
Stateful operators write the state changes to that log (logging the state), in addition to applying them to the state tables in RocksDB or the in-mem Hashtable.
An operator can acknowledge a checkpoint as soon as the changes in the log have reached the durable checkpoint storage.
The state tables are persisted periodically, independent of the checkpoints. We call this the materialization of the state on the checkpoint storage.
Once the state is materialized on checkpoint storage, the state changelog can be truncated to the corresponding point.
It establish a way to drastically reduce the checkpoint interval for streaming applications across state backends. For more details please check the FLIP-158.
If this method is not called explicitly, it means no preference for enabling the change log. Configs for change log enabling will override in different config levels (job/local/cluster).
See also
- Parameters
enabled – True if enable the change log for state backend explicitly, otherwise disable the change log.
- Returns
This object.
New in version 1.14.0.
-
enable_checkpointing
(interval: int, mode: pyflink.datastream.checkpointing_mode.CheckpointingMode = None) → pyflink.datastream.stream_execution_environment.StreamExecutionEnvironment[source]¶ Enables checkpointing for the streaming job. The distributed state of the streaming dataflow will be periodically snapshotted. In case of a failure, the streaming dataflow will be restarted from the latest completed checkpoint.
The job draws checkpoints periodically, in the given interval. The system uses the given
CheckpointingMode
for the checkpointing (“exactly once” vs “at least once”). The state will be stored in the configured state backend.Note
Checkpointing iterative streaming dataflows in not properly supported at the moment. For that reason, iterative jobs will not be started if used with enabled checkpointing.
Example:
>>> env.enable_checkpointing(300000, CheckpointingMode.AT_LEAST_ONCE)
- Parameters
interval – Time interval between state checkpoints in milliseconds.
mode – The checkpointing mode, selecting between “exactly once” and “at least once” guaranteed.
- Returns
This object.
-
execute
(job_name: str = None) → pyflink.common.job_execution_result.JobExecutionResult[source]¶ Triggers the program execution. The environment will execute all parts of the program that have resulted in a “sink” operation. Sink operations are for example printing results or forwarding them to a message queue.
The program execution will be logged and displayed with the provided name
- Parameters
job_name – Desired name of the job, optional.
- Returns
The result of the job execution, containing elapsed time and accumulators.
-
execute_async
(job_name: str = 'Flink Streaming Job') → pyflink.common.job_client.JobClient[source]¶ Triggers the program asynchronously. The environment will execute all parts of the program that have resulted in a “sink” operation. Sink operations are for example printing results or forwarding them to a message queue. The program execution will be logged and displayed with a generated default name.
- Parameters
job_name – Desired name of the job.
- Returns
A JobClient that can be used to communicate with the submitted job, completed on submission succeeded.
-
from_collection
(collection: List[Any], type_info: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Creates a data stream from the given non-empty collection. The type of the data stream is that of the elements in the collection.
Note that this operation will result in a non-parallel data stream source, i.e. a data stream source with parallelism one.
- Parameters
collection – The collection of elements to create the data stream from.
type_info – The TypeInformation for the produced data stream
- Returns
the data stream representing the given collection.
-
from_source
(source: pyflink.datastream.connectors.base.Source, watermark_strategy: pyflink.common.watermark_strategy.WatermarkStrategy, source_name: str, type_info: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Adds a data
Source
to the environment to get aDataStream
.The result will be either a bounded data stream (that can be processed in a batch way) or an unbounded data stream (that must be processed in a streaming way), based on the boundedness property of the source.
This method takes an explicit type information for the produced data stream, so that callers can define directly what type/serializer will be used for the produced stream. For sources that describe their produced type, the parameter type_info should not be specified to avoid specifying the produced type redundantly.
New in version 1.13.0.
-
get_buffer_timeout
() → int[source]¶ Gets the maximum time frequency (milliseconds) for the flushing of the output buffers. For clarification on the extremal values see
set_buffer_timeout()
.- Returns
The timeout of the buffer.
-
get_checkpoint_config
() → pyflink.datastream.checkpoint_config.CheckpointConfig[source]¶ Gets the checkpoint config, which defines values like checkpoint interval, delay between checkpoints, etc.
- Returns
The
CheckpointConfig
.
-
get_checkpoint_interval
() → int[source]¶ Returns the checkpointing interval or -1 if checkpointing is disabled.
Shorthand for get_checkpoint_config().get_checkpoint_interval().
- Returns
The checkpointing interval or -1.
-
get_checkpointing_mode
() → pyflink.datastream.checkpointing_mode.CheckpointingMode[source]¶ Returns the checkpointing mode (exactly-once vs. at-least-once).
Shorthand for get_checkpoint_config().get_checkpointing_mode().
- Returns
The
CheckpointingMode
.
-
get_config
() → pyflink.common.execution_config.ExecutionConfig[source]¶ Gets the config object.
- Returns
The
ExecutionConfig
object.
-
get_default_local_parallelism
() → int[source]¶ Gets the default parallelism that will be used for the local execution environment.
- Returns
The default local parallelism.
-
get_default_savepoint_directory
() → Optional[str][source]¶ Gets the default savepoint directory for this Job.
-
static
get_execution_environment
(configuration: pyflink.common.configuration.Configuration = None) → pyflink.datastream.stream_execution_environment.StreamExecutionEnvironment[source]¶ Creates an execution environment that represents the context in which the program is currently executed. If the program is invoked standalone, this method returns a local execution environment.
When executed from the command line the given configuration is stacked on top of the global configuration which comes from the flink-conf.yaml, potentially overriding duplicated options.
- Parameters
configuration – The configuration to instantiate the environment with.
- Returns
The execution environment of the context in which the program is executed.
-
get_execution_plan
() → str[source]¶ Creates the plan with which the system will execute the program, and returns it as a String using a JSON representation of the execution data flow graph. Note that this needs to be called, before the plan is executed.
If the compiler could not be instantiated, or the master could not be contacted to retrieve information relevant to the execution planning, an exception will be thrown.
- Returns
The execution plan of the program, as a JSON String.
-
get_max_parallelism
() → int[source]¶ Gets the maximum degree of parallelism defined for the program.
The maximum degree of parallelism specifies the upper limit for dynamic scaling. It also defines the number of key groups used for partitioned state.
- Returns
Maximum degree of parallelism.
-
get_parallelism
() → int[source]¶ Gets the parallelism with which operation are executed by default. Operations can individually override this value to use a specific parallelism.
- Returns
The parallelism used by operations, unless they override that value.
-
get_restart_strategy
() → pyflink.common.restart_strategy.RestartStrategyConfiguration[source]¶ Returns the specified restart strategy configuration.
- Returns
The restart strategy configuration to be used.
-
get_state_backend
() → pyflink.datastream.state_backend.StateBackend[source]¶ Gets the state backend that defines how to store and checkpoint state.
See also
- Returns
The
StateBackend
.
-
get_stream_time_characteristic
() → pyflink.datastream.time_characteristic.TimeCharacteristic[source]¶ Gets the time characteristic.
See also
- Returns
The
TimeCharacteristic
.
-
is_chaining_enabled
() → bool[source]¶ Returns whether operator chaining is enabled.
- Returns
True if chaining is enabled, false otherwise.
-
is_changelog_state_backend_enabled
() → Optional[bool][source]¶ Gets the enable status of change log for state backend.
See also
- Returns
An
Optional[bool]
for the enable status of change log for state backend. Could be None if user never specify this by callingenable_changelog_state_backend()
.
New in version 1.14.0.
-
read_text_file
(file_path: str, charset_name: str = 'UTF-8') → pyflink.datastream.data_stream.DataStream[source]¶ Reads the given file line-by-line and creates a DataStream that contains a string with the contents of each such line. The charset with the given name will be used to read the files.
Note that this interface is not fault tolerant that is supposed to be used for test purpose.
- Parameters
file_path – The path of the file, as a URI (e.g., “file:///some/local/file” or “hdfs://host:port/file/path”)
charset_name – The name of the character set used to read the file.
- Returns
The DataStream that represents the data read from the given file as text lines.
-
register_cached_file
(file_path: str, name: str, executable: bool = False)[source]¶ Registers a file at the distributed cache under the given name. The file will be accessible from any user-defined function in the (distributed) runtime under a local path. Files may be local files (which will be distributed via BlobServer), or files in a distributed file system. The runtime will copy the files temporarily to a local cache, if needed.
- Parameters
file_path – The path of the file, as a URI (e.g. “file:///some/path” or hdfs://host:port/and/path”).
name – The name under which the file is registered.
executable – Flag indicating whether the file should be executable.
New in version 1.16.0.
-
register_slot_sharing_group
(slot_sharing_group: pyflink.datastream.slot_sharing_group.SlotSharingGroup) → pyflink.datastream.stream_execution_environment.StreamExecutionEnvironment[source]¶ Register a slot sharing group with its resource spec.
Note that a slot sharing group hints the scheduler that the grouped operators CAN be deployed into a shared slot. There’s no guarantee that the scheduler always deploy the grouped operators together. In cases grouped operators are deployed into separate slots, the slot resources will be derived from the specified group requirements.
- Parameters
slot_sharing_group – Which contains name and its resource spec.
- Returns
This object.
-
register_type
(type_class_name: str)[source]¶ Registers the given type with the serialization stack. If the type is eventually serialized as a POJO, then the type is registered with the POJO serializer. If the type ends up being serialized with Kryo, then it will be registered at Kryo to make sure that only tags are written.
Example:
>>> env.register_type("com.aaa.bbb.TypeClass")
- Parameters
type_class_name – The full-qualified java class name of the type to register.
-
register_type_with_kryo_serializer
(type_class_name: str, serializer_class_name: str)[source]¶ Registers the given Serializer via its class as a serializer for the given type at the KryoSerializer.
Example:
>>> env.register_type_with_kryo_serializer("com.aaa.bbb.TypeClass", ... "com.aaa.bbb.Serializer")
- Parameters
type_class_name – The full-qualified java class name of the types serialized with the given serializer.
serializer_class_name – The full-qualified java class name of the serializer to use.
-
set_buffer_timeout
(timeout_millis: int) → pyflink.datastream.stream_execution_environment.StreamExecutionEnvironment[source]¶ Sets the maximum time frequency (milliseconds) for the flushing of the output buffers. By default the output buffers flush frequently to provide low latency and to aid smooth developer experience. Setting the parameter can result in three logical modes:
A positive integer triggers flushing periodically by that integer
0 triggers flushing after every record thus minimizing latency
-1 triggers flushing only when the output buffer is full thus maximizing throughput
- Parameters
timeout_millis – The maximum time between two output flushes.
- Returns
This object.
-
set_default_local_parallelism
(parallelism: int)[source]¶ Sets the default parallelism that will be used for the local execution environment.
- Parameters
parallelism – The parallelism to use as the default local parallelism.
-
set_default_savepoint_directory
(directory: str) → pyflink.datastream.stream_execution_environment.StreamExecutionEnvironment[source]¶ Sets the default savepoint directory, where savepoints will be written to if none is explicitly provided when triggered.
Example:
>>> env.set_default_savepoint_directory("hdfs://savepoints")
:param directory The savepoint directory :return: This object.
-
set_max_parallelism
(max_parallelism: int) → pyflink.datastream.stream_execution_environment.StreamExecutionEnvironment[source]¶ Sets the maximum degree of parallelism defined for the program. The upper limit (inclusive) is 32767.
The maximum degree of parallelism specifies the upper limit for dynamic scaling. It also defines the number of key groups used for partitioned state.
- Parameters
max_parallelism – Maximum degree of parallelism to be used for the program, with 0 < maxParallelism <= 2^15 - 1.
- Returns
This object.
-
set_parallelism
(parallelism: int) → pyflink.datastream.stream_execution_environment.StreamExecutionEnvironment[source]¶ Sets the parallelism for operations executed through this environment. Setting a parallelism of x here will cause all operators (such as map, batchReduce) to run with x parallel instances. This method overrides the default parallelism for this environment. The LocalStreamEnvironment uses by default a value equal to the number of hardware contexts (CPU cores / threads). When executing the program via the command line client from a JAR file, the default degree of parallelism is the one configured for that setup.
- Parameters
parallelism – The parallelism.
- Returns
This object.
-
set_python_executable
(python_exec: str)[source]¶ Sets the path of the python interpreter which is used to execute the python udf workers.
e.g. “/usr/local/bin/python3”.
If python UDF depends on a specific python version which does not exist in the cluster, the method
pyflink.datastream.StreamExecutionEnvironment.add_python_archive()
can be used to upload a virtual environment. The path of the python interpreter contained in the uploaded environment can be specified via this method.Example:
# command executed in shell # assume that the relative path of python interpreter is py_env/bin/python $ zip -r py_env.zip py_env # python code >>> stream_env.add_python_archive("py_env.zip") >>> stream_env.set_python_executable("py_env.zip/py_env/bin/python")
Note
Please make sure the uploaded python environment matches the platform that the cluster is running on and that the python version must be 3.6 or higher.
Note
The python udf worker depends on Apache Beam (version == 2.38.0). Please ensure that the specified environment meets the above requirements.
- Parameters
python_exec – The path of python interpreter.
-
set_python_requirements
(requirements_file_path: str, requirements_cache_dir: str = None)[source]¶ Specifies a requirements.txt file which defines the third-party dependencies. These dependencies will be installed to a temporary directory and added to the PYTHONPATH of the python UDF worker.
For the dependencies which could not be accessed in the cluster, a directory which contains the installation packages of these dependencies could be specified using the parameter “requirements_cached_dir”. It will be uploaded to the cluster to support offline installation.
Example:
# commands executed in shell $ echo numpy==1.16.5 > requirements.txt $ pip download -d cached_dir -r requirements.txt --no-binary :all: # python code >>> stream_env.set_python_requirements("requirements.txt", "cached_dir")
Note
Please make sure the installation packages matches the platform of the cluster and the python version used. These packages will be installed using pip, so also make sure the version of Pip (version >= 20.3) and the version of SetupTools (version >= 37.0.0).
- Parameters
requirements_file_path – The path of “requirements.txt” file.
requirements_cache_dir – The path of the local directory which contains the installation packages.
-
set_restart_strategy
(restart_strategy_configuration: pyflink.common.restart_strategy.RestartStrategyConfiguration)[source]¶ Sets the restart strategy configuration. The configuration specifies which restart strategy will be used for the execution graph in case of a restart.
Example:
>>> env.set_restart_strategy(RestartStrategies.no_restart())
- Parameters
restart_strategy_configuration – Restart strategy configuration to be set.
- Returns
-
set_runtime_mode
(execution_mode: pyflink.datastream.execution_mode.RuntimeExecutionMode)[source]¶ Sets the runtime execution mode for the application
RuntimeExecutionMode
. This is equivalent to setting the execution.runtime-mode in your application’s configuration file.We recommend users to NOT use this method but set the execution.runtime-mode using the command-line when submitting the application. Keeping the application code configuration-free allows for more flexibility as the same application will be able to be executed in any execution mode.
- Parameters
execution_mode – The desired execution mode.
- Returns
The execution environment of your application.
New in version 1.13.0.
-
set_state_backend
(state_backend: pyflink.datastream.state_backend.StateBackend) → pyflink.datastream.stream_execution_environment.StreamExecutionEnvironment[source]¶ Sets the state backend that describes how to store and checkpoint operator state. It defines both which data structures hold state during execution (for example hash tables, RockDB, or other data stores) as well as where checkpointed data will be persisted.
The
MemoryStateBackend
for example maintains the state in heap memory, as objects. It is lightweight without extra dependencies, but can checkpoint only small states(some counters).In contrast, the
FsStateBackend
stores checkpoints of the state (also maintained as heap objects) in files. When using a replicated file system (like HDFS, S3, Alluxio, etc) this will guarantee that state is not lost upon failures of individual nodes and that streaming program can be executed highly available and strongly consistent(assuming that Flink is run in high-availability mode).- The build-in state backend includes:
See also
Example:
>>> env.set_state_backend(EmbeddedRocksDBStateBackend())
- Parameters
state_backend – The
StateBackend
.- Returns
This object.
-
set_stream_time_characteristic
(characteristic: pyflink.datastream.time_characteristic.TimeCharacteristic)[source]¶ Sets the time characteristic for all streams create from this environment, e.g., processing time, event time, or ingestion time.
If you set the characteristic to IngestionTime of EventTime this will set a default watermark update interval of 200 ms. If this is not applicable for your application you should change it using
pyflink.common.ExecutionConfig.set_auto_watermark_interval()
.Example:
>>> env.set_stream_time_characteristic(TimeCharacteristic.EventTime)
- Parameters
characteristic – The time characteristic, which could be
TimeCharacteristic.ProcessingTime
,TimeCharacteristic.IngestionTime
,TimeCharacteristic.EventTime
.
-
-
class
pyflink.datastream.
TimeCharacteristic
(value)[source]¶ Bases:
enum.Enum
The time characteristic defines how the system determines time for time-dependent order and operations that depend on time (such as time windows).
Processing time for operators means that the operator uses the system clock of the machine to determine the current time of the data stream. Processing-time windows trigger based on wall-clock time and include whatever elements happen to have arrived at the operator at that point in time.
Using processing time for window operations results in general in quite non-deterministic results, because the contents of the windows depends on the speed in which elements arrive. It is, however, the cheapest method of forming windows and the method that introduces the least latency.
Ingestion time means that the time of each individual element in the stream is determined when the element enters the Flink streaming data flow. Operations like windows group the elements based on that time, meaning that processing speed within the streaming dataflow does not affect windowing, but only the speed at which sources receive elements.
Ingestion time is often a good compromise between processing time and event time. It does not need any special manual form of watermark generation, and events are typically not too much out-or-order when they arrive at operators; in fact, out-of-orderness can only be introduced by streaming shuffles or split/join/union operations. The fact that elements are not very much out-of-order means that the latency increase is moderate, compared to event time.
Event time means that the time of each individual element in the stream (also called event) is determined by the event’s individual custom timestamp. These timestamps either exist in the elements from before they entered the Flink streaming dataflow, or are user-assigned at the sources. The big implication of this is that it allows for elements to arrive in the sources and in all operators out of order, meaning that elements with earlier timestamps may arrive after elements with later timestamps.
Operators that window or order data with respect to event time must buffer data until they can be sure that all timestamps for a certain time interval have been received. This is handled by the so called “time watermarks”.
Operations based on event time are very predictable - the result of windowing operations is typically identical no matter when the window is executed and how fast the streams operate. At the same time, the buffering and tracking of event time is also costlier than operating with processing time, and typically also introduces more latency. The amount of extra cost depends mostly on how much out of order the elements arrive, i.e., how long the time span between the arrival of early and late elements is. With respect to the “time watermarks”, this means that the cost typically depends on how early or late the watermarks can be generated for their timestamp.
In relation to
IngestionTime
, the event time is similar, but refers the event’s original time, rather than the time assigned at the data source. Practically, that means that event time has generally more meaning, but also that it takes longer to determine that all elements for a certain time have arrived.-
EventTime
= 2¶
-
IngestionTime
= 1¶
-
ProcessingTime
= 0¶
-
-
class
pyflink.datastream.
TimeDomain
(value)[source]¶ Bases:
enum.Enum
TimeDomain specifies whether a firing timer is based on event time or processing time.
EVENT_TIME: Time is based on timestamp of events. PROCESSING_TIME: Time is based on the current processing-time of a machine where processing happens.
-
EVENT_TIME
= 0¶
-
PROCESSING_TIME
= 1¶
-
-
class
pyflink.datastream.
TimeWindow
(start: int, end: int)[source]¶ Bases:
pyflink.datastream.window.Window
Window that represents a time interval from start (inclusive) to end (exclusive).
-
cover
(other: pyflink.datastream.window.TimeWindow) → pyflink.datastream.window.TimeWindow[source]¶ Returns the minimal window covers both this window and the given window.
-
static
get_window_start_with_offset
(timestamp: int, offset: int, window_size: int)[source]¶ Method to get the window start for a timestamp.
- Parameters
timestamp – epoch millisecond to get the window start.
offset – The offset which window start would be shifted by.
window_size – The size of the generated windows.
- Returns
window start
-
intersects
(other: pyflink.datastream.window.TimeWindow) → bool[source]¶ Returns True if this window intersects the given window.
-
static
merge_windows
(windows: Iterable[TimeWindow], callback: pyflink.datastream.window.MergingWindowAssigner.MergeCallback[pyflink.datastream.window.TimeWindow]) → None[source]¶ Merge overlapping :class`TimeWindow`.
-
-
class
pyflink.datastream.
TimerService
[source]¶ Bases:
abc.ABC
Interface for working with time and timers.
-
delete_event_time_timer
(timestamp: int)[source]¶ Deletes the event-time timer with the given trigger time. This method has only an effect if such a timer was previously registered and did not already expire.
Timers can internally be scoped to keys and/or windows. When you delete a timer, it is removed from the current keyed context.
- Parameters
timestamp – The given trigger time of timer to be deleted.
-
delete_processing_time_timer
(timestamp: int)[source]¶ Deletes the processing-time timer with the given trigger time. This method has only an effect if such a timer was previously registered and did not already expire.
Timers can internally be scoped to keys and/or windows. When you delete a timer, it is removed from the current keyed context.
- Parameters
timestamp – The given trigger time of timer to be deleted.
-
abstract
register_event_time_timer
(timestamp: int)[source]¶ Registers a timer tobe fired when the event time watermark passes the given time.
Timers can internally be scoped to keys and/or windows. When you set a timer in a keyed context, such as in an operation on KeyedStream then that context will so be active when you receive the timer notification.
- Parameters
timestamp – The event time of the timer to be registered.
-
abstract
register_processing_time_timer
(timestamp: int)[source]¶ Registers a timer to be fired when processing time passes the given time.
Timers can internally be scoped to keys and/or windows. When you set a timer in a keyed context, such as in an operation on KeyedStream then that context will so be active when you receive the timer notification.
- Parameters
timestamp – The processing time of the timer to be registered.
-
-
class
pyflink.datastream.
Trigger
(*args, **kwds)[source]¶ Bases:
abc.ABC
,typing.Generic
A Trigger determines when a pane of a window should be evaluated to emit the results for that part of the window.
A pane is the bucket of elements that have the same key (assigned by the KeySelector) and same Window. An element can be in multiple panes if it was assigned to multiple windows by the WindowAssigner. These panes all have their own instance of the Trigger.
Triggers must not maintain state internally since they can be re-created or reused for different keys. All necessary state should be persisted using the state abstraction available on the TriggerContext.
When used with a MergingWindowAssigner the Trigger must return true from
can_merge()
andon_merge()
most be properly implemented.-
class
OnMergeContext
[source]¶ Bases:
pyflink.datastream.window.Trigger.TriggerContext
Extension of
TriggerContext
that is given toon_merge()
.-
abstract
delete_event_time_timer
(time: int) → None¶ Delete the event-time trigger for the given time.
-
abstract
delete_processing_time_timer
(time: int) → None¶ Delete the processing time trigger for the given time.
-
abstract
get_current_processing_time
() → int¶ - Returns
The current processing time.
-
abstract
get_current_watermark
() → int¶ - Returns
The current watermark time.
-
abstract
get_metric_group
() → pyflink.metrics.metricbase.MetricGroup¶ Returns the metric group for this
Trigger
. This is the same metric group that would be returned fromget_metric_group()
in a user function.- Returns
The metric group.
-
abstract
get_partitioned_state
(state_descriptor: pyflink.datastream.state.StateDescriptor) → pyflink.datastream.state.State¶ Retrieves a
State
object that can be used to interact with fault-tolerant state that is scoped to the window and key of the current trigger invocation.- Parameters
state_descriptor – The StateDescriptor that contains the name and type of the state that is being accessed.
- Returns
The partitioned state object.
-
abstract
merge_partitioned_state
(state_descriptor: pyflink.datastream.state.StateDescriptor) → None[source]¶
-
abstract
register_event_time_timer
(time: int) → None¶ Register an event-time callback. When the current watermark passes the specified time
on_event_time()
is called with the time specified here.- Parameters
time – The watermark at which to invoke
on_event_time()
.
-
abstract
register_processing_time_timer
(time: int) → None¶ Register a system time callback. When the current system time passes the specified time
on_processing_time()
is called with the time specified here.- Parameters
time – The time at which to invoke
on_processing_time()
.
-
abstract
-
class
TriggerContext
[source]¶ Bases:
abc.ABC
A context object that is given to
Trigger
methods to allow them to register timer callbacks and deal with state.-
abstract
delete_event_time_timer
(time: int) → None[source]¶ Delete the event-time trigger for the given time.
-
abstract
delete_processing_time_timer
(time: int) → None[source]¶ Delete the processing time trigger for the given time.
-
abstract
get_metric_group
() → pyflink.metrics.metricbase.MetricGroup[source]¶ Returns the metric group for this
Trigger
. This is the same metric group that would be returned fromget_metric_group()
in a user function.- Returns
The metric group.
-
abstract
get_partitioned_state
(state_descriptor: pyflink.datastream.state.StateDescriptor) → pyflink.datastream.state.State[source]¶ Retrieves a
State
object that can be used to interact with fault-tolerant state that is scoped to the window and key of the current trigger invocation.- Parameters
state_descriptor – The StateDescriptor that contains the name and type of the state that is being accessed.
- Returns
The partitioned state object.
-
abstract
register_event_time_timer
(time: int) → None[source]¶ Register an event-time callback. When the current watermark passes the specified time
on_event_time()
is called with the time specified here.- Parameters
time – The watermark at which to invoke
on_event_time()
.
-
abstract
register_processing_time_timer
(time: int) → None[source]¶ Register a system time callback. When the current system time passes the specified time
on_processing_time()
is called with the time specified here.- Parameters
time – The time at which to invoke
on_processing_time()
.
-
abstract
-
can_merge
() → bool[source]¶ Note
If this returns true you must properly implement
on_merge()
- Returns
True if this trigger supports merging of trigger state and can therefore be used with a MergingWindowAssigner.
-
abstract
clear
(window: W, ctx: pyflink.datastream.window.Trigger.TriggerContext) → None[source]¶ Clears any state that the trigger might still hold for the given window. This is called when a window is purged. Timers set using
register_event_time_timer()
andregister_processing_time_timer()
should be deleted here as well as state acquired usingget_partitioned_state()
.
-
abstract
on_element
(element: T, timestamp: int, window: W, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called for every element that gets added to a pane. The result of this will determine whether the pane is evaluated to emit results.
- Parameters
element – The element that arrived.
timestamp – The timestamp of the element that arrived.
window – The window to which the element is being added.
ctx – A context object that can be used to register timer callbacks.
-
abstract
on_event_time
(time: int, window: W, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when an event-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
abstract
on_merge
(window: W, ctx: pyflink.datastream.window.Trigger.OnMergeContext) → None[source]¶ Called when several windows have been merged into one window by the
WindowAssigner
.- Parameters
window – The new window that results from the merge.
ctx – A context object that can be used to register timer callbacks and access state.
-
abstract
on_processing_time
(time: int, window: W, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when a processing-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
class
-
class
pyflink.datastream.
TriggerResult
(value)[source]¶ Bases:
enum.Enum
Result type for trigger methods. This determines what happens with the window, for example whether the window function should be called, or the window should be discarded.
If a
Trigger
returns TriggerResult.FIRE or TriggerResult.FIRE_AND_PURGE but the window does not contain any data the window function will not be invoked, i.e. no data will be produced for the window.CONTINUE: No action is taken on the window.
FIRE_AND_PURGE: Evaluates the window function and emits the ‘window result’.
- FIRE: On FIRE, the window is evaluated and results are emitted. The window is not purged
though, all elements are retained.
- PURGE: All elements in the window are cleared and the window is discarded, without
evaluating the window function or emitting any elements.
-
CONTINUE
= (False, False)¶
-
FIRE
= (True, False)¶
-
FIRE_AND_PURGE
= (True, True)¶
-
PURGE
= (False, True)¶
-
class
pyflink.datastream.
Window
[source]¶ Bases:
abc.ABC
Window is a grouping of elements into finite buckets. Windows have a maximum timestamp which means that, at some point, all elements that go into one window will have arrived.
-
class
pyflink.datastream.
WindowAssigner
(*args, **kwds)[source]¶ Bases:
abc.ABC
,typing.Generic
A
WindowAssigner
assigns zero or moreWindow
to an element.In a window operation, elements are grouped by their key (if available) and by the windows to which it was assigned. The set of elements with the same key and window is called a pane. When a
Trigger
decides that a certain pane should fire the WindowFunction is applied to produce output elements for that pane.-
class
WindowAssignerContext
[source]¶ Bases:
abc.ABC
A context provided to the
WindowAssigner
that allows it to query the current processing time.
-
abstract
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[W][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
abstract
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, W][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
class
-
class
pyflink.datastream.
WindowFunction
(*args, **kwds)[source]¶ Bases:
pyflink.datastream.functions.Function
,typing.Generic
Base interface for functions that are evaluated over keyed (grouped) windows.
-
abstract
apply
(key: KEY, window: W, inputs: Iterable[IN]) → Iterable[OUT][source]¶ Evaluates the window and outputs none or several elements.
- Parameters
key – The key for which this window is evaluated.
window – The window that is being evaluated.
inputs – The elements in the window being evaluated.
-
close
()¶
-
open
(runtime_context: pyflink.datastream.functions.RuntimeContext)¶
-
abstract
-
class
pyflink.datastream.
WindowedStream
(keyed_stream: pyflink.datastream.data_stream.KeyedStream, window_assigner: pyflink.datastream.window.WindowAssigner)[source]¶ Bases:
object
A WindowedStream represents a data stream where elements are grouped by key, and for each key, the stream of elements is split into windows based on a WindowAssigner. Window emission is triggered based on a Trigger.
The windows are conceptually evaluated for each key individually, meaning windows can trigger at different points for each key.
Note that the WindowedStream is purely an API construct, during runtime the WindowedStream will be collapsed together with the KeyedStream and the operation over the window into one single operation.
-
aggregate
(aggregate_function: pyflink.datastream.functions.AggregateFunction, window_function: Union[pyflink.datastream.functions.WindowFunction, pyflink.datastream.functions.ProcessWindowFunction] = None, accumulator_type: pyflink.common.typeinfo.TypeInformation = None, output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Applies the given window function to each window. The window function is called for each evaluation of the window for each key individually. The output of the window function is interpreted as a regular non-windowed stream.
Arriving data is incrementally aggregated using the given aggregate function. This means that the window function typically has only a single value to process when called.
Example:
>>> class AverageAggregate(AggregateFunction): ... def create_accumulator(self) -> Tuple[int, int]: ... return 0, 0 ... ... def add(self, value: Tuple[str, int], accumulator: Tuple[int, int]) \ ... -> Tuple[int, int]: ... return accumulator[0] + value[1], accumulator[1] + 1 ... ... def get_result(self, accumulator: Tuple[int, int]) -> float: ... return accumulator[0] / accumulator[1] ... ... def merge(self, a: Tuple[int, int], b: Tuple[int, int]) -> Tuple[int, int]: ... return a[0] + b[0], a[1] + b[1] >>> ds.key_by(lambda x: x[1]) \ ... .window(TumblingEventTimeWindows.of(Time.seconds(5))) \ ... .aggregate(AverageAggregate(), ... accumulator_type=Types.TUPLE([Types.LONG(), Types.LONG()]), ... output_type=Types.DOUBLE())
- Parameters
aggregate_function – The aggregation function that is used for incremental aggregation.
window_function – The window function.
accumulator_type – Type information for the internal accumulator type of the aggregation function.
output_type – Type information for the result type of the window function.
- Returns
The data stream that is the result of applying the window function to the window.
New in version 1.16.0.
-
allowed_lateness
(time_ms: int) → pyflink.datastream.data_stream.WindowedStream[source]¶ Sets the time by which elements are allowed to be late. Elements that arrive behind the watermark by more than the specified time will be dropped. By default, the allowed lateness is 0.
Setting an allowed lateness is only valid for event-time windows.
-
apply
(window_function: pyflink.datastream.functions.WindowFunction, output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Applies the given window function to each window. The window function is called for each evaluation of the window for each key individually. The output of the window function is interpreted as a regular non-windowed stream.
Note that this function requires that all data in the windows is buffered until the window is evaluated, as the function provides no means of incremental aggregation.
- Parameters
window_function – The window function.
output_type – Type information for the result type of the window function.
- Returns
The data stream that is the result of applying the window function to the window.
-
process
(process_window_function: pyflink.datastream.functions.ProcessWindowFunction, output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Applies the given window function to each window. The window function is called for each evaluation of the window for each key individually. The output of the window function is interpreted as a regular non-windowed stream.
Note that this function requires that all data in the windows is buffered until the window is evaluated, as the function provides no means of incremental aggregation.
- Parameters
process_window_function – The window function.
output_type – Type information for the result type of the window function.
- Returns
The data stream that is the result of applying the window function to the window.
-
reduce
(reduce_function: Union[Callable, pyflink.datastream.functions.ReduceFunction], window_function: Union[pyflink.datastream.functions.WindowFunction, pyflink.datastream.functions.ProcessWindowFunction] = None, output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶ Applies a reduce function to the window. The window function is called for each evaluation of the window for each key individually. The output of the reduce function is interpreted as a regular non-windowed stream.
This window will try and incrementally aggregate data as much as the window policies permit. For example, tumbling time windows can aggregate the data, meaning that only one element per key is stored. Sliding time windows will aggregate on the granularity of the slide interval, so a few elements are stored per key (one per slide interval). Custom windows may not be able to incrementally aggregate, or may need to store extra values in an aggregation tree.
Example:
>>> ds.key_by(lambda x: x[1]) \ ... .window(TumblingEventTimeWindows.of(Time.seconds(5))) \ ... .reduce(lambda a, b: a[0] + b[0], b[1])
- Parameters
reduce_function – The reduce function.
window_function – The window function.
output_type – Type information for the result type of the window function.
- Returns
The data stream that is the result of applying the reduce function to the window.
New in version 1.16.0.
-
side_output_late_data
(output_tag: pyflink.datastream.output_tag.OutputTag) → pyflink.datastream.data_stream.WindowedStream[source]¶ Send late arriving data to the side output identified by the given
OutputTag
. Data is considered late after the watermark has passed the end of the window plus the allowed lateness set usingallowed_lateness()
.You can get the stream of late data using
get_side_output()
on theDataStream
resulting from the windowed operation with the sameOutputTag
.Example:
>>> tag = OutputTag("late-data", Types.TUPLE([Types.INT(), Types.STRING()])) >>> main_stream = ds.key_by(lambda x: x[1]) \ ... .window(TumblingEventTimeWindows.of(Time.seconds(5))) \ ... .side_output_late_data(tag) \ ... .reduce(lambda a, b: a[0] + b[0], b[1]) >>> late_stream = main_stream.get_side_output(tag)
New in version 1.16.0.
-
trigger
(trigger: pyflink.datastream.window.Trigger) → pyflink.datastream.data_stream.WindowedStream[source]¶ Sets the Trigger that should be used to trigger window emission.
-
pyflink.datastream.window module¶
-
class
pyflink.datastream.window.
ContinuousEventTimeTrigger
(*args, **kwds)[source]¶ A Trigger that continuously fires based on a given time interval. This fires based Watermarks.
-
can_merge
() → bool[source]¶ Note
If this returns true you must properly implement
on_merge()
- Returns
True if this trigger supports merging of trigger state and can therefore be used with a MergingWindowAssigner.
-
clear
(window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → None[source]¶ Clears any state that the trigger might still hold for the given window. This is called when a window is purged. Timers set using
register_event_time_timer()
andregister_processing_time_timer()
should be deleted here as well as state acquired usingget_partitioned_state()
.
-
static
of
(interval: pyflink.common.time.Time) → pyflink.datastream.window.ContinuousEventTimeTrigger[source]¶
-
on_element
(element: T, timestamp: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called for every element that gets added to a pane. The result of this will determine whether the pane is evaluated to emit results.
- Parameters
element – The element that arrived.
timestamp – The timestamp of the element that arrived.
window – The window to which the element is being added.
ctx – A context object that can be used to register timer callbacks.
-
on_event_time
(time: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when an event-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
on_merge
(window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.OnMergeContext) → None[source]¶ Called when several windows have been merged into one window by the
WindowAssigner
.- Parameters
window – The new window that results from the merge.
ctx – A context object that can be used to register timer callbacks and access state.
-
on_processing_time
(time: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when a processing-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
register_next_fire_timestamp
(time: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext, fire_timestamp_state: pyflink.datastream.state.ReducingState)[source]¶
-
-
class
pyflink.datastream.window.
ContinuousProcessingTimeTrigger
(*args, **kwds)[source]¶ A Trigger that continuously fires based on a given time interval as measured by the clock of the machine on which the job is running.
-
can_merge
() → bool[source]¶ Note
If this returns true you must properly implement
on_merge()
- Returns
True if this trigger supports merging of trigger state and can therefore be used with a MergingWindowAssigner.
-
clear
(window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → None[source]¶ Clears any state that the trigger might still hold for the given window. This is called when a window is purged. Timers set using
register_event_time_timer()
andregister_processing_time_timer()
should be deleted here as well as state acquired usingget_partitioned_state()
.
-
static
of
(interval: pyflink.common.time.Time) → pyflink.datastream.window.ContinuousProcessingTimeTrigger[source]¶
-
on_element
(element: T, timestamp: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called for every element that gets added to a pane. The result of this will determine whether the pane is evaluated to emit results.
- Parameters
element – The element that arrived.
timestamp – The timestamp of the element that arrived.
window – The window to which the element is being added.
ctx – A context object that can be used to register timer callbacks.
-
on_event_time
(time: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when an event-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
on_merge
(window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.OnMergeContext) → None[source]¶ Called when several windows have been merged into one window by the
WindowAssigner
.- Parameters
window – The new window that results from the merge.
ctx – A context object that can be used to register timer callbacks and access state.
-
on_processing_time
(time: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when a processing-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
register_next_fire_timestamp
(time: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext, fire_timestamp_state: pyflink.datastream.state.ReducingState)[source]¶
-
-
class
pyflink.datastream.window.
CountSlidingWindowAssigner
(*args, **kwds)[source]¶ A WindowAssigner that windows elements into sliding windows based on the count number of the elements. Windows can possibly overlap.
-
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[pyflink.datastream.window.CountWindow][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, pyflink.datastream.window.CountWindow][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
get_window_serializer
() → pyflink.common.serializer.TypeSerializer[pyflink.datastream.window.CountWindow][source]¶ - Returns
A TypeSerializer for serializing windows that are assigned by this WindowAssigner.
-
is_event_time
() → bool[source]¶ - Returns
True if elements are assigned to windows based on event time, false otherwise.
-
static
of
(window_size: int, window_slide: int) → pyflink.datastream.window.CountSlidingWindowAssigner[source]¶
-
-
class
pyflink.datastream.window.
CountTrigger
(*args, **kwds)[source]¶ A Trigger that fires once the count of elements in a pane reaches the given count.
-
can_merge
() → bool[source]¶ Note
If this returns true you must properly implement
on_merge()
- Returns
True if this trigger supports merging of trigger state and can therefore be used with a MergingWindowAssigner.
-
clear
(window: pyflink.datastream.window.CountWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → None[source]¶ Clears any state that the trigger might still hold for the given window. This is called when a window is purged. Timers set using
register_event_time_timer()
andregister_processing_time_timer()
should be deleted here as well as state acquired usingget_partitioned_state()
.
-
static
of
(window_size: int) → pyflink.datastream.window.CountTrigger[source]¶
-
on_element
(element: T, timestamp: int, window: pyflink.datastream.window.CountWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called for every element that gets added to a pane. The result of this will determine whether the pane is evaluated to emit results.
- Parameters
element – The element that arrived.
timestamp – The timestamp of the element that arrived.
window – The window to which the element is being added.
ctx – A context object that can be used to register timer callbacks.
-
on_event_time
(time: int, window: pyflink.datastream.window.CountWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when an event-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
on_merge
(window: pyflink.datastream.window.CountWindow, ctx: pyflink.datastream.window.Trigger.OnMergeContext) → None[source]¶ Called when several windows have been merged into one window by the
WindowAssigner
.- Parameters
window – The new window that results from the merge.
ctx – A context object that can be used to register timer callbacks and access state.
-
on_processing_time
(time: int, window: pyflink.datastream.window.CountWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when a processing-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
-
class
pyflink.datastream.window.
CountTumblingWindowAssigner
(*args, **kwds)[source]¶ A WindowAssigner that windows elements into fixed-size windows based on the count number of the elements. Windows cannot overlap.
-
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[pyflink.datastream.window.CountWindow][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, pyflink.datastream.window.CountWindow][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
get_window_serializer
() → pyflink.common.serializer.TypeSerializer[pyflink.datastream.window.CountWindow][source]¶ - Returns
A TypeSerializer for serializing windows that are assigned by this WindowAssigner.
-
is_event_time
() → bool[source]¶ - Returns
True if elements are assigned to windows based on event time, false otherwise.
-
static
of
(window_size: int) → pyflink.datastream.window.CountTumblingWindowAssigner[source]¶
-
-
class
pyflink.datastream.window.
CountWindow
(id: int)[source]¶ A Window that represents a count window. For each count window, we will assign a unique id. Thus this CountWindow can act as namespace part in state. We can attach data to each different CountWindow.
-
class
pyflink.datastream.window.
CountWindowSerializer
(*args, **kwds)[source]¶ -
deserialize
(stream: _io.BytesIO) → pyflink.datastream.window.CountWindow[source]¶ Returns a deserialized element from the input stream.
-
serialize
(element: pyflink.datastream.window.CountWindow, stream: _io.BytesIO) → None[source]¶ Serializes an element to the output stream.
-
-
class
pyflink.datastream.window.
DynamicEventTimeSessionWindows
(*args, **kwds)[source]¶ A
WindowAssigner
that windows elements into sessions based on the timestamp of the elements. Windows cannot overlap.For example, in order to window into windows with a dynamic time gap:
>>> data_stream.key_by(lambda x: x[0], key_type=Types.STRING()) \ ... .window(DynamicEventTimeSessionWindows.with_dynamic_gap(extractor))
-
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[pyflink.datastream.window.TimeWindow][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, pyflink.datastream.window.TimeWindow][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
get_window_serializer
() → pyflink.common.serializer.TypeSerializer[pyflink.datastream.window.TimeWindow][source]¶ - Returns
A TypeSerializer for serializing windows that are assigned by this WindowAssigner.
-
is_event_time
() → bool[source]¶ - Returns
True if elements are assigned to windows based on event time, false otherwise.
-
merge_windows
(windows: Iterable[pyflink.datastream.window.TimeWindow], callback: pyflink.datastream.window.MergingWindowAssigner.MergeCallback[pyflink.datastream.window.TimeWindow]) → None[source]¶ Determines which windows (if any) should be merged.
- Parameters
windows – The window candidates.
callback – A callback that can be invoked to signal which windows should be merged.
-
static
with_dynamic_gap
(extractor: pyflink.datastream.window.SessionWindowTimeGapExtractor) → pyflink.datastream.window.DynamicEventTimeSessionWindows[source]¶ Creates a new SessionWindows WindowAssigner that assigns elements to sessions based on the element timestamp.
- Parameters
extractor – The extractor to use to extract the time gap from the input elements.
- Returns
The policy.
-
-
class
pyflink.datastream.window.
DynamicProcessingTimeSessionWindows
(*args, **kwds)[source]¶ A WindowAssigner that windows elements into sessions based on the current processing time. Windows cannot overlap.
For example, in order to window into windows with a dynamic time gap:
>>> data_stream.key_by(lambda x: x[0], key_type=Types.STRING()) \ ... .window(DynamicProcessingTimeSessionWindows.with_dynamic_gap(extractor))
-
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[pyflink.datastream.window.TimeWindow][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, pyflink.datastream.window.TimeWindow][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
get_window_serializer
() → pyflink.common.serializer.TypeSerializer[pyflink.datastream.window.TimeWindow][source]¶ - Returns
A TypeSerializer for serializing windows that are assigned by this WindowAssigner.
-
is_event_time
() → bool[source]¶ - Returns
True if elements are assigned to windows based on event time, false otherwise.
-
merge_windows
(windows: Iterable[pyflink.datastream.window.TimeWindow], callback: pyflink.datastream.window.MergingWindowAssigner.MergeCallback[pyflink.datastream.window.TimeWindow]) → None[source]¶ Determines which windows (if any) should be merged.
- Parameters
windows – The window candidates.
callback – A callback that can be invoked to signal which windows should be merged.
-
static
with_dynamic_gap
(extractor: pyflink.datastream.window.SessionWindowTimeGapExtractor) → pyflink.datastream.window.DynamicProcessingTimeSessionWindows[source]¶ Creates a new SessionWindows WindowAssigner that assigns elements to sessions based on the element timestamp.
- Parameters
extractor – The extractor to use to extract the time gap from the input elements.
- Returns
The policy.
-
-
class
pyflink.datastream.window.
EventTimeSessionWindows
(*args, **kwds)[source]¶ A
WindowAssigner
that windows elements into sessions based on the timestamp of the elements. Windows cannot overlap.For example, Set the timestamp of the element to 1 minutes:
>>> data_stream.key_by(lambda x: x[0], key_type=Types.STRING()) \ ... .window(EventTimeSessionWindows.with_gap(Time.minutes(1)))
-
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[pyflink.datastream.window.TimeWindow][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, pyflink.datastream.window.TimeWindow][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
get_window_serializer
() → pyflink.common.serializer.TypeSerializer[pyflink.datastream.window.TimeWindow][source]¶ - Returns
A TypeSerializer for serializing windows that are assigned by this WindowAssigner.
-
is_event_time
() → bool[source]¶ - Returns
True if elements are assigned to windows based on event time, false otherwise.
-
merge_windows
(windows: Iterable[pyflink.datastream.window.TimeWindow], callback: pyflink.datastream.window.MergingWindowAssigner.MergeCallback[pyflink.datastream.window.TimeWindow]) → None[source]¶ Determines which windows (if any) should be merged.
- Parameters
windows – The window candidates.
callback – A callback that can be invoked to signal which windows should be merged.
-
static
with_dynamic_gap
(extractor: pyflink.datastream.window.SessionWindowTimeGapExtractor) → pyflink.datastream.window.DynamicEventTimeSessionWindows[source]¶ Creates a new SessionWindows WindowAssigner that assigns elements to sessions based on the element timestamp.
- Parameters
extractor – The extractor to use to extract the time gap from the input elements.
- Returns
The policy.
-
static
with_gap
(size: pyflink.common.time.Time) → pyflink.datastream.window.EventTimeSessionWindows[source]¶ Creates a new SessionWindows WindowAssigner that assigns elements to sessions based on the element timestamp.
- Parameters
size – The session timeout, i.e. the time gap between sessions.
- Returns
The policy.
-
-
class
pyflink.datastream.window.
EventTimeTrigger
(*args, **kwds)[source]¶ A Trigger that fires once the watermark passes the end of the window to which a pane belongs.
-
can_merge
() → bool[source]¶ Note
If this returns true you must properly implement
on_merge()
- Returns
True if this trigger supports merging of trigger state and can therefore be used with a MergingWindowAssigner.
-
clear
(window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → None[source]¶ Clears any state that the trigger might still hold for the given window. This is called when a window is purged. Timers set using
register_event_time_timer()
andregister_processing_time_timer()
should be deleted here as well as state acquired usingget_partitioned_state()
.
-
static
create
() → pyflink.datastream.window.EventTimeTrigger[source]¶
-
on_element
(element: T, timestamp: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called for every element that gets added to a pane. The result of this will determine whether the pane is evaluated to emit results.
- Parameters
element – The element that arrived.
timestamp – The timestamp of the element that arrived.
window – The window to which the element is being added.
ctx – A context object that can be used to register timer callbacks.
-
on_event_time
(time: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when an event-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
on_merge
(window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.OnMergeContext) → None[source]¶ Called when several windows have been merged into one window by the
WindowAssigner
.- Parameters
window – The new window that results from the merge.
ctx – A context object that can be used to register timer callbacks and access state.
-
on_processing_time
(time: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when a processing-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
-
class
pyflink.datastream.window.
GlobalWindow
[source]¶ The default window into which all data is placed GlobalWindows.
-
static
get
() → pyflink.datastream.window.GlobalWindow[source]¶
-
static
-
class
pyflink.datastream.window.
GlobalWindowSerializer
(*args, **kwds)[source]¶ A TypeSerializer for GlobalWindow.
-
deserialize
(stream: _io.BytesIO) → pyflink.datastream.window.GlobalWindow[source]¶ Returns a deserialized element from the input stream.
-
serialize
(element: pyflink.datastream.window.GlobalWindow, stream: _io.BytesIO) → None[source]¶ Serializes an element to the output stream.
-
-
class
pyflink.datastream.window.
GlobalWindows
(*args, **kwds)[source]¶ A WindowAssigner that assigns all elements to the same GlobalWindow.
-
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[pyflink.datastream.window.GlobalWindow][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
static
create
() → pyflink.datastream.window.GlobalWindows[source]¶ Creates a new GlobalWindows WindowAssigner that assigns all elements to the same GlobalWindow.
-
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, pyflink.datastream.window.GlobalWindow][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
get_window_serializer
() → pyflink.common.serializer.TypeSerializer[pyflink.datastream.window.GlobalWindow][source]¶ - Returns
A TypeSerializer for serializing windows that are assigned by this WindowAssigner.
-
-
class
pyflink.datastream.window.
MergingWindowAssigner
(*args, **kwds)[source]¶ A WindowAssigner that can merge windows.
-
class
MergeCallback
(*args, **kwds)[source]¶ Callback to be used in
merge_windows()
for specifying which windows should be merged.
-
abstract
merge_windows
(windows: Iterable[W], callback: pyflink.datastream.window.MergingWindowAssigner.MergeCallback[W]) → None[source]¶ Determines which windows (if any) should be merged.
- Parameters
windows – The window candidates.
callback – A callback that can be invoked to signal which windows should be merged.
-
class
-
class
pyflink.datastream.window.
NeverTrigger
(*args, **kwds)[source]¶ A trigger that never fires, as default Trigger for GlobalWindows.
-
clear
(window: pyflink.datastream.window.GlobalWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → None[source]¶ Clears any state that the trigger might still hold for the given window. This is called when a window is purged. Timers set using
register_event_time_timer()
andregister_processing_time_timer()
should be deleted here as well as state acquired usingget_partitioned_state()
.
-
on_element
(element: T, timestamp: int, window: pyflink.datastream.window.GlobalWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called for every element that gets added to a pane. The result of this will determine whether the pane is evaluated to emit results.
- Parameters
element – The element that arrived.
timestamp – The timestamp of the element that arrived.
window – The window to which the element is being added.
ctx – A context object that can be used to register timer callbacks.
-
on_event_time
(time: int, window: pyflink.datastream.window.GlobalWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when an event-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
on_merge
(window: pyflink.datastream.window.GlobalWindow, ctx: pyflink.datastream.window.Trigger.OnMergeContext) → None[source]¶ Called when several windows have been merged into one window by the
WindowAssigner
.- Parameters
window – The new window that results from the merge.
ctx – A context object that can be used to register timer callbacks and access state.
-
on_processing_time
(time: int, window: pyflink.datastream.window.GlobalWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when a processing-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
-
class
pyflink.datastream.window.
ProcessingTimeSessionWindows
(*args, **kwds)[source]¶ A WindowAssigner that windows elements into sessions based on the current processing time. Windows cannot overlap.
For example, the processing interval is set to 1 minutes:
>>> data_stream.key_by(lambda x: x[0], key_type=Types.STRING()) \ ... .window(ProcessingTimeSessionWindows.with_gap(Time.minutes(1)))
-
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[pyflink.datastream.window.TimeWindow][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, pyflink.datastream.window.TimeWindow][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
get_window_serializer
() → pyflink.common.serializer.TypeSerializer[pyflink.datastream.window.TimeWindow][source]¶ - Returns
A TypeSerializer for serializing windows that are assigned by this WindowAssigner.
-
is_event_time
() → bool[source]¶ - Returns
True if elements are assigned to windows based on event time, false otherwise.
-
merge_windows
(windows: Iterable[pyflink.datastream.window.TimeWindow], callback: pyflink.datastream.window.MergingWindowAssigner.MergeCallback[pyflink.datastream.window.TimeWindow]) → None[source]¶ Determines which windows (if any) should be merged.
- Parameters
windows – The window candidates.
callback – A callback that can be invoked to signal which windows should be merged.
-
static
with_dynamic_gap
(extractor: pyflink.datastream.window.SessionWindowTimeGapExtractor) → pyflink.datastream.window.DynamicProcessingTimeSessionWindows[source]¶ Creates a new SessionWindows WindowAssigner that assigns elements to sessions based on the element timestamp.
- Parameters
extractor – The extractor to use to extract the time gap from the input elements.
- Returns
The policy.
-
static
with_gap
(size: pyflink.common.time.Time) → pyflink.datastream.window.ProcessingTimeSessionWindows[source]¶ Creates a new SessionWindows WindowAssigner that assigns elements to sessions based on the element timestamp.
- Parameters
size – The session timeout, i.e. the time gap between sessions
- Returns
The policy.
-
-
class
pyflink.datastream.window.
ProcessingTimeTrigger
(*args, **kwds)[source]¶ A Trigger that fires once the current system time passes the end of the window to which a pane belongs.
-
can_merge
() → bool[source]¶ Note
If this returns true you must properly implement
on_merge()
- Returns
True if this trigger supports merging of trigger state and can therefore be used with a MergingWindowAssigner.
-
clear
(window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → None[source]¶ Clears any state that the trigger might still hold for the given window. This is called when a window is purged. Timers set using
register_event_time_timer()
andregister_processing_time_timer()
should be deleted here as well as state acquired usingget_partitioned_state()
.
-
static
create
() → pyflink.datastream.window.ProcessingTimeTrigger[source]¶
-
on_element
(element: T, timestamp: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called for every element that gets added to a pane. The result of this will determine whether the pane is evaluated to emit results.
- Parameters
element – The element that arrived.
timestamp – The timestamp of the element that arrived.
window – The window to which the element is being added.
ctx – A context object that can be used to register timer callbacks.
-
on_event_time
(time: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when an event-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
on_merge
(window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.OnMergeContext) → None[source]¶ Called when several windows have been merged into one window by the
WindowAssigner
.- Parameters
window – The new window that results from the merge.
ctx – A context object that can be used to register timer callbacks and access state.
-
on_processing_time
(time: int, window: pyflink.datastream.window.TimeWindow, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when a processing-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
-
class
pyflink.datastream.window.
PurgingTrigger
(*args, **kwds)[source]¶ A trigger that can turn any Trigger into a purging Trigger. When the nested trigger fires, this will return a FIRE_AND_PURGE TriggerResult.
-
can_merge
() → bool[source]¶ Note
If this returns true you must properly implement
on_merge()
- Returns
True if this trigger supports merging of trigger state and can therefore be used with a MergingWindowAssigner.
-
clear
(window: pyflink.datastream.window.Window, ctx: pyflink.datastream.window.Trigger.TriggerContext) → None[source]¶ Clears any state that the trigger might still hold for the given window. This is called when a window is purged. Timers set using
register_event_time_timer()
andregister_processing_time_timer()
should be deleted here as well as state acquired usingget_partitioned_state()
.
-
static
of
(nested_trigger: pyflink.datastream.window.Trigger[T, pyflink.datastream.window.Window]) → pyflink.datastream.window.PurgingTrigger[source]¶
-
on_element
(element: T, timestamp: int, window: pyflink.datastream.window.Window, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called for every element that gets added to a pane. The result of this will determine whether the pane is evaluated to emit results.
- Parameters
element – The element that arrived.
timestamp – The timestamp of the element that arrived.
window – The window to which the element is being added.
ctx – A context object that can be used to register timer callbacks.
-
on_event_time
(time: int, window: pyflink.datastream.window.Window, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when an event-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
on_merge
(window: pyflink.datastream.window.Window, ctx: pyflink.datastream.window.Trigger.OnMergeContext) → None[source]¶ Called when several windows have been merged into one window by the
WindowAssigner
.- Parameters
window – The new window that results from the merge.
ctx – A context object that can be used to register timer callbacks and access state.
-
on_processing_time
(time: int, window: pyflink.datastream.window.Window, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when a processing-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
-
class
pyflink.datastream.window.
SessionWindowTimeGapExtractor
[source]¶ A SessionWindowTimeGapExtractor extracts session time gaps for Dynamic Session Window Assigners.
-
class
pyflink.datastream.window.
SlidingEventTimeWindows
(*args, **kwds)[source]¶ A WindowAssigner that windows elements into sliding windows based on the timestamp of the elements. Windows can possibly overlap.
For example, in order to window into windows of 1 minute, every 10 seconds:
>>> data_stream.key_by(lambda x: x[0], key_type=Types.STRING()) \ ... .window(SlidingEventTimeWindows.of(Time.minutes(1), Time.seconds(10)))
-
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[pyflink.datastream.window.TimeWindow][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, pyflink.datastream.window.TimeWindow][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
get_window_serializer
() → pyflink.common.serializer.TypeSerializer[pyflink.datastream.window.TimeWindow][source]¶ - Returns
A TypeSerializer for serializing windows that are assigned by this WindowAssigner.
-
is_event_time
() → bool[source]¶ - Returns
True if elements are assigned to windows based on event time, false otherwise.
-
static
of
(size: pyflink.common.time.Time, slide: pyflink.common.time.Time, offset: pyflink.common.time.Time = None) → pyflink.datastream.window.SlidingEventTimeWindows[source]¶ Creates a new
SlidingEventTimeWindows
WindowAssigner
that assigns elements to time windows based on the element timestamp and offset.For example, if you want window a stream by hour,but window begins at the 15th minutes of each hour, you can use of(Time.hours(1),Time.minutes(15)),then you will get time windows start at 0:15:00,1:15:00,2:15:00,etc.
Rather than that, if you are living in somewhere which is not using UTC±00:00 time, such as China which is using UTC+08:00, and you want a time window with size of one day, and window begins at every 00:00:00 of local time,you may use of(Time.days(1),Time.hours(-8)). The parameter of offset is Time.hours(-8) since UTC+08:00 is 8 hours earlier than UTC time.
:param size The size of the generated windows. :param slide The slide interval of the generated windows. :param offset The offset which window start would be shifted by. :return The time policy.
-
-
class
pyflink.datastream.window.
SlidingProcessingTimeWindows
(*args, **kwds)[source]¶ A WindowAssigner that windows elements into sliding windows based on the current system time of the machine the operation is running on. Windows can possibly overlap.
For example, in order to window into windows of 1 minute, every 10 seconds:
>>> data_stream.key_by(lambda x: x[0], key_type=Types.STRING()) \ ... .window(SlidingProcessingTimeWindows.of(Time.minutes(1), Time.seconds(10)))
-
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[pyflink.datastream.window.TimeWindow][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, pyflink.datastream.window.TimeWindow][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
get_window_serializer
() → pyflink.common.serializer.TypeSerializer[pyflink.datastream.window.TimeWindow][source]¶ - Returns
A TypeSerializer for serializing windows that are assigned by this WindowAssigner.
-
is_event_time
() → bool[source]¶ - Returns
True if elements are assigned to windows based on event time, false otherwise.
-
static
of
(size: pyflink.common.time.Time, slide: pyflink.common.time.Time, offset: pyflink.common.time.Time = None) → pyflink.datastream.window.SlidingProcessingTimeWindows[source]¶ Creates a new
SlidingProcessingTimeWindows
WindowAssigner
that assigns elements to time windows based on the element timestamp and offset.For example, if you want window a stream by hour,but window begins at the 15th minutes of each hour, you can use of(Time.hours(1),Time.minutes(15)),then you will get time windows start at 0:15:00,1:15:00,2:15:00,etc.
Rather than that, if you are living in somewhere which is not using UTC±00:00 time, such as China which is using UTC+08:00, and you want a time window with size of one day, and window begins at every 00:00:00 of local time,you may use of(Time.days(1),Time.hours(-8)). The parameter of offset is Time.hours(-8) since UTC+08:00 is 8 hours earlier than UTC time.
:param size The size of the generated windows. :param slide The slide interval of the generated windows. :param offset The offset which window start would be shifted by. :return The time policy.
-
-
class
pyflink.datastream.window.
TimeWindow
(start: int, end: int)[source]¶ Window that represents a time interval from start (inclusive) to end (exclusive).
-
cover
(other: pyflink.datastream.window.TimeWindow) → pyflink.datastream.window.TimeWindow[source]¶ Returns the minimal window covers both this window and the given window.
-
static
get_window_start_with_offset
(timestamp: int, offset: int, window_size: int)[source]¶ Method to get the window start for a timestamp.
- Parameters
timestamp – epoch millisecond to get the window start.
offset – The offset which window start would be shifted by.
window_size – The size of the generated windows.
- Returns
window start
-
intersects
(other: pyflink.datastream.window.TimeWindow) → bool[source]¶ Returns True if this window intersects the given window.
-
static
merge_windows
(windows: Iterable[TimeWindow], callback: pyflink.datastream.window.MergingWindowAssigner.MergeCallback[pyflink.datastream.window.TimeWindow]) → None[source]¶ Merge overlapping :class`TimeWindow`.
-
-
class
pyflink.datastream.window.
TimeWindowSerializer
(*args, **kwds)[source]¶ The serializer used to write the TimeWindow type.
-
deserialize
(stream: _io.BytesIO) → pyflink.datastream.window.TimeWindow[source]¶ Returns a deserialized element from the input stream.
-
serialize
(element: pyflink.datastream.window.TimeWindow, stream: _io.BytesIO) → None[source]¶ Serializes an element to the output stream.
-
-
class
pyflink.datastream.window.
Trigger
(*args, **kwds)[source]¶ A Trigger determines when a pane of a window should be evaluated to emit the results for that part of the window.
A pane is the bucket of elements that have the same key (assigned by the KeySelector) and same Window. An element can be in multiple panes if it was assigned to multiple windows by the WindowAssigner. These panes all have their own instance of the Trigger.
Triggers must not maintain state internally since they can be re-created or reused for different keys. All necessary state should be persisted using the state abstraction available on the TriggerContext.
When used with a MergingWindowAssigner the Trigger must return true from
can_merge()
andon_merge()
most be properly implemented.-
class
OnMergeContext
[source]¶ Extension of
TriggerContext
that is given toon_merge()
.
-
class
TriggerContext
[source]¶ A context object that is given to
Trigger
methods to allow them to register timer callbacks and deal with state.-
abstract
delete_event_time_timer
(time: int) → None[source]¶ Delete the event-time trigger for the given time.
-
abstract
delete_processing_time_timer
(time: int) → None[source]¶ Delete the processing time trigger for the given time.
-
abstract
get_metric_group
() → pyflink.metrics.metricbase.MetricGroup[source]¶ Returns the metric group for this
Trigger
. This is the same metric group that would be returned fromget_metric_group()
in a user function.- Returns
The metric group.
-
abstract
get_partitioned_state
(state_descriptor: pyflink.datastream.state.StateDescriptor) → pyflink.datastream.state.State[source]¶ Retrieves a
State
object that can be used to interact with fault-tolerant state that is scoped to the window and key of the current trigger invocation.- Parameters
state_descriptor – The StateDescriptor that contains the name and type of the state that is being accessed.
- Returns
The partitioned state object.
-
abstract
register_event_time_timer
(time: int) → None[source]¶ Register an event-time callback. When the current watermark passes the specified time
on_event_time()
is called with the time specified here.- Parameters
time – The watermark at which to invoke
on_event_time()
.
-
abstract
register_processing_time_timer
(time: int) → None[source]¶ Register a system time callback. When the current system time passes the specified time
on_processing_time()
is called with the time specified here.- Parameters
time – The time at which to invoke
on_processing_time()
.
-
abstract
-
can_merge
() → bool[source]¶ Note
If this returns true you must properly implement
on_merge()
- Returns
True if this trigger supports merging of trigger state and can therefore be used with a MergingWindowAssigner.
-
abstract
clear
(window: W, ctx: pyflink.datastream.window.Trigger.TriggerContext) → None[source]¶ Clears any state that the trigger might still hold for the given window. This is called when a window is purged. Timers set using
register_event_time_timer()
andregister_processing_time_timer()
should be deleted here as well as state acquired usingget_partitioned_state()
.
-
abstract
on_element
(element: T, timestamp: int, window: W, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called for every element that gets added to a pane. The result of this will determine whether the pane is evaluated to emit results.
- Parameters
element – The element that arrived.
timestamp – The timestamp of the element that arrived.
window – The window to which the element is being added.
ctx – A context object that can be used to register timer callbacks.
-
abstract
on_event_time
(time: int, window: W, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when an event-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
abstract
on_merge
(window: W, ctx: pyflink.datastream.window.Trigger.OnMergeContext) → None[source]¶ Called when several windows have been merged into one window by the
WindowAssigner
.- Parameters
window – The new window that results from the merge.
ctx – A context object that can be used to register timer callbacks and access state.
-
abstract
on_processing_time
(time: int, window: W, ctx: pyflink.datastream.window.Trigger.TriggerContext) → pyflink.datastream.window.TriggerResult[source]¶ Called when a processing-time timer that was set using the trigger context fires.
- Parameters
time – The timestamp at which the timer fired.
window – The window for which the timer fired.
ctx – A context object that can be used to register timer callbacks.
-
class
-
class
pyflink.datastream.window.
TriggerResult
(value)[source]¶ Result type for trigger methods. This determines what happens with the window, for example whether the window function should be called, or the window should be discarded.
If a
Trigger
returns TriggerResult.FIRE or TriggerResult.FIRE_AND_PURGE but the window does not contain any data the window function will not be invoked, i.e. no data will be produced for the window.CONTINUE: No action is taken on the window.
FIRE_AND_PURGE: Evaluates the window function and emits the ‘window result’.
- FIRE: On FIRE, the window is evaluated and results are emitted. The window is not purged
though, all elements are retained.
- PURGE: All elements in the window are cleared and the window is discarded, without
evaluating the window function or emitting any elements.
-
CONTINUE
= (False, False)¶
-
FIRE
= (True, False)¶
-
FIRE_AND_PURGE
= (True, True)¶
-
PURGE
= (False, True)¶
-
class
pyflink.datastream.window.
TumblingEventTimeWindows
(*args, **kwds)[source]¶ A WindowAssigner that windows elements into windows based on the timestamp of the elements. Windows cannot overlap.
For example, in order to window into windows of 1 minute:
>>> data_stream.key_by(lambda x: x[0], key_type=Types.STRING()) \ ... .window(TumblingEventTimeWindows.of(Time.minutes(1)))
-
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[pyflink.datastream.window.TimeWindow][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, pyflink.datastream.window.TimeWindow][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
get_window_serializer
() → pyflink.common.serializer.TypeSerializer[pyflink.datastream.window.TimeWindow][source]¶ - Returns
A TypeSerializer for serializing windows that are assigned by this WindowAssigner.
-
is_event_time
() → bool[source]¶ - Returns
True if elements are assigned to windows based on event time, false otherwise.
-
static
of
(size: pyflink.common.time.Time, offset: pyflink.common.time.Time = None) → pyflink.datastream.window.TumblingEventTimeWindows[source]¶ Creates a new TumblingEventTimeWindows WindowAssigner that assigns elements to time windows based on the element timestamp, offset and a staggering offset, depending on the staggering policy.
:param size The size of the generated windows. :param offset The globalOffset which window start would be shifted by.
-
-
class
pyflink.datastream.window.
TumblingProcessingTimeWindows
(*args, **kwds)[source]¶ A WindowAssigner that windows elements into windows based on the current system time of the machine the operation is running on. Windows cannot overlap.
For example, in order to window into windows of 1 minute, every 10 seconds:
>>> data_stream.key_by(lambda x: x[0], key_type=Types.STRING()) \ ... .window(TumblingProcessingTimeWindows.of(Time.minutes(1), Time.seconds(10)))
-
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[pyflink.datastream.window.TimeWindow][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, pyflink.datastream.window.TimeWindow][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
get_window_serializer
() → pyflink.common.serializer.TypeSerializer[pyflink.datastream.window.TimeWindow][source]¶ - Returns
A TypeSerializer for serializing windows that are assigned by this WindowAssigner.
-
is_event_time
() → bool[source]¶ - Returns
True if elements are assigned to windows based on event time, false otherwise.
-
static
of
(size: pyflink.common.time.Time, offset: pyflink.common.time.Time = None) → pyflink.datastream.window.TumblingProcessingTimeWindows[source]¶ Creates a new
TumblingProcessingTimeWindows
WindowAssigner
that assigns elements to time windows based on the element timestamp and offset.For example, if you want window a stream by hour, but window begins at the 15th minutes of each hour, you can use of(Time.hours(1), Time.minutes(15)), then you will get time windows start at 0:15:00,1:15:00,2:15:00,etc.
Rather than that, if you are living in somewhere which is not using UTC±00:00 time, such as China which is using UTC+08:00, and you want a time window with size of one day, and window begins at every 00:00:00 of local time, you may use of(Time.days(1), Time.hours(-8)). The parameter of offset is Time.hours(-8) since UTC+08:00 is 8 hours earlier than UTC time.
:param size The size of the generated windows. :param offset The offset which window start would be shifted by. :return The time policy.
-
-
class
pyflink.datastream.window.
Window
[source]¶ Window is a grouping of elements into finite buckets. Windows have a maximum timestamp which means that, at some point, all elements that go into one window will have arrived.
-
class
pyflink.datastream.window.
WindowAssigner
(*args, **kwds)[source]¶ A
WindowAssigner
assigns zero or moreWindow
to an element.In a window operation, elements are grouped by their key (if available) and by the windows to which it was assigned. The set of elements with the same key and window is called a pane. When a
Trigger
decides that a certain pane should fire the WindowFunction is applied to produce output elements for that pane.-
class
WindowAssignerContext
[source]¶ A context provided to the
WindowAssigner
that allows it to query the current processing time.
-
abstract
assign_windows
(element: T, timestamp: int, context: pyflink.datastream.window.WindowAssigner.WindowAssignerContext) → Collection[W][source]¶ - Parameters
element – The element to which windows should be assigned.
timestamp – The timestamp of the element.
context – The
WindowAssignerContext
in which the assigner operates.
- Returns
A collection of windows that should be assigned to the element.
-
abstract
get_default_trigger
(env) → pyflink.datastream.window.Trigger[T, W][source]¶ - Parameters
env – The StreamExecutionEnvironment used to compile the DataStream job.
- Returns
The default trigger associated with this
WindowAssigner
.
-
class
pyflink.datastream.state module¶
-
class
pyflink.datastream.state.
AggregatingState
(*args, **kwds)[source]¶ State
interface for aggregating state, based on anAggregateFunction
. Elements that are added to this type of state will be eagerly pre-aggregated using a given AggregateFunction.The state holds internally always the accumulator type of the AggregateFunction. When accessing the result of the state, the function’s
get_result()
method.The state is accessed and modified by user functions, and checkpointed consistently by the system as part of the distributed snapshots.
The state is only accessible by functions applied on a KeyedStream. The key is automatically supplied by the system, so the function always sees the value mapped to the key of the current element. That way, the system can handle stream and state partitioning consistently together.
-
abstract
add
(value: IN) → None¶ Adding the given value to the tail of this list state.
-
abstract
clear
() → None¶ Removes the value mapped under the current key.
-
abstract
get
() → OUT¶ Returns the elements under the current key.
-
abstract
-
class
pyflink.datastream.state.
AggregatingStateDescriptor
(name: str, agg_function, state_type_info)[source]¶ A StateDescriptor for AggregatingState.
The type internally stored in the state is the type of the Accumulator of the
AggregateFunction()
.-
enable_time_to_live
(ttl_config: pyflink.datastream.state.StateTtlConfig)¶ Configures optional activation of state time-to-live (TTL).
State user value will expire, become unavailable and be cleaned up in storage depending on configured StateTtlConfig.
- Parameters
ttl_config – Configuration of state TTL
-
get_name
() → str¶ Get the name of the state.
- Returns
The name of the state.
-
-
class
pyflink.datastream.state.
BroadcastState
(*args, **kwds)[source]¶ A type of state that can be created to store the state of a
BroadcastStream
. This state assumes that the same elements are sent to all instances of an operator. CAUTION: the user has to guarantee that all task instances store the same elements in this type of state. Each operator instance individually maintains and stores elements in the broadcast state. The fact that the incoming stream is a broadcast one guarantees that all instances see all the elements. Upon recovery or re-scaling, the same state is given to each of the instances. To avoid hotspots, each task reads its previous partition, and if there are more tasks (scale up ), then the new instances read from the old instances in a round-robin fashion. This is why each instance has to guarantee that it stores the same elements as the rest. If not, upon recovery or rescaling you may have unpredictable redistribution of the partitions, thus unpredictable results.-
abstract
clear
() → None¶ Removes the value mapped under the current key.
-
abstract
contains
(key: K) → bool¶ Returns whether there exists the given mapping.
-
abstract
get
(key: K) → V¶ Returns the current value associated with the given key.
-
abstract
is_empty
() → bool¶ Returns true if this state contains no key-value mappings, otherwise false.
-
abstract
items
() → Iterable[Tuple[K, V]]¶ Returns all the mappings in the state.
-
abstract
keys
() → Iterable[K]¶ Returns all the keys in the state.
-
abstract
put_all
(dict_value: Dict[K, V]) → None[source]¶ Copies all of the mappings from the given map into the state.
-
abstract
values
() → Iterable[V]¶ Returns all the values in the state.
-
abstract
-
class
pyflink.datastream.state.
ListState
(*args, **kwds)[source]¶ State
interface for partitioned list state in Operations. The state is accessed and modified by user functions, and checkpointed consistently by the system as part of the distributed snapshots.Currently only keyed list state is supported.
When it is a keyed list state, the state key is automatically supplied by the system, so the user function always sees the value mapped to the key of the current element. That way, the system can handle stream and state partitioning consistently together.
-
abstract
add
(value: IN) → None¶ Adding the given value to the tail of this list state.
-
abstract
add_all
(values: List[T]) → None[source]¶ Adding the given values to the tail of this list state.
-
abstract
clear
() → None¶ Removes the value mapped under the current key.
-
abstract
get
() → OUT¶ Returns the elements under the current key.
-
abstract
-
class
pyflink.datastream.state.
ListStateDescriptor
(name: str, elem_type_info: pyflink.common.typeinfo.TypeInformation)[source]¶ StateDescriptor for ListState. This can be used to create state where the type is a list that can be appended and iterated over.
-
enable_time_to_live
(ttl_config: pyflink.datastream.state.StateTtlConfig)¶ Configures optional activation of state time-to-live (TTL).
State user value will expire, become unavailable and be cleaned up in storage depending on configured StateTtlConfig.
- Parameters
ttl_config – Configuration of state TTL
-
get_name
() → str¶ Get the name of the state.
- Returns
The name of the state.
-
-
class
pyflink.datastream.state.
MapState
(*args, **kwds)[source]¶ State
interface for partitioned key-value state. The key-value pair can be added, updated and retrieved. The state is accessed and modified by user functions, and checkpointed consistently by the system as part of the distributed snapshots.The state key is automatically supplied by the system, so the function always sees the value mapped to the key of the current element. That way, the system can handle stream and state partitioning consistently together.
-
abstract
clear
() → None¶ Removes the value mapped under the current key.
-
abstract
is_empty
() → bool[source]¶ Returns true if this state contains no key-value mappings, otherwise false.
-
abstract
-
class
pyflink.datastream.state.
MapStateDescriptor
(name: str, key_type_info: pyflink.common.typeinfo.TypeInformation, value_type_info: pyflink.common.typeinfo.TypeInformation)[source]¶ StateDescriptor for MapState. This can be used to create state where the type is a map that can be updated and iterated over.
-
enable_time_to_live
(ttl_config: pyflink.datastream.state.StateTtlConfig)¶ Configures optional activation of state time-to-live (TTL).
State user value will expire, become unavailable and be cleaned up in storage depending on configured StateTtlConfig.
- Parameters
ttl_config – Configuration of state TTL
-
get_name
() → str¶ Get the name of the state.
- Returns
The name of the state.
-
-
class
pyflink.datastream.state.
OperatorStateStore
[source]¶ Interface for getting operator states. Currently, only
BroadcastState
is supported. .. versionadded:: 1.16.0-
abstract
get_broadcast_state
(state_descriptor: pyflink.datastream.state.MapStateDescriptor) → pyflink.datastream.state.BroadcastState[source]¶ Fetches the
BroadcastState
described byMapStateDescriptor
, which has read/write access to the broadcast operator state.
-
abstract
-
class
pyflink.datastream.state.
ReadOnlyBroadcastState
(*args, **kwds)[source]¶ A read-only view of the
BroadcastState
. Although read-only, the user code should not modify the value returned by theget()
or the items returned byitems()
, as this can lead to inconsistent states. The reason for this is that we do not create extra copies of the elements for performance reasons.-
abstract
clear
() → None¶ Removes the value mapped under the current key.
-
abstract
-
class
pyflink.datastream.state.
ReducingState
(*args, **kwds)[source]¶ State
interface for reducing state. Elements can be added to the state, they will be combined using a reduce function. The current state can be inspected.The state is accessed and modified by user functions, and checkpointed consistently by the system as part of the distributed snapshots.
The state is only accessible by functions applied on a KeyedStream. The key is automatically supplied by the system, so the function always sees the value mapped to the key of the current element. That way, the system can handle stream and state partitioning consistently together.
-
abstract
add
(value: IN) → None¶ Adding the given value to the tail of this list state.
-
abstract
clear
() → None¶ Removes the value mapped under the current key.
-
abstract
get
() → OUT¶ Returns the elements under the current key.
-
abstract
-
class
pyflink.datastream.state.
ReducingStateDescriptor
(name: str, reduce_function, type_info: pyflink.common.typeinfo.TypeInformation)[source]¶ StateDescriptor for ReducingState. This can be used to create partitioned reducing state using RuntimeContext.get_reducing_state(ReducingStateDescriptor).
-
enable_time_to_live
(ttl_config: pyflink.datastream.state.StateTtlConfig)¶ Configures optional activation of state time-to-live (TTL).
State user value will expire, become unavailable and be cleaned up in storage depending on configured StateTtlConfig.
- Parameters
ttl_config – Configuration of state TTL
-
get_name
() → str¶ Get the name of the state.
- Returns
The name of the state.
-
-
class
pyflink.datastream.state.
StateTtlConfig
(update_type: pyflink.datastream.state.StateTtlConfig.UpdateType, state_visibility: pyflink.datastream.state.StateTtlConfig.StateVisibility, ttl_time_characteristic: pyflink.datastream.state.StateTtlConfig.TtlTimeCharacteristic, ttl: pyflink.common.time.Time, cleanup_strategies: pyflink.datastream.state.StateTtlConfig.CleanupStrategies)[source]¶ -
class
Builder
(ttl: pyflink.common.time.Time)[source]¶ Builder for the StateTtlConfig.
-
cleanup_full_snapshot
() → pyflink.datastream.state.StateTtlConfig.Builder[source]¶ Cleanup expired state in full snapshot on checkpoint.
-
cleanup_in_rocksdb_compact_filter
(query_time_after_num_entries) → pyflink.datastream.state.StateTtlConfig.Builder[source]¶ Cleanup expired state while Rocksdb compaction is running.
RocksDB compaction filter will query current timestamp, used to check expiration, from Flink every time after processing {@code queryTimeAfterNumEntries} number of state entries. Updating the timestamp more often can improve cleanup speed but it decreases compaction performance because it uses JNI call from native code.
- Parameters
query_time_after_num_entries – number of state entries to process by compaction filter before updating current timestamp
- Returns
-
cleanup_incrementally
(cleanup_size: int, run_cleanup_for_every_record) → pyflink.datastream.state.StateTtlConfig.Builder[source]¶ Cleanup expired state incrementally cleanup local state.
Upon every state access this cleanup strategy checks a bunch of state keys for expiration and cleans up expired ones. It keeps a lazy iterator through all keys with relaxed consistency if backend supports it. This way all keys should be regularly checked and cleaned eventually over time if any state is constantly being accessed.
Additionally to the incremental cleanup upon state access, it can also run per every record. Caution: if there are a lot of registered states using this option, they all will be iterated for every record to check if there is something to cleanup.
if no access happens to this state or no records are processed in case of run_cleanup_for_every_record, expired state will persist.
Time spent for the incremental cleanup increases record processing latency.
Note:
At the moment incremental cleanup is implemented only for Heap state backend. Setting it for RocksDB will have no effect.
Note:
If heap state backend is used with synchronous snapshotting, the global iterator keeps a copy of all keys while iterating because of its specific implementation which does not support concurrent modifications. Enabling of this feature will increase memory consumption then. Asynchronous snapshotting does not have this problem.
- Parameters
cleanup_size – max number of keys pulled from queue for clean up upon state touch for any key
run_cleanup_for_every_record – run incremental cleanup per each processed record
-
disable_cleanup_in_background
() → pyflink.datastream.state.StateTtlConfig.Builder[source]¶ Disable default cleanup of expired state in background (enabled by default).
If some specific cleanup is configured, e.g.
cleanup_incrementally()
orcleanup_in_rocksdb_compact_filter()
, this setting does not disable it.
-
never_return_expired
() → pyflink.datastream.state.StateTtlConfig.Builder[source]¶
-
return_expired_if_not_cleaned_up
() → pyflink.datastream.state.StateTtlConfig.Builder[source]¶
-
set_state_visibility
(state_visibility: pyflink.datastream.state.StateTtlConfig.StateVisibility) → pyflink.datastream.state.StateTtlConfig.Builder[source]¶ Sets the state visibility.
- Parameters
state_visibility – The state visibility configures whether expired user value can be returned or not.
-
set_ttl
(ttl: pyflink.common.time.Time) → pyflink.datastream.state.StateTtlConfig.Builder[source]¶ Sets the ttl time.
- Parameters
ttl – The ttl time.
-
set_ttl_time_characteristic
(ttl_time_characteristic: pyflink.datastream.state.StateTtlConfig.TtlTimeCharacteristic) → pyflink.datastream.state.StateTtlConfig.Builder[source]¶ Sets the time characteristic.
- Parameters
ttl_time_characteristic – The time characteristic configures time scale to use for ttl.
-
set_update_type
(update_type: pyflink.datastream.state.StateTtlConfig.UpdateType) → pyflink.datastream.state.StateTtlConfig.Builder[source]¶ Sets the ttl update type.
- Parameters
update_type – The ttl update type configures when to update last access timestamp which prolongs state TTL.
-
update_ttl_on_create_and_write
() → pyflink.datastream.state.StateTtlConfig.Builder[source]¶
-
update_ttl_on_read_and_write
() → pyflink.datastream.state.StateTtlConfig.Builder[source]¶
-
use_processing_time
() → pyflink.datastream.state.StateTtlConfig.Builder[source]¶
-
-
class
CleanupStrategies
(strategies: Dict[pyflink.datastream.state.StateTtlConfig.CleanupStrategies.Strategies, pyflink.datastream.state.StateTtlConfig.CleanupStrategies.CleanupStrategy], is_cleanup_in_background: bool)[source]¶ TTL cleanup strategies.
This class configures when to cleanup expired state with TTL. By default, state is always cleaned up on explicit read access if found expired. Currently cleanup of state full snapshot can be additionally activated.
-
EMPTY_STRATEGY
= <pyflink.datastream.state.StateTtlConfig.CleanupStrategies.EmptyCleanupStrategy object>¶
-
class
IncrementalCleanupStrategy
(cleanup_size: int, run_cleanup_for_every_record: bool)[source]¶ Configuration of cleanup strategy while taking the full snapshot.
-
class
RocksdbCompactFilterCleanupStrategy
(query_time_after_num_entries: int)[source]¶ Configuration of cleanup strategy using custom compaction filter in RocksDB.
-
class
Strategies
(value)[source]¶ Fixed strategies ordinals in strategies config field.
-
FULL_STATE_SCAN_SNAPSHOT
= 0¶
-
INCREMENTAL_CLEANUP
= 1¶
-
ROCKSDB_COMPACTION_FILTER
= 2¶
-
-
get_incremental_cleanup_strategy
() → pyflink.datastream.state.StateTtlConfig.CleanupStrategies.IncrementalCleanupStrategy[source]¶
-
get_rocksdb_compact_filter_cleanup_strategy
() → pyflink.datastream.state.StateTtlConfig.CleanupStrategies.RocksdbCompactFilterCleanupStrategy[source]¶
-
-
class
StateVisibility
(value)[source]¶ This option configures whether expired user value can be returned or not.
-
NeverReturnExpired
= 1¶ Never return expired user value.
-
ReturnExpiredIfNotCleanedUp
= 0¶ Return expired user value if it is not cleaned up yet.
-
-
class
TtlTimeCharacteristic
(value)[source]¶ This option configures time scale to use for ttl.
-
ProcessingTime
= 0¶ Processing time
-
-
class
UpdateType
(value)[source]¶ This option value configures when to update last access timestamp which prolongs state TTL.
-
Disabled
= 0¶ TTL is disabled. State does not expire.
-
OnCreateAndWrite
= 1¶ Last access timestamp is initialised when state is created and updated on every write operation.
-
OnReadAndWrite
= 2¶ The same as OnCreateAndWrite but also updated on read.
-
-
get_cleanup_strategies
() → pyflink.datastream.state.StateTtlConfig.CleanupStrategies[source]¶
-
get_state_visibility
() → pyflink.datastream.state.StateTtlConfig.StateVisibility[source]¶
-
get_ttl_time_characteristic
() → pyflink.datastream.state.StateTtlConfig.TtlTimeCharacteristic[source]¶
-
get_update_type
() → pyflink.datastream.state.StateTtlConfig.UpdateType[source]¶
-
class
-
class
pyflink.datastream.state.
ValueState
(*args, **kwds)[source]¶ State
interface for partitioned single-value state. The value can be retrieved or updated.The state is accessed and modified by user functions, and checkpointed consistently by the system as part of the distributed snapshots.
-
abstract
clear
() → None¶ Removes the value mapped under the current key.
-
abstract
update
(value: T) → None[source]¶ Updates the operator state accessible by
value()
to the given value. The next timevalue()
is called (for the same state partition) the returned state will represent the updated value. When a partitioned state is updated with null, the state for the current key will be removed and the default value is returned on the next access.
-
abstract
value
() → T[source]¶ Returns the current value for the state. When the state is not partitioned the returned value is the same for all inputs in a given operator instance. If state partitioning is applied, the value returned depends on the current operator input, as the operator maintains an independent state for each partition.
-
abstract
-
class
pyflink.datastream.state.
ValueStateDescriptor
(name: str, value_type_info: pyflink.common.typeinfo.TypeInformation)[source]¶ StateDescriptor for ValueState. This can be used to create partitioned value state using RuntimeContext.get_state(ValueStateDescriptor).
-
enable_time_to_live
(ttl_config: pyflink.datastream.state.StateTtlConfig)¶ Configures optional activation of state time-to-live (TTL).
State user value will expire, become unavailable and be cleaned up in storage depending on configured StateTtlConfig.
- Parameters
ttl_config – Configuration of state TTL
-
get_name
() → str¶ Get the name of the state.
- Returns
The name of the state.
-
pyflink.datastream.connectors module¶
-
class
pyflink.datastream.connectors.
DeliveryGuarantee
(value)[source]¶ DeliverGuarantees that can be chosen. In general your pipeline can only offer the lowest delivery guarantee which is supported by your sources and sinks.
- Data
EXACTLY_ONCE:
Records are only delivered exactly-once also under failover scenarios. To build a complete exactly-once pipeline is required that the source and sink support exactly-once and are properly configured.
- Data
AT_LEAST_ONCE:
Records are ensured to be delivered but it may happen that the same record is delivered multiple times. Usually, this guarantee is faster than the exactly-once delivery.
- Data
NONE:
Records are delivered on a best effort basis. It is often the fastest way to process records but it may happen that records are lost or duplicated.
-
AT_LEAST_ONCE
= (1,)¶
-
EXACTLY_ONCE
= (0,)¶
-
NONE
= 2¶
pyflink.datastream.connectors.kafka module¶
-
class
pyflink.datastream.connectors.kafka.
FlinkKafkaConsumer
(topics: Union[str, List[str]], deserialization_schema: pyflink.common.serialization.DeserializationSchema, properties: Dict)[source]¶ The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from Apache Kafka. The consumer can run in multiple parallel instances, each of which will pull data from one or more Kafka partitions.
The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost during a failure, and that the computation processes elements ‘exactly once. (These guarantees naturally assume that Kafka itself does not lose any data.)
Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets committed to Kafka / Zookeeper are only to bring the outside view of progress in sync with Flink’s view of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer has consumed a topic.
Please refer to Kafka’s documentation for the available configuration properties: http://kafka.apache.org/documentation.html#newconsumerconfigs
-
class
pyflink.datastream.connectors.kafka.
FlinkKafkaProducer
(topic: str, serialization_schema: pyflink.common.serialization.SerializationSchema, producer_config: Dict, kafka_producer_pool_size: int = 5, semantic=<Semantic.AT_LEAST_ONCE: (1, )>)[source]¶ Flink Sink to produce data into a Kafka topic. By default producer will use AT_LEAST_ONCE semantic. Before using EXACTLY_ONCE please refer to Flink’s Kafka connector documentation.
-
ignore_failures_after_transaction_timeout
() → pyflink.datastream.connectors.kafka.FlinkKafkaProducer[source]¶ Disables the propagation of exceptions thrown when committing presumably timed out Kafka transactions during recovery of the job. If a Kafka transaction is timed out, a commit will never be successful. Hence, use this feature to avoid recovery loops of the Job. Exceptions will still be logged to inform the user that data loss might have occurred.
Note that we use the System.currentTimeMillis() to track the age of a transaction. Moreover, only exceptions thrown during the recovery are caught, i.e., the producer will attempt at least one commit of the transaction before giving up.
- Returns
This FlinkKafkaProducer.
-
-
class
pyflink.datastream.connectors.kafka.
KafkaOffsetResetStrategy
(value)[source]¶ Corresponding to Java
org.apache.kafka.client.consumer.OffsetResetStrategy
class.New in version 1.16.0.
-
EARLIEST
= 1¶
-
LATEST
= 0¶
-
NONE
= 2¶
-
-
class
pyflink.datastream.connectors.kafka.
KafkaOffsetsInitializer
(j_initializer: py4j.java_gateway.JavaObject)[source]¶ An interface for users to specify the starting / stopping offset of a KafkaPartitionSplit.
New in version 1.16.0.
-
static
committed_offsets
(offset_reset_strategy: pyflink.datastream.connectors.kafka.KafkaOffsetResetStrategy = <KafkaOffsetResetStrategy.NONE: 2>) → pyflink.datastream.connectors.kafka.KafkaOffsetsInitializer[source]¶ Get an
KafkaOffsetsInitializer
which initializes the offsets to the committed offsets. An exception will be thrown at runtime if there is no committed offsets.An optional
KafkaOffsetResetStrategy
can be specified to initialize the offsets if the committed offsets does not exist.- Parameters
offset_reset_strategy – the offset reset strategy to use when the committed offsets do not exist.
- Returns
an offset initializer which initialize the offsets to the committed offsets.
-
static
earliest
() → pyflink.datastream.connectors.kafka.KafkaOffsetsInitializer[source]¶ Get an
KafkaOffsetsInitializer
which initializes the offsets to the earliest available offsets of each partition.- Returns
an
KafkaOffsetsInitializer
which initializes the offsets to the earliest available offsets.
-
static
latest
() → pyflink.datastream.connectors.kafka.KafkaOffsetsInitializer[source]¶ Get an
KafkaOffsetsInitializer
which initializes the offsets to the latest offsets of each partition.- Returns
an
KafkaOffsetsInitializer
which initializes the offsets to the latest offsets.
-
static
offsets
(offsets: Dict[KafkaTopicPartition, int], offset_reset_strategy: pyflink.datastream.connectors.kafka.KafkaOffsetResetStrategy = <KafkaOffsetResetStrategy.EARLIEST: 1>) → pyflink.datastream.connectors.kafka.KafkaOffsetsInitializer[source]¶ Get an
KafkaOffsetsInitializer
which initializes the offsets to the specified offsets.An optional
KafkaOffsetResetStrategy
can be specified to initialize the offsets in case the specified offset is out of range.Example:
>>> KafkaOffsetsInitializer.offsets({ ... KafkaTopicPartition('TOPIC1', 0): 0, ... KafkaTopicPartition('TOPIC1', 1): 10000 ... }, KafkaOffsetResetStrategy.EARLIEST)
- Parameters
offsets – the specified offsets for each partition.
offset_reset_strategy – the
KafkaOffsetResetStrategy
to use when the specified offset is out of range.
- Returns
an
KafkaOffsetsInitializer
which initializes the offsets to the specified offsets.
-
static
timestamp
(timestamp: int) → pyflink.datastream.connectors.kafka.KafkaOffsetsInitializer[source]¶ Get an
KafkaOffsetsInitializer
which initializes the offsets in each partition so that the initialized offset is the offset of the first record whose record timestamp is greater than or equals the give timestamp.- Parameters
timestamp – the timestamp to start the consumption.
- Returns
an
OffsetsInitializer
which initializes the offsets based on the given timestamp.
-
static
-
class
pyflink.datastream.connectors.kafka.
KafkaRecordSerializationSchema
(j_serialization_schema, topic_selector: Optional[KafkaTopicSelector] = None)[source]¶ A serialization schema which defines how to convert the stream record to kafka producer record.
New in version 1.16.0.
-
static
builder
() → pyflink.datastream.connectors.kafka.KafkaRecordSerializationSchemaBuilder[source]¶ Creates a default schema builder to provide common building blocks i.e. key serialization, value serialization, topic selection.
-
static
-
class
pyflink.datastream.connectors.kafka.
KafkaRecordSerializationSchemaBuilder
[source]¶ Builder to construct
KafkaRecordSerializationSchema
.Example:
>>> KafkaRecordSerializationSchema.builder() \ ... .set_topic('topic') \ ... .set_key_serialization_schema(SimpleStringSchema()) \ ... .set_value_serialization_schema(SimpleStringSchema()) \ ... .build()
And the sink topic can be calculated dynamically from each record:
>>> KafkaRecordSerializationSchema.builder() \ ... .set_topic_selector(lambda row: 'topic-' + row['category']) \ ... .set_value_serialization_schema( ... JsonRowSerializationSchema.builder().with_type_info(ROW_TYPE).build()) \ ... .build()
It is necessary to configure exactly one serialization method for the value and a topic.
New in version 1.16.0.
-
build
() → pyflink.datastream.connectors.kafka.KafkaRecordSerializationSchema[source]¶ Constructs the
KafkaRecordSerializationSchemaBuilder
with the configured properties.
-
set_key_serialization_schema
(key_serialization_schema: pyflink.common.serialization.SerializationSchema) → pyflink.datastream.connectors.kafka.KafkaRecordSerializationSchemaBuilder[source]¶ Sets a
SerializationSchema
which is used to serialize the incoming element to the key of the producer record. The key serialization is optional, if not set, the key of the producer record will be null.- Parameters
key_serialization_schema – The
SerializationSchema
to serialize each incoming record as the key of producer record.
-
set_topic
(topic: str) → pyflink.datastream.connectors.kafka.KafkaRecordSerializationSchemaBuilder[source]¶ Sets a fixed topic which used as destination for all records.
- Parameters
topic – The fixed topic.
-
set_topic_selector
(topic_selector: Union[Callable[[Any], str], KafkaTopicSelector]) → pyflink.datastream.connectors.kafka.KafkaRecordSerializationSchemaBuilder[source]¶ Sets a topic selector which computes the target topic for every incoming record.
- Parameters
topic_selector – A
KafkaTopicSelector
implementation or a function that consumes each incoming record and return the topic string.
-
set_value_serialization_schema
(value_serialization_schema: pyflink.common.serialization.SerializationSchema) → pyflink.datastream.connectors.kafka.KafkaRecordSerializationSchemaBuilder[source]¶ Sets a
SerializationSchema
which is used to serialize the incoming element to the value of the producer record. The value serialization is required.- Parameters
value_serialization_schema – The
SerializationSchema
to serialize each data record as the key of producer record.
-
-
class
pyflink.datastream.connectors.kafka.
KafkaSink
(j_kafka_sink, transformer: Optional[pyflink.datastream.connectors.base.StreamTransformer] = None)[source]¶ Flink Sink to produce data into a Kafka topic. The sink supports all delivery guarantees described by
DeliveryGuarantee
.DeliveryGuarantee.NONE
does not provide any guarantees: messages may be lost in case of issues on the Kafka broker and messages may be duplicated in case of a Flink failure.DeliveryGuarantee.AT_LEAST_ONCE
the sink will wait for all outstanding records in the Kafka buffers to be acknowledged by the Kafka producer on a checkpoint. No messages will be lost in case of any issue with the Kafka brokers but messages may be duplicated when Flink restarts.DeliveryGuarantee.EXACTLY_ONCE
: In this mode the KafkaSink will write all messages in a Kafka transaction that will be committed to Kafka on a checkpoint. Thus, if the consumer reads only committed data (see Kafka consumer configisolation.level
), no duplicates will be seen in case of a Flink restart. However, this delays record writing effectively until a checkpoint is written, so adjust the checkpoint duration accordingly. Please ensure that you use unique transactional id prefixes across your applications running on the same Kafka cluster such that multiple running jobs do not interfere in their transactions! Additionally, it is highly recommended to tweak Kafka transaction timeout (link) >> maximum checkpoint duration + maximum restart duration or data loss may happen when Kafka expires an uncommitted transaction.
New in version 1.16.0.
-
static
builder
() → pyflink.datastream.connectors.kafka.KafkaSinkBuilder[source]¶ Create a
KafkaSinkBuilder
to constructKafkaSink
.
-
class
pyflink.datastream.connectors.kafka.
KafkaSinkBuilder
[source]¶ Builder to construct
KafkaSink
.The following example shows the minimum setup to create a KafkaSink that writes String values to a Kafka topic.
>>> record_serializer = KafkaRecordSerializationSchema.builder() \ ... .set_topic(MY_SINK_TOPIC) \ ... .set_value_serialization_schema(SimpleStringSchema()) \ ... .build() >>> sink = KafkaSink.builder() \ ... .set_bootstrap_servers(MY_BOOTSTRAP_SERVERS) \ ... .set_record_serializer(record_serializer) \ ... .build()
One can also configure different
DeliveryGuarantee
by usingset_delivery_guarantee()
but keep in mind when usingDeliveryGuarantee.EXACTLY_ONCE
, one must set the transactional id prefixset_transactional_id_prefix()
.New in version 1.16.0.
-
build
() → pyflink.datastream.connectors.kafka.KafkaSink[source]¶ Constructs the
KafkaSink
with the configured properties.
-
set_bootstrap_servers
(bootstrap_servers: str) → pyflink.datastream.connectors.kafka.KafkaSinkBuilder[source]¶ Sets the Kafka bootstrap servers.
- Parameters
bootstrap_servers – A comma separated list of valid URIs to reach the Kafka broker.
-
set_delivery_guarantee
(delivery_guarantee: pyflink.datastream.connectors.base.DeliveryGuarantee) → pyflink.datastream.connectors.kafka.KafkaSinkBuilder[source]¶ Sets the wanted
DeliveryGuarantee
. The default delivery guarantee isDeliveryGuarantee.NONE
.- Parameters
delivery_guarantee – The wanted
DeliveryGuarantee
.
-
set_property
(key: str, value: str) → pyflink.datastream.connectors.kafka.KafkaSinkBuilder[source]¶ Sets kafka producer config.
- Parameters
key – Kafka producer config key.
value – Kafka producer config value.
-
set_record_serializer
(record_serializer: pyflink.datastream.connectors.kafka.KafkaRecordSerializationSchema) → pyflink.datastream.connectors.kafka.KafkaSinkBuilder[source]¶ Sets the
KafkaRecordSerializationSchema
that transforms incoming records to kafka producer records.- Parameters
record_serializer – The
KafkaRecordSerializationSchema
.
-
set_transactional_id_prefix
(transactional_id_prefix: str) → pyflink.datastream.connectors.kafka.KafkaSinkBuilder[source]¶ Sets the prefix for all created transactionalIds if
DeliveryGuarantee.EXACTLY_ONCE
is configured.It is mandatory to always set this value with
DeliveryGuarantee.EXACTLY_ONCE
to prevent corrupted transactions if multiple jobs using the KafkaSink run against the same Kafka Cluster. The default prefix is"kafka-sink"
.The size of the prefix is capped by MAXIMUM_PREFIX_BYTES (6400) formatted with UTF-8.
It is important to keep the prefix stable across application restarts. If the prefix changes it might happen that lingering transactions are not correctly aborted and newly written messages are not immediately consumable until transactions timeout.
- Parameters
transactional_id_prefix – The transactional id prefix.
-
-
class
pyflink.datastream.connectors.kafka.
KafkaSource
(j_kafka_source: py4j.java_gateway.JavaObject)[source]¶ The Source implementation of Kafka. Please use a
KafkaSourceBuilder
to construct aKafkaSource
. The following example shows how to create a KafkaSource emitting records of String type.>>> source = KafkaSource \ ... .builder() \ ... .set_bootstrap_servers('MY_BOOTSTRAP_SERVERS') \ ... .set_group_id('MY_GROUP') \ ... .set_topics('TOPIC1', 'TOPIC2') \ ... .set_value_only_deserializer(SimpleStringSchema()) \ ... .set_starting_offsets(KafkaOffsetsInitializer.earliest()) \ ... .build()
New in version 1.16.0.
-
static
builder
() → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ Get a kafkaSourceBuilder to build a
KafkaSource
.- Returns
a Kafka source builder.
-
static
-
class
pyflink.datastream.connectors.kafka.
KafkaSourceBuilder
[source]¶ The builder class for
KafkaSource
to make it easier for the users to construct aKafkaSource
.The following example shows the minimum setup to create a KafkaSource that reads the String values from a Kafka topic.
>>> source = KafkaSource.builder() \ ... .set_bootstrap_servers('MY_BOOTSTRAP_SERVERS') \ ... .set_topics('TOPIC1', 'TOPIC2') \ ... .set_value_only_deserializer(SimpleStringSchema()) \ ... .build()
The bootstrap servers, topics/partitions to consume, and the record deserializer are required fields that must be set.
To specify the starting offsets of the KafkaSource, one can call
set_starting_offsets()
.By default, the KafkaSource runs in an CONTINUOUS_UNBOUNDED mode and never stops until the Flink job is canceled or fails. To let the KafkaSource run in CONTINUOUS_UNBOUNDED but stops at some given offsets, one can call
set_stopping_offsets()
. For example the following KafkaSource stops after it consumes up to the latest partition offsets at the point when the Flink started.>>> source = KafkaSource.builder() \ ... .set_bootstrap_servers('MY_BOOTSTRAP_SERVERS') \ ... .set_topics('TOPIC1', 'TOPIC2') \ ... .set_value_only_deserializer(SimpleStringSchema()) \ ... .set_unbounded(KafkaOffsetsInitializer.latest()) \ ... .build()
New in version 1.16.0.
-
set_bootstrap_servers
(bootstrap_servers: str) → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ Sets the bootstrap servers for the KafkaConsumer of the KafkaSource.
- Parameters
bootstrap_servers – the bootstrap servers of the Kafka cluster.
- Returns
this KafkaSourceBuilder.
-
set_bounded
(stopping_offsets_initializer: pyflink.datastream.connectors.kafka.KafkaOffsetsInitializer) → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ By default, the KafkaSource is set to run in CONTINUOUS_UNBOUNDED manner and thus never stops until the Flink job fails or is canceled. To let the KafkaSource run in BOUNDED manner and stop at some point, one can set an
KafkaOffsetsInitializer
to specify the stopping offsets for each partition. When all the partitions have reached their stopping offsets, the KafkaSource will then exit.This method is different from
set_unbounded()
that after setting the stopping offsets with this method,KafkaSource.get_boundedness()
will return BOUNDED instead of CONTINUOUS_UNBOUNDED.The following
KafkaOffsetsInitializer
s are commonly used and provided out of the box. Currently, customized offset initializer is not supported in PyFlink.KafkaOffsetsInitializer.latest()
- starting from the latest offsets.KafkaOffsetsInitializer.committedOffsets()
- starting from the committed offsets of the consumer group. If there is no committed offsets, starting from the offsets specified by theKafkaOffsetResetStrategy
.KafkaOffsetsInitializer.offsets()
- starting from the specified offsets for each partition.KafkaOffsetsInitializer.timestamp()
- starting from the specified timestamp for each partition. Note that the guarantee here is that all the records in Kafka whose timestamp is greater than the given starting timestamp will be consumed. However, it is possible that some consumer records whose timestamp is smaller than the given starting timestamp are also consumed.
- Parameters
stopping_offsets_initializer – the
KafkaOffsetsInitializer
to specify the stopping offsets.- Returns
this KafkaSourceBuilder
-
set_client_id_prefix
(prefix: str) → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ Sets the client id prefix of this KafkaSource.
- Parameters
prefix – the client id prefix to use for this KafkaSource.
- Returns
this KafkaSourceBuilder.
-
set_group_id
(group_id: str) → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ Sets the consumer group id of the KafkaSource.
- Parameters
group_id – the group id of the KafkaSource.
- Returns
this KafkaSourceBuilder.
-
set_partitions
(partitions: Set[KafkaTopicPartition]) → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ Set a set of partitions to consume from.
Example:
>>> KafkaSource.builder().set_partitions({ ... KafkaTopicPartition('TOPIC1', 0), ... KafkaTopicPartition('TOPIC1', 1), ... })
- Parameters
partitions – the set of partitions to consume from.
- Returns
this KafkaSourceBuilder.
-
set_properties
(props: Dict) → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ Set arbitrary properties for the KafkaSource and KafkaConsumer. The valid keys can be found in ConsumerConfig and KafkaSourceOptions.
Note that the following keys will be overridden by the builder when the KafkaSource is created.
key.deserializer
is always set to ByteArrayDeserializer.value.deserializer
is always set to ByteArrayDeserializer.auto.offset.reset.strategy
is overridden by AutoOffsetResetStrategy returned byKafkaOffsetsInitializer
for the starting offsets, which is by defaultKafkaOffsetsInitializer.earliest()
.partition.discovery.interval.ms
is overridden to -1 whenset_bounded()
has been invoked.client.id
is overridden to “client.id.prefix-RANDOM_LONG”, or “group.id-RANDOM_LONG” if the client id prefix is not set.
- Parameters
props – the properties to set for the KafkaSource.
- Returns
this KafkaSourceBuilder.
-
set_property
(key: str, value: str) → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ Set an arbitrary property for the KafkaSource and KafkaConsumer. The valid keys can be found in ConsumerConfig and KafkaSourceOptions.
Note that the following keys will be overridden by the builder when the KafkaSource is created.
key.deserializer
is always set to ByteArrayDeserializer.value.deserializer
is always set to ByteArrayDeserializer.auto.offset.reset.strategy
is overridden by AutoOffsetResetStrategy returned byKafkaOffsetsInitializer
for the starting offsets, which is by defaultKafkaOffsetsInitializer.earliest()
.partition.discovery.interval.ms
is overridden to -1 whenset_bounded()
has been invoked.
- Parameters
key – the key of the property.
value – the value of the property.
- Returns
this KafkaSourceBuilder.
-
set_starting_offsets
(starting_offsets_initializer: pyflink.datastream.connectors.kafka.KafkaOffsetsInitializer) → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ Specify from which offsets the KafkaSource should start consume from by providing an
KafkaOffsetsInitializer
.The following
KafkaOffsetsInitializer
s are commonly used and provided out of the box. Currently, customized offset initializer is not supported in PyFlink.KafkaOffsetsInitializer.earliest()
- starting from the earliest offsets. This is also the default offset initializer of the KafkaSource for starting offsets.KafkaOffsetsInitializer.latest()
- starting from the latest offsets.KafkaOffsetsInitializer.committedOffsets()
- starting from the committed offsets of the consumer group. If there is no committed offsets, starting from the offsets specified by theKafkaOffsetResetStrategy
.KafkaOffsetsInitializer.offsets()
- starting from the specified offsets for each partition.KafkaOffsetsInitializer.timestamp()
- starting from the specified timestamp for each partition. Note that the guarantee here is that all the records in Kafka whose timestamp is greater than the given starting timestamp will be consumed. However, it is possible that some consumer records whose timestamp is smaller than the given starting timestamp are also consumed.
- Parameters
starting_offsets_initializer – the
KafkaOffsetsInitializer
setting the starting offsets for the Source.- Returns
this KafkaSourceBuilder.
-
set_topic_pattern
(topic_pattern: str) → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ Set a topic pattern to consume from use the java Pattern. For grammar, check out JavaDoc .
- Parameters
topic_pattern – the pattern of the topic name to consume from.
- Returns
this KafkaSourceBuilder.
-
set_topics
(*topics: str) → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ Set a list of topics the KafkaSource should consume from. All the topics in the list should have existed in the Kafka cluster. Otherwise, an exception will be thrown. To allow some topics to be created lazily, please use
set_topic_pattern()
instead.- Parameters
topics – the list of topics to consume from.
- Returns
this KafkaSourceBuilder.
-
set_unbounded
(stopping_offsets_initializer: pyflink.datastream.connectors.kafka.KafkaOffsetsInitializer) → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ By default, the KafkaSource is set to run in CONTINUOUS_UNBOUNDED manner and thus never stops until the Flink job fails or is canceled. To let the KafkaSource run as a streaming source but still stops at some point, one can set an
KafkaOffsetsInitializer
to specify the stopping offsets for each partition. When all the partitions have reached their stopping offsets, the KafkaSource will then exit.This method is different from
set_bounded()
that after setting the stopping offsets with this method, KafkaSource will still be CONTINUOUS_UNBOUNDED even though it will stop at the stopping offsets specified by the stopping offset initializer.The following
KafkaOffsetsInitializer
s are commonly used and provided out of the box. Currently, customized offset initializer is not supported in PyFlink.KafkaOffsetsInitializer.latest()
- starting from the latest offsets.KafkaOffsetsInitializer.committedOffsets()
- starting from the committed offsets of the consumer group. If there is no committed offsets, starting from the offsets specified by theKafkaOffsetResetStrategy
.KafkaOffsetsInitializer.offsets()
- starting from the specified offsets for each partition.KafkaOffsetsInitializer.timestamp()
- starting from the specified timestamp for each partition. Note that the guarantee here is that all the records in Kafka whose timestamp is greater than the given starting timestamp will be consumed. However, it is possible that some consumer records whose timestamp is smaller than the given starting timestamp are also consumed.
- Parameters
stopping_offsets_initializer – the
KafkaOffsetsInitializer
to specify the stopping offsets.- Returns
this KafkaSourceBuilder
-
set_value_only_deserializer
(deserialization_schema: pyflink.common.serialization.DeserializationSchema) → pyflink.datastream.connectors.kafka.KafkaSourceBuilder[source]¶ Sets the
DeserializationSchema
for deserializing the value of Kafka’s ConsumerRecord. The other information (e.g. key) in a ConsumerRecord will be ignored.- Parameters
deserialization_schema – the
DeserializationSchema
to use for deserialization.- Returns
this KafkaSourceBuilder.
-
-
class
pyflink.datastream.connectors.kafka.
KafkaTopicPartition
(topic: str, partition: int)[source]¶ Corresponding to Java
org.apache.kafka.common.TopicPartition
class.Example:
>>> topic_partition = KafkaTopicPartition('TOPIC1', 0)
New in version 1.16.0.
-
class
pyflink.datastream.connectors.kafka.
KafkaTopicSelector
[source]¶ Select topic for an incoming record
New in version 1.16.0.
-
class
pyflink.datastream.connectors.kafka.
Semantic
(value)[source]¶ Semantics that can be chosen.
- Data
EXACTLY_ONCE:
The Flink producer will write all messages in a Kafka transaction that will be committed to the Kafka on a checkpoint. In this mode FlinkKafkaProducer sets up a pool of FlinkKafkaProducer. Between each checkpoint there is created new Kafka transaction, which is being committed on FlinkKafkaProducer.notifyCheckpointComplete(long). If checkpoint complete notifications are running late, FlinkKafkaProducer can run out of FlinkKafkaProducers in the pool. In that case any subsequent FlinkKafkaProducer.snapshot- State() requests will fail and the FlinkKafkaProducer will keep using the FlinkKafkaProducer from previous checkpoint. To decrease chances of failing checkpoints there are four options:
decrease number of max concurrent checkpoints
make checkpoints mre reliable (so that they complete faster)
increase delay between checkpoints
increase size of FlinkKafkaProducers pool
- Data
AT_LEAST_ONCE:
The Flink producer will wait for all outstanding messages in the Kafka buffers to be acknowledged by the Kafka producer on a checkpoint.
- Data
NONE:
Means that nothing will be guaranteed. Messages can be lost and/or duplicated in case of failure.
-
AT_LEAST_ONCE
= (1,)¶
-
EXACTLY_ONCE
= (0,)¶
-
NONE
= 2¶
pyflink.datastream.connectors.file_system module¶
-
class
pyflink.datastream.connectors.file_system.
BucketAssigner
(j_bucket_assigner)[source]¶ A BucketAssigner is used with a file sink to determine the bucket each incoming element should be put into.
The StreamingFileSink can be writing to many buckets at a time, and it is responsible for managing a set of active buckets. Whenever a new element arrives it will ask the BucketAssigner for the bucket the element should fall in. The BucketAssigner can, for example, determine buckets based on system time.
-
static
base_path_bucket_assigner
() → pyflink.datastream.connectors.file_system.BucketAssigner[source]¶ Creates a BucketAssigner that does not perform any bucketing of files. All files are written to the base path.
-
static
date_time_bucket_assigner
(format_str: str = 'yyyy-MM-dd--HH', timezone_id: str = None)[source]¶ Creates a BucketAssigner that assigns to buckets based on current system time.
It will create directories of the following form: /{basePath}/{dateTimePath}/}. The basePath is the path that was specified as a base path when creating the new bucket. The dateTimePath is determined based on the current system time and the user provided format string.
The Java DateTimeFormatter is used to derive a date string from the current system time and the date format string. The default format string is “yyyy-MM-dd–HH” so the rolling files will have a granularity of hours.
- Parameters
format_str – The format string used to determine the bucket id.
timezone_id – The timezone id, either an abbreviation such as “PST”, a full name such as “America/Los_Angeles”, or a custom timezone_id such as “GMT-08:00”. Th e default time zone will b used if it’s None.
-
static
-
class
pyflink.datastream.connectors.file_system.
BulkFormat
(j_bulk_format)[source]¶ The BulkFormat reads and decodes batches of records at a time. Examples of bulk formats are formats like ORC or Parquet.
Internally in the file source, the readers pass batches of records from the reading threads (that perform the typically blocking I/O operations) to the async mailbox threads that do the streaming and batch data processing. Passing records in batches (rather than one-at-a-time) much reduce the thread-to-thread handover overhead.
For the BulkFormat, one batch is handed over as one.
New in version 1.16.0.
-
class
pyflink.datastream.connectors.file_system.
FileCompactStrategy
(j_file_compact_strategy)[source]¶ Strategy for compacting the files written in {@link FileSink} before committing.
New in version 1.16.0.
-
class
Builder
[source]¶ -
-
enable_compaction_on_checkpoint
(num_checkpoints_before_compaction: int) → pyflink.datastream.connectors.file_system.FileCompactStrategy.Builder[source]¶ Optional, compaction will be triggered when N checkpoints passed since the last triggering, -1 by default indicating no compaction on checkpoint.
-
set_num_compact_threads
(num_compact_threads: int) → pyflink.datastream.connectors.file_system.FileCompactStrategy.Builder[source]¶ Optional, the count of compacting threads in a compactor operator, 1 by default.
-
set_size_threshold
(size_threshold: int) → pyflink.datastream.connectors.file_system.FileCompactStrategy.Builder[source]¶ Optional, compaction will be triggered when the total size of compacting files reaches the threshold. -1 by default, indicating the size is unlimited.
-
-
class
-
class
pyflink.datastream.connectors.file_system.
FileCompactor
(j_file_compactor)[source]¶ The FileCompactor is responsible for compacting files into one file.
New in version 1.16.0.
-
class
pyflink.datastream.connectors.file_system.
FileEnumeratorProvider
(j_file_enumerator_provider)[source]¶ Factory for FileEnumerator which task is to discover all files to be read and to split them into a set of file source splits. This includes possibly, path traversals, file filtering (by name or other patterns) and deciding whether to split files into multiple splits, and how to split them.
-
static
default_non_splittable_file_enumerator
() → pyflink.datastream.connectors.file_system.FileEnumeratorProvider[source]¶ The default file enumerator used for non-splittable formats. The enumerator recursively enumerates files, creates one split for the file, and filters hidden files (files starting with ‘.’ or ‘_’).
-
static
default_splittable_file_enumerator
() → pyflink.datastream.connectors.file_system.FileEnumeratorProvider[source]¶ The default file enumerator used for splittable formats. The enumerator recursively enumerates files, split files that consist of multiple distributed storage blocks into multiple splits, and filters hidden files (files starting with ‘.’ or ‘_’). Files with suffixes of common compression formats (for example ‘.gzip’, ‘.bz2’, ‘.xy’, ‘.zip’, …) will not be split.
-
static
-
class
pyflink.datastream.connectors.file_system.
FileSink
(j_file_sink, transformer: Optional[pyflink.datastream.connectors.base.StreamTransformer] = None)[source]¶ A unified sink that emits its input elements to FileSystem files within buckets. This sink achieves exactly-once semantics for both BATCH and STREAMING.
When creating the sink a basePath must be specified. The base directory contains one directory for every bucket. The bucket directories themselves contain several part files, with at least one for each parallel subtask of the sink which is writing data to that bucket. These part files contain the actual output data.
The sink uses a BucketAssigner to determine in which bucket directory each element should be written to inside the base directory. The BucketAssigner can, for example, roll on every checkpoint or use time or a property of the element to determine the bucket directory. The default BucketAssigner is a DateTimeBucketAssigner which will create one new bucket every hour. You can specify a custom BucketAssigner using the
with_bucket_assigner()
, after callingfor_row_format
.The names of the part files could be defined using OutputFileConfig. This configuration contains a part prefix and a part suffix that will be used with a random uid assigned to each subtask of the sink and a rolling counter to determine the file names. For example with a prefix “prefix” and a suffix “.ext”, a file named {@code “prefix-81fc4980-a6af-41c8-9937-9939408a734b-17.ext”} contains the data from subtask with uid {@code 81fc4980-a6af-41c8-9937-9939408a734b} of the sink and is the {@code 17th} part-file created by that subtask.
Part files roll based on the user-specified RollingPolicy. By default, a DefaultRollingPolicy is used for row-encoded sink output; a OnCheckpointRollingPolicy is used for bulk-encoded sink output.
In some scenarios, the open buckets are required to change based on time. In these cases, the user can specify a bucket_check_interval (by default 1m) and the sink will check periodically and roll the part file if the specified rolling policy says so.
Part files can be in one of three states: in-progress, pending or finished. The reason for this is how the sink works to provide exactly-once semantics and fault-tolerance. The part file that is currently being written to is in-progress. Once a part file is closed for writing it becomes pending. When a checkpoint is successful (for STREAMING) or at the end of the job (for BATCH) the currently pending files will be moved to finished.
For STREAMING in order to guarantee exactly-once semantics in case of a failure, the sink should roll back to the state it had when that last successful checkpoint occurred. To this end, when restoring, the restored files in pending state are transferred into the finished state while any in-progress files are rolled back, so that they do not contain data that arrived after the checkpoint from which we restore.
-
class
BaseBuilder
(j_builder)[source]¶ -
-
enable_compact
(strategy: pyflink.datastream.connectors.file_system.FileCompactStrategy, compactor: pyflink.datastream.connectors.file_system.FileCompactor)[source]¶
-
with_bucket_assigner
(bucket_assigner: pyflink.datastream.connectors.file_system.BucketAssigner)[source]¶
-
with_bucket_check_interval
(interval: int)[source]¶ - Parameters
interval – The check interval in milliseconds.
-
with_output_file_config
(output_file_config: pyflink.datastream.connectors.file_system.OutputFileConfig)[source]¶
-
-
class
BulkFormatBuilder
(j_bulk_format_builder)[source]¶ Builder for the vanilla FileSink using a bulk format.
New in version 1.16.0.
-
class
RowFormatBuilder
(j_row_format_builder)[source]¶ Builder for the vanilla FileSink using a row format.
Changed in version 1.16.0: Support compaction.
-
with_rolling_policy
(rolling_policy: pyflink.datastream.connectors.file_system.RollingPolicy)[source]¶
-
-
static
for_bulk_format
(base_path: str, writer_factory: pyflink.common.serialization.BulkWriterFactory) → pyflink.datastream.connectors.file_system.FileSink.BulkFormatBuilder[source]¶
-
static
for_row_format
(base_path: str, encoder: pyflink.common.serialization.Encoder) → pyflink.datastream.connectors.file_system.FileSink.RowFormatBuilder[source]¶
-
class
-
class
pyflink.datastream.connectors.file_system.
FileSource
(j_file_source)[source]¶ A unified data source that reads files - both in batch and in streaming mode.
This source supports all (distributed) file systems and object stores that can be accessed via the Flink’s FileSystem class.
Start building a file source via one of the following calls:
This creates a
FileSourceBuilder
on which you can configure all the properties of the file source.<h2>Batch and Streaming</h2>
This source supports both bounded/batch and continuous/streaming data inputs. For the bounded/batch case, the file source processes all files under the given path(s). In the continuous/streaming case, the source periodically checks the paths for new files and will start reading those.
When you start creating a file source (via the
FileSourceBuilder
created through one of the above-mentioned methods) the source is by default in bounded/batch mode. Callmonitor_continuously()
to put the source into continuous streaming mode.<h2>Format Types</h2>
The reading of each file happens through file readers defined by <i>file formats</i>. These define the parsing logic for the contents of the file. There are multiple classes that the source supports. Their interfaces trade of simplicity of implementation and flexibility/efficiency.
A
StreamFormat
reads the contents of a file from a file stream. It is the simplest format to implement, and provides many features out-of-the-box (like checkpointing logic) but is limited in the optimizations it can apply (such as object reuse, batching, etc.).
<h2>Discovering / Enumerating Files</h2>
The way that the source lists the files to be processes is defined by the
FileEnumeratorProvider
. The FileEnumeratorProvider is responsible to select the relevant files (for example filter out hidden files) and to optionally splits files into multiple regions (= file source splits) that can be read in parallel).-
static
for_bulk_file_format
(bulk_format: pyflink.datastream.connectors.file_system.BulkFormat, *paths: str) → pyflink.datastream.connectors.file_system.FileSourceBuilder[source]¶
-
static
for_record_stream_format
(stream_format: pyflink.datastream.connectors.file_system.StreamFormat, *paths: str) → pyflink.datastream.connectors.file_system.FileSourceBuilder[source]¶ Builds a new FileSource using a
StreamFormat
to read record-by-record from a file stream.When possible, stream-based formats are generally easier (preferable) to file-based formats, because they support better default behavior around I/O batching or progress tracking (checkpoints).
Stream formats also automatically de-compress files based on the file extension. This supports files ending in “.deflate” (Deflate), “.xz” (XZ), “.bz2” (BZip2), “.gz”, “.gzip” (GZip).
-
class
pyflink.datastream.connectors.file_system.
FileSourceBuilder
(j_file_source_builder)[source]¶ The builder for the
FileSource
, to configure the various behaviors.Start building the source via one of the following methods:
-
build
() → pyflink.datastream.connectors.file_system.FileSource[source]¶ Creates the file source with the settings applied to this builder.
-
monitor_continuously
(discovery_interval: pyflink.common.time.Duration) → pyflink.datastream.connectors.file_system.FileSourceBuilder[source]¶ Sets this source to streaming (“continuous monitoring”) mode.
This makes the source a “continuous streaming” source that keeps running, monitoring for new files, and reads these files when they appear and are discovered by the monitoring.
The interval in which the source checks for new files is the discovery_interval. Shorter intervals mean that files are discovered more quickly, but also imply more frequent listing or directory traversal of the file system / object store.
-
process_static_file_set
() → pyflink.datastream.connectors.file_system.FileSourceBuilder[source]¶ Sets this source to bounded (batch) mode.
In this mode, the source processes the files that are under the given paths when the application is started. Once all files are processed, the source will finish.
This setting is also the default behavior. This method is mainly here to “switch back” to bounded (batch) mode, or to make it explicit in the source construction.
-
set_file_enumerator
(file_enumerator: pyflink.datastream.connectors.file_system.FileEnumeratorProvider) → pyflink.datastream.connectors.file_system.FileSourceBuilder[source]¶ Configures the FileEnumerator for the source. The File Enumerator is responsible for selecting from the input path the set of files that should be processed (and which to filter out). Furthermore, the File Enumerator may split the files further into sub-regions, to enable parallelization beyond the number of files.
-
set_split_assigner
(split_assigner: pyflink.datastream.connectors.file_system.FileSplitAssignerProvider) → pyflink.datastream.connectors.file_system.FileSourceBuilder[source]¶ Configures the FileSplitAssigner for the source. The File Split Assigner determines which parallel reader instance gets which {@link FileSourceSplit}, and in which order these splits are assigned.
-
-
class
pyflink.datastream.connectors.file_system.
FileSplitAssignerProvider
(j_file_split_assigner)[source]¶ Factory for FileSplitAssigner which is responsible for deciding what split should be processed next by which node. It determines split processing order and locality.
-
static
locality_aware_split_assigner
() → pyflink.datastream.connectors.file_system.FileSplitAssignerProvider[source]¶ A FileSplitAssigner that assigns to each host preferably splits that are local, before assigning splits that are not local.
-
static
-
class
pyflink.datastream.connectors.file_system.
OutputFileConfig
(part_prefix: str, part_suffix: str)[source]¶ Part file name configuration. This allow to define a prefix and a suffix to the part file name.
-
class
OutputFileConfigBuilder
[source]¶ A builder to create the part file configuration.
-
with_part_prefix
(prefix) → pyflink.datastream.connectors.file_system.OutputFileConfig.OutputFileConfigBuilder[source]¶
-
with_part_suffix
(suffix) → pyflink.datastream.connectors.file_system.OutputFileConfig.OutputFileConfigBuilder[source]¶
-
-
class
-
class
pyflink.datastream.connectors.file_system.
RollingPolicy
(j_rolling_policy)[source]¶ The policy based on which a Bucket in the FileSink rolls its currently open part file and opens a new one.
-
static
default_rolling_policy
(part_size: int = 134217728, rollover_interval: int = 60000, inactivity_interval: int = 60000) → pyflink.datastream.connectors.file_system.DefaultRollingPolicy[source]¶ Returns the default implementation of the RollingPolicy.
This policy rolls a part file if:
there is no open part file,
the current file has reached the maximum bucket size (by default 128MB),
the current file is older than the roll over interval (by default 60 sec), or
the current file has not been written to for more than the allowed inactivityTime (by default 60 sec).
- Parameters
part_size – The maximum part file size before rolling.
rollover_interval – The maximum time duration a part file can stay open before rolling.
inactivity_interval – The time duration of allowed inactivity after which a part file will have to roll.
-
static
-
class
pyflink.datastream.connectors.file_system.
StreamFormat
(j_stream_format)[source]¶ A reader format that reads individual records from a stream.
Compared to the
BulkFormat
, the stream format handles a few things out-of-the-box, like deciding how to batch records or dealing with compression.Internally in the file source, the readers pass batches of records from the reading threads (that perform the typically blocking I/O operations) to the async mailbox threads that do the streaming and batch data processing. Passing records in batches (rather than one-at-a-time) much reduces the thread-to-thread handover overhead.
This batching is by default based on I/O fetch size for the StreamFormat, meaning the set of records derived from one I/O buffer will be handed over as one. See config option source.file.stream.io-fetch-size to configure that fetch size.
-
static
text_line_format
(charset_name: str = 'UTF-8') → pyflink.datastream.connectors.file_system.StreamFormat[source]¶ Creates a reader format that text lines from a file.
The reader uses Java’s built-in java.io.InputStreamReader to decode the byte stream using various supported charset encodings.
This format does not support optimized recovery from checkpoints. On recovery, it will re-read and discard the number of lined that were processed before the last checkpoint. That is due to the fact that the offsets of lines in the file cannot be tracked through the charset decoders with their internal buffering of stream input and charset decoder state.
- Parameters
charset_name – The charset to decode the byte stream.
-
static
-
class
pyflink.datastream.connectors.file_system.
StreamingFileSink
(j_obj)[source]¶ Sink that emits its input elements to FileSystem files within buckets. This is integrated with the checkpointing mechanism to provide exactly once semantics.
When creating the sink a basePath must be specified. The base directory contains one directory for every bucket. The bucket directories themselves contain several part files, with at least one for each parallel subtask of the sink which is writing data to that bucket. These part files contain the actual output data.
-
class
BaseBuilder
(j_builder)[source]¶ -
-
with_bucket_assigner
(bucket_assigner: pyflink.datastream.connectors.file_system.BucketAssigner)[source]¶
-
with_output_file_config
(output_file_config: pyflink.datastream.connectors.file_system.OutputFileConfig)[source]¶
-
-
class
DefaultRowFormatBuilder
(j_default_row_format_builder)[source]¶ Builder for the vanilla StreamingFileSink using a row format.
-
with_rolling_policy
(policy: pyflink.datastream.connectors.file_system.RollingPolicy)[source]¶
-
-
class
pyflink.datastream.connectors.kinesis module¶
-
class
pyflink.datastream.connectors.kinesis.
FlinkKinesisConsumer
(streams: Union[str, List[str]], deserializer: Union[pyflink.common.serialization.DeserializationSchema, pyflink.datastream.connectors.kinesis.KinesisDeserializationSchema], config_props: Dict)[source]¶ The Flink Kinesis Consumer is an exactly-once parallel streaming data source that subscribes to multiple AWS Kinesis streams within the same AWS service region, and can handle resharding of streams. Each subtask of the consumer is responsible for fetching data records from multiple Kinesis shards. The number of shards fetched by each subtask will change as shards are closed and created by Kinesis.
To leverage Flink’s checkpointing mechanics for exactly-once streaming processing guarantees, the Flink Kinesis consumer is implemented with the AWS Java SDK, instead of the officially recommended AWS Kinesis Client Library, for low-level control on the management of stream state. The Flink Kinesis Connector also supports setting the initial starting points of Kinesis streams, namely TRIM_HORIZON and LATEST.
Kinesis and the Flink consumer support dynamic re-sharding and shard IDs, while sequential, cannot be assumed to be consecutive. There is no perfect generic default assignment function. Default shard to subtask assignment, which is based on hash code, may result in skew, with some subtasks having many shards assigned and others none.
It is recommended to monitor the shard distribution and adjust assignment appropriately. A custom assigner implementation can be set via setShardAssigner(KinesisShardAssigner) to optimize the hash function or use static overrides to limit skew.
In order for the consumer to emit watermarks, a timestamp assigner needs to be set via setPeriodicWatermarkAssigner(AssignerWithPeriodicWatermarks) and the auto watermark emit interval configured via ExecutionConfig.setAutoWatermarkInterval(long).
Watermarks can only advance when all shards of a subtask continuously deliver records. To avoid an inactive or closed shard to block the watermark progress, the idle timeout should be configured via configuration property ConsumerConfigConstants.SHARD_IDLE_INTERVAL_MILLIS. By default, shards won’t be considered idle and watermark calculation will wait for newer records to arrive from all shards.
Note that re-sharding of the Kinesis stream while an application (that relies on the Kinesis records for watermarking) is running can lead to incorrect late events. This depends on how shards are assigned to subtasks and applies regardless of whether watermarks are generated in the source or a downstream operator.
-
set_periodic_watermark_assigner
(periodic_watermark_assigner: pyflink.common.watermark_strategy.AssignerWithPeriodicWatermarksWrapper) → pyflink.datastream.connectors.kinesis.FlinkKinesisConsumer[source]¶ Set the assigner that will extract the timestamp from T and calculate the watermark.
-
set_shard_assigner
(shard_assigner: pyflink.datastream.connectors.kinesis.KinesisShardAssigner) → pyflink.datastream.connectors.kinesis.FlinkKinesisConsumer[source]¶ Provide a custom assigner to influence how shards are distributed over subtasks.
-
set_watermark_tracker
(watermark_tracker: pyflink.datastream.connectors.kinesis.WatermarkTracker) → pyflink.datastream.connectors.kinesis.FlinkKinesisConsumer[source]¶ Set the global watermark tracker. When set, it will be used by the fetcher to align the shard consumers by event time.
-
-
class
pyflink.datastream.connectors.kinesis.
KinesisDeserializationSchema
(j_kinesis_deserialization_schema)[source]¶ This is a deserialization schema specific for the Flink Kinesis Consumer. Different from the basic DeserializationSchema, this schema offers additional Kinesis-specific information about the record that may be useful to the user application.
-
class
pyflink.datastream.connectors.kinesis.
KinesisFirehoseSink
(j_kinesis_firehose_sink)[source]¶ A Kinesis Data Firehose (KDF) Sink that performs async requests against a destination delivery stream using the buffering protocol.
-
static
builder
() → pyflink.datastream.connectors.kinesis.KinesisFirehoseSinkBuilder[source]¶
-
static
-
class
pyflink.datastream.connectors.kinesis.
KinesisFirehoseSinkBuilder
[source]¶ Builder to construct KinesisFirehoseSink.
The following example shows the minimum setup to create a KinesisFirehoseSink that writes String values to a Kinesis Data Firehose delivery stream named delivery-stream-name.
Example:
>>> from pyflink.common.serialization import SimpleStringSchema >>> sink_properties = {"aws.region": "eu-west-1"} >>> sink = KinesisFirehoseSink.builder() \ ... .set_firehose_client_properties(sink_properties) \ ... .set_delivery_stream_name("delivery-stream-name") \ ... .set_serialization_schema(SimpleStringSchema()) \ ... .set_max_batch_size(20) \ ... .build()
If the following parameters are not set in this builder, the following defaults will be used: - maxBatchSize will be 500 - maxInFlightRequests will be 50 - maxBufferedRequests will be 10000 - maxBatchSizeInBytes will be 4 MB i.e. 4 * 1024 * 1024 - maxTimeInBufferMS will be 5000ms - maxRecordSizeInBytes will be 1000 KB i.e. 1000 * 1024 - failOnError will be false
-
build
() → pyflink.datastream.connectors.kinesis.KinesisFirehoseSink[source]¶ Build thd KinesisFirehoseSink.
-
set_delivery_stream_name
(delivery_stream_name: str) → pyflink.datastream.connectors.kinesis.KinesisFirehoseSinkBuilder[source]¶ Sets the name of the KDF delivery stream that the sink will connect to. There is no default for this parameter, therefore, this must be provided at sink creation time otherwise the build will fail.
-
set_fail_on_error
(fail_on_error: bool) → pyflink.datastream.connectors.kinesis.KinesisFirehoseSinkBuilder[source]¶ If writing to Kinesis Data Firehose results in a partial or full failure being returned, the job will fail
-
set_firehose_client_properties
(firehose_client_properties: Dict) → pyflink.datastream.connectors.kinesis.KinesisFirehoseSinkBuilder[source]¶ A set of properties used by the sink to create the firehose client. This may be used to set the aws region, credentials etc. See the docs for usage and syntax.
-
set_max_batch_size
(max_batch_size: int) → pyflink.datastream.connectors.kinesis.KinesisFirehoseSinkBuilder[source]¶ Maximum number of elements that may be passed in a list to be written downstream.
-
set_max_batch_size_in_bytes
(max_batch_size_in_bytes: int) → pyflink.datastream.connectors.kinesis.KinesisFirehoseSinkBuilder[source]¶ The flush will be attempted if the most recent call to write introduces an element to the buffer such that the total size of the buffer is greater than or equal to this threshold value. If this happens, the maximum number of elements from the head of the buffer will be selected, that is smaller than maxBatchSizeInBytes in size will be flushed.
-
set_max_buffered_requests
(max_buffered_requests: int) → pyflink.datastream.connectors.kinesis.KinesisFirehoseSinkBuilder[source]¶ The maximum buffer length. Callbacks to add elements to the buffer and calls to write will block if this length has been reached and will only unblock if elements from the buffer have been removed for flushing.
-
set_max_in_flight_requests
(max_in_flight_requests: int) → pyflink.datastream.connectors.kinesis.KinesisFirehoseSinkBuilder[source]¶ Maximum number of uncompleted calls to submitRequestEntries that the SinkWriter will allow at any given point. Once this point has reached, writes and callbacks to add elements to the buffer may block until one or more requests to submitRequestEntries completes.
-
set_max_record_size_in_bytes
(max_record_size_in_bytes: int) → pyflink.datastream.connectors.kinesis.KinesisFirehoseSinkBuilder[source]¶ The maximum size of each records in bytes. If a record larger than this is passed to the sink, it will throw an IllegalArgumentException.
-
set_max_time_in_buffer_ms
(max_time_in_buffer_ms: int) → pyflink.datastream.connectors.kinesis.KinesisFirehoseSinkBuilder[source]¶ The maximum amount of time an element may remain in the buffer. In most cases elements are flushed as a result of the batch size (in bytes or number) being reached or during a snapshot. However, there are scenarios where an element may remain in the buffer forever or a long period of time. To mitigate this, a timer is constantly active in the buffer such that: while the buffer is not empty, it will flush every maxTimeInBufferMS milliseconds.
-
set_serialization_schema
(serialization_schema: pyflink.common.serialization.SerializationSchema) → pyflink.datastream.connectors.kinesis.KinesisFirehoseSinkBuilder[source]¶ Allows the user to specify a serialization schema to serialize each record to persist to Firehose.
-
-
class
pyflink.datastream.connectors.kinesis.
KinesisShardAssigner
(j_kinesis_shard_assigner)[source]¶ Utility to map Kinesis shards to Flink subtask indices. Users can provide a Java KinesisShardAssigner in Python if they want to provide custom shared assigner.
-
static
default_shard_assigner
() → pyflink.datastream.connectors.kinesis.KinesisShardAssigner[source]¶ A Default KinesisShardAssigner that maps Kinesis shard hash-key ranges to Flink subtasks.
-
static
uniform_shard_assigner
() → pyflink.datastream.connectors.kinesis.KinesisShardAssigner[source]¶ A KinesisShardAssigner that maps Kinesis shard hash-key ranges to Flink subtasks. It creates a more uniform distribution of shards across subtasks than org.apache.flink. streaming.connectors.kinesis.internals.KinesisDataFetcher.DEFAULT_SHARD_ASSIGNER when the Kinesis records in the stream have hash keys that are uniformly distributed over all possible hash keys, which is the case if records have randomly-generated partition keys. (This is the same assumption made if you use the Kinesis UpdateShardCount operation with UNIFORM_SCALING.)
-
static
-
class
pyflink.datastream.connectors.kinesis.
KinesisStreamsSink
(j_kinesis_streams_sink)[source]¶ A Kinesis Data Streams (KDS) Sink that performs async requests against a destination stream using the buffering protocol.
The sink internally uses a software.amazon.awssdk.services.kinesis.KinesisAsyncClient to communicate with the AWS endpoint.
The behaviour of the buffering may be specified by providing configuration during the sink build time.
maxBatchSize: the maximum size of a batch of entries that may be sent to KDS
- maxInFlightRequests: the maximum number of in flight requests that may exist, if any more in
flight requests need to be initiated once the maximum has been reached, then it will be blocked until some have completed
- maxBufferedRequests: the maximum number of elements held in the buffer, requests to add
elements will be blocked while the number of elements in the buffer is at the maximum
- maxBatchSizeInBytes: the maximum size of a batch of entries that may be sent to KDS
measured in bytes
- maxTimeInBufferMS: the maximum amount of time an entry is allowed to live in the buffer,
if any element reaches this age, the entire buffer will be flushed immediately
- maxRecordSizeInBytes: the maximum size of a record the sink will accept into the buffer,
a record of size larger than this will be rejected when passed to the sink
- failOnError: when an exception is encountered while persisting to Kinesis Data Streams,
the job will fail immediately if failOnError is set
-
static
builder
() → pyflink.datastream.connectors.kinesis.KinesisStreamsSinkBuilder[source]¶
-
class
pyflink.datastream.connectors.kinesis.
KinesisStreamsSinkBuilder
[source]¶ Builder to construct KinesisStreamsSink.
The following example shows the minimum setup to create a KinesisStreamsSink that writes String values to a Kinesis Data Streams stream named your_stream_here.
Example:
>>> from pyflink.common.serialization import SimpleStringSchema >>> sink_properties = {"aws.region": "eu-west-1"} >>> sink = KinesisStreamsSink.builder() \ ... .set_kinesis_client_properties(sink_properties) \ ... .set_stream_name("your_stream_name") \ ... .set_serialization_schema(SimpleStringSchema()) \ ... .set_partition_key_generator(PartitionKeyGenerator.random()) \ ... .build()
If the following parameters are not set in this builder, the following defaults will be used:
maxBatchSize will be 500
maxInFlightRequests will be 50
maxBufferedRequests will be 10000
maxBatchSizeInBytes will be 5 MB i.e. 5 * 1024 * 1024
maxTimeInBufferMS will be 5000ms
maxRecordSizeInBytes will be 1 MB i.e. 1 * 1024 * 1024
failOnError will be false
-
build
() → pyflink.datastream.connectors.kinesis.KinesisStreamsSink[source]¶ Build thd KinesisStreamsSink.
-
set_fail_on_error
(fail_on_error: bool) → pyflink.datastream.connectors.kinesis.KinesisStreamsSinkBuilder[source]¶ Sets the failOnError of the KinesisSinkBuilder. If failOnError is on, then a runtime exception will be raised. Otherwise, those records will be requested in the buffer for retry.
-
set_kinesis_client_properties
(kinesis_client_properties: Dict) → pyflink.datastream.connectors.kinesis.KinesisStreamsSinkBuilder[source]¶ Sets the kinesisClientProperties of the KinesisSinkBuilder.
-
set_max_batch_size
(max_batch_size: int) → pyflink.datastream.connectors.kinesis.KinesisStreamsSinkBuilder[source]¶ Maximum number of elements that may be passed in a list to be written downstream.
-
set_max_batch_size_in_bytes
(max_batch_size_in_bytes: int) → pyflink.datastream.connectors.kinesis.KinesisStreamsSinkBuilder[source]¶ The flush will be attempted if the most recent call to write introduces an element to the buffer such that the total size of the buffer is greater than or equal to this threshold value. If this happens, the maximum number of elements from the head of the buffer will be selected, that is smaller than maxBatchSizeInBytes in size will be flushed.
-
set_max_buffered_requests
(max_buffered_requests: int) → pyflink.datastream.connectors.kinesis.KinesisStreamsSinkBuilder[source]¶ The maximum buffer length. Callbacks to add elements to the buffer and calls to write will block if this length has been reached and will only unblock if elements from the buffer have been removed for flushing.
-
set_max_in_flight_requests
(max_in_flight_requests: int) → pyflink.datastream.connectors.kinesis.KinesisStreamsSinkBuilder[source]¶ Maximum number of uncompleted calls to submitRequestEntries that the SinkWriter will allow at any given point. Once this point has reached, writes and callbacks to add elements to the buffer may block until one or more requests to submitRequestEntries completes.
-
set_max_record_size_in_bytes
(max_record_size_in_bytes: int) → pyflink.datastream.connectors.kinesis.KinesisStreamsSinkBuilder[source]¶ The maximum size of each records in bytes. If a record larger than this is passed to the sink, it will throw an IllegalArgumentException.
-
set_max_time_in_buffer_ms
(max_time_in_buffer_ms: int) → pyflink.datastream.connectors.kinesis.KinesisStreamsSinkBuilder[source]¶ The maximum amount of time an element may remain in the buffer. In most cases elements are flushed as a result of the batch size (in bytes or number) being reached or during a snapshot. However, there are scenarios where an element may remain in the buffer forever or a long period of time. To mitigate this, a timer is constantly active in the buffer such that: while the buffer is not empty, it will flush every maxTimeInBufferMS milliseconds.
-
set_partition_key_generator
(partition_key_generator: pyflink.datastream.connectors.kinesis.PartitionKeyGenerator) → pyflink.datastream.connectors.kinesis.KinesisStreamsSinkBuilder[source]¶ Sets the PartitionKeyGenerator of the KinesisSinkBuilder.
-
set_serialization_schema
(serialization_schema: pyflink.common.serialization.SerializationSchema) → pyflink.datastream.connectors.kinesis.KinesisStreamsSinkBuilder[source]¶ Sets the SerializationSchema of the KinesisSinkBuilder.
-
set_stream_name
(stream_name: Union[str, List[str]]) → pyflink.datastream.connectors.kinesis.KinesisStreamsSinkBuilder[source]¶ Sets the name of the KDS stream that the sink will connect to. There is no default for this parameter, therefore, this must be provided at sink creation time otherwise the build will fail.
-
class
pyflink.datastream.connectors.kinesis.
PartitionKeyGenerator
(j_partition_key_generator)[source]¶ This is a generator convert from an input element to the partition key, a string.
-
static
fixed
() → pyflink.datastream.connectors.kinesis.PartitionKeyGenerator[source]¶ A partitioner ensuring that each internal Flink partition ends up in the same Kinesis partition. This is achieved by using the index of the producer task as a PartitionKey.
-
static
random
() → pyflink.datastream.connectors.kinesis.PartitionKeyGenerator[source]¶ A PartitionKeyGenerator that maps an arbitrary input element to a random partition ID.
-
static
-
class
pyflink.datastream.connectors.kinesis.
WatermarkTracker
(j_watermark_tracker)[source]¶ The watermark tracker is responsible for aggregating watermarks across distributed operators. It can be used for sub tasks of a single Flink source as well as multiple heterogeneous sources or other operators.The class essentially functions like a distributed hash table that enclosing operators can use to adopt their processing / IO rates
-
static
job_manager_watermark_tracker
(aggregate_name: str, log_accumulator_interval_millis: int = - 1) → pyflink.datastream.connectors.kinesis.WatermarkTracker[source]¶
-
static
pyflink.datastream.connectors.number_seq module¶
-
class
pyflink.datastream.connectors.number_seq.
NumberSequenceSource
(start: int, end: int)[source]¶ A data source that produces a sequence of numbers (longs). This source is useful for testing and for cases that just need a stream of N events of any kind.
The source splits the sequence into as many parallel sub-sequences as there are parallel source readers. Each sub-sequence will be produced in order. Consequently, if the parallelism is limited to one, this will produce one sequence in order.
This source is always bounded. For very long sequences (for example over the entire domain of long integer values), user may want to consider executing the application in a streaming manner, because, despite the fact that the produced stream is bounded, the end bound is pretty far away.
pyflink.datastream.connectors.jdbc module¶
-
class
pyflink.datastream.connectors.jdbc.
JdbcConnectionOptions
(j_jdbc_connection_options)[source]¶ JDBC connection options.
-
class
JdbcConnectionOptionsBuilder
[source]¶ Builder for JdbcConnectionOptions.
-
with_driver_name
(driver_name: str) → pyflink.datastream.connectors.jdbc.JdbcConnectionOptions.JdbcConnectionOptionsBuilder[source]¶
-
with_password
(password: str) → pyflink.datastream.connectors.jdbc.JdbcConnectionOptions.JdbcConnectionOptionsBuilder[source]¶
-
with_url
(url: str) → pyflink.datastream.connectors.jdbc.JdbcConnectionOptions.JdbcConnectionOptionsBuilder[source]¶
-
with_user_name
(user_name: str) → pyflink.datastream.connectors.jdbc.JdbcConnectionOptions.JdbcConnectionOptionsBuilder[source]¶
-
-
class
-
class
pyflink.datastream.connectors.jdbc.
JdbcExecutionOptions
(j_jdbc_execution_options)[source]¶ JDBC sink batch options.
-
class
Builder
[source]¶ Builder for JdbcExecutionOptions.
-
with_batch_interval_ms
(interval_ms: int) → pyflink.datastream.connectors.jdbc.JdbcExecutionOptions.Builder[source]¶
-
with_batch_size
(size: int) → pyflink.datastream.connectors.jdbc.JdbcExecutionOptions.Builder[source]¶
-
with_max_retries
(max_retries: int) → pyflink.datastream.connectors.jdbc.JdbcExecutionOptions.Builder[source]¶
-
-
static
defaults
() → pyflink.datastream.connectors.jdbc.JdbcExecutionOptions[source]¶
-
class
-
class
pyflink.datastream.connectors.jdbc.
JdbcSink
(j_jdbc_sink)[source]¶ -
static
sink
(sql: str, type_info: pyflink.common.typeinfo.RowTypeInfo, jdbc_connection_options: pyflink.datastream.connectors.jdbc.JdbcConnectionOptions, jdbc_execution_options: Optional[pyflink.datastream.connectors.jdbc.JdbcExecutionOptions] = None)[source]¶ Create a JDBC sink.
- Parameters
sql – arbitrary DML query (e.g. insert, update, upsert)
type_info – A RowTypeInfo for query field types.
jdbc_execution_options – parameters of execution, such as batch size and maximum retries.
jdbc_connection_options – parameters of connection, such as JDBC URL.
- Returns
A JdbcSink.
-
static
pyflink.datastream.connectors.elasticsearch module¶
-
class
pyflink.datastream.connectors.elasticsearch.
Elasticsearch6SinkBuilder
[source]¶ Builder to construct an Elasticsearch 6 compatible ElasticsearchSink.
The following example shows the minimal setup to create a ElasticsearchSink that submits actions on checkpoint or the default number of actions was buffered (1000).
Example:
>>> sink = Elasticsearch6SinkBuilder() \ ... .set_hosts('localhost:9200') \ ... .set_emitter(ElasticsearchEmitter.static_index("user", "key_col")) \ ... .build()
-
class
pyflink.datastream.connectors.elasticsearch.
Elasticsearch7SinkBuilder
[source]¶ Builder to construct an Elasticsearch 7 compatible ElasticsearchSink.
The following example shows the minimal setup to create a ElasticsearchSink that submits actions on checkpoint or the default number of actions was buffered (1000).
Example:
>>> sink = Elasticsearch7SinkBuilder() \ ... .set_hosts('localhost:9200') \ ... .set_emitter(ElasticsearchEmitter.dynamic_index("index_col", "key_col")) \ ... .build()
-
class
pyflink.datastream.connectors.elasticsearch.
ElasticsearchEmitter
(j_emitter)[source]¶ Emitter which is used by sinks to prepare elements for sending them to Elasticsearch.
-
static
dynamic_index
(index_field: str, key_field: str = None, doc_type: str = None) → pyflink.datastream.connectors.elasticsearch.ElasticsearchEmitter[source]¶ Creates an emitter with dynamic index which is invoked on every record to convert it to Elasticsearch actions.
-
static
static_index
(index: str, key_field: str = None, doc_type: str = None) → pyflink.datastream.connectors.elasticsearch.ElasticsearchEmitter[source]¶ Creates an emitter with static index which is invoked on every record to convert it to Elasticsearch actions.
-
static
-
class
pyflink.datastream.connectors.elasticsearch.
ElasticsearchSink
(j_elasticsearch_sink)[source]¶ Flink Sink to insert or update data in an Elasticsearch index. The sink supports the following delivery guarantees.
DeliveryGuarantee.NONE does not provide any guarantees: actions are flushed to Elasticsearch only depending on the configurations of the bulk processor. In case of a failure, it might happen that actions are lost if the bulk processor still has buffered actions.
DeliveryGuarantee.AT_LEAST_ONCE on a checkpoint the sink will wait until all buffered actions are flushed to and acknowledged by Elasticsearch. No actions will be lost but actions might be sent to Elasticsearch multiple times when Flink restarts. These additional requests may cause inconsistent data in ElasticSearch right after the restart, but eventually everything will be consistent again.
-
class
pyflink.datastream.connectors.elasticsearch.
FlushBackoffType
(value)[source]¶ Used to control whether the sink should retry failed requests at all or with which kind back off strategy.
- Data
CONSTANT:
After every failure, it waits a configured time until the retries are exhausted.
- Data
EXPONENTIAL:
After every failure, it waits initially the configured time and increases the waiting time exponentially until the retries are exhausted.
- Data
NONE:
The failure is not retried.
-
CONSTANT
= (0,)¶
-
EXPONENTIAL
= (1,)¶
-
NONE
= (2,)¶
pyflink.datastream.connectors.rabbitmq module¶
-
class
pyflink.datastream.connectors.rabbitmq.
RMQConnectionConfig
(j_rmq_connection_config)[source]¶ Connection Configuration for RMQ.
-
class
Builder
[source]¶ Builder for RMQConnectionConfig.
-
set_automatic_recovery
(automatic_recovery: bool) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_connection_timeout
(connection_timeout: int) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_host
(host: str) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_network_recovery_interval
(network_recovery_interval: int) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_password
(password: str) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_port
(port: int) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_prefetch_count
(prefetch_count: int) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_requested_channel_max
(requested_channel_max: int) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_requested_frame_max
(requested_frame_max: int) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_requested_heartbeat
(requested_heartbeat: int) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_topology_recovery_enabled
(topology_recovery_enabled: bool) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_uri
(uri: str) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_user_name
(user_name: str) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
set_virtual_host
(vhost: str) → pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig.Builder[source]¶
-
-
class
-
class
pyflink.datastream.connectors.rabbitmq.
RMQSink
(connection_config: pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig, queue_name: str, serialization_schema: pyflink.common.serialization.SerializationSchema)[source]¶
-
class
pyflink.datastream.connectors.rabbitmq.
RMQSource
(connection_config: pyflink.datastream.connectors.rabbitmq.RMQConnectionConfig, queue_name: str, use_correlation_id: bool, deserialization_schema: pyflink.common.serialization.DeserializationSchema)[source]¶
pyflink.datastream.connectors.pulsar module¶
-
class
pyflink.datastream.connectors.pulsar.
MessageDelayer
(_j_message_delayer)[source]¶ A delayer for Pulsar broker passing the sent message to the downstream consumer. This is only works in
SubscriptionType.Shared
subscription.Read delayed message delivery https://pulsar.apache.org/docs/en/next/concepts-messaging/#delayed-message-delivery for better understanding this feature.
-
static
fixed
(duration: pyflink.common.time.Duration) → pyflink.datastream.connectors.pulsar.MessageDelayer[source]¶ All the messages should be consumed in a fixed duration.
-
static
never
() → pyflink.datastream.connectors.pulsar.MessageDelayer[source]¶ All the messages should be consumed immediately.
-
static
-
class
pyflink.datastream.connectors.pulsar.
PulsarDeserializationSchema
(_j_pulsar_deserialization_schema)[source]¶ A schema bridge for deserializing the pulsar’s Message into a flink managed instance. We support both the pulsar’s self managed schema and flink managed schema.
-
static
flink_schema
(deserialization_schema: pyflink.common.serialization.DeserializationSchema) → pyflink.datastream.connectors.pulsar.PulsarDeserializationSchema[source]¶ Create a PulsarDeserializationSchema by using the flink’s DeserializationSchema. It would consume the pulsar message as byte array and decode the message by using flink’s logic.
-
static
flink_type_info
(type_information: pyflink.common.typeinfo.TypeInformation, execution_config: pyflink.common.execution_config.ExecutionConfig = None) → pyflink.datastream.connectors.pulsar.PulsarDeserializationSchema[source]¶ Create a PulsarDeserializationSchema by using the given TypeInformation. This method is only used for treating message that was written into pulsar by TypeInformation.
-
static
-
class
pyflink.datastream.connectors.pulsar.
PulsarSerializationSchema
(_j_pulsar_serialization_schema)[source]¶ The serialization schema for how to serialize records into Pulsar.
-
static
flink_schema
(serialization_schema: pyflink.common.serialization.SerializationSchema) → pyflink.datastream.connectors.pulsar.PulsarSerializationSchema[source]¶ Create a PulsarSerializationSchema by using the flink’s SerializationSchema. It would serialize the message into byte array and send it to Pulsar with Schema#BYTES.
-
static
-
class
pyflink.datastream.connectors.pulsar.
PulsarSink
(j_pulsar_sink)[source]¶ The Sink implementation of Pulsar. Please use a PulsarSinkBuilder to construct a PulsarSink. The following example shows how to create a PulsarSink receiving records of String type.
Example:
>>> sink = PulsarSink.builder() \ ... .set_service_url(PULSAR_BROKER_URL) \ ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \ ... .set_topics(topic) \ ... .set_serialization_schema( ... PulsarSerializationSchema.flink_schema(SimpleStringSchema())) \ ... .build()
The sink supports all delivery guarantees described by DeliveryGuarantee.
DeliveryGuarantee#NONE does not provide any guarantees: messages may be lost in case of issues on the Pulsar broker and messages may be duplicated in case of a Flink failure.
DeliveryGuarantee#AT_LEAST_ONCE the sink will wait for all outstanding records in the Pulsar buffers to be acknowledged by the Pulsar producer on a checkpoint. No messages will be lost in case of any issue with the Pulsar brokers but messages may be duplicated when Flink restarts.
DeliveryGuarantee#EXACTLY_ONCE: In this mode the PulsarSink will write all messages in a Pulsar transaction that will be committed to Pulsar on a checkpoint. Thus, no duplicates will be seen in case of a Flink restart. However, this delays record writing effectively until a checkpoint is written, so adjust the checkpoint duration accordingly. Additionally, it is highly recommended to tweak Pulsar transaction timeout (link) >> maximum checkpoint duration + maximum restart duration or data loss may happen when Pulsar expires an uncommitted transaction.
See PulsarSinkBuilder for more details.
-
static
builder
() → pyflink.datastream.connectors.pulsar.PulsarSinkBuilder[source]¶ Get a PulsarSinkBuilder to builder a PulsarSink.
-
static
-
class
pyflink.datastream.connectors.pulsar.
PulsarSinkBuilder
[source]¶ The builder class for PulsarSink to make it easier for the users to construct a PulsarSink.
The following example shows the minimum setup to create a PulsarSink that reads the String values from a Pulsar topic.
Example:
>>> sink = PulsarSink.builder() \ ... .set_service_url(PULSAR_BROKER_URL) \ ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \ ... .set_topics([TOPIC1, TOPIC2]) \ ... .set_serialization_schema( ... PulsarSerializationSchema.flink_schema(SimpleStringSchema())) \ ... .build()
The service url, admin url, and the record serializer are required fields that must be set. If you don’t set the topics, make sure you have provided a custom TopicRouter. Otherwise, you must provide the topics to produce.
To specify the delivery guarantees of PulsarSink, one can call #setDeliveryGuarantee(DeliveryGuarantee). The default value of the delivery guarantee is DeliveryGuarantee#NONE, and it wouldn’t promise the consistence when write the message into Pulsar.
Example:
>>> sink = PulsarSink.builder() \ ... .set_service_url(PULSAR_BROKER_URL) \ ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \ ... .set_topics([TOPIC1, TOPIC2]) \ ... .set_serialization_schema( ... PulsarSerializationSchema.flink_schema(SimpleStringSchema())) \ ... .set_delivery_guarantee(DeliveryGuarantee.EXACTLY_ONCE) ... .build()
-
build
() → pyflink.datastream.connectors.pulsar.PulsarSink[source]¶ Build the PulsarSink.
-
delay_sending_message
(message_delayer: pyflink.datastream.connectors.pulsar.MessageDelayer) → pyflink.datastream.connectors.pulsar.PulsarSinkBuilder[source]¶ Set a message delayer for enable Pulsar message delay delivery.
-
set_admin_url
(admin_url: str) → pyflink.datastream.connectors.pulsar.PulsarSinkBuilder[source]¶ Sets the admin endpoint for the PulsarAdmin of the PulsarSink.
-
set_config
(key: str, value) → pyflink.datastream.connectors.pulsar.PulsarSinkBuilder[source]¶ Set an arbitrary property for the PulsarSink and Pulsar Producer. The valid keys can be found in PulsarSinkOptions and PulsarOptions.
Make sure the option could be set only once or with same value.
-
set_delivery_guarantee
(delivery_guarantee: pyflink.datastream.connectors.base.DeliveryGuarantee) → pyflink.datastream.connectors.pulsar.PulsarSinkBuilder[source]¶ Sets the wanted the DeliveryGuarantee. The default delivery guarantee is DeliveryGuarantee#NONE.
-
set_producer_name
(producer_name: str) → pyflink.datastream.connectors.pulsar.PulsarSinkBuilder[source]¶ The producer name is informative, and it can be used to identify a particular producer instance from the topic stats.
-
set_properties
(config: Dict) → pyflink.datastream.connectors.pulsar.PulsarSinkBuilder[source]¶ Set an arbitrary property for the PulsarSink and Pulsar Producer. The valid keys can be found in PulsarSinkOptions and PulsarOptions.
-
set_serialization_schema
(pulsar_serialization_schema: pyflink.datastream.connectors.pulsar.PulsarSerializationSchema) → pyflink.datastream.connectors.pulsar.PulsarSinkBuilder[source]¶ Sets the PulsarSerializationSchema that transforms incoming records to bytes.
-
set_service_url
(service_url: str) → pyflink.datastream.connectors.pulsar.PulsarSinkBuilder[source]¶ Sets the server’s link for the PulsarProducer of the PulsarSink.
-
set_topic_router
(topic_router_class_name: str) → pyflink.datastream.connectors.pulsar.PulsarSinkBuilder[source]¶ Use a custom topic router instead predefine topic routing.
-
set_topic_routing_mode
(topic_routing_mode: pyflink.datastream.connectors.pulsar.TopicRoutingMode) → pyflink.datastream.connectors.pulsar.PulsarSinkBuilder[source]¶ Set a routing mode for choosing right topic partition to send messages.
-
set_topics
(topics: Union[str, List[str]]) → pyflink.datastream.connectors.pulsar.PulsarSinkBuilder[source]¶ Set a pulsar topic list for flink sink. Some topic may not exist currently, write to this non-existed topic wouldn’t throw any exception.
-
-
class
pyflink.datastream.connectors.pulsar.
PulsarSource
(j_pulsar_source)[source]¶ The Source implementation of Pulsar. Please use a PulsarSourceBuilder to construct a PulsarSource. The following example shows how to create a PulsarSource emitting records of String type.
Example:
>>> source = PulsarSource() \ ... .builder() \ ... .set_topics([TOPIC1, TOPIC2]) \ ... .set_service_url(get_service_url()) \ ... .set_admin_url(get_admin_url()) \ ... .set_subscription_name("test") \ ... .set_deserialization_schema( ... PulsarDeserializationSchema.flink_schema(SimpleStringSchema())) \ ... .set_bounded_stop_cursor(StopCursor.default_stop_cursor()) \ ... .build()
See PulsarSourceBuilder for more details.
-
static
builder
() → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ Get a PulsarSourceBuilder to builder a PulsarSource.
-
static
-
class
pyflink.datastream.connectors.pulsar.
PulsarSourceBuilder
[source]¶ The builder class for PulsarSource to make it easier for the users to construct a PulsarSource.
The following example shows the minimum setup to create a PulsarSource that reads the String values from a Pulsar topic.
Example:
>>> source = PulsarSource() \ ... .builder() \ ... .set_service_url(PULSAR_BROKER_URL) \ ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \ ... .set_subscription_name("flink-source-1") \ ... .set_topics([TOPIC1, TOPIC2]) \ ... .set_deserialization_schema( ... PulsarDeserializationSchema.flink_schema(SimpleStringSchema())) \ ... .build()
The service url, admin url, subscription name, topics to consume, and the record deserializer are required fields that must be set.
To specify the starting position of PulsarSource, one can call set_start_cursor(StartCursor).
By default the PulsarSource runs in an Boundedness.CONTINUOUS_UNBOUNDED mode and never stop until the Flink job is canceled or fails. To let the PulsarSource run in Boundedness.CONTINUOUS_UNBOUNDED but stops at some given offsets, one can call set_unbounded_stop_cursor(StopCursor).
For example the following PulsarSource stops after it consumes up to a event time when the Flink started.
Example:
>>> source = PulsarSource() \ ... .builder() \ ... .set_service_url(PULSAR_BROKER_URL) \ ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \ ... .set_subscription_name("flink-source-1") \ ... .set_topics([TOPIC1, TOPIC2]) \ ... .set_deserialization_schema( ... PulsarDeserializationSchema.flink_schema(SimpleStringSchema())) \ ... .set_bounded_stop_cursor(StopCursor.at_publish_time(int(time.time() * 1000))) ... .build()
-
build
() → pyflink.datastream.connectors.pulsar.PulsarSource[source]¶ Build the PulsarSource.
-
set_admin_url
(admin_url: str) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ Sets the admin endpoint for the PulsarAdmin of the PulsarSource.
-
set_bounded_stop_cursor
(stop_cursor: pyflink.datastream.connectors.pulsar.StopCursor) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ By default the PulsarSource is set to run in Boundedness.CONTINUOUS_UNBOUNDED manner and thus never stops until the Flink job fails or is canceled. To let the PulsarSource run in Boundedness.BOUNDED manner and stops at some point, one can set an StopCursor to specify the stopping offsets for each partition. When all the partitions have reached their stopping offsets, the PulsarSource will then exit.
This method is different from set_unbounded_stop_cursor(StopCursor) that after setting the stopping offsets with this method, PulsarSource.getBoundedness() will return Boundedness.BOUNDED instead of Boundedness.CONTINUOUS_UNBOUNDED.
-
set_config
(key: Union[str, pyflink.common.config_options.ConfigOption], value) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ Set arbitrary properties for the PulsarSource and PulsarConsumer. The valid keys can be found in PulsarSourceOptions and PulsarOptions.
Make sure the option could be set only once or with same value.
-
set_config_with_dict
(config: Dict) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ Set arbitrary properties for the PulsarSource and PulsarConsumer. The valid keys can be found in PulsarSourceOptions and PulsarOptions.
-
set_deserialization_schema
(pulsar_deserialization_schema: pyflink.datastream.connectors.pulsar.PulsarDeserializationSchema) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ DeserializationSchema is required for getting the Schema for deserialize message from pulsar and getting the TypeInformation for message serialization in flink.
We have defined a set of implementations, using PulsarDeserializationSchema#flink_type_info or PulsarDeserializationSchema#flink_schema for creating the desired schema.
-
set_properties
(config: Dict) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ Set arbitrary properties for the PulsarSource and PulsarConsumer. The valid keys can be found in PulsarSourceOptions and PulsarOptions.
-
set_service_url
(service_url: str) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ Sets the server’s link for the PulsarConsumer of the PulsarSource.
-
set_start_cursor
(start_cursor: pyflink.datastream.connectors.pulsar.StartCursor) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ Specify from which offsets the PulsarSource should start consume from by providing an StartCursor.
-
set_subscription_name
(subscription_name: str) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ Sets the name for this pulsar subscription.
-
set_subscription_type
(subscription_type: pyflink.datastream.connectors.pulsar.SubscriptionType) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ SubscriptionType is the consuming behavior for pulsar, we would generator different split by the given subscription type. Please take some time to consider which subscription type matches your application best. Default is SubscriptionType.Shared.
-
set_topic_pattern
(topic_pattern: str) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ Set a topic pattern to consume from the java regex str. You can set topics once either with set_topics or set_topic_pattern in this builder.
-
set_topics
(topics: Union[str, List[str]]) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ Set a pulsar topic list for flink source. Some topic may not exist currently, consuming this non-existed topic wouldn’t throw any exception. But the best solution is just consuming by using a topic regex. You can set topics once either with setTopics or setTopicPattern in this builder.
-
set_topics_pattern
(topics_pattern: str) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ Set a topic pattern to consume from the java regex str. You can set topics once either with set_topics or set_topic_pattern in this builder.
-
set_unbounded_stop_cursor
(stop_cursor: pyflink.datastream.connectors.pulsar.StopCursor) → pyflink.datastream.connectors.pulsar.PulsarSourceBuilder[source]¶ By default the PulsarSource is set to run in Boundedness.CONTINUOUS_UNBOUNDED manner and thus never stops until the Flink job fails or is canceled. To let the PulsarSource run as a streaming source but still stops at some point, one can set an StopCursor to specify the stopping offsets for each partition. When all the partitions have reached their stopping offsets, the PulsarSource will then exit.
This method is different from set_bounded_stop_cursor(StopCursor) that after setting the stopping offsets with this method, PulsarSource.getBoundedness() will still return Boundedness.CONTINUOUS_UNBOUNDED even though it will stop at the stopping offsets specified by the stopping offsets StopCursor.
-
-
class
pyflink.datastream.connectors.pulsar.
StartCursor
(_j_start_cursor)[source]¶ A factory class for users to specify the start position of a pulsar subscription. Since it would be serialized into split. The implementation for this interface should be well considered. I don’t recommend adding extra internal state for this implementation.
This class would be used only for SubscriptionType.Exclusive and SubscriptionType.Failover.
-
static
default_start_cursor
() → pyflink.datastream.connectors.pulsar.StartCursor[source]¶
-
static
earliest
() → pyflink.datastream.connectors.pulsar.StartCursor[source]¶
-
static
from_message_id
(message_id: bytes, inclusive: bool = True) → pyflink.datastream.connectors.pulsar.StartCursor[source]¶ Find the available message id and start consuming from it. User could call pulsar Python library serialize method to cover messageId bytes.
Example:
>>> from pulsar import MessageId >>> message_id_bytes = MessageId().serialize() >>> start_cursor = StartCursor.from_message_id(message_id_bytes)
-
static
from_message_time
(timestamp: int) → pyflink.datastream.connectors.pulsar.StartCursor[source]¶ This method is designed for seeking message from event time. But Pulsar didn’t support seeking from message time, instead, it would seek the position from publish time. We only keep this method for backward compatible.
-
static
from_publish_time
(timestamp: int) → pyflink.datastream.connectors.pulsar.StartCursor[source]¶ Seek the start position by using message publish time.
-
static
latest
() → pyflink.datastream.connectors.pulsar.StartCursor[source]¶
-
static
-
class
pyflink.datastream.connectors.pulsar.
StopCursor
(_j_stop_cursor)[source]¶ A factory class for users to specify the stop position of a pulsar subscription. Since it would be serialized into split. The implementation for this interface should be well considered. I don’t recommend adding extra internal state for this implementation.
-
static
after_event_time
(timestamp: int) → pyflink.datastream.connectors.pulsar.StopCursor[source]¶ Stop consuming when message eventTime is greater than the specified timestamp.
-
static
after_message_id
(message_id: bytes) → pyflink.datastream.connectors.pulsar.StopCursor[source]¶ Stop consuming when the messageId is greater than the specified messageId. Message that is equal to the specified messageId will be consumed. User could call pulsar Python library serialize method to cover messageId bytes.
Example:
>>> from pulsar import MessageId >>> message_id_bytes = MessageId().serialize() >>> stop_cursor = StopCursor.after_message_id(message_id_bytes)
-
static
after_publish_time
(timestamp: int) → pyflink.datastream.connectors.pulsar.StopCursor[source]¶ Stop consuming when message publishTime is greater than the specified timestamp.
-
static
at_event_time
(timestamp: int) → pyflink.datastream.connectors.pulsar.StopCursor[source]¶ Stop consuming when message eventTime is greater than or equals the specified timestamp.
-
static
at_message_id
(message_id: bytes) → pyflink.datastream.connectors.pulsar.StopCursor[source]¶ Stop consuming when the messageId is equal or greater than the specified messageId. Message that is equal to the specified messageId will not be consumed. User could call pulsar Python library serialize method to cover messageId bytes.
Example:
>>> from pulsar import MessageId >>> message_id_bytes = MessageId().serialize() >>> stop_cursor = StopCursor.at_message_id(message_id_bytes)
-
static
at_publish_time
(timestamp: int) → pyflink.datastream.connectors.pulsar.StopCursor[source]¶ Stop consuming when message publishTime is greater than or equals the specified timestamp.
-
static
default_stop_cursor
() → pyflink.datastream.connectors.pulsar.StopCursor[source]¶
-
static
latest
() → pyflink.datastream.connectors.pulsar.StopCursor[source]¶
-
static
never
() → pyflink.datastream.connectors.pulsar.StopCursor[source]¶
-
static
-
class
pyflink.datastream.connectors.pulsar.
SubscriptionType
(value)[source]¶ Types of subscription supported by Pulsar.
- Data
Exclusive:
There can be only 1 consumer on the same topic with the same subscription name.
- Data
Shared:
Multiple consumer will be able to use the same subscription name and the messages will be dispatched according to a round-robin rotation between the connected consumers. In this mode, the consumption order is not guaranteed.
- Data
Failover:
Multiple consumer will be able to use the same subscription name but only 1 consumer will receive the messages. If that consumer disconnects, one of the other connected consumers will start receiving messages. In failover mode, the consumption ordering is guaranteed. In case of partitioned topics, the ordering is guaranteed on a per-partition basis. The partitions assignments will be split across the available consumers. On each partition, at most one consumer will be active at a given point in time.
- Data
Key_Shared:
Multiple consumer will be able to use the same subscription and all messages with the same key will be dispatched to only one consumer. Use ordering_key to overwrite the message key for message ordering.
-
Exclusive
= (0,)¶
-
Failover
= (2,)¶
-
class
pyflink.datastream.connectors.pulsar.
TopicRoutingMode
(value)[source]¶ The routing policy for choosing the desired topic by the given message.
- Data
ROUND_ROBIN:
The producer will publish messages across all partitions in a round-robin fashion to achieve maximum throughput. Please note that round-robin is not done per individual message but rather it’s set to the same boundary of batching delay, to ensure batching is effective.
- Data
MESSAGE_KEY_HASH:
If no key is provided, The partitioned producer will randomly pick one single topic partition and publish all the messages into that partition. If a key is provided on the message, the partitioned producer will hash the key and assign the message to a particular partition.
- Data
CUSTOM:
Use custom topic router implementation that will be called to determine the partition for a particular message.
-
CUSTOM
= 2¶
-
MESSAGE_KEY_HASH
= 1¶
-
ROUND_ROBIN
= 0¶
pyflink.datastream.connectors.hybrid_source module¶
-
class
pyflink.datastream.connectors.hybrid_source.
HybridSource
(j_hybrid_source: py4j.java_gateway.JavaObject)[source]¶ Hybrid source that switches underlying sources based on configured source chain.
A simple example with FileSource and KafkaSource with fixed Kafka start position:
>>> file_source = FileSource \ ... .for_record_stream_format(StreamFormat.text_line_format(), test_dir) \ ... .build() >>> kafka_source = KafkaSource \ ... .builder() \ ... .set_bootstrap_servers('localhost:9092') \ ... .set_group_id('MY_GROUP') \ ... .set_topics('quickstart-events') \ ... .set_value_only_deserializer(SimpleStringSchema()) \ ... .set_starting_offsets(KafkaOffsetsInitializer.earliest()) \ ... .build() >>> hybrid_source = HybridSource.builder(file_source).add_source(kafka_source).build()
-
static
builder
(first_source: pyflink.datastream.connectors.base.Source) → pyflink.datastream.connectors.hybrid_source.HybridSourceBuilder[source]¶
-
static
-
class
pyflink.datastream.connectors.hybrid_source.
HybridSourceBuilder
(j_hybrid_source_builder)[source]¶ -
add_source
(source: pyflink.datastream.connectors.base.Source) → pyflink.datastream.connectors.hybrid_source.HybridSourceBuilder[source]¶
-
pyflink.datastream.connectors.cassandra module¶
-
class
pyflink.datastream.connectors.cassandra.
CassandraCommitter
(j_checkpoint_committer)[source]¶ CheckpointCommitter that saves information about completed checkpoints within a separate table in a cassandra database.
-
static
default_checkpoint_committer
(builder: pyflink.datastream.connectors.cassandra.ClusterBuilder, key_space: str = None) → pyflink.datastream.connectors.cassandra.CassandraCommitter[source]¶ CheckpointCommitter that saves information about completed checkpoints within a separate table in a cassandra database.
Entries are in the form: | operator_id | subtask_id | last_completed_checkpoint |
-
static
-
class
pyflink.datastream.connectors.cassandra.
CassandraFailureHandler
(j_cassandra_failure_handler)[source]¶ Handle a failed Throwable.
-
static
no_op
() → pyflink.datastream.connectors.cassandra.CassandraFailureHandler[source]¶ A CassandraFailureHandler that simply fails the sink on any failures. This is also the default failure handler if not specified.
-
static
-
class
pyflink.datastream.connectors.cassandra.
CassandraSink
(j_cassandra_sink)[source]¶ Sets the ClusterBuilder for this sink. A ClusterBuilder is used to configure the connection to cassandra.
-
class
CassandraSinkBuilder
(j_cassandra_sink_builder)[source]¶ Builder for a CassandraSink.
-
build
() → pyflink.datastream.connectors.cassandra.CassandraSink[source]¶ Finalizes the configuration of this sink.
-
enable_ignore_null_fields
() → pyflink.datastream.connectors.cassandra.CassandraSink.CassandraSinkBuilder[source]¶ Enables ignoring null values, treats null values as unset and avoids writing null fields and creating tombstones. This call has no effect if CassandraSinkBuilder.enableWriteAheadLog() is called.
-
enable_write_ahead_log
(committer: pyflink.datastream.connectors.cassandra.CassandraCommitter = None) → pyflink.datastream.connectors.cassandra.CassandraSink.CassandraSinkBuilder[source]¶ Enables the write-ahead log, which allows exactly-once processing for non-deterministic algorithms that use idempotent updates.
-
set_cluster_builder
(builder: pyflink.datastream.connectors.cassandra.ClusterBuilder) → pyflink.datastream.connectors.cassandra.CassandraSink.CassandraSinkBuilder[source]¶ Sets the ClusterBuilder for this sink. A ClusterBuilder is used to configure the connection to cassandra.
-
set_failure_handler
(failure_handler: pyflink.datastream.connectors.cassandra.CassandraFailureHandler) → pyflink.datastream.connectors.cassandra.CassandraSink.CassandraSinkBuilder[source]¶ Sets the failure handler for this sink. The failure handler is used to provide custom error handling.
-
set_host
(host: str, port: int = 9042) → pyflink.datastream.connectors.cassandra.CassandraSink.CassandraSinkBuilder[source]¶ Sets the cassandra host/port to connect to.
-
set_mapper_options
(options: pyflink.datastream.connectors.cassandra.MapperOptions) → pyflink.datastream.connectors.cassandra.CassandraSink.CassandraSinkBuilder[source]¶ Sets the mapper options for this sink. The mapper options are used to configure the DataStax com.datastax.driver.mapping.Mapper when writing POJOs. This call has no effect if the input DataStream for this sink does not contain POJOs.
-
set_max_concurrent_requests
(max_concurrent_requests: int, duration: pyflink.common.time.Duration = None) → pyflink.datastream.connectors.cassandra.CassandraSink.CassandraSinkBuilder[source]¶ Sets the maximum allowed number of concurrent requests for this sink.
-
set_query
(query: str) → pyflink.datastream.connectors.cassandra.CassandraSink.CassandraSinkBuilder[source]¶ Sets the query that is to be executed for every record.
-
-
static
add_sink
(input) → CassandraSinkBuilder[source]¶ Writes a DataStream into a Cassandra database.
-
disable_chaining
() → pyflink.datastream.connectors.cassandra.CassandraSink[source]¶ Turns off chaining for this operator so thread co-location will not be used as an optimization.
-
name
(name: str) → pyflink.datastream.connectors.cassandra.CassandraSink[source]¶ Set the name of this sink. This name is used by the visualization and logging during runtime.
-
set_parallelism
(parallelism: int) → pyflink.datastream.connectors.cassandra.CassandraSink[source]¶ Sets the parallelism for this sink. The degree must be higher than zero.
-
set_uid_hash
(uid_hash: str) → pyflink.datastream.connectors.cassandra.CassandraSink[source]¶ Sets an user provided hash for this operator. This will be used AS IS the create the JobVertexID.
The user provided hash is an alternative to the generated hashes, that is considered when identifying an operator through the default hash mechanics fails (e.g. because of changes between Flink versions).
Note that this should be used as a workaround or for trouble shooting. The provided hash needs to be unique per transformation and job. Otherwise, job submission will fail. Furthermore, you cannot assign user-specified hash to intermediate nodes in an operator chain and trying so will let your job fail.
A use case for this is in migration between Flink versions or changing the jobs in a way that changes the automatically generated hashes. In this case, providing the previous hashes directly through this method (e.g. obtained from old logs) can help to reestablish a lost mapping from states to their target operator.
-
slot_sharing_group
(slot_sharing_group: str) → pyflink.datastream.connectors.cassandra.CassandraSink[source]¶ Sets the slot sharing group of this operation. Parallel instances of operations that are in the same slot sharing group will be co-located in the same TaskManager slot, if possible.
Operations inherit the slot sharing group of input operations if all input operations are in the same slot sharing group and no slot sharing group was explicitly specified.
Initially an operation is in the default slot sharing group. An operation can be put into the default group explicitly by setting the slot sharing group to {@code “default”}.
-
uid
(uid: str) → pyflink.datastream.connectors.cassandra.CassandraSink[source]¶ Sets an ID for this operator. The specified ID is used to assign the same operator ID across job submissions (for example when starting a job from a savepoint). Note that this ID needs to be unique per transformation and job. Otherwise, job submission will fail.
-
class
-
class
pyflink.datastream.connectors.cassandra.
ClusterBuilder
(j_cluster_builder)[source]¶ This class is used to configure a Cluster after deployment. The cluster represents the connection that will be established to Cassandra.
-
class
pyflink.datastream.connectors.cassandra.
ConsistencyLevel
(value)[source]¶ The consistency level
-
ALL
= 5¶
-
ANY
= 0¶
-
EACH_QUORUM
= 7¶
-
LOCAL_ONE
= 10¶
-
LOCAL_QUORUM
= 6¶
-
LOCAL_SERIAL
= 9¶
-
ONE
= 1¶
-
QUORUM
= 4¶
-
SERIAL
= 8¶
-
THREE
= 3¶
-
TWO
= 2¶
-
-
class
pyflink.datastream.connectors.cassandra.
MapperOptions
[source]¶ This class is used to configure a Mapper after deployment.
-
consistency_level
(cl: pyflink.datastream.connectors.cassandra.ConsistencyLevel) → pyflink.datastream.connectors.cassandra.MapperOptions[source]¶ Creates a new Option object to add a consistency level value to a mapper operation. This is valid for save, delete and get operations.
-
if_not_exists
(enabled: bool) → pyflink.datastream.connectors.cassandra.MapperOptions[source]¶ Creates a new Option object to specify whether an IF NOT EXISTS clause should be included in insert queries. This option is valid only for save operations.
If this option is not specified, it defaults to false (IF NOT EXISTS statements are not used).
-
save_null_fields
(enabled: bool) → pyflink.datastream.connectors.cassandra.MapperOptions[source]¶ Creates a new Option object to specify whether null entity fields should be included in insert queries. This option is valid only for save operations.
-
timestamp
(timestamp: int) → pyflink.datastream.connectors.cassandra.MapperOptions[source]¶ Creates a new Option object to add a timestamp to a mapper operation. This is only valid for save and delete operations.
-
tracing
(enabled: bool) → pyflink.datastream.connectors.cassandra.MapperOptions[source]¶ Creates a new Option object to enable query tracing for a mapper operation. This is valid for save, delete and get operations.
-
ttl
(ttl: int) → pyflink.datastream.connectors.cassandra.MapperOptions[source]¶ Creates a new Option object to add time-to-live to a mapper operation. This is only valid for save operations.
-
pyflink.datastream.formats.csv module¶
-
class
pyflink.datastream.formats.csv.
CsvBulkWriters
[source]¶ CsvBulkWriter is for building
BulkWriterFactory
to write Rows with a predefined CSV schema to partitioned files in a bulk fashion.Example:
>>> schema = CsvSchema.builder() \ ... .add_number_column('id', number_type=DataTypes.INT()) \ ... .add_string_column('name') \ ... .add_array_column('list', ',', element_type=DataTypes.STRING()) \ ... .set_column_separator('|') \ ... .build() >>> sink = FileSink.for_bulk_format( ... OUTPUT_DIR, CsvBulkWriters.for_schema(schema)).build() >>> ds.sink_to(sink)
New in version 1.16.0.
-
static
for_schema
(schema: pyflink.datastream.formats.csv.CsvSchema) → pyflink.common.serialization.BulkWriterFactory[source]¶ Creates a
BulkWriterFactory
for writing records to files in CSV format.
-
static
-
class
pyflink.datastream.formats.csv.
CsvReaderFormat
(j_csv_format)[source]¶ The
StreamFormat
for reading csv files.Example:
>>> schema = CsvSchema.builder() \ ... .add_number_column('id', number_type=DataTypes.INT()) \ ... .add_string_column('name') \ ... .add_array_column('list', ',', element_type=DataTypes.STRING()) \ ... .set_column_separator('|') \ ... .set_escape_char('\\') \ ... .set_use_header() \ ... .set_strict_headers() \ ... .build() >>> source = FileSource.for_record_stream_format( ... CsvReaderFormat.for_schema(schema), CSV_FILE_PATH).build() >>> ds = env.from_source(source, WatermarkStrategy.no_watermarks(), 'csv-source') >>> # the type of records is Types.ROW_NAMED(['id', 'name', 'list'], >>> # [Types.INT(), Types.STRING(), Types.LIST(Types.STRING())])
New in version 1.16.0.
-
static
for_schema
(schema: pyflink.datastream.formats.csv.CsvSchema) → pyflink.datastream.formats.csv.CsvReaderFormat[source]¶ Builds a
CsvReaderFormat
using CsvSchema.
-
static
-
class
pyflink.datastream.formats.csv.
CsvRowDeserializationSchema
(j_deserialization_schema)[source]¶ Deserialization schema from CSV to Flink types. Deserializes a byte[] message as a JsonNode and converts it to Row.
Failure during deserialization are forwarded as wrapped IOException.
-
class
pyflink.datastream.formats.csv.
CsvRowSerializationSchema
(j_csv_row_serialization_schema)[source]¶ Serialization schema that serializes an object of Flink types into a CSV bytes. Serializes the input row into an ObjectNode and converts it into byte[].
Result byte[] messages can be deserialized using CsvRowDeserializationSchema.
-
class
pyflink.datastream.formats.csv.
CsvSchema
(j_schema, row_type: RowType)[source]¶ CsvSchema holds schema information of a csv file, corresponding to Java
com.fasterxml.jackson.dataformat.csv.CsvSchema
class.New in version 1.16.0.
-
static
builder
() → pyflink.datastream.formats.csv.CsvSchemaBuilder[source]¶ Returns a
CsvSchemaBuilder
.
-
static
-
class
pyflink.datastream.formats.csv.
CsvSchemaBuilder
[source]¶ CsvSchemaBuilder is for building a
CsvSchema
, corresponding to Javacom.fasterxml.jackson.dataformat.csv.CsvSchema.Builder
class.New in version 1.16.0.
-
add_array_column
(name: str, separator: str = ';', element_type: Optional[DataType] = None) → CsvSchemaBuilder[source]¶ Add an array column to schema, the type of elements could be specified via
element_type
, which should be primitive types.- Parameters
name – Name of the column.
separator – Text separator of array elements, default to
;
.element_type – DataType of array elements, default to
DataTypes.STRING()
.
-
add_boolean_column
(name: str) → pyflink.datastream.formats.csv.CsvSchemaBuilder[source]¶ Add a boolean column to schema, with type as
DataTypes.BOOLEAN()
.- Parameters
name – Name of the column.
-
add_columns_from
(schema: pyflink.datastream.formats.csv.CsvSchema) → pyflink.datastream.formats.csv.CsvSchemaBuilder[source]¶ Add all columns in
schema
to current schema.- Parameters
schema – Another
CsvSchema
.
-
add_number_column
(name: str, number_type: Optional[NumericType] = None) → CsvSchemaBuilder[source]¶ Add a number column to schema, the type of number could be specified via
number_type
.- Parameters
name – Name of the column.
number_type – DataType of the number, default to
DataTypes.BIGINT()
.
-
add_string_column
(name: str) → pyflink.datastream.formats.csv.CsvSchemaBuilder[source]¶ Add a string column to schema, with type as
DataTypes.STRING()
.- Parameters
name – Name of the column.
-
build
() → pyflink.datastream.formats.csv.CsvSchema[source]¶ Build the
CsvSchema
.
-
remove_array_element_separator
(index: int)[source]¶ Set array element separator of a column specified by
index
to""
.
-
set_array_element_separator
(separator: str)[source]¶ Set global array element separator, default to
;
.
-
set_column_separator
(char: str)[source]¶ Set column separator,
char
should be a single char, default to,
.
-
set_escape_char
(char: str)[source]¶ Set escape char,
char
should be a single char, default to no-escaping.
-
set_line_separator
(separator: str)[source]¶ Set line separator, default to
\n
. This is only configurable for writing, for reading,\n
,\r
,\r\n
are recognized.
-
pyflink.datastream.formats.avro module¶
-
class
pyflink.datastream.formats.avro.
AvroBulkWriters
[source]¶ Convenience builder to create
BulkWriterFactory
for Avro types.New in version 1.16.0.
-
static
for_generic_record
(schema: pyflink.datastream.formats.avro.AvroSchema) → pyflink.common.serialization.BulkWriterFactory[source]¶ Creates an AvroWriterFactory that accepts and writes Avro generic types. The Avro writers will use the given schema to build and write the records.
Note that to make this works in PyFlink, you need to declare the output type of the predecessor before FileSink to be
GenericRecordAvroTypeInfo
, and the predecessor cannot beStreamExecutionEnvironment.from_collection()
, you can add a pass-through map function before the sink, as the example shown below.The Python data records should match the Avro schema, and have the same behavior with vanilla Python data structure, e.g. an object for Avro array should behave like Python list, an object for Avro map should behave like Python dict.
Example:
>>> env = StreamExecutionEnvironment.get_execution_environment() >>> schema = AvroSchema(JSON_SCHEMA) >>> avro_type_info = GenericRecordAvroTypeInfo(schema) >>> ds = env.from_collection([{'array': [1, 2]}], type_info=Types.PICKLED_BYTE_ARRAY()) >>> sink = FileSink.for_bulk_format( ... OUTPUT_DIR, AvroBulkWriters.for_generic_record(schema)).build() >>> # A map to indicate its Avro type info is necessary for serialization >>> ds.map(lambda e: e, output_type=GenericRecordAvroTypeInfo(schema)) \ ... .sink_to(sink)
- Parameters
schema – The avro schema.
- Returns
The BulkWriterFactory to write generic records into avro files.
-
static
-
class
pyflink.datastream.formats.avro.
AvroInputFormat
(path: str, schema: pyflink.datastream.formats.avro.AvroSchema)[source]¶ Provides a FileInputFormat for Avro records.
Example:
>>> env = StreamExecutionEnvironment.get_execution_environment() >>> schema = AvroSchema.parse_string(JSON_SCHEMA) >>> ds = env.create_input(AvroInputFormat(FILE_PATH, schema))
New in version 1.16.0.
-
get_produced_type
() → pyflink.datastream.formats.avro.GenericRecordAvroTypeInfo[source]¶
-
-
class
pyflink.datastream.formats.avro.
AvroRowDeserializationSchema
(record_class: str = None, avro_schema_string: str = None)[source]¶ Deserialization schema from Avro bytes to Row. Deserializes the byte[] messages into (nested) Flink rows. It converts Avro types into types that are compatible with Flink’s Table & SQL API.
Projects with Avro records containing logical date/time types need to add a JodaTime dependency.
-
class
pyflink.datastream.formats.avro.
AvroRowSerializationSchema
(record_class: str = None, avro_schema_string: str = None)[source]¶ Serialization schema that serializes to Avro binary format.
-
class
pyflink.datastream.formats.avro.
AvroSchema
(j_schema)[source]¶ Avro Schema class contains Java org.apache.avro.Schema.
New in version 1.16.0.
-
static
parse_file
(file_path: str) → pyflink.datastream.formats.avro.AvroSchema[source]¶ Parse a schema definition file as Avro Schema.
- Parameters
file_path – path to schema definition file.
- Returns
the Avro Schema.
-
static
parse_string
(json_schema: str) → pyflink.datastream.formats.avro.AvroSchema[source]¶ Parse JSON string as Avro Schema.
- Parameters
json_schema – JSON represented schema string.
- Returns
the Avro Schema.
-
static
-
class
pyflink.datastream.formats.avro.
GenericRecordAvroTypeInfo
(schema: pyflink.datastream.formats.avro.AvroSchema)[source]¶ A
TypeInformation
of Avro’s GenericRecord, including the schema. This is a wrapper of Java org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo.Note that this type cannot be used as the type_info of data in
StreamExecutionEnvironment.from_collection()
.
pyflink.datastream.formats.parquet module¶
-
class
pyflink.datastream.formats.parquet.
AvroParquetReaders
[source]¶ A convenience builder to create reader format that reads individual Avro records from a Parquet stream. Only GenericRecord is supported in PyFlink.
New in version 1.16.0.
-
static
for_generic_record
(schema: pyflink.datastream.formats.avro.AvroSchema) → pyflink.datastream.connectors.file_system.StreamFormat[source]¶ Creates a new AvroParquetRecordFormat that reads the parquet file into Avro GenericRecords.
To read into GenericRecords, this method needs an Avro Schema. That is because Flink needs to be able to serialize the results in its data flow, which is very inefficient without the schema. And while the Schema is stored in the Avro file header, Flink needs this schema during ‘pre-flight’ time when the data flow is set up and wired, which is before there is access to the files.
Example:
>>> env = StreamExecutionEnvironment.get_execution_environment() >>> schema = AvroSchema.parse_string(JSON_SCHEMA) >>> source = FileSource.for_record_stream_format( ... AvroParquetReaders.for_generic_record(schema), ... PARQUET_FILE_PATH ... ).build() >>> ds = env.from_source(source, WatermarkStrategy.no_watermarks(), "parquet-source")
- Parameters
schema – the Avro Schema.
- Returns
StreamFormat for reading Avro GenericRecords.
-
static
-
class
pyflink.datastream.formats.parquet.
AvroParquetWriters
[source]¶ Convenient builder to create Parquet BulkWriterFactory instances for Avro types. Only GenericRecord is supported at present.
New in version 1.16.0.
-
static
for_generic_record
(schema: pyflink.datastream.formats.avro.AvroSchema) → pyflink.common.serialization.BulkWriterFactory[source]¶ Creates a ParquetWriterFactory that accepts and writes Avro generic types. The Parquet writers will use the given schema to build and write the columnar data.
Note that to make this works in PyFlink, you need to declare the output type of the predecessor before FileSink to be
GenericRecordAvroTypeInfo
, and the predecessor cannot beStreamExecutionEnvironment.from_collection()
, you can add a pass-through map function before the sink, as the example shown below.The Python data records should match the Avro schema, and have the same behavior with vanilla Python data structure, e.g. an object for Avro array should behave like Python list, an object for Avro map should behave like Python dict.
Example:
>>> env = StreamExecutionEnvironment.get_execution_environment() >>> schema = AvroSchema(JSON_SCHEMA) >>> avro_type_info = GenericRecordAvroTypeInfo(schema) >>> ds = env.from_collection([{'array': [1, 2]}], type_info=Types.PICKLED_BYTE_ARRAY()) >>> sink = FileSink.for_bulk_format( ... OUTPUT_DIR, AvroParquetWriters.for_generic_record(schema)).build() >>> # A map to indicate its Avro type info is necessary for serialization >>> ds.map(lambda e: e, output_type=GenericRecordAvroTypeInfo(schema)) \ ... .sink_to(sink)
- Parameters
schema – The avro schema.
- Returns
The BulkWriterFactory to write generic records into parquet files.
-
static
-
class
pyflink.datastream.formats.parquet.
ParquetBulkWriters
[source]¶ Convenient builder to create a
BulkWriterFactory
that writes records with a predefined schema into Parquet files in a batch fashion.Example:
>>> row_type = DataTypes.ROW([ ... DataTypes.FIELD('string', DataTypes.STRING()), ... DataTypes.FIELD('int_array', DataTypes.ARRAY(DataTypes.INT())) ... ]) >>> sink = FileSink.for_bulk_format( ... OUTPUT_DIR, ParquetBulkWriters.for_row_type( ... row_type, ... hadoop_config=Configuration(), ... utc_timestamp=True, ... ) ... ).build() >>> ds.sink_to(sink)
New in version 1.16.0.
-
static
for_row_type
(row_type: RowType, hadoop_config: Optional[pyflink.common.configuration.Configuration] = None, utc_timestamp: bool = False) → BulkWriterFactory[source]¶ Create a
BulkWriterFactory
that writes records with a predefined schema into Parquet files in a batch fashion.- Parameters
row_type – The RowType of records, it should match the RowTypeInfo of Row records.
hadoop_config – Hadoop configuration.
utc_timestamp – Use UTC timezone or local timezone to the conversion between epoch time and LocalDateTime. Hive 0.x/1.x/2.x use local timezone. But Hive 3.x use UTC timezone.
-
static
-
class
pyflink.datastream.formats.parquet.
ParquetColumnarRowInputFormat
(row_type: RowType, hadoop_config: Optional[pyflink.common.configuration.Configuration] = None, batch_size: int = 2048, is_utc_timestamp: bool = False, is_case_sensitive: bool = True)[source]¶ A ParquetVectorizedInputFormat to provide RowData iterator. Using ColumnarRowData to provide a row view of column batch. Only primitive types are supported for a column, composite types such as array, map are not supported.
Example:
>>> row_type = DataTypes.ROW([ ... DataTypes.FIELD('a', DataTypes.INT()), ... DataTypes.FIELD('b', DataTypes.STRING()), ... ]) >>> source = FileSource.for_bulk_file_format(ParquetColumnarRowInputFormat( ... row_type=row_type, ... hadoop_config=Configuration(), ... batch_size=2048, ... is_utc_timestamp=False, ... is_case_sensitive=True, ... ), PARQUET_FILE_PATH).build() >>> ds = env.from_source(source, WatermarkStrategy.no_watermarks(), "parquet-source")
New in version 1.16.0.