Navigation

  • next
  • previous |
  • PyFlink 1.15.dev0 documentation »
  • pyflink package »

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 a DataStream 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.

Functions used to transform a DataStream into another DataStream:

  • MapFunction: Performs a map transformation of a DataStream at element wise.

  • CoMapFunction: Performs a map transformation over two connected streams.

  • FlatMapFunction: Performs a flatmap transformation of a DataStream 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 to FlatMapFunction, except that it could access the current timestamp and watermark in ProcessFunction.

  • KeyedProcessFunction: Similar to ProcessFunction, except that it was applied to a KeyedStream and could register event-time and processing-time timers.

  • CoProcessFunction: Similar to CoFlatMapFunction, except that it could access the current timestamp and watermark in CoProcessFunction.

  • KeyedCoProcessFunction: Similar to CoProcessFunction, except that it was applied to a keyed ConnectedStreams and could register event-time and processing-time timers.

  • WindowFunction: Base interface for functions that are evaluated over keyed (grouped) windows.

  • ProcessWindowFunction: Similar to WindowFunction, 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.

  • 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).

  • WindowAssigner: Assigns zero or more Window to an element.

  • MergingWindowAssigner: A WindowAssigner 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 configured CheckpointStorage.

  • 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 a ReduceFunction. The current state can be inspected.

  • state.AggregatingState: Interface for aggregating state, based on an AggregateFunction. Elements that are added to this type of state will be eagerly pre-aggregated using a given AggregateFunction.

  • state.StateTtlConfig: Configuration of state TTL logic.

Classes to define source & sink:

  • connectors.FlinkKafkaConsumer: A streaming data source that pulls a parallel data stream from Apache Kafka.

  • connectors.FlinkKafkaProducer: A streaming data sink to produce data into a Kafka topic.

  • connectors.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.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.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.JdbcSink: A data sink to produce data into an external storage using JDBC.

  • connectors.StreamingFileSink: Sink that emits its input elements to files within buckets. This is integrated with the checkpointing mechanism to provide exactly once semantics.

  • connectors.RMQSource: A streaming data source that pulls a parallel data stream from RabbitMQ.

  • connectors.RMQSink: A Sink for publishing data into RabbitMQ.

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.

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.

configure(configuration: pyflink.common.configuration.Configuration)[source]¶

Sets all relevant options contained in the Configuration. such as e.g. pipeline.time-characteristic. It will reconfigure StreamExecutionEnvironment, ExecutionConfig and CheckpointConfig.

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.

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

is_changelog_state_backend_enabled()

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.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 a DataStream.

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() → 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.

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

set_state_backend()

Returns

The StateBackend.

get_stream_time_characteristic() → pyflink.datastream.time_characteristic.TimeCharacteristic[source]¶

Gets the time characteristic.

See also

set_stream_time_characteristic()

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

enable_changelog_state_backend()

Returns

An Optional[bool] for the enable status of change log for state backend. Could be None if user never specify this by calling enable_changelog_state_backend().

New in version 1.14.0.

is_force_unaligned_checkpoints()[source]¶

Returns whether Unaligned Checkpoints are force-enabled.

is_unaligned_checkpoints_enabled()[source]¶

Returns whether Unaligned Checkpoints are enabled.

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_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.27.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 >= 7.1.0) 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:

MemoryStateBackend, FsStateBackend and RocksDBStateBackend.

See also

get_state_backend()

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.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() → pyflink.datastream.data_stream.DataStream[source]¶

Sets the partitioning of the DataStream so that the output elements are broadcasted to every parallel instance of the next operation.

Returns

The DataStream with broadcast partitioning set.

connect(ds: pyflink.datastream.data_stream.DataStream) → pyflink.datastream.data_stream.ConnectedStreams[source]¶

Creates a new ‘ConnectedStreams’ 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.

Parameters

ds – The DataStream with which this stream will be connected.

Returns

The ConnectedStreams.

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_config() → pyflink.common.execution_config.ExecutionConfig[source]¶
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_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.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.

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.

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.

broadcast() → pyflink.datastream.data_stream.DataStream[source]¶

Sets the partitioning of the DataStream so that the output elements are broadcasted to every parallel instance of the next operation.

Returns

The DataStream with broadcast partitioning set.

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.

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_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.

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.

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_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.

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)[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.

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.

allowed_lateness(time_ms: int)[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, result_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.

  • result_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.

get_execution_environment()[source]¶
get_input_type()[source]¶
process(process_window_function: pyflink.datastream.functions.ProcessWindowFunction, result_type: pyflink.common.typeinfo.TypeInformation = None)[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.

  • result_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.

trigger(trigger: pyflink.datastream.window.Trigger)[source]¶

Sets the Trigger that should be used to trigger window emission.

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

process(func: Union[pyflink.datastream.functions.CoProcessFunction, pyflink.datastream.functions.KeyedCoProcessFunction], output_type: pyflink.common.typeinfo.TypeInformation = None) → pyflink.datastream.data_stream.DataStream[source]¶
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.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())
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.

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())
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

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())
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

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())
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

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())
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.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.

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.

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).

abstract timer_service() → pyflink.datastream.timerservice.TimerService[source]¶

A Timer service for querying time and registering timers.

abstract timestamp() → int[source]¶

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 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 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

abstract get_current_key()[source]¶
abstract timer_service() → pyflink.datastream.timerservice.TimerService[source]¶

A Timer service for querying time and registering timers.

abstract timestamp() → int[source]¶

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.

class OnTimerContext[source]¶

Bases: pyflink.datastream.functions.Context

abstract time_domain() → pyflink.datastream.time_domain.TimeDomain[source]¶

The TimeDomain of the firing timer. :return: The TimeDomain of current fired timer.

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.

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 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 provided CoProcessFunction.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

abstract timer_service() → pyflink.datastream.timerservice.TimerService[source]¶

A Timer service for querying time and registering timers.

abstract timestamp() → int[source]¶

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 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 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

abstract get_current_key()[source]¶
abstract timer_service() → pyflink.datastream.timerservice.TimerService[source]¶

A Timer service for querying time and registering timers.

abstract timestamp() → int[source]¶

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.

class OnTimerContext[source]¶

Bases: pyflink.datastream.functions.Context

abstract time_domain() → pyflink.datastream.time_domain.TimeDomain[source]¶

The TimeDomain of the firing timer. :return: The TimeDomain of current fired timer.

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.

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 pyflink.datastream.WindowFunction[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.

class pyflink.datastream.ProcessWindowFunction[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[source]¶

Bases: abc.ABC, typing.Generic

The context holding window metadata.

abstract current_processing_time() → int[source]¶
Returns

The current processing time.

abstract current_watermark() → int[source]¶
Returns

The current event-time watermark.

abstract global_state() → pyflink.datastream.functions.KeyedStateStore[source]¶

State accessor for per-key global state.

abstract window() → W2[source]¶
Returns

The window that is being evaluated.

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.

abstract process(key: KEY, content: 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.

  • content – 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 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.

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.

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_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_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_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 be get_number_of_parallel_subtasks().

class pyflink.datastream.TimerService[source]¶

Bases: abc.ABC

Interface for working with time and timers.

abstract current_processing_time()[source]¶

Returns the current processing time.

abstract current_watermark()[source]¶

Returns the current event-time watermark.

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.CheckpointingMode[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)

EXACTLY_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.

AT_LEAST_ONCE:

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.CheckpointConfig(j_checkpoint_config)[source]¶

Bases: object

Configuration that captures all checkpointing related settings.

DEFAULT_MODE:

The default checkpoint mode: exactly once.

DEFAULT_TIMEOUT:

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() is CheckpointingMode.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 or SUSPENDED). 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 status CANCELED).

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 or ExternalizedCheckpointCleanup.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() is CheckpointingMode.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() or org.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() and get_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

set_checkpointing_mode()

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 to 0, checkpoints will always start unaligned. If alignment_timeout has value greater then 0, checkpoints will start aligned. If during checkpointing, checkpoint start delay exceeds this alignment_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() and set_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 or SUSPENDED). 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 status CANCELED).

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 or ExternalizedCheckpointCleanup.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.ExternalizedCheckpointCleanup[source]¶

Bases: enum.Enum

Cleanup behaviour for externalized checkpoints when the job is cancelled.

DELETE_ON_CANCELLATION:

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_ON_CANCELLATION:

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.

NO_EXTERNALIZED_CHECKPOINTS:

Externalized checkpoints are disabled completely.

DELETE_ON_CANCELLATION = 0¶
NO_EXTERNALIZED_CHECKPOINTS = 2¶
RETAIN_ON_CANCELLATION = 1¶
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.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.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() and set_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()) are PredefinedOptions.DEFAULT.

If user-configured options within RocksDBConfigurableOptions is set (through flink-conf.yaml) or a user-defined options factory is set (via setOptions()), then the options from the factory are applied on top of the predefined and customized options.

See also

set_predefined_options()

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 in RocksDBConfigurableOptions.

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 (via setOptions()), 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.CustomStateBackend(j_custom_state_backend)[source]¶

Bases: pyflink.datastream.state_backend.StateBackend

A wrapper of customized java state backend.

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. The FsStateBackend 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.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() and set_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()) are PredefinedOptions.DEFAULT.

If user-configured options within RocksDBConfigurableOptions is set (through flink-conf.yaml) or a user-defined options factory is set (via setOptions()), then the options from the factory are applied on top of the predefined and customized options.

See also

set_predefined_options()

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 (via setOptions()), 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.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.PredefinedOptions[source]¶

Bases: enum.Enum

The PredefinedOptions are configuration settings for the RocksDBStateBackend. 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:

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.

SPINNING_DISK_OPTIMIZED:

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.

FLASH_SSD_OPTIMIZED:

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.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.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.

get_savepoint_path() → Optional[str][source]¶

Gets the base directory where all the savepoints are stored. The job-specific savepoint directory is created inside this directory.

Returns

The base directory for savepoints.

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.

get_savepoint_path() → Optional[str][source]¶

Gets the base directory where all the savepoints are stored. The job-specific savepoint directory is created inside this directory.

Returns

The base directory for savepoints.

get_write_buffer_size() → int[source]¶

Gets the write buffer size for created checkpoint streams.

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.RuntimeExecutionMode[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.

STREAMING:

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.

BATCH:

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.

AUTOMATIC:

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.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.

abstract max_timestamp() → int[source]¶
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.

max_timestamp() → int[source]¶
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.

max_timestamp() → int[source]¶
class pyflink.datastream.WindowAssigner[source]¶

Bases: abc.ABC, typing.Generic

A WindowAssigner assigns zero or more Window 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 get_current_processing_time() → int[source]¶
Returns

The current processing time.

abstract get_runtime_context() → pyflink.datastream.functions.RuntimeContext[source]¶
Returns

The current runtime context.

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][T, W][source]¶
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][W][source]¶
Returns

A TypeSerializer for serializing windows that are assigned by this WindowAssigner.

abstract is_event_time() → bool[source]¶
Returns

True if elements are assigned to windows based on event time, false otherwise.

class pyflink.datastream.MergingWindowAssigner[source]¶

Bases: pyflink.datastream.window.WindowAssigner

A WindowAssigner that can merge windows.

class MergeCallback[source]¶

Bases: abc.ABC, typing.Generic

Callback to be used in merge_windows() for specifying which windows should be merged.

abstract merge(to_be_merged: Iterable[W2], merge_result: W2) → None[source]¶

Specifies that the given windows should be merged into the result window.

Parameters
  • to_be_merged – The list of windows that should be merged into one window.

  • merge_result – The resulting merged window.

abstract merge_windows(windows: Iterable[W], callback: pyflink.datastream.window.MergingWindowAssigner.MergeCallback[~W][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 pyflink.datastream.TriggerResult[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)¶
is_fire() → bool[source]¶
is_purge() → bool[source]¶
class pyflink.datastream.Trigger[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() and on_merge() most be properly implemented.

class OnMergeContext[source]¶

Bases: pyflink.datastream.window.TriggerContext

Extension of TriggerContext that is given to on_merge().

abstract merge_partitioned_state(state_descriptor: pyflink.datastream.state.StateDescriptor) → None[source]¶
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_current_processing_time() → int[source]¶
Returns

The current processing time.

abstract get_current_watermark() → int[source]¶
Returns

The current watermark 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 from get_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().

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() and register_processing_time_timer() should be deleted here as well as state acquired using get_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 pyflink.datastream.TimeCharacteristic[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).

ProcessingTime:

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.

IngestionTime:

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.

EventTime:

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[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.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.

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.

class pyflink.datastream.Partitioner[source]¶

Bases: pyflink.datastream.functions.Function

Function to implement a custom partition assignment for keys.

abstract partition(key: Any, num_partitions: int) → int[source]¶

Computes the partition for the given key.

Parameters
  • key – The key.

  • num_partitions – The number of partitions to partition into.

Returns

The partition index.

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.

class pyflink.datastream.SinkFunction(sink_func: Union[str, py4j.java_gateway.JavaObject])[source]¶

Bases: pyflink.datastream.functions.JavaFunctionWrapper

The base class for SinkFunctions.

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.

get_task_heap_memory() → Optional[pyflink.datastream.slot_sharing_group.MemorySize][source]¶

Gets the task heap memory for this SlotSharingGroup.

Returns

The task heap memory of the SlotSharingGroup.

get_task_off_heap_memory() → Optional[pyflink.datastream.slot_sharing_group.MemorySize][source]¶

Gets the task off-heap memory for this SlotSharingGroup.

Returns

The task off-heap memory of the SlotSharingGroup.

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_bytes() → int[source]¶

Gets the memory size in bytes.

Returns

The memory size in bytes.

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.

get_tebi_bytes() → int[source]¶

Gets the memory size in Tebibytes (= 1024 Gibibytes).

Returns

The memory size in Tebibytes.

static of_mebi_bytes(mebi_bytes: int) → pyflink.datastream.slot_sharing_group.MemorySize[source]¶

pyflink.datastream.state module¶

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).

class pyflink.datastream.state.ValueState[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 update(value: T) → None[source]¶

Updates the operator state accessible by value() to the given value. The next time value() 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.

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.

class pyflink.datastream.state.ListState[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_all(values: List[T]) → None[source]¶

Adding the given values to the tail of this list state.

abstract update(values: List[T]) → None[source]¶

Updating existing values to to the given list of values.

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.

class pyflink.datastream.state.MapState[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 contains(key: K) → bool[source]¶

Returns whether there exists the given mapping.

abstract get(key: K) → V[source]¶

Returns the current value associated with the given key.

abstract is_empty() → bool[source]¶

Returns true if this state contains no key-value mappings, otherwise false.

abstract items() → Iterable[Tuple[K, V]][source]¶

Returns all the mappings in the state.

abstract keys() → Iterable[K][source]¶

Returns all the keys in the state.

abstract put(key: K, value: V) → None[source]¶

Associates a new value with the given key.

abstract put_all(dict_value: Dict[K, V]) → None[source]¶

Copies all of the mappings from the given map into the state.

abstract remove(key: K) → None[source]¶

Deletes the mapping of the given key.

abstract values() → Iterable[V][source]¶

Returns all the values in the state.

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).

get_reduce_function()[source]¶
class pyflink.datastream.state.ReducingState[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.

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().

get_agg_function()[source]¶
class pyflink.datastream.state.AggregatingState[source]¶

State interface for aggregating state, based on an AggregateFunction. 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.

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.

build() → pyflink.datastream.state.StateTtlConfig[source]¶
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() or cleanup_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.

class CleanupStrategy[source]¶

Base interface for cleanup strategies configurations.

EMPTY_STRATEGY = <pyflink.datastream.state.StateTtlConfig.CleanupStrategies.EmptyCleanupStrategy object>¶
class EmptyCleanupStrategy[source]¶
class IncrementalCleanupStrategy(cleanup_size: int, run_cleanup_for_every_record: int)[source]¶

Configuration of cleanup strategy while taking the full snapshot.

get_cleanup_size() → int[source]¶
run_cleanup_for_every_record() → int[source]¶
class RocksdbCompactFilterCleanupStrategy(query_time_after_num_entries: int)[source]¶

Configuration of cleanup strategy using custom compaction filter in RocksDB.

get_query_time_after_num_entries() → int[source]¶
class Strategies[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]¶
in_full_snapshot() → bool[source]¶
is_cleanup_in_background() → bool[source]¶
class StateVisibility[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[source]¶

This option configures time scale to use for ttl.

ProcessingTime = 0¶

Processing time

class UpdateType[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() → pyflink.common.time.Time[source]¶
get_ttl_time_characteristic() → pyflink.datastream.state.StateTtlConfig.TtlTimeCharacteristic[source]¶
get_update_type() → pyflink.datastream.state.StateTtlConfig.UpdateType[source]¶
is_enabled() → bool[source]¶
static new_builder(ttl: pyflink.common.time.Time)[source]¶

pyflink.datastream.connectors module¶

class pyflink.datastream.connectors.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.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.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.

class pyflink.datastream.connectors.FileSink(j_file_sink)[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 calling for_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 RowFormatBuilder(j_row_format_builder)[source]¶

Builder for the vanilla FileSink using a row format.

build()[source]¶
with_bucket_assigner(bucket_assigner: pyflink.datastream.connectors.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.OutputFileConfig)[source]¶
with_rolling_policy(rolling_policy: pyflink.datastream.connectors.RollingPolicy)[source]¶
static for_row_format(base_path: str, encoder: pyflink.common.serialization.Encoder) → pyflink.datastream.connectors.FileSink.RowFormatBuilder[source]¶
class pyflink.datastream.connectors.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:

  • for_record_stream_format()

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. Call monitor_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_record_stream_format(stream_format: pyflink.datastream.connectors.StreamFormat, *paths: str) → pyflink.datastream.connectors.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.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:

  • for_record_stream_format()

build() → pyflink.datastream.connectors.FileSource[source]¶

Creates the file source with the settings applied to this builder.

monitor_continuously(discovery_interval: pyflink.common.time.Duration) → pyflink.datastream.connectors.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.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.FileEnumeratorProvider) → pyflink.datastream.connectors.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.FileSplitAssignerProvider) → pyflink.datastream.connectors.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.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.FileSplitAssignerProvider[source]¶

A FileSplitAssigner that assigns to each host preferably splits that are local, before assigning splits that are not local.

class pyflink.datastream.connectors.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.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.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.JdbcSink(j_jdbc_sink)[source]¶
static sink(sql: str, type_info: pyflink.common.typeinfo.RowTypeInfo, jdbc_connection_options: pyflink.datastream.connectors.JdbcConnectionOptions, jdbc_execution_options: Optional[pyflink.datastream.connectors.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.

class pyflink.datastream.connectors.JdbcConnectionOptions(j_jdbc_connection_options)[source]¶

JDBC connection options.

class JdbcConnectionOptionsBuilder[source]¶

Builder for JdbcConnectionOptions.

build() → pyflink.datastream.connectors.JdbcConnectionOptions[source]¶
with_driver_name(driver_name: str) → pyflink.datastream.connectors.JdbcConnectionOptions.JdbcConnectionOptionsBuilder[source]¶
with_password(password: str) → pyflink.datastream.connectors.JdbcConnectionOptions.JdbcConnectionOptionsBuilder[source]¶
with_url(url: str) → pyflink.datastream.connectors.JdbcConnectionOptions.JdbcConnectionOptionsBuilder[source]¶
with_user_name(user_name: str) → pyflink.datastream.connectors.JdbcConnectionOptions.JdbcConnectionOptionsBuilder[source]¶
get_db_url() → str[source]¶
get_driver_name() → str[source]¶
get_password() → str[source]¶
get_user_name() → str[source]¶
class pyflink.datastream.connectors.JdbcExecutionOptions(j_jdbc_execution_options)[source]¶

JDBC sink batch options.

class Builder[source]¶

Builder for JdbcExecutionOptions.

build() → pyflink.datastream.connectors.JdbcExecutionOptions[source]¶
with_batch_interval_ms(interval_ms: int) → pyflink.datastream.connectors.JdbcExecutionOptions.Builder[source]¶
with_batch_size(size: int) → pyflink.datastream.connectors.JdbcExecutionOptions.Builder[source]¶
with_max_retries(max_retries: int) → pyflink.datastream.connectors.JdbcExecutionOptions.Builder[source]¶
static builder() → Builder[source]¶
static defaults() → pyflink.datastream.connectors.JdbcExecutionOptions[source]¶
get_batch_interval_ms() → int[source]¶
get_batch_size() → int[source]¶
get_max_retries() → int[source]¶
class pyflink.datastream.connectors.NumberSequenceSource(start, end)[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.

class pyflink.datastream.connectors.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.

build() → pyflink.datastream.connectors.OutputFileConfig[source]¶
with_part_prefix(prefix) → pyflink.datastream.connectors.OutputFileConfig.OutputFileConfigBuilder[source]¶
with_part_suffix(suffix) → pyflink.datastream.connectors.OutputFileConfig.OutputFileConfigBuilder[source]¶
static builder()[source]¶
get_part_prefix() → str[source]¶

The prefix for the part name.

get_part_suffix() → str[source]¶

The suffix for the part name.

class pyflink.datastream.connectors.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.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.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.

class pyflink.datastream.connectors.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.PulsarSourceBuilder[source]¶

Get a PulsarSourceBuilder to builder a PulsarSource.

class pyflink.datastream.connectors.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_event_time(int(time.time() * 1000)))
...     .build()
build() → pyflink.datastream.connectors.PulsarSource[source]¶

Build the PulsarSource.

set_admin_url(admin_url: str) → pyflink.datastream.connectors.PulsarSourceBuilder[source]¶

Sets the admin endpoint for the PulsarAdmin of the PulsarSource.

set_bounded_stop_cursor(stop_cursor: pyflink.datastream.connectors.StopCursor) → pyflink.datastream.connectors.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: pyflink.common.config_options.ConfigOption, value) → pyflink.datastream.connectors.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.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.PulsarDeserializationSchema) → pyflink.datastream.connectors.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_service_url(service_url: str) → pyflink.datastream.connectors.PulsarSourceBuilder[source]¶

Sets the server’s link for the PulsarConsumer of the PulsarSource.

set_start_cursor(start_cursor: pyflink.datastream.connectors.StartCursor) → pyflink.datastream.connectors.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.PulsarSourceBuilder[source]¶

Sets the name for this pulsar subscription.

set_subscription_type(subscription_type: pyflink.datastream.connectors.SubscriptionType) → pyflink.datastream.connectors.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.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.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 set_topics or set_topic_pattern in this builder.

set_topics_pattern(topics_pattern: str) → pyflink.datastream.connectors.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.StopCursor) → pyflink.datastream.connectors.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.RMQConnectionConfig(j_rmq_connection_config)[source]¶

Connection Configuration for RMQ.

class Builder[source]¶

Builder for RMQConnectionConfig.

build() → pyflink.datastream.connectors.RMQConnectionConfig[source]¶
set_automatic_recovery(automatic_recovery: bool) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_connection_timeout(connection_timeout: int) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_host(host: str) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_network_recovery_interval(network_recovery_interval: int) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_password(password: str) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_port(port: int) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_prefetch_count(prefetch_count: int) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_requested_channel_max(requested_channel_max: int) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_requested_frame_max(requested_frame_max: int) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_requested_heartbeat(requested_heartbeat: int) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_topology_recovery_enabled(topology_recovery_enabled: bool) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_uri(uri: str) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_user_name(user_name: str) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
set_virtual_host(vhost: str) → pyflink.datastream.connectors.RMQConnectionConfig.Builder[source]¶
get_connection_timeout() → int[source]¶
get_host() → str[source]¶
get_network_recovery_interval() → int[source]¶
get_password() → str[source]¶
get_port() → int[source]¶
get_requested_channel_max() → int[source]¶
get_requested_frame_max() → int[source]¶
get_requested_heartbeat() → int[source]¶
get_uri() → str[source]¶
get_user_name() → str[source]¶
get_virtual_host() → str[source]¶
is_automatic_recovery() → bool[source]¶
is_topology_recovery() → bool[source]¶
class pyflink.datastream.connectors.RMQSource(connection_config: pyflink.datastream.connectors.RMQConnectionConfig, queue_name: str, use_correlation_id: bool, deserialization_schema: pyflink.common.serialization.DeserializationSchema)[source]¶
class pyflink.datastream.connectors.RMQSink(connection_config: pyflink.datastream.connectors.RMQConnectionConfig, queue_name: str, serialization_schema: pyflink.common.serialization.SerializationSchema)[source]¶
class pyflink.datastream.connectors.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.RollingPolicy[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 on_checkpoint_rolling_policy() → pyflink.datastream.connectors.RollingPolicy[source]¶

Returns a RollingPolicy which rolls (ONLY) on every checkpoint.

class pyflink.datastream.connectors.Sink(sink: Union[str, py4j.java_gateway.JavaObject])[source]¶

Base class for all unified data sink in Flink.

class pyflink.datastream.connectors.Source(source: Union[str, py4j.java_gateway.JavaObject])[source]¶

Base class for all unified data source in Flink.

class pyflink.datastream.connectors.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.StartCursor[source]¶
static earliest() → pyflink.datastream.connectors.StartCursor[source]¶
static from_message_time(timestamp: int) → pyflink.datastream.connectors.StartCursor[source]¶
static latest() → pyflink.datastream.connectors.StartCursor[source]¶
class pyflink.datastream.connectors.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 at_event_time(timestamp: int) → pyflink.datastream.connectors.StopCursor[source]¶
static default_stop_cursor() → pyflink.datastream.connectors.StopCursor[source]¶
static latest() → pyflink.datastream.connectors.StopCursor[source]¶
static never() → pyflink.datastream.connectors.StopCursor[source]¶
class pyflink.datastream.connectors.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.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.

class pyflink.datastream.connectors.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 DefaultRowFormatBuilder(j_default_row_format_builder)[source]¶

Builder for the vanilla StreamingFileSink using a row format.

build() → pyflink.datastream.connectors.StreamingFileSink[source]¶
with_bucket_assigner(bucket_assigner: pyflink.datastream.connectors.BucketAssigner) → pyflink.datastream.connectors.StreamingFileSink.DefaultRowFormatBuilder[source]¶
with_bucket_check_interval(interval: int) → pyflink.datastream.connectors.StreamingFileSink.DefaultRowFormatBuilder[source]¶
with_output_file_config(output_file_config: pyflink.datastream.connectors.OutputFileConfig) → pyflink.datastream.connectors.StreamingFileSink.DefaultRowFormatBuilder[source]¶
with_rolling_policy(policy: pyflink.datastream.connectors.RollingPolicy) → pyflink.datastream.connectors.StreamingFileSink.DefaultRowFormatBuilder[source]¶
static for_row_format(base_path: str, encoder: pyflink.common.serialization.Encoder) → DefaultRowFormatBuilder[source]¶
class pyflink.datastream.connectors.SubscriptionType[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,)¶
Key_Shared = 3¶
Shared = (1,)¶

pyflink.datastream.window module¶

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.

abstract max_timestamp() → int[source]¶
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.

max_timestamp() → int[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.

max_timestamp() → int[source]¶
class pyflink.datastream.window.WindowAssigner[source]¶

A WindowAssigner assigns zero or more Window 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 get_current_processing_time() → int[source]¶
Returns

The current processing time.

abstract get_runtime_context() → pyflink.datastream.functions.RuntimeContext[source]¶
Returns

The current runtime context.

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][T, W][source]¶
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][W][source]¶
Returns

A TypeSerializer for serializing windows that are assigned by this WindowAssigner.

abstract is_event_time() → bool[source]¶
Returns

True if elements are assigned to windows based on event time, false otherwise.

class pyflink.datastream.window.MergingWindowAssigner[source]¶

A WindowAssigner that can merge windows.

class MergeCallback[source]¶

Callback to be used in merge_windows() for specifying which windows should be merged.

abstract merge(to_be_merged: Iterable[W2], merge_result: W2) → None[source]¶

Specifies that the given windows should be merged into the result window.

Parameters
  • to_be_merged – The list of windows that should be merged into one window.

  • merge_result – The resulting merged window.

abstract merge_windows(windows: Iterable[W], callback: pyflink.datastream.window.MergingWindowAssigner.MergeCallback[~W][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 pyflink.datastream.window.TriggerResult[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)¶
is_fire() → bool[source]¶
is_purge() → bool[source]¶
class pyflink.datastream.window.Trigger[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() and on_merge() most be properly implemented.

class OnMergeContext[source]¶

Extension of TriggerContext that is given to on_merge().

abstract merge_partitioned_state(state_descriptor: pyflink.datastream.state.StateDescriptor) → None[source]¶
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_current_processing_time() → int[source]¶
Returns

The current processing time.

abstract get_current_watermark() → int[source]¶
Returns

The current watermark 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 from get_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().

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() and register_processing_time_timer() should be deleted here as well as state acquired using get_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 pyflink.datastream.window.TimeWindowSerializer[source]¶
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.CountWindowSerializer[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.

Logo

Table of Contents

  • pyflink.datastream package
    • Module contents
    • pyflink.datastream.state module
    • pyflink.datastream.connectors module
    • pyflink.datastream.window module

Previous topic

pyflink.table package

Next topic

pyflink.metrics package

This Page

  • Show Source

Quick search

Navigation

  • next
  • previous |
  • PyFlink 1.15.dev0 documentation »
  • pyflink package »
© Copyright . Created using Sphinx 2.4.4.