Navigation

  • previous
  • PyFlink 1.10.dev0 documentation »
  • pyflink package »

pyflink.datastream package¶

Module contents¶

Important classes of Flink Streaming API:

  • StreamExecutionEnvironment: The context in which a streaming program is executed.

  • CheckpointConfig: Configuration that captures all checkpointing related settings.

  • StateBackend: Defines how the state of a streaming application is stored and checkpointed.

class pyflink.datastream.StreamExecutionEnvironment(j_stream_execution_environment)[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_default_kryo_serializer(type_class_name, serializer_class_name)[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.

disable_operator_chaining()[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_checkpointing(interval, mode=None)[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=None)[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.

get_buffer_timeout()[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()[source]¶

Gets the checkpoint config, which defines values like checkpoint interval, delay between checkpoints, etc.

Returns

The CheckpointConfig.

get_checkpoint_interval()[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()[source]¶

Returns the checkpointing mode (exactly-once vs. at-least-once).

Shorthand for get_checkpoint_config().get_checkpointing_mode().

Returns

The CheckpointingMode.

get_config()[source]¶

Gets the config object.

Returns

The ExecutionConfig object.

get_default_local_parallelism()[source]¶

Gets the default parallelism that will be used for the local execution environment.

Returns

The default local parallelism.

static get_execution_environment()[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()[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()[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()[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()[source]¶

Returns the specified restart strategy configuration.

Returns

The restart strategy configuration to be used.

get_state_backend()[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()[source]¶

Gets the time characteristic.

See also

set_stream_time_characteristic()

Returns

The TimeCharacteristic.

is_chaining_enabled()[source]¶

Returns whether operator chaining is enabled.

Returns

True if chaining is enabled, false otherwise.

register_type(type_class_name)[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, serializer_class_name)[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)[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)[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_max_parallelism(max_parallelism)[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)[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_restart_strategy(restart_strategy_configuration)[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_state_backend(state_backend)[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, MapR FS, 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(RocksDBStateBackend("file://var/checkpoints/"))
Parameters

state_backend – The StateBackend.

Returns

This object.

set_stream_time_characteristic(characteristic)[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.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¶
enable_externalized_checkpoints(cleanup_mode)[source]¶

Enables checkpoints to be persisted externally.

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 you 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 cleanup behaviour, the mode could be ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION or ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION

get_checkpoint_interval()[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_timeout()[source]¶

Gets the maximum time that a checkpoint may take before being discarded.

Returns

The checkpoint timeout, in milliseconds.

get_checkpointing_mode()[source]¶

Gets the checkpointing mode (exactly-once vs. at-least-once).

See also

set_checkpointing_mode()

Returns

The CheckpointingMode.

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

is_checkpointing_enabled()[source]¶

Checks whether checkpointing is enabled.

Returns

True if checkpointing is enables, false otherwise.

is_externalized_checkpoints_enabled()[source]¶

Returns whether checkpoints should be persisted externally.

Returns

True if checkpoints should be externalized, false otherwise.

is_fail_on_checkpointing_errors()[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_prefer_checkpoint_for_recovery()[source]¶

Returns whether a job recovery should fallback to checkpoint when there is a more recent savepoint.

Returns

True if a job recovery should fallback to checkpoint, false otherwise.

set_checkpoint_interval(checkpoint_interval)[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_timeout(checkpoint_timeout)[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)[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_fail_on_checkpointing_errors(fail_on_checkpointing_errors)[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_max_concurrent_checkpoints(max_concurrent_checkpoints)[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)[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_prefer_checkpoint_for_recovery(prefer_checkpoint_for_recovery)[source]¶

Sets whether a job recovery should fallback to checkpoint when there is a more recent savepoint.

Parameters

prefer_checkpoint_for_recovery – True if a job recovery should fallback to checkpoint, false otherwise.

class pyflink.datastream.CheckpointingMode[source]¶

Bases: object

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.StateBackend(j_state_backend)[source]¶

Bases: object

A State Backend defines how the state of a streaming application is stored and checkpointed. Different State Backends store their state in different fashions, and use different data structures to hold the state of a running application.

For example, the MemoryStateBackend keeps working state in the memory of the TaskManager and stores checkpoints in the memory of the JobManager. The backend is lightweight and without additional dependencies, but not highly available and supports only small state.

The FsStateBackend 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 RocksDBStateBackend stores working state in RocksDB, and checkpoints the state by default to a filesystem (similar to the FsStateBackend).

Raw Bytes Storage and Backends

The StateBackend creates services for raw bytes storage and for keyed state and operator state.

The raw bytes storage (through the org.apache.flink.runtime.state.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 checkpointed 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.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

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

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

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

disable_ttl_compaction_filter()[source]¶

Disable compaction filter to cleanup state with TTL.

Note

This is an advanced option and the method should only be used when experiencing serious performance degradations during compaction in RocksDB.

Note

Deprecated in 1.10. Enabled by default and will be removed in the future.

enable_ttl_compaction_filter()[source]¶

Enable compaction filter to cleanup state with TTL.

Note

User can still decide in state TTL configuration in state descriptor whether the filter is active for particular state or not.

Note

Deprecated in 1.10. Enabled by default and will be removed in the future.

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

Gets whether incremental checkpoints are enabled for this state backend.

Returns

True if incremental checkpoints are enabled, false otherwise.

is_ttl_compaction_filter_enabled()[source]¶

Gets whether compaction filter to cleanup state with TTL is enabled.

Returns

True if enabled, false otherwise.

Note

Deprecated in 1.10. Enabled by default and will be removed in the future.

set_db_storage_paths(*paths)[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)[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)[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)[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 created from the provided StateBackendFactory.

class pyflink.datastream.PredefinedOptions[source]¶

Bases: object

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

Bases: object

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.

DELETE_ON_CANCELLATION = 0¶
RETAIN_ON_CANCELLATION = 1¶
class pyflink.datastream.TimeCharacteristic[source]¶

Bases: object

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

Logo

Table of Contents

  • pyflink.datastream package
    • Module contents

Previous topic

pyflink.dataset package

This Page

  • Show Source

Quick search

Navigation

  • previous
  • PyFlink 1.10.dev0 documentation »
  • pyflink package »
© Copyright . Created using Sphinx 2.4.4.