本文档是 Apache Flink 的旧版本。建议访问 最新的稳定版本

Release Notes - Flink 1.9

These release notes discuss important aspects, such as configuration, behavior, or dependencies, that changed between Flink 1.8 and Flink 1.9. It also provides an overview on known shortcoming or limitations with new experimental features introduced in 1.9.

Please read these notes carefully if you are planning to upgrade your Flink version to 1.9.

Known shortcomings or limitations for new features

Flink 1.9.0 provides support for two planners for the Table API, namely Flink’s original planner and the new Blink planner. The original planner maintains same behaviour as previous releases, while the new Blink planner is still considered experimental and has the following limitations:

  • The Blink planner can not be used with BatchTableEnvironment, and therefore Table programs ran with the planner can not be transformed to DataSet programs. This is by design and will also not be supported in the future. Therefore, if you want to run a batch job with the Blink planner, please use the new TableEnvironment. For streaming jobs, both StreamTableEnvironment and TableEnvironment works.
  • Implementations of StreamTableSink should implement the consumeDataStream method instead of emitDataStream if it is used with the Blink planner. Both methods work with the original planner. This is by design to make the returned DataStreamSink accessible for the planner.
  • Due to a bug with how transformations are not being cleared on execution, TableEnvironment instances should not be reused across multiple SQL statements when using the Blink planner.
  • Table.flatAggregate is not supported
  • Session and count windows are not supported when running batch jobs.
  • The Blink planner only supports the new Catalog API, and does not support ExternalCatalog which is now deprecated.

Related issues:

SQL DDL

In Flink 1.9.0, the community also added a preview feature about SQL DDL, but only for batch style DDLs. Therefore, all streaming related concepts are not supported yet, for example watermarks.

Related issues:

Java 9 support

Since Flink 1.9.0, Flink can now be compiled and run on Java 9. Note that certain components interacting with external systems (connectors, filesystems, metric reporters, etc.) may not work since the respective projects may have skipped Java 9 support.

Related issues:

Memory management

In Fink 1.9.0 and prior version, the managed memory fraction of taskmanager is controlled by taskmanager.memory.fraction, and with 0.7 as the default value. However, sometimes this will cause OOMs due to the fact that the default value of JVM parameter NewRatio is 2, which means the old generation occupied only 2/3 (0.66) of the heap memory. So if you run into this case, please manually change this value to a lower value.

Related issues:

Deprecations and breaking changes

Since 1.9.0, the implicit conversions for the Scala expression DSL for the Table API has been moved to flink-table-api-scala. This requires users to update the imports in their Table programs.

Users of pure Table programs should define their imports like:

import org.apache.flink.table.api._

TableEnvironment.create(...)

Users of the DataStream API should define their imports like:

import org.apache.flink.table.api._
import org.apache.flink.table.api.scala._

StreamTableEnvironment.create(...)

Related issues:

Failover strategies

As a result of completing fine-grained recovery (FLIP-1), Flink will now attempt to only restart tasks that are connected to failed tasks through a pipelined connection. By default, the region failover strategy is used.

Users who were not using a restart strategy or have already configured a failover strategy should not be affected. Moreover, users who already enabled the region failover strategy, along with a restart strategy that enforces a certain number of restarts or introduces a restart delay, will see changes in behavior. The region failover strategy now correctly respects constraints that are defined by the restart strategy.

Streaming users who were not using a failover strategy may be affected if their jobs are embarrassingly parallel or contain multiple independent jobs. In this case, only the failed parallel pipeline or affected jobs will be restarted.

Batch users may be affected if their job contains blocking exchanges (usually happens for shuffles) or the ExecutionMode was set to BATCH or BATCH_FORCED via the ExecutionConfig.

Overall, users should see an improvement in performance.

Related issues:

Job termination via CLI

With the support of graceful job termination with savepoints for semantic correctness (FLIP-34), a few changes related to job termination has been made to the CLI.

From now on, the stop command with no further arguments stops the job with a savepoint targeted at the default savepoint location (as configured via the state.savepoints.dir property in the job configuration), or a location explicitly specified using the -p <savepoint-path> option. Please make sure to configure the savepoint path using either one of these options.

Since job terminations are now always accompanied with a savepoint, stopping jobs is expected to take longer now.

Related issues:

Network stack

A few changes in the network stack related to changes in the threading model of StreamTask to a mailbox-based approach requires close attention to some related configuration:

  • Due to changes in the lifecycle management of result partitions, partition requests as well as re-triggers will now happen sooner. Therefore, it is possible that some jobs with long deployment times and large state might start failing more frequently with PartitionNotFound exceptions compared to previous versions. If that’s the case, users should increase the value of taskmanager.network.request-backoff.max in order to have the same effective partition request timeout as it was prior to 1.9.0.

  • To avoid a potential deadlock, a timeout has been added for how long a task will wait for assignment of exclusive memory segments. The default timeout is 30 seconds, and is configurable via taskmanager.network.memory.exclusive-buffers-request-timeout-ms. It is possible that for some previously working deployments this default timeout value is too low and might have to be increased.

Please also notice that several network I/O metrics have had their scope changed. See the 1.9 metrics documentation for which metrics are affected. In 1.9.0, these metrics will still be available under their previous scopes, but this may no longer be the case in future versions.

Related issues:

AsyncIO

Due to a bug in the AsyncWaitOperator, in 1.9.0 the default chaining behaviour of the operator is now changed so that it is never chained after another operator. This should not be problematic for migrating from older version snapshots as long as an uid was assigned to the operator. If an uid was not assigned to the operator, please see the instructions here for a possible workaround.

Related issues:

Connectors and Libraries

Introduced KafkaSerializationSchema to fully replace KeyedSerializationSchema

The universal FlinkKafkaProducer (in flink-connector-kafka) supports a new KafkaSerializationSchema that will fully replace KeyedSerializationSchema in the long run. This new schema allows directly generating Kafka ProducerRecords for sending to Kafka, therefore enabling the user to use all available Kafka features (in the context of Kafka records).

Dropped connectors and libraries

  • The Elasticsearch 1 connector has been dropped and will no longer receive patches. Users may continue to use the connector from a previous series (like 1.8) with newer versions of Flink. It is being dropped due to being used significantly less than more recent versions (Elasticsearch versions 2.x and 5.x are downloaded 4 to 5 times more), and hasn’t seen any development for over a year.

  • The older Python APIs for batch and streaming have been removed and will no longer receive new patches. A new API is being developed based on the Table API as part of FLINK-12308: Support python language in Flink Table API. Existing users may continue to use these older APIs with future versions of Flink by copying both the flink-streaming-python and flink-python jars into the /lib directory of the distribution and the corresponding start scripts pyflink-stream.sh and pyflink.sh into the /bin directory of the distribution.

  • The older machine learning libraries have been removed and will no longer receive new patches. This is due to efforts towards a new Table-based machine learning library (FLIP-39). Users can still use the 1.8 version of the legacy library if their projects still rely on it.

Related issues:

MapR dependency removed

Dependency on MapR vendor-specific artifacts has been removed, by changing the MapR filesystem connector to work purely based on reflection. This does not introduce any regession in the support for the MapR filesystem. The decision to remove hard dependencies on the MapR artifacts was made due to very flaky access to the secure https endpoint of the MapR artifact repository, and affected build stability of Flink.

Related issues:

StateDescriptor interface change

Access to the state serializer in StateDescriptor is now modified from protected to private access. Subclasses should use the StateDescriptor#getSerializer() method as the only means to obtain the wrapped state serializer.

Related issues:

Web UI dashboard

The web frontend of Flink has been updated to use the latest Angular version (7.x). The old frontend remains available in Flink 1.9.x, but will be removed in a later Flink release once the new frontend is considered stable.

Related issues: