Apache Flink 2.0.0: A new Era of Real-Time Data Processing
March 24, 2025 - Xintong SongToday, the Flink PMC is proud to announce the official release of Apache Flink 2.0.0! This marks the first release in the Flink 2.x series and is the first major release since Flink 1.0 launched nine years ago. This version is the culmination of two years of meticulous preparation and collaboration, signifying a new chapter in the evolution of Flink.
In this release, 165 contributors have come together to complete 25 FLIPs (Flink Improvement Proposals) and 369 issues. We extend our heartfelt gratitude to all contributors for their invaluable contributions to this milestone release!
Over the past decade, Apache Flink has undergone transformative evolution. In the 1.0 era, Flink pioneered Stateful Computations over Data Streams, making end-to-end exactly-once stateful stream processing a reality. Today, real-time processing with sub-second latency has become a standard expectation. However, users of real-time computing now face new challenges that hinder broader adoption. The costs of real-time computing have remained prohibitively high, both in terms of expensive resource consumption and the steep learning curve required to master complex distributed stream processing concepts. These barriers limit the application of real-time computing across more diverse use cases. Meanwhile, the rapid emergence of modern trends such as cloud-native architectures, data lakes, and AI LLMs has introduced new requirements for real-time systems. In the 2.0 era, Flink is tackling these challenges head-on. By addressing these pain points, Flink aims to deliver more accessible and scalable real-time computing solutions, empowering organizations to fully embrace real-time capabilities across the entire spectrum of big data and AI applications. This new chapter represents Flink’s commitment to making real-time computing more practical, efficient, and widely applicable than ever before.
In the 2.0 release, Flink introduces several innovative features that address key challenges in real-time data processing and align with the growing demands of modern applications, including AI-driven workflows.
- The Disaggregated State Management architecture enables more efficient resource utilization in cloud-native environments, ensuring high-performance real-time processing while minimizing resource overhead.
- The introduction and refinement of Materialized Tables empower users to focus on business logic without needing to understand the complexities of stream processing or the differences between stream and batch execution modes, simplifying development and enhances productivity for users across various domains. Optimizations in Batch Execution mode provide a cost-effective alternative for scenarios where near-real-time or non-real-time processing is sufficient, expanding Flink’s versatility for diverse use cases.
- Additionally, the deep integration with Apache Paimon strengthens the Streaming Lakehouse architecture, making Flink a leading solution for real-time data lake use cases.
- As AI and LLMs continue to gain prominence, the demand for scalable, real-time data processing solutions grows. Flink 2.0’s advancements in performance, resource efficiency, and ease of use position it as a strong foundation for AI workflows, ensuring that Flink remains at the forefront of real-time data processing innovations.
These enhancements collectively demonstrate Flink’s commitment to addressing the evolving needs of modern data applications, including the integration of real-time processing capabilities with AI-driven systems.
In addition to the new features introduced in Flink 2.0, the release also includes a comprehensive cleanup of deprecated APIs and configurations, which may result in backward-incompatible changes in certain interfaces and behaviors. Users upgrading to this version should pay special attention to these changes to ensure a smooth transition.
Highlights of New Features #
Disaggregated State Management #
The past decade has seen a transformative evolution in Flink’s deployment paradigms, workload patterns, and hardware advancements. From the tightly coupled compute-storage nodes of the map-reduce era, we have transitioned to a cloud-native world where containerized deployments on Kubernetes are now the norm. To fully embrace this shift, Flink 2.0 introduces Disaggregated State Storage and Management, leveraging Distributed File Systems (DFS) as the primary storage medium. This architectural innovation addresses critical challenges posed by the cloud-native environment while enabling new levels of scalability, performance, and flexibility.
This new architecture solves the following challenges brought in the cloud-native era for Flink.
- Local Disk Constraints in containerization
- Spiky Resource Usage caused by compaction in the current state model
- Fast Rescaling for jobs with large states (hundreds of Terabytes)
- Light and Fast Checkpoint in a native way
While extending the state store to interact with remote DFS seems like a straightforward solution, it is insufficient due to Flink’s existing blocking execution model. To overcome this limitation, Flink 2.0 introduces an asynchronous execution model alongside a disaggregated state backend, as well as newly designed SQL operators performing asynchronous state access in parallel.
Flink 2.0 delivers a comprehensive end-to-end experience for disaggregated state management, encompassing both the runtime and SQL operator layers:
Asynchronous Execution Model #
- Out-of-Order Record Processing: Decouples state access from computation to enable parallel execution.
- Asynchronous State APIs: Full support for non-blocking state operations during checkpointing, reducing latency and improving resource utilization.
- Semantic Preservation: Maintains core Flink guarantees (e.g., watermark propagation, timer handling, and key ordering) to ensure that users can adopt the new architecture without worrying about behavioral changes in their applications.
Enhanced SQL Operators #
- Leveraging the new asynchronous state APIs, Flink 2.0 re-implements seven critical SQL operators, including stateful operations like Joins and Aggregates (e.g., Window Aggregation, Group Aggregation). These optimizations target high-latency state access scenarios, enabling non-blocking execution to maximize throughput.
- Users can enable this feature by setting the configuration parameter
table.exec.async-state.enabled. Once activated, all supported SQL operators within a job automatically switch to asynchronous state access mode without requiring code changes. - In the Nexmark benchmark, 11 out of 14 stateful queries are now fully compatible with the asynchronous execution model, demonstrating significant performance improvements. Efforts are underway to extend support to the remaining stateful operators.
ForSt - A Disaggregated State Backend #
- ForSt, which stands for “For Streaming”, is a purpose-built, disaggregated state backend designed to meet the unique demands of cloud-native deployments. By decoupling state storage from compute resources, ForSt removes the limitations of local disk usage and supports parallel multi-I/O operations, effectively mitigating the impact of increased latency.
- Forst’s integration with DFS ensures durability and fault tolerance while maintaining high performance through optimized read/write operations. It could perform the checkpoint and recovery very light and fast.
Performance Evaluation (on Nexmark) #
Nexmark is a widely recognized benchmark for evaluating the performance of stream processing systems. In our evaluation, we compared Flink 2.0’s Disaggregated State Management with the traditional local state store solution across a range of stateful queries.
- For stateful queries with heavy I/O (q5,q7,q18,q19,q20), Flink 2.0 with disaggregated state and 1GB cache could achieve 75% ~ 120% in throughput comparing to the traditional local state store solution. Notably, the state sizes for these queries range from 1.2GB to 4.8GB, and even under constrained caching conditions, the disaggregated state architecture with limited local cache demonstrates competitive performance against the fully local state setup. Remarkably, even without any caching, the asynchronous model ensures approximately 50% of the throughput achieved by the local state store.
- For stateful queries with small state ranging in 10MB to 400MB (q3,q4,q5,q8,q12,q17), states fully reside in the memory block cache, rendering disk I/O negligible. The disaggregated state store’s performance trails the local state store by an average of no more than 10% in these cases.
- Benchmark results confirm the disaggregated state architecture’s capability to efficiently handle large-scale stateful workloads. It emerges as a seamless, high-performance alternative to traditional aggregated state storage, without significant performance trade-offs.
Flink 2.0’s Disaggregated State Management represents a pivotal step toward a truly cloud-native future. By addressing key challenges such as local disk constraints, spiky resource usage, and the need for fast rescaling, this architecture empowers users to build scalable, high-performance streaming applications. With the introduction of the asynchronous execution model and ForSt, along with enhanced SQL operator capabilities, we expect Flink 2.0 to be a new standard for stateful stream processing in the cloud-native era.
Stream-Batch Unification #
Materialized Table #
Materialized Tables represent a cornerstone of our vision to unify stream and batch processing paradigms. These tables enable users to declaratively manage both real-time and historical data through a single pipeline, eliminating the need for separate codebases or workflows.
In this release, with a focus on production-grade operability, we have done critical enhancements to simplify lifecycle management and execution in real-world environments:
Query Modifications - Materialized Tables now support schema and query updates, enabling seamless iteration of business logic without reprocessing historical data. This is vital for production scenarios requiring rapid schema evolution and computational adjustments.
Kubernetes/Yarn Submission - Beyond standalone clusters, Flink 2.0 extends native support for submitting Materialized Table refresh jobs to YARN and Kubernetes clusters. This allows users to seamlessly integrate refresh workflows into their production-grade infrastructure, leveraging standardized resource management, fault tolerance, and scalability.
Ecosystem Integration - Collaborating with the Apache Paimon community, Materialized Tables now integrate natively with Paimon’s lake storage format, combining Flink’s stream-batch compute with Paimon’s high-performance ACID transactions for unified data serving.
By streamlining modifications and execution on production infrastructure, Materialized Tables empower teams to unify streaming and batch pipelines with higher reliability. Future iterations will deepen production support, including integration with a production-ready schedulers to enable policy-driven refresh automation.
Adaptive Batch Execution #
Flink possesses adaptive batch execution capabilities that optimize execution plans based on runtime information to enhance performance. Key features include dynamic partition pruning, runtime filter, and automatic parallelism adjustment based on data volume. In Flink 2.0, we have further strengthened these capabilities with two new optimizations:
Adaptive Broadcast Join - Compared to Shuffled Hash Join and Sort Merge Join, Broadcast Join eliminates the need for large-scale data shuffling and sorting, delivering superior execution efficiency. However, its applicability depends on one side of the input being sufficiently small; otherwise, performance or stability issues may arise. During the static SQL optimization phase, accurately estimating the input data volume of a Join operator is challenging, making it difficult to determine whether Broadcast Join is suitable. By enabling adaptive execution optimization, Flink dynamically captures the actual input conditions of Join operators at runtime and automatically switches to Broadcast Join when criteria are met, significantly improving execution efficiency.
Automatic Join Skew Optimization - In Join operations, frequent occurrences of specific keys may lead to significant disparities in data volumes processed by downstream Join tasks. Tasks handling larger data volumes can become long-tail bottlenecks, severely delaying overall job execution. Through the Adaptive Skewed Join optimization, Flink leverages runtime statistical information from Join operator inputs to dynamically split skewed data partitions while ensuring the integrity of Join results. This effectively mitigates long-tail latency caused by data skew.
See more details about the capabilities and usages of Flink’s Adaptive Batch Execution.
Performance #
Through the optimizations mentioned above, the batch processing performance of Flink 2.0 has been further improved. We conducted benchmark via 10TB TPC-DS: with additional statistical information generated via the ANALYZE TABLE statement, Flink 2.0 achieves an 8% performance improvement compared to Flink 1.20; without additional statistical information, it achieves a 16% performance improvement.
Streaming Lakehouse #
The lakehouse architecture has emerged as a transformative trend in recent years. By leveraging Flink as a stream-batch unified processing engine and Paimon as a stream-batch unified lake format, the Streaming Lakehouse architecture has enabled real-time data freshness for lakehouse. In Flink 2.0, the Flink community has partnered closely with the Paimon community, leveraging each other’s strengths and cutting-edge features, resulting in significant enhancements and optimizations.
- Nested projection pushdown is now supported when interacting with Paimon data sources, significantly reducing IO overhead and enhancing performance in scenarios involving complex data structures.
- Lookup join performance has been substantially improved when utilizing Paimon as the dimensional table. This enhancement is achieved by aligning data with the bucketing mechanism of the Paimon table, thereby significantly reducing the volume of data each lookup join task needs to retrieve, cache, and process from Paimon.
- All Paimon maintenance actions (such as compaction, managing snapshots/branches/tags, etc.) are now easily executable via Flink SQL call procedures, enhanced with named parameter support that can work with any subset of optional parameters.
- Writing data into Paimon in batch mode with automatic parallelism deciding used to be problematic. This issue has been resolved by ensuring correct bucketing through a fixed parallelism strategy, while applying the automatic parallelism strategy in scenarios where bucketing is irrelevant.
- For Materialized Table, the new stream-batch unified table type in Flink SQL, Paimon serves as the first and sole supported catalog, providing a consistent development experience.
AI #
With the rapid evolution of AI and large language model technologies, artificial intelligence is increasingly shifting from training to inference and practical applications, driving a growing demand for real-time processing of large-scale data. As the leading engine for real-time big data processing, Flink has been actively exploring innovative ways to address the opportunities and challenges posed by the AI era and better support real-time AI applications.
The Flink CDC 3.3 release introduces dynamic AI model invocation capabilities within Transform expressions, with native support for OpenAI chat and embedding models. After capturing database data changes in real time, users can immediately leverage these AI models for intelligent sorting, semantic analysis, or anomaly detection. This integration enables Flink CDC to effectively combine stream processing with Retrieval-Augmented Generation (RAG) technology, delivering end-to-end low-latency processing in scenarios such as real-time risk control, personalized recommendations, and intelligent log parsing, thereby unlocking real-time AI value in data streams.
Furthermore, Flink SQL has introduced specialized syntax for AI models, allowing users to define AI models as easily as defining a Catalog and invoke them like Functions or TableFunctions in SQL statements. Compared to Flink CDC, Flink SQL supports more complex relational data processing logic, seamlessly integrating intricate data processing workflows with AI model invocation. This initiative is currently under active development and refinement.
Misc #
DataStream V2 API #
The DataStream API is one of the two main APIs that Flink provides for writing data processing programs. As an API that was introduced practically since day-1 of the project and has been evolved for nearly a decade, we are observing more and more problems of it. Improvements on these problems require significant breaking changes, which makes in-place refactor impractical. Therefore, we propose to introduce a new set of APIs, the DataStream API V2, to gradually replace the original DataStream API.
In Flink 2.0, we provide the MVP version of the new DataStream V2 API. It contains the low-level building blocks (DataStream, ProcessFunction, Partitioning), context and primitives like state, time service, watermark processing. At the same time, we also provide some high-level extensions, such as window and join. They are more like short-cuts / sugars, without which users can probably still achieve the same behavior by working with the fundamental APIs, but would be a lot easier with the builtin supports.
See documentations for more details
NOTICE: The new DataStream API is currently in the experimental stage and is not yet stable, thus not recommended for production usage at the moment.
SQL gateway supports application mode #
SQL gateway now supports executing SQL jobs in application mode, serving as a replacement of the removed per-job deployment mode.
SQL Syntax Enhancements #
Flink SQL now supports C-style escape strings. See the documentation for more details.
A new QUALIFY clause has been added as a more concise syntax for filtering outputs of window functions. Demonstrations on this can be found in the Top-N and Deduplication examples.
For table function calls it is now possible to use them without TABLE() wrapper in FROM. Examples could be found in updated documentation for Windowing table-valued functions. Queries with such wrapper will continue working as before.
Java Supports #
Starting the 2.0 version, Flink officially supports Java 21.
The default and recommended Java version is changed to Java 17 (previously Java 11). This change mainly affect the docker images and building Flink from sources.
Meanwhile, Java 8 is no longer supported.
Serialization Improvements #
Flink 2.0 introduces much more efficient built-in serializers for collection types (i.e., Map / List / Set), which is enabled by default.
We have also upgraded Kryo to version 5.6, which is faster and more memory efficient, and has better supports for newer Java versions.
Breaking Changes #
API #
The following sets of APIs have been completely removed.
- DataSet API. Please migrate to DataStream API, or Table API/SQL if applicable. See also How to Migrate from DataSet to DataStream.
- Scala DataStream and DataSet API. Please migrate to the Java DataStream API.
- SourceFuction, SinkFunction and Sink V1. Please migrate to Source and Sink V2.
- TableSoure and TableSink. Please migrate to DynamicTableSource and DynamicTableSink. See also User-defined Sources & Sinks.
- TableSchema, TableColumn and Types. Please migrate to Schema, Column and DataTypes respectively.
Some deprecated methods have been removed from DataStream API. See also the list of breaking programming APIs.
Some deprecated fields have been removed from REST API. See also the list of breaking REST APIs.
NOTICE: You may find some of the removed APIs still exist in the code base, usually in a different package. They are for internal usages only and can be changed / removed anytime without notifications. Please DO NOT USE them.
Connector Adaption Plan #
As SourceFunction, SinkFunction and SinkV1 being removed, existing connectors depending on these APIs will not work on the Flink 2.x series. Here’s the plan for adapting the first-party connectors.
- A new version of Kafka, Paimon, JDBC and ElasticSearch connectors, adapted to the API changes, will be released right after the release of Flink 2.0.0.
- We plan to gradually migrate the remaining first-party connectors within 3 subsequent minor releases (i.e., by Flink 2.3).
Configuration #
Configuration options meet the following criteria are removed. See also the list of removed configuration options.
- Annotated as
@Publicand have been deprecated for at least 2 minor releases. - Annotated as
@PublicEvolvingand have been deprecated for at least 1 minor releases.
The legacy configuration file flink-conf.yaml is no longer supported. Please use config.yaml with standard YAML format instead. A migration tool is provided to convert a legacy flink-conf.yaml into a new config.yaml. See Migrate from flink-conf.yaml to config.yaml for more details.
Configuration APIs that takes java objects as arguments are removed from StreamExecutionEnvironment and ExecutionConfig. They should now be set via Configuration and ConfigOption. See also the list of breaking programming APIs.
To avoid exposing internal interfaces, User-Defined Functions no longer have full access to ExecutionConfig. Instead, necessary functions such as createSerializer(), getGlobalJobParameters() and isObjectReuseEnabled() can now be accessed from RuntimeContext directly.
Misc #
- State Compatibility is not guaranteed between 1.x and 2.x.
- Java 8 is no longer supported. The minimum Java version supported by Flink now is Java 11.
- The Per-job deployment mode is no longer supported. Please use the Application mode instead.
- Legacy Mode of Hybrid Shuffle is removed.
Release notes #
For a comprehensive list of features, improvements, bug-fixes, as well as adjustments to make and issues to check during the upgrading process, please refer to the release notes.
List of Contributors #
The Apache Flink community would like to thank each one of the contributors that have made this release possible:
Alan Sheinberg, Aleksandr Pilipenko, Alex Sorokoumov, AlexYinHan, Alexander Fedulov, Ammu, Andrei Kaigorodov, Andrey Gaskov, Arkadiusz Dankiewicz, Arvid Heise, BoShuai Li, Brisk Wong, Cancai Cai, Chesnay Schepler, Chester, Chris, CuiYanxiang, Danny Cranmer, David Anderson, David Moravek, David Radley, Dawid Wysakowicz, Dian Fu, Dmitriy Linevich, Eaugene Thomas, Fabian Hueske, Feng Jin, Ferenc Csaky, Francesco Di Chiara, Gabor Somogyi, Gantigmaa Selenge, Grace Grimwood, Grzegorz Kołakowski, Gustavo de Morais, Hanyu Zheng, Hao Li, Hong Teoh, Hyungstler, Jacky Lau, James Hughes, Jeyassri Balachandran, Jiangjie (Becket) Qin, Jim Hughes, Jingsong, Joern Kottmann, Joery, JunRuiLee, Junrui Lee, Kaitian Hu, Kartikey Pant, Kunni, Kurt Ostfeld, Lajith, Lei Yang, Lorenzo Affetti, Luke Chen, Marc Aurel Fritz, Martijn Visser, Márton Balassi, Mate Czagany, Matt Braymer-Hayes, Matthias Pohl, Mina Asham, Myracle, Paul Zhang, Peng Lu, Peter Huang, Piotr Nowojski, Piotr Przybylski, Qingsheng Ren, Ran Tao, Robin Moffatt, Roc Marshal, Roman Khachatryan, Ron, Rui Fan, Ryan Skraba, Sam Barker, Samrat, Sergei Morozov, Sergey Nuyanzin, Sergio Pena, Sergio Peña, Shengkai, Shuyi Chen, Stefan Richter, Sud0x67, Tamas Sule, Thomas Cooper, Timo Walther, Vincent-Woo, Wang FeiFan, Wang Qian, WangQian, Weijie Guo, Wenchao Wu, Wenjun Ruan, Xia Sun, Xiangyu Feng, Xintong Song, Xu Huang, XuHao41, XuShuai, Xuannan, Xuyang, Yanfei Lei, Yi Zhang, Yiyu Tian, Yu Chen, Yubin Li, Yuxin Tan, Zakelly, Zdenek Tison, Zhanghao Chen, Zhen Wang, anupamaggarwal, argoyal2212, auroflow, candaccc, clarax, codenohup, drymatini, dylanhz, fengli, fredia, gongzhongqiang, haishui, huyuanfeng, jectpro7, lexluo09, lincoln lee, liuyongvs, lvyanquan, lz, mayuehappy, mehdid93, morazow, naferx, nateab, noorall, r-sidd, shalini, simplejason, slankka, sullis, sunxia, sxnan, tison, wangfeifan, xaniasd, xiarui, xincheng.ljr, xuyang, xuzifu666, yinhan.yh, yunfengzhou-hub, zbz, zhangmang, zhaorongsheng, zhengchenyu, zhuanshenbsj1, 余良, 皆非, 马越, 林尚泉
Appendix #
List of breaking change programming APIs #
Removed Classes #
org.apache.flink.api.common.ExecutionConfig$SerializableSerializerorg.apache.flink.api.common.ExecutionModeorg.apache.flink.api.common.InputDependencyConstraintorg.apache.flink.api.common.restartstrategy.RestartStrategies$ExponentialDelayRestartStrategyConfigurationorg.apache.flink.api.common.restartstrategy.RestartStrategies$FailureRateRestartStrategyConfigurationorg.apache.flink.api.common.restartstrategy.RestartStrategies$FallbackRestartStrategyConfigurationorg.apache.flink.api.common.restartstrategy.RestartStrategies$FixedDelayRestartStrategyConfigurationorg.apache.flink.api.common.restartstrategy.RestartStrategies$NoRestartStrategyConfigurationorg.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfigurationorg.apache.flink.api.common.restartstrategy.RestartStrategiesorg.apache.flink.api.common.time.Timeorg.apache.flink.api.connector.sink.Committerorg.apache.flink.api.connector.sink.GlobalCommitterorg.apache.flink.api.connector.sink.Sink$InitContextorg.apache.flink.api.connector.sink.Sink$ProcessingTimeService$ProcessingTimeCallbackorg.apache.flink.api.connector.sink.Sink$ProcessingTimeServiceorg.apache.flink.api.connector.sink.SinkWriter$Contextorg.apache.flink.api.connector.sink.SinkWriterorg.apache.flink.api.connector.sink.Sinkorg.apache.flink.api.connector.sink2.Sink$InitContextWrapperorg.apache.flink.api.connector.sink2.Sink$InitContextorg.apache.flink.api.connector.sink2.StatefulSink$StatefulSinkWriterorg.apache.flink.api.connector.sink2.StatefulSink$WithCompatibleStateorg.apache.flink.api.connector.sink2.StatefulSinkorg.apache.flink.api.connector.sink2.TwoPhaseCommittingSink$PrecommittingSinkWriterorg.apache.flink.api.connector.sink2.TwoPhaseCommittingSinkorg.apache.flink.api.java.CollectionEnvironmentorg.apache.flink.api.java.DataSetorg.apache.flink.api.java.ExecutionEnvironmentFactoryorg.apache.flink.api.java.ExecutionEnvironmentorg.apache.flink.api.java.LocalEnvironmentorg.apache.flink.api.java.RemoteEnvironmentorg.apache.flink.api.java.aggregation.Aggregationsorg.apache.flink.api.java.aggregation.UnsupportedAggregationTypeExceptionorg.apache.flink.api.java.functions.FlatMapIteratororg.apache.flink.api.java.functions.FunctionAnnotation$ForwardedFieldsFirstorg.apache.flink.api.java.functions.FunctionAnnotation$ForwardedFieldsSecondorg.apache.flink.api.java.functions.FunctionAnnotation$ForwardedFieldsorg.apache.flink.api.java.functions.FunctionAnnotation$NonForwardedFieldsFirstorg.apache.flink.api.java.functions.FunctionAnnotation$NonForwardedFieldsSecondorg.apache.flink.api.java.functions.FunctionAnnotation$NonForwardedFieldsorg.apache.flink.api.java.functions.FunctionAnnotation$ReadFieldsFirstorg.apache.flink.api.java.functions.FunctionAnnotation$ReadFieldsSecondorg.apache.flink.api.java.functions.FunctionAnnotation$ReadFieldsorg.apache.flink.api.java.functions.FunctionAnnotationorg.apache.flink.api.java.functions.GroupReduceIteratororg.apache.flink.api.java.io.CollectionInputFormatorg.apache.flink.api.java.io.CsvOutputFormatorg.apache.flink.api.java.io.CsvReaderorg.apache.flink.api.java.io.DiscardingOutputFormatorg.apache.flink.api.java.io.IteratorInputFormatorg.apache.flink.api.java.io.LocalCollectionOutputFormatorg.apache.flink.api.java.io.ParallelIteratorInputFormatorg.apache.flink.api.java.io.PrimitiveInputFormatorg.apache.flink.api.java.io.PrintingOutputFormatorg.apache.flink.api.java.io.RowCsvInputFormatorg.apache.flink.api.java.io.SplitDataProperties$SourcePartitionerMarkerorg.apache.flink.api.java.io.SplitDataPropertiesorg.apache.flink.api.java.io.TextInputFormatorg.apache.flink.api.java.io.TextOutputFormat$TextFormatterorg.apache.flink.api.java.io.TextOutputFormatorg.apache.flink.api.java.io.TextValueInputFormatorg.apache.flink.api.java.io.TypeSerializerInputFormatorg.apache.flink.api.java.io.TypeSerializerOutputFormatorg.apache.flink.api.java.operators.AggregateOperatororg.apache.flink.api.java.operators.CoGroupOperator$CoGroupOperatorSetsorg.apache.flink.api.java.operators.CoGroupOperatororg.apache.flink.api.java.operators.CrossOperator$DefaultCrossorg.apache.flink.api.java.operators.CrossOperator$ProjectCrossorg.apache.flink.api.java.operators.CrossOperatororg.apache.flink.api.java.operators.CustomUnaryOperationorg.apache.flink.api.java.operators.DataSinkorg.apache.flink.api.java.operators.DataSourceorg.apache.flink.api.java.operators.DeltaIteration$SolutionSetPlaceHolderorg.apache.flink.api.java.operators.DeltaIteration$WorksetPlaceHolderorg.apache.flink.api.java.operators.DeltaIterationResultSetorg.apache.flink.api.java.operators.DeltaIterationorg.apache.flink.api.java.operators.DistinctOperatororg.apache.flink.api.java.operators.FilterOperatororg.apache.flink.api.java.operators.FlatMapOperatororg.apache.flink.api.java.operators.GroupCombineOperatororg.apache.flink.api.java.operators.GroupReduceOperatororg.apache.flink.api.java.operators.Groupingorg.apache.flink.api.java.operators.IterativeDataSetorg.apache.flink.api.java.operators.JoinOperator$DefaultJoinorg.apache.flink.api.java.operators.JoinOperator$EquiJoinorg.apache.flink.api.java.operators.JoinOperator$JoinOperatorSets$JoinOperatorSetsPredicateorg.apache.flink.api.java.operators.JoinOperator$JoinOperatorSetsorg.apache.flink.api.java.operators.JoinOperator$ProjectJoinorg.apache.flink.api.java.operators.JoinOperatororg.apache.flink.api.java.operators.MapOperatororg.apache.flink.api.java.operators.MapPartitionOperatororg.apache.flink.api.java.operators.Operatororg.apache.flink.api.java.operators.PartitionOperatororg.apache.flink.api.java.operators.ProjectOperatororg.apache.flink.api.java.operators.ReduceOperatororg.apache.flink.api.java.operators.SingleInputOperatororg.apache.flink.api.java.operators.SingleInputUdfOperatororg.apache.flink.api.java.operators.SortPartitionOperatororg.apache.flink.api.java.operators.SortedGroupingorg.apache.flink.api.java.operators.TwoInputOperatororg.apache.flink.api.java.operators.TwoInputUdfOperatororg.apache.flink.api.java.operators.UdfOperatororg.apache.flink.api.java.operators.UnionOperatororg.apache.flink.api.java.operators.UnsortedGroupingorg.apache.flink.api.java.operators.join.JoinFunctionAssignerorg.apache.flink.api.java.operators.join.JoinOperatorSetsBase$JoinOperatorSetsPredicateBaseorg.apache.flink.api.java.operators.join.JoinOperatorSetsBaseorg.apache.flink.api.java.operators.join.JoinTypeorg.apache.flink.api.java.summarize.BooleanColumnSummaryorg.apache.flink.api.java.summarize.ColumnSummaryorg.apache.flink.api.java.summarize.NumericColumnSummaryorg.apache.flink.api.java.summarize.ObjectColumnSummaryorg.apache.flink.api.java.summarize.StringColumnSummaryorg.apache.flink.api.java.utils.AbstractParameterToolorg.apache.flink.api.java.utils.DataSetUtilsorg.apache.flink.api.java.utils.MultipleParameterToolorg.apache.flink.api.java.utils.ParameterToolorg.apache.flink.configuration.AkkaOptionsorg.apache.flink.connector.file.src.reader.FileRecordFormat$Readerorg.apache.flink.connector.file.src.reader.FileRecordFormatorg.apache.flink.connector.testframe.external.sink.DataStreamSinkV1ExternalContextorg.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend$PriorityQueueStateTypeorg.apache.flink.core.execution.RestoreModeorg.apache.flink.datastream.api.stream.KeyedPartitionStream$TwoKeyedPartitionStreamsorg.apache.flink.datastream.api.stream.NonKeyedPartitionStream$TwoNonKeyedPartitionStreamsorg.apache.flink.formats.avro.AvroRowDeserializationSchemaorg.apache.flink.formats.csv.CsvRowDeserializationSchema$Builderorg.apache.flink.formats.csv.CsvRowDeserializationSchemaorg.apache.flink.formats.csv.CsvRowSerializationSchema$Builderorg.apache.flink.formats.csv.CsvRowSerializationSchemaorg.apache.flink.formats.json.JsonRowDeserializationSchema$Builderorg.apache.flink.formats.json.JsonRowDeserializationSchemaorg.apache.flink.formats.json.JsonRowSerializationSchema$Builderorg.apache.flink.formats.json.JsonRowSerializationSchemaorg.apache.flink.metrics.reporter.InstantiateViaFactoryorg.apache.flink.metrics.reporter.InterceptInstantiationViaReflectionorg.apache.flink.runtime.jobgraph.SavepointConfigOptionsorg.apache.flink.runtime.state.CheckpointListenerorg.apache.flink.runtime.state.filesystem.FsStateBackendFactoryorg.apache.flink.runtime.state.filesystem.FsStateBackendorg.apache.flink.runtime.state.memory.MemoryStateBackendFactoryorg.apache.flink.runtime.state.memory.MemoryStateBackendorg.apache.flink.state.api.BootstrapTransformationorg.apache.flink.state.api.EvictingWindowReaderorg.apache.flink.state.api.ExistingSavepointorg.apache.flink.state.api.KeyedOperatorTransformationorg.apache.flink.state.api.NewSavepointorg.apache.flink.state.api.OneInputOperatorTransformationorg.apache.flink.state.api.Savepointorg.apache.flink.state.api.WindowReaderorg.apache.flink.state.api.WindowedOperatorTransformationorg.apache.flink.state.api.WritableSavepointorg.apache.flink.state.forst.fs.ByteBufferReadableFSDataInputStreamorg.apache.flink.state.forst.fs.ByteBufferWritableFSDataOutputStreamorg.apache.flink.state.forst.fs.ForStFlinkFileSystemorg.apache.flink.streaming.api.TimeCharacteristicorg.apache.flink.streaming.api.checkpoint.ExternallyInducedSource$CheckpointTriggerorg.apache.flink.streaming.api.checkpoint.ExternallyInducedSourceorg.apache.flink.streaming.api.connector.sink2.WithPostCommitTopologyorg.apache.flink.streaming.api.connector.sink2.WithPreCommitTopologyorg.apache.flink.streaming.api.connector.sink2.WithPreWriteTopologyorg.apache.flink.streaming.api.datastream.IterativeStream$ConnectedIterativeStreamsorg.apache.flink.streaming.api.environment.CheckpointConfig$ExternalizedCheckpointCleanuporg.apache.flink.streaming.api.environment.ExecutionCheckpointingOptionsorg.apache.flink.streaming.api.environment.StreamPipelineOptionsorg.apache.flink.streaming.api.functions.AscendingTimestampExtractororg.apache.flink.streaming.api.functions.sink.DiscardingSinkorg.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunctionorg.apache.flink.streaming.api.functions.sink.PrintSinkFunctionorg.apache.flink.streaming.api.functions.sink.RichSinkFunctionorg.apache.flink.streaming.api.functions.sink.SinkFunction$Contextorg.apache.flink.streaming.api.functions.sink.SinkFunctionorg.apache.flink.streaming.api.functions.sink.SocketClientSinkorg.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunctionorg.apache.flink.streaming.api.functions.sink.WriteFormatAsCsvorg.apache.flink.streaming.api.functions.sink.WriteFormatAsTextorg.apache.flink.streaming.api.functions.sink.WriteFormatorg.apache.flink.streaming.api.functions.sink.WriteSinkFunctionByMillisorg.apache.flink.streaming.api.functions.sink.WriteSinkFunctionorg.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink$BulkFormatBuilderorg.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink$DefaultBulkFormatBuilderorg.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink$DefaultRowFormatBuilderorg.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink$RowFormatBuilderorg.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSinkorg.apache.flink.streaming.api.functions.source.FromElementsFunctionorg.apache.flink.streaming.api.functions.source.FromIteratorFunctionorg.apache.flink.streaming.api.functions.source.FromSplittableIteratorFunctionorg.apache.flink.streaming.api.functions.source.MessageAcknowledgingSourceBaseorg.apache.flink.streaming.api.functions.source.MultipleIdsMessageAcknowledgingSourceBaseorg.apache.flink.streaming.api.functions.source.ParallelSourceFunctionorg.apache.flink.streaming.api.functions.source.RichParallelSourceFunctionorg.apache.flink.streaming.api.functions.source.RichSourceFunctionorg.apache.flink.streaming.api.functions.source.SocketTextStreamFunctionorg.apache.flink.streaming.api.functions.source.SourceFunction$SourceContextorg.apache.flink.streaming.api.functions.source.SourceFunctionorg.apache.flink.streaming.api.functions.source.StatefulSequenceSourceorg.apache.flink.streaming.api.functions.source.datagen.DataGeneratorSourceorg.apache.flink.streaming.api.functions.windowing.RichProcessAllWindowFunctionorg.apache.flink.streaming.api.functions.windowing.RichProcessWindowFunctionorg.apache.flink.streaming.api.operators.SetupableStreamOperatororg.apache.flink.streaming.api.operators.YieldingOperatorFactoryorg.apache.flink.streaming.api.windowing.time.Timeorg.apache.flink.streaming.util.serialization.AbstractDeserializationSchemaorg.apache.flink.streaming.util.serialization.DeserializationSchemaorg.apache.flink.streaming.util.serialization.SerializationSchemaorg.apache.flink.streaming.util.serialization.SimpleStringSchemaorg.apache.flink.streaming.util.serialization.TypeInformationSerializationSchemaorg.apache.flink.table.api.TableColumn$ComputedColumnorg.apache.flink.table.api.TableColumn$MetadataColumnorg.apache.flink.table.api.TableColumn$PhysicalColumnorg.apache.flink.table.api.TableColumnorg.apache.flink.table.api.TableSchema$Builderorg.apache.flink.table.api.TableSchemaorg.apache.flink.table.api.constraints.Constraint$ConstraintTypeorg.apache.flink.table.api.constraints.Constraintorg.apache.flink.table.api.constraints.UniqueConstraintorg.apache.flink.table.connector.sink.SinkFunctionProviderorg.apache.flink.table.connector.sink.SinkProviderorg.apache.flink.table.connector.source.AsyncTableFunctionProviderorg.apache.flink.table.connector.source.SourceFunctionProviderorg.apache.flink.table.connector.source.TableFunctionProviderorg.apache.flink.table.descriptors.Descriptororg.apache.flink.table.descriptors.RowtimeValidatororg.apache.flink.table.descriptors.Rowtimeorg.apache.flink.table.descriptors.SchemaValidatororg.apache.flink.table.descriptors.Schemaorg.apache.flink.table.factories.StreamTableSinkFactoryorg.apache.flink.table.factories.StreamTableSourceFactoryorg.apache.flink.table.factories.TableFactoryorg.apache.flink.table.factories.TableSinkFactory$Contextorg.apache.flink.table.factories.TableSinkFactoryorg.apache.flink.table.factories.TableSourceFactory$Contextorg.apache.flink.table.factories.TableSourceFactoryorg.apache.flink.table.planner.codegen.agg.batch.HashAggCodeGenerator$org.apache.flink.table.planner.plan.metadata.FlinkRelMdRowCount$org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder$org.apache.flink.table.planner.plan.rules.logical.JoinDeriveNullFilterRule$org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalJoinRuleBase$org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalSortMergeJoinRule$org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalSortRule$org.apache.flink.table.planner.plan.rules.physical.stream.IncrementalAggregateRule$org.apache.flink.table.planner.plan.utils.FlinkRexUtil$org.apache.flink.table.sinks.AppendStreamTableSinkorg.apache.flink.table.sinks.OutputFormatTableSinkorg.apache.flink.table.sinks.OverwritableTableSinkorg.apache.flink.table.sinks.PartitionableTableSinkorg.apache.flink.table.sinks.RetractStreamTableSinkorg.apache.flink.table.sinks.TableSinkorg.apache.flink.table.sinks.UpsertStreamTableSinkorg.apache.flink.table.sources.DefinedFieldMappingorg.apache.flink.table.sources.DefinedProctimeAttributeorg.apache.flink.table.sources.DefinedRowtimeAttributesorg.apache.flink.table.sources.FieldComputerorg.apache.flink.table.sources.InputFormatTableSourceorg.apache.flink.table.sources.LimitableTableSourceorg.apache.flink.table.sources.LookupableTableSourceorg.apache.flink.table.sources.NestedFieldsProjectableTableSourceorg.apache.flink.table.sources.PartitionableTableSourceorg.apache.flink.table.sources.ProjectableTableSourceorg.apache.flink.table.sources.TableSourceorg.apache.flink.table.sources.tsextractors.ExistingFieldorg.apache.flink.table.sources.tsextractors.StreamRecordTimestamporg.apache.flink.table.sources.tsextractors.TimestampExtractororg.apache.flink.table.types.logical.TypeInformationRawTypeorg.apache.flink.table.utils.TypeStringUtilsorg.apache.flink.walkthrough.common.sink.AlertSinkorg.apache.flink.walkthrough.common.source.TransactionSource
Modified Classes #
org.apache.flink.table.api.bridge.java.StreamTableEnvironment- method removed:
void registerDataStream(java.lang.String, org.apache.flink.streaming.api.datastream.DataStream<T>)void registerFunction(java.lang.String, org.apache.flink.table.functions.TableFunction<T>)void registerFunction(java.lang.String, org.apache.flink.table.functions.AggregateFunction<T,ACC>)void registerFunction(java.lang.String, org.apache.flink.table.functions.TableAggregateFunction<T,ACC>)
- method removed:
org.apache.flink.table.api.config.ExecutionConfigOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Boolean> TABLE_EXEC_LEGACY_TRANSFORMATION_UIDSorg.apache.flink.configuration.ConfigOption<java.lang.String> TABLE_EXEC_SHUFFLE_MODE
- field removed:
org.apache.flink.table.api.config.LookupJoinHintOptions- method modified:
org.apache.flink.shaded.guava32.com.google.common.collect.ImmutableSet<org.apache.flink.configuration.ConfigOption><org.apache.flink.configuration.ConfigOption> (<-org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableSet<org.apache.flink.configuration.ConfigOption><org.apache.flink.configuration.ConfigOption>) getRequiredOptions()org.apache.flink.shaded.guava32.com.google.common.collect.ImmutableSet<org.apache.flink.configuration.ConfigOption><org.apache.flink.configuration.ConfigOption> (<-org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableSet<org.apache.flink.configuration.ConfigOption><org.apache.flink.configuration.ConfigOption>) getSupportedOptions()
- method modified:
org.apache.flink.table.api.config.OptimizerConfigOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Boolean> TABLE_OPTIMIZER_SOURCE_PREDICATE_PUSHDOWN_ENABLEDorg.apache.flink.configuration.ConfigOption<java.lang.Boolean> TABLE_OPTIMIZER_SOURCE_AGGREGATE_PUSHDOWN_ENABLED
- field removed:
org.apache.flink.table.api.dataview.ListView- field removed:
TRANSIENT(-) org.apache.flink.api.common.typeinfo.TypeInformation<?> elementType
- constructor removed:
ListView(org.apache.flink.api.common.typeinfo.TypeInformation<?>)
- field removed:
org.apache.flink.table.api.dataview.MapView- field removed:
TRANSIENT(-) org.apache.flink.api.common.typeinfo.TypeInformation<?> valueTypeTRANSIENT(-) org.apache.flink.api.common.typeinfo.TypeInformation<?> keyType
- constructor removed:
MapView(org.apache.flink.api.common.typeinfo.TypeInformation<?>, org.apache.flink.api.common.typeinfo.TypeInformation<?>)
- field removed:
org.apache.flink.table.api.EnvironmentSettings- method removed:
org.apache.flink.table.api.EnvironmentSettings fromConfiguration(org.apache.flink.configuration.ReadableConfig)org.apache.flink.configuration.Configuration toConfiguration()
- method removed:
org.apache.flink.table.api.internal.BaseExpressions- method removed:
java.lang.Object cast(org.apache.flink.api.common.typeinfo.TypeInformation<?>)
- method removed:
org.apache.flink.table.api.OverWindow- method modified:
java.util.Optional<org.apache.flink.table.expressions.Expression> (<-org.apache.flink.table.expressions.Expression<org.apache.flink.table.expressions.Expression>) getPreceding()
- method modified:
org.apache.flink.table.api.Table- method modified:
org.apache.flink.table.legacy.api.TableSchema (<-org.apache.flink.table.api.TableSchema) getSchema()
- method modified:
org.apache.flink.table.api.TableConfig- constructor modified:
PRIVATE (<- PUBLIC) TableConfig()
- method removed:
long getMaxIdleStateRetentionTime()long getMinIdleStateRetentionTime()void setIdleStateRetentionTime(org.apache.flink.api.common.time.Time, org.apache.flink.api.common.time.Time)
- constructor modified:
org.apache.flink.table.api.TableDescriptor- method removed:
org.apache.flink.table.api.TableDescriptor$Builder forManaged()
- method removed:
org.apache.flink.table.api.TableResult- method removed:
org.apache.flink.table.api.TableSchema getTableSchema()
- method removed:
org.apache.flink.table.catalog.Catalog- method removed:
java.util.Optional<org.apache.flink.table.factories.TableFactory> getTableFactory()boolean supportsManagedTable()
- method removed:
org.apache.flink.table.catalog.CatalogBaseTable- method modified:
org.apache.flink.table.legacy.api.TableSchema (<-org.apache.flink.table.api.TableSchema) getSchema()
- method modified:
org.apache.flink.table.catalog.CatalogFunction- method removed:
boolean isGeneric()
- method removed:
org.apache.flink.table.catalog.CatalogTable- method removed:
org.apache.flink.table.catalog.CatalogTable of(org.apache.flink.table.api.Schema, java.lang.String, java.util.List<java.lang.String>, java.util.Map<java.lang.String,java.lang.String>)org.apache.flink.table.catalog.CatalogTable of(org.apache.flink.table.api.Schema, java.lang.String, java.util.List<java.lang.String>, java.util.Map<java.lang.String,java.lang.String>, java.lang.Long)java.util.Map<java.lang.String,java.lang.String> toProperties()
- method removed:
org.apache.flink.table.catalog.ResolvedCatalogBaseTable- method modified:
org.apache.flink.table.legacy.api.TableSchema (<-org.apache.flink.table.api.TableSchema) getSchema()
- method modified:
org.apache.flink.table.connector.sink.DataStreamSinkProvider- method modified:
(<- NON_ABSTRACT) org.apache.flink.streaming.api.datastream.DataStreamSink<?><?> consumeDataStream(org.apache.flink.table.connector.ProviderContext, org.apache.flink.streaming.api.datastream.DataStream<org.apache.flink.table.data.RowData><org.apache.flink.table.data.RowData>)
- method removed:
org.apache.flink.streaming.api.datastream.DataStreamSink<?> consumeDataStream(org.apache.flink.streaming.api.datastream.DataStream<org.apache.flink.table.data.RowData>)
- method modified:
org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown- method removed:
void applyProjection(int[][])
- method modified:
(<- NON_ABSTRACT) void applyProjection(int[][], org.apache.flink.table.types.DataType)
- method removed:
org.apache.flink.table.connector.source.DataStreamScanProvider- method modified:
(<- NON_ABSTRACT) org.apache.flink.streaming.api.datastream.DataStream<org.apache.flink.table.data.RowData><org.apache.flink.table.data.RowData> produceDataStream(org.apache.flink.table.connector.ProviderContext, org.apache.flink.streaming.api.environment.StreamExecutionEnvironment)
- method removed:
org.apache.flink.streaming.api.datastream.DataStream<org.apache.flink.table.data.RowData> produceDataStream(org.apache.flink.streaming.api.environment.StreamExecutionEnvironment)
- method modified:
org.apache.flink.table.expressions.CallExpression- constructor removed:
CallExpression(org.apache.flink.table.functions.FunctionIdentifier, org.apache.flink.table.functions.FunctionDefinition, java.util.List<org.apache.flink.table.expressions.ResolvedExpression>, org.apache.flink.table.types.DataType)CallExpression(org.apache.flink.table.functions.FunctionDefinition, java.util.List<org.apache.flink.table.expressions.ResolvedExpression>, org.apache.flink.table.types.DataType)
- constructor removed:
org.apache.flink.table.factories.FactoryUtil- method removed:
org.apache.flink.table.connector.sink.DynamicTableSink createDynamicTableSink(org.apache.flink.table.factories.DynamicTableSinkFactory, org.apache.flink.table.catalog.ObjectIdentifier, org.apache.flink.table.catalog.ResolvedCatalogTable, org.apache.flink.configuration.ReadableConfig, java.lang.ClassLoader, boolean)org.apache.flink.table.connector.source.DynamicTableSource createDynamicTableSource(org.apache.flink.table.factories.DynamicTableSourceFactory, org.apache.flink.table.catalog.ObjectIdentifier, org.apache.flink.table.catalog.ResolvedCatalogTable, org.apache.flink.configuration.ReadableConfig, java.lang.ClassLoader, boolean)org.apache.flink.table.connector.sink.DynamicTableSink createTableSink(org.apache.flink.table.catalog.Catalog, org.apache.flink.table.catalog.ObjectIdentifier, org.apache.flink.table.catalog.ResolvedCatalogTable, org.apache.flink.configuration.ReadableConfig, java.lang.ClassLoader, boolean)org.apache.flink.table.connector.source.DynamicTableSource createTableSource(org.apache.flink.table.catalog.Catalog, org.apache.flink.table.catalog.ObjectIdentifier, org.apache.flink.table.catalog.ResolvedCatalogTable, org.apache.flink.configuration.ReadableConfig, java.lang.ClassLoader, boolean)
- method removed:
org.apache.flink.table.factories.FunctionDefinitionFactory- method removed:
org.apache.flink.table.functions.FunctionDefinition createFunctionDefinition(java.lang.String, org.apache.flink.table.catalog.CatalogFunction)
- method modified:
(<- NON_ABSTRACT) org.apache.flink.table.functions.FunctionDefinition createFunctionDefinition(java.lang.String, org.apache.flink.table.catalog.CatalogFunction, org.apache.flink.table.factories.FunctionDefinitionFactory$Context)
- method removed:
org.apache.flink.table.functions.FunctionContext- constructor removed:
FunctionContext(org.apache.flink.api.common.functions.RuntimeContext, java.lang.ClassLoader, org.apache.flink.configuration.Configuration)
- constructor removed:
org.apache.flink.table.plan.stats.ColumnStats- constructor removed:
ColumnStats(java.lang.Long, java.lang.Long, java.lang.Double, java.lang.Integer, java.lang.Number, java.lang.Number)
- method removed:
java.lang.Number getMaxValue()java.lang.Number getMinValue()
- constructor removed:
org.apache.flink.table.types.logical.SymbolType- constructor removed:
SymbolType(boolean, java.lang.Class<T>)SymbolType(java.lang.Class<T>)
- constructor removed:
org.apache.flink.table.types.logical.utils.LogicalTypeParser- method removed:
org.apache.flink.table.types.logical.LogicalType parse(java.lang.String)
- method removed:
org.apache.flink.api.common.state.v2.StateIterator- method removed:
org.apache.flink.api.common.state.v2.StateFuture<java.util.Collection<U>> onNext(java.util.function.Function<T,org.apache.flink.api.common.state.v2.StateFuture<U>>)org.apache.flink.api.common.state.v2.StateFuture<java.lang.Void> onNext(java.util.function.Consumer<T>)
- method removed:
org.apache.flink.table.api.ImplicitExpressionConversions- method removed:
org.apache.flink.table.expressions.Expression toTimestampLtz(org.apache.flink.table.expressions.Expression, org.apache.flink.table.expressions.Expression)SYNTHETIC(-) org.apache.flink.table.expressions.Expression toTimestampLtz$(org.apache.flink.table.api.ImplicitExpressionConversions, org.apache.flink.table.expressions.Expression, org.apache.flink.table.expressions.Expression)
- method removed:
org.apache.flink.api.common.eventtime.WatermarksWithIdleness- constructor removed:
WatermarksWithIdleness(org.apache.flink.api.common.eventtime.WatermarkGenerator<T>, java.time.Duration)
- constructor removed:
org.apache.flink.api.common.ExecutionConfig- field removed:
int PARALLELISM_AUTO_MAX
- method removed:
void addDefaultKryoSerializer(java.lang.Class<?>, com.esotericsoftware.kryo.Serializer<?>)void addDefaultKryoSerializer(java.lang.Class<?>, java.lang.Class<? extends com.esotericsoftware.kryo.Serializer<? extends ?>>)boolean canEqual(java.lang.Object)void disableAutoTypeRegistration()void disableForceAvro()void disableForceKryo()void disableGenericTypes()void enableForceAvro()void enableForceKryo()void enableGenericTypes()int getAsyncInflightRecordsLimit()int getAsyncStateBufferSize()long getAsyncStateBufferTimeout()org.apache.flink.api.common.InputDependencyConstraint getDefaultInputDependencyConstraint()java.util.LinkedHashMap<java.lang.Class<?>,java.lang.Class<com.esotericsoftware.kryo.Serializer<? extends ?>>> getDefaultKryoSerializerClasses()java.util.LinkedHashMap<java.lang.Class<?>,org.apache.flink.api.common.ExecutionConfig$SerializableSerializer<?>> getDefaultKryoSerializers()org.apache.flink.api.common.ExecutionMode getExecutionMode()long getExecutionRetryDelay()int getNumberOfExecutionRetries()java.util.LinkedHashSet<java.lang.Class<?>> getRegisteredKryoTypes()java.util.LinkedHashSet<java.lang.Class<?>> getRegisteredPojoTypes()java.util.LinkedHashMap<java.lang.Class<?>,java.lang.Class<com.esotericsoftware.kryo.Serializer<? extends ?>>> getRegisteredTypesWithKryoSerializerClasses()java.util.LinkedHashMap<java.lang.Class<?>,org.apache.flink.api.common.ExecutionConfig$SerializableSerializer<?>> getRegisteredTypesWithKryoSerializers()org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration getRestartStrategy()boolean hasGenericTypesDisabled()boolean isAutoTypeRegistrationDisabled()boolean isForceAvroEnabled()boolean isForceKryoEnabled()void registerKryoType(java.lang.Class<?>)void registerPojoType(java.lang.Class<?>)void registerTypeWithKryoSerializer(java.lang.Class<?>, com.esotericsoftware.kryo.Serializer<?>)void registerTypeWithKryoSerializer(java.lang.Class<?>, java.lang.Class<? extends com.esotericsoftware.kryo.Serializer>)org.apache.flink.api.common.ExecutionConfig setAsyncInflightRecordsLimit(int)org.apache.flink.api.common.ExecutionConfig setAsyncStateBufferSize(int)org.apache.flink.api.common.ExecutionConfig setAsyncStateBufferTimeout(long)void setDefaultInputDependencyConstraint(org.apache.flink.api.common.InputDependencyConstraint)void setExecutionMode(org.apache.flink.api.common.ExecutionMode)org.apache.flink.api.common.ExecutionConfig setExecutionRetryDelay(long)org.apache.flink.api.common.ExecutionConfig setNumberOfExecutionRetries(int)void setRestartStrategy(org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration)
- field removed:
org.apache.flink.api.common.functions.RichFunction- method removed:
void open(org.apache.flink.configuration.Configuration)
- method modified:
(<- NON_ABSTRACT) void open(org.apache.flink.api.common.functions.OpenContext)
- method removed:
org.apache.flink.api.common.functions.RuntimeContext- method removed:
int getAttemptNumber()org.apache.flink.api.common.ExecutionConfig getExecutionConfig()int getIndexOfThisSubtask()org.apache.flink.api.common.JobID getJobId()int getMaxNumberOfParallelSubtasks()int getNumberOfParallelSubtasks()java.lang.String getTaskName()java.lang.String getTaskNameWithSubtasks()
- method removed:
org.apache.flink.api.common.io.BinaryInputFormat- field removed:
java.lang.String BLOCK_SIZE_PARAMETER_KEY
- field removed:
org.apache.flink.api.common.io.BinaryOutputFormat- field removed:
java.lang.String BLOCK_SIZE_PARAMETER_KEY
- field removed:
org.apache.flink.api.common.io.FileInputFormat- field removed:
java.lang.String ENUMERATE_NESTED_FILES_FLAG
- method removed:
org.apache.flink.core.fs.Path getFilePath()boolean supportsMultiPaths()
- field removed:
org.apache.flink.api.common.io.FileOutputFormat- field removed:
java.lang.String FILE_PARAMETER_KEY
- field removed:
org.apache.flink.api.common.io.FinalizeOnMaster- method removed:
void finalizeGlobal(int)
- method modified:
(<- NON_ABSTRACT) void finalizeGlobal(org.apache.flink.api.common.io.FinalizeOnMaster$FinalizationContext)
- method removed:
org.apache.flink.api.common.io.OutputFormat- method removed:
void open(int, int)
- method modified:
(<- NON_ABSTRACT) void open(org.apache.flink.api.common.io.OutputFormat$InitializationContext)
- method removed:
org.apache.flink.api.common.JobExecutionResult- method removed:
org.apache.flink.api.common.JobExecutionResult fromJobSubmissionResult(org.apache.flink.api.common.JobSubmissionResult)java.lang.Integer getIntCounterResult(java.lang.String)
- method removed:
org.apache.flink.api.common.serialization.SerializerConfig- method removed:
java.util.LinkedHashMap<java.lang.Class<?>,org.apache.flink.api.common.ExecutionConfig$SerializableSerializer<?>> getDefaultKryoSerializers()java.util.LinkedHashMap<java.lang.Class<?>,org.apache.flink.api.common.ExecutionConfig$SerializableSerializer<?>> getRegisteredTypesWithKryoSerializers()
- method removed:
org.apache.flink.api.common.state.StateTtlConfig- method removed:
org.apache.flink.api.common.time.Time getTtl()org.apache.flink.api.common.state.StateTtlConfig$Builder newBuilder(org.apache.flink.api.common.time.Time)
- method removed:
org.apache.flink.api.common.state.StateTtlConfig$Builder- constructor removed:
StateTtlConfig$Builder(org.apache.flink.api.common.time.Time)
- method removed:
org.apache.flink.api.common.state.StateTtlConfig$Builder setTtl(org.apache.flink.api.common.time.Time)
- constructor removed:
org.apache.flink.api.common.typeinfo.TypeInformation- method modified:
(<- NON_ABSTRACT) org.apache.flink.api.common.typeutils.TypeSerializer<T><T> createSerializer(org.apache.flink.api.common.serialization.SerializerConfig)
- method removed:
org.apache.flink.api.common.typeutils.TypeSerializer<T> createSerializer(org.apache.flink.api.common.ExecutionConfig)
- method modified:
org.apache.flink.api.common.typeutils.TypeSerializerSnapshot- method removed:
org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility(org.apache.flink.api.common.typeutils.TypeSerializer<T>)
- method modified:
(<- NON_ABSTRACT) org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility<T><T> resolveSchemaCompatibility(org.apache.flink.api.common.typeutils.TypeSerializerSnapshot<T><T>)
- method removed:
org.apache.flink.api.connector.sink2.Sink- method removed:
org.apache.flink.api.connector.sink2.SinkWriter<InputT> createWriter(org.apache.flink.api.connector.sink2.Sink$InitContext)
- method modified:
(<- NON_ABSTRACT) org.apache.flink.api.connector.sink2.SinkWriter<InputT><InputT> createWriter(org.apache.flink.api.connector.sink2.WriterInitContext)
- method removed:
org.apache.flink.api.java.typeutils.PojoTypeInfo- method removed:
org.apache.flink.api.java.typeutils.runtime.PojoSerializer<T> createPojoSerializer(org.apache.flink.api.common.ExecutionConfig)
- method removed:
org.apache.flink.api.java.typeutils.RowTypeInfo- method removed:
org.apache.flink.api.common.typeutils.TypeSerializer<org.apache.flink.types.Row> createLegacySerializer(org.apache.flink.api.common.serialization.SerializerConfig)
- method removed:
org.apache.flink.configuration.CheckpointingOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Boolean> LOCAL_RECOVERYorg.apache.flink.configuration.ConfigOption<java.lang.String> STATE_BACKENDorg.apache.flink.configuration.ConfigOption<java.lang.Boolean> ASYNC_SNAPSHOTS
- field removed:
org.apache.flink.configuration.ClusterOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Boolean> FINE_GRAINED_SHUFFLE_MODE_ALL_BLOCKINGorg.apache.flink.configuration.ConfigOption<java.lang.Boolean> EVENLY_SPREAD_OUT_SLOTS_STRATEGY
- field removed:
org.apache.flink.configuration.ConfigConstants- field removed:
java.lang.String HA_ZOOKEEPER_LEADER_PATHdouble DEFAULT_AKKA_WATCH_THRESHOLDint DEFAULT_JOB_MANAGER_IPC_PORTjava.lang.String JOB_MANAGER_WEB_TMPDIR_KEYint DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZEjava.lang.String METRICS_SCOPE_NAMING_TASKjava.lang.String ZOOKEEPER_NAMESPACE_KEYint DEFAULT_AKKA_DISPATCHER_THROUGHPUTjava.lang.String RESTART_STRATEGY_FIXED_DELAY_ATTEMPTSjava.lang.String MESOS_MASTER_URLjava.lang.String FLINK_BASE_DIR_PATH_KEYjava.lang.String JOB_MANAGER_WEB_SSL_ENABLEDjava.lang.String YARN_APPLICATION_TAGSjava.lang.String HDFS_SITE_CONFIGjava.lang.String EXECUTION_RETRY_DELAY_KEYint DEFAULT_MESOS_ARTIFACT_SERVER_PORTboolean DEFAULT_SECURITY_SSL_VERIFY_HOSTNAMEjava.lang.String CONTAINERIZED_HEAP_CUTOFF_MINjava.lang.String YARN_HEARTBEAT_DELAY_SECONDSjava.lang.String AKKA_SSL_ENABLEDjava.lang.String HA_MODEjava.lang.String ZOOKEEPER_MESOS_WORKERS_PATHboolean DEFAULT_ZOOKEEPER_SASL_DISABLEjava.lang.String METRICS_SCOPE_DELIMITERjava.lang.String LOCAL_NUMBER_RESOURCE_MANAGERjava.lang.String AKKA_TCP_TIMEOUTjava.lang.String METRICS_SCOPE_NAMING_OPERATORjava.lang.String ZOOKEEPER_RECOVERY_PATHint DEFAULT_ZOOKEEPER_LEADER_PORTjava.lang.String DEFAULT_ZOOKEEPER_LATCH_PATHint DEFAULT_ZOOKEEPER_PEER_PORTjava.lang.String METRICS_SCOPE_NAMING_TM_JOBint DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_NUM_SAMPLESjava.lang.String HA_ZOOKEEPER_SESSION_TIMEOUTjava.lang.String FLINK_JVM_OPTIONSjava.lang.String HA_ZOOKEEPER_CHECKPOINT_COUNTER_PATHjava.lang.String METRICS_SCOPE_NAMING_JMjava.lang.String DEFAULT_YARN_JOB_MANAGER_PORTboolean DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLEjava.lang.String HA_ZOOKEEPER_QUORUM_KEYboolean DEFAULT_JOB_MANAGER_WEB_SUBMIT_ENABLEDjava.lang.String JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZEjava.lang.String ZOOKEEPER_JOBGRAPHS_PATHjava.lang.String ZOOKEEPER_SASL_SERVICE_NAMEjava.lang.String DEFAULT_AKKA_LOOKUP_TIMEOUTjava.lang.String RESTART_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVALjava.lang.String JOB_MANAGER_WEB_PORT_KEYjava.lang.String METRICS_LATENCY_HISTORY_SIZEint DEFAULT_BLOB_FETCH_BACKLOGjava.lang.String JOB_MANAGER_WEB_BACK_PRESSURE_REFRESH_INTERVALfloat DEFAULT_SORT_SPILLING_THRESHOLDjava.lang.String DEFAULT_AKKA_TRANSPORT_HEARTBEAT_INTERVALjava.lang.String CONTAINERIZED_MASTER_ENV_PREFIXint DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNTjava.lang.String TASK_MANAGER_HOSTNAME_KEYjava.lang.String AKKA_WATCH_HEARTBEAT_INTERVALjava.lang.String DEFAULT_TASK_MANAGER_TMP_PATHint DEFAULT_EXECUTION_RETRIESint DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORTjava.lang.String JOB_MANAGER_WEB_LOG_PATH_KEYjava.lang.String TASK_MANAGER_MEMORY_SIZE_KEYjava.lang.String DEFAULT_MESOS_RESOURCEMANAGER_FRAMEWORK_NAMEjava.lang.String TASK_MANAGER_DATA_PORT_KEYjava.lang.String ZOOKEEPER_CHECKPOINTS_PATHjava.lang.String HA_JOB_MANAGER_PORTjava.lang.String TASK_MANAGER_REFUSED_REGISTRATION_PAUSEjava.lang.String CONTAINERIZED_HEAP_CUTOFF_RATIOjava.lang.String DEFAULT_SORT_SPILLING_THRESHOLD_KEYjava.lang.String YARN_CONTAINER_START_COMMAND_TEMPLATEboolean DEFAULT_JOB_MANAGER_WEB_SSL_ENABLEDjava.lang.String LIBRARY_CACHE_MANAGER_CLEANUP_INTERVALjava.lang.String JOB_MANAGER_WEB_CHECKPOINTS_DISABLEjava.lang.String DEFAULT_ZOOKEEPER_LEADER_PATHint DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_DELAYjava.lang.String DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSEjava.lang.String METRICS_REPORTERS_LISTjava.lang.String DEFAULT_RECOVERY_MODEint DEFAULT_METRICS_LATENCY_HISTORY_SIZEjava.lang.String TASK_MANAGER_INITIAL_REGISTRATION_PAUSEjava.lang.String DEFAULT_MESOS_RESOURCEMANAGER_FRAMEWORK_ROLEint DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZEjava.lang.String YARN_PROPERTIES_FILE_LOCATIONjava.lang.String RECOVERY_JOB_MANAGER_PORTboolean DEFAULT_SECURITY_SSL_ENABLEDjava.lang.String MESOS_FAILOVER_TIMEOUT_SECONDSjava.lang.String RUNTIME_HASH_JOIN_BLOOM_FILTERS_KEYjava.lang.String ZOOKEEPER_LEADER_PATHjava.lang.String ZOOKEEPER_MAX_RETRY_ATTEMPTSjava.lang.String HA_ZOOKEEPER_CHECKPOINTS_PATHjava.lang.String MESOS_RESOURCEMANAGER_FRAMEWORK_ROLEint DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_REFRESH_INTERVALjava.lang.String DEFAULT_ZOOKEEPER_MESOS_WORKERS_PATHjava.lang.String JOB_MANAGER_IPC_PORT_KEYjava.lang.String AKKA_WATCH_HEARTBEAT_PAUSEjava.lang.String MESOS_RESOURCEMANAGER_FRAMEWORK_NAMEjava.lang.String DELIMITED_FORMAT_MAX_SAMPLE_LENGTH_KEYjava.lang.String STATE_BACKENDjava.lang.String MESOS_RESOURCEMANAGER_FRAMEWORK_PRINCIPALlong DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MSjava.lang.String DEFAULT_AKKA_CLIENT_TIMEOUTint DEFAULT_SPILLING_MAX_FANjava.lang.String TASK_MANAGER_IPC_PORT_KEYjava.lang.String TASK_MANAGER_MEMORY_OFF_HEAP_KEYboolean DEFAULT_FILESYSTEM_OVERWRITEboolean DEFAULT_USE_LARGE_RECORD_HANDLERjava.lang.String HA_ZOOKEEPER_JOBGRAPHS_PATHboolean DEFAULT_BLOB_SERVICE_SSL_ENABLEDjava.lang.String ZOOKEEPER_SESSION_TIMEOUTjava.lang.String TASK_MANAGER_NETWORK_DEFAULT_IO_MODEjava.lang.String SECURITY_SSL_TRUSTSTORE_PASSWORDint DEFAULT_ZOOKEEPER_MAX_RETRY_ATTEMPTSjava.lang.String AKKA_STARTUP_TIMEOUTjava.lang.String TASK_MANAGER_TMP_DIR_KEYjava.lang.String USE_LARGE_RECORD_HANDLER_KEYjava.lang.String DEFAULT_ZOOKEEPER_DIR_KEYint DEFAULT_YARN_MIN_HEAP_CUTOFFjava.lang.String TASK_MANAGER_DATA_SSL_ENABLEDjava.lang.String HDFS_DEFAULT_CONFIGboolean DEFAULT_TASK_MANAGER_DATA_SSL_ENABLEDjava.lang.String DEFAULT_ZOOKEEPER_JOBGRAPHS_PATHjava.lang.String HA_ZOOKEEPER_MESOS_WORKERS_PATHjava.lang.String BLOB_STORAGE_DIRECTORY_KEYjava.lang.String DEFAULT_STATE_BACKENDjava.lang.String HA_ZOOKEEPER_RETRY_WAITjava.lang.String AKKA_ASK_TIMEOUTjava.lang.String JOB_MANAGER_WEB_SUBMIT_ENABLED_KEYjava.lang.String DEFAULT_ZOOKEEPER_NAMESPACE_KEYjava.lang.String DEFAULT_ZOOKEEPER_CHECKPOINTS_PATHint DEFAULT_LOCAL_NUMBER_JOB_MANAGERjava.lang.String AKKA_TRANSPORT_HEARTBEAT_INTERVALjava.lang.String DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATHjava.lang.String FS_STREAM_OPENING_TIMEOUT_KEYjava.lang.String SECURITY_SSL_TRUSTSTOREjava.lang.String METRICS_SCOPE_NAMING_JM_JOBjava.lang.String MESOS_INITIAL_TASKSjava.lang.String AKKA_FRAMESIZEint DEFAULT_ZOOKEEPER_INIT_LIMITjava.lang.String JOB_MANAGER_WEB_BACK_PRESSURE_CLEAN_UP_INTERVALjava.lang.String SECURITY_SSL_KEYSTOREboolean DEFAULT_MESOS_ARTIFACT_SERVER_SSL_ENABLEDjava.lang.String HA_ZOOKEEPER_MAX_RETRY_ATTEMPTSint DEFAULT_PARALLELISMjava.lang.String RECOVERY_MODEjava.lang.String EXECUTION_RETRIES_KEYjava.lang.String METRICS_REPORTER_SCOPE_DELIMITERjava.lang.String LOCAL_START_WEBSERVERjava.lang.String LOCAL_NUMBER_JOB_MANAGERjava.lang.String RESTART_STRATEGYjava.lang.String ZOOKEEPER_QUORUM_KEYint DEFAULT_MESOS_FAILOVER_TIMEOUT_SECSboolean DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATEint DEFAULT_LOCAL_NUMBER_RESOURCE_MANAGERjava.lang.String HA_ZOOKEEPER_CLIENT_ACLjava.lang.String METRICS_REPORTER_FACTORY_CLASS_SUFFIXboolean DEFAULT_FILESYSTEM_ALWAYS_CREATE_DIRECTORYjava.lang.String BLOB_FETCH_CONCURRENT_KEYjava.lang.String FILESYSTEM_DEFAULT_OVERWRITE_KEYjava.lang.String RESOURCE_MANAGER_IPC_PORT_KEYjava.lang.String DEFAULT_AKKA_ASK_TIMEOUTint DEFAULT_ZOOKEEPER_CLIENT_PORTdouble DEFAULT_AKKA_TRANSPORT_THRESHOLDjava.lang.String DEFAULT_AKKA_FRAMESIZEjava.lang.String TASK_MANAGER_NUM_TASK_SLOTSjava.lang.String YARN_APPLICATION_MASTER_ENV_PREFIXjava.lang.String JOB_MANAGER_WEB_BACK_PRESSURE_DELAYlong DEFAULT_TASK_CANCELLATION_INTERVAL_MILLISjava.lang.String TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEYjava.lang.String FILESYSTEM_SCHEMEjava.lang.String TASK_MANAGER_MAX_REGISTRATION_DURATIONjava.lang.String HA_ZOOKEEPER_DIR_KEYjava.lang.String DEFAULT_MESOS_RESOURCEMANAGER_FRAMEWORK_USERjava.lang.String DEFAULT_FILESYSTEM_SCHEMEjava.lang.String MESOS_RESOURCEMANAGER_FRAMEWORK_SECRETint DEFAULT_DELIMITED_FORMAT_MAX_SAMPLE_LENjava.lang.String ENV_FLINK_BIN_DIRfloat DEFAULT_YARN_HEAP_CUTOFF_RATIOjava.lang.String SAVEPOINT_FS_DIRECTORY_KEYjava.lang.String AKKA_JVM_EXIT_ON_FATAL_ERRORjava.lang.String ZOOKEEPER_RETRY_WAITjava.lang.String HA_ZOOKEEPER_NAMESPACE_KEYjava.lang.String ZOOKEEPER_CONNECTION_TIMEOUTjava.lang.String TASK_MANAGER_NETWORK_NUM_BUFFERS_KEYjava.lang.String JOB_MANAGER_WEB_ARCHIVE_COUNTint DEFAULT_RESOURCE_MANAGER_IPC_PORTint DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_CLEAN_UP_INTERVALjava.lang.String YARN_REALLOCATE_FAILED_CONTAINERSjava.lang.String SECURITY_SSL_KEYSTORE_PASSWORDjava.lang.String DEFAULT_HA_JOB_MANAGER_PORTjava.lang.String BLOB_FETCH_RETRIES_KEYjava.lang.String METRICS_REPORTER_EXCLUDED_VARIABLESjava.lang.String DEFAULT_SECURITY_SSL_PROTOCOLjava.lang.String RECOVERY_JOB_DELAYjava.lang.String TASK_CANCELLATION_INTERVAL_MILLISjava.lang.String YARN_APPLICATION_MASTER_PORTint DEFAULT_TASK_MANAGER_DATA_PORTjava.lang.String RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVALjava.lang.String YARN_TASK_MANAGER_ENV_PREFIXint DEFAULT_DELIMITED_FORMAT_MIN_LINE_SAMPLESjava.lang.String AKKA_LOG_LIFECYCLE_EVENTSboolean DEFAULT_AKKA_LOG_LIFECYCLE_EVENTSjava.lang.String SECURITY_SSL_ENABLEDint DEFAULT_DELIMITED_FORMAT_MAX_LINE_SAMPLESjava.lang.String LOCAL_NUMBER_TASK_MANAGERjava.lang.String DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSEjava.lang.String DEFAULT_SECURITY_SSL_ALGORITHMSjava.lang.String MESOS_MAX_FAILED_TASKSint DEFAULT_TASK_MANAGER_IPC_PORTorg.apache.flink.configuration.ConfigOption<java.lang.String> DEFAULT_JOB_MANAGER_WEB_FRONTEND_ADDRESSjava.lang.String SECURITY_SSL_ALGORITHMSint DEFAULT_ZOOKEEPER_CONNECTION_TIMEOUTjava.lang.String YARN_HEAP_CUTOFF_RATIOjava.lang.String HA_ZOOKEEPER_LATCH_PATHint DEFAULT_ZOOKEEPER_SESSION_TIMEOUTjava.lang.String DEFAULT_SPILLING_MAX_FAN_KEYjava.lang.String AKKA_WATCH_THRESHOLDjava.lang.String TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MSjava.lang.String HA_ZOOKEEPER_STORAGE_PATHjava.lang.String DEFAULT_BLOB_SERVER_PORTjava.lang.String AKKA_TRANSPORT_THRESHOLDjava.lang.String ZOOKEEPER_CHECKPOINT_COUNTER_PATHboolean DEFAULT_RUNTIME_HASH_JOIN_BLOOM_FILTERSint DEFAULT_BLOB_FETCH_CONCURRENTjava.lang.String BLOB_SERVER_PORTorg.apache.flink.configuration.ConfigOption<java.lang.String> RESTART_STRATEGY_FIXED_DELAY_DELAYjava.lang.String METRICS_REPORTER_CLASS_SUFFIXjava.lang.String ZOOKEEPER_DIR_KEYjava.lang.String JOB_MANAGER_IPC_ADDRESS_KEYint DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERSjava.lang.String DEFAULT_AKKA_TRANSPORT_HEARTBEAT_PAUSEjava.lang.String MESOS_ARTIFACT_SERVER_SSL_ENABLEDjava.lang.String RESTART_STRATEGY_FAILURE_RATE_DELAYjava.lang.String DELIMITED_FORMAT_MIN_LINE_SAMPLES_KEYjava.lang.String BLOB_FETCH_BACKLOG_KEYjava.lang.String FILESYSTEM_OUTPUT_ALWAYS_CREATE_DIRECTORY_KEYjava.lang.String DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATIONjava.lang.String TASK_MANAGER_LOG_PATH_KEYjava.lang.String DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODEint DEFAULT_YARN_HEAP_CUTOFFjava.lang.String SECURITY_SSL_PROTOCOLjava.lang.String JOB_MANAGER_WEB_BACK_PRESSURE_NUM_SAMPLESjava.lang.String CHECKPOINTS_DIRECTORY_KEYjava.lang.String DELIMITED_FORMAT_MAX_LINE_SAMPLES_KEYjava.lang.String PATH_HADOOP_CONFIGjava.lang.String ZOOKEEPER_SASL_DISABLEjava.lang.String AKKA_LOOKUP_TIMEOUTjava.lang.String YARN_HEAP_CUTOFF_MINjava.lang.String AKKA_CLIENT_TIMEOUTint DEFAULT_ZOOKEEPER_SYNC_LIMITjava.lang.String DEFAULT_HA_MODEjava.lang.String CONTAINERIZED_TASK_MANAGER_ENV_PREFIXjava.lang.String HA_ZOOKEEPER_CONNECTION_TIMEOUTjava.lang.String METRICS_REPORTER_ADDITIONAL_VARIABLESjava.lang.String MESOS_ARTIFACT_SERVER_PORT_KEYjava.lang.String TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREADjava.lang.String TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEYjava.lang.String YARN_APPLICATION_ATTEMPTSjava.lang.String AKKA_TRANSPORT_HEARTBEAT_PAUSEjava.lang.String DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSEjava.lang.String SECURITY_SSL_VERIFY_HOSTNAMEjava.lang.String DEFAULT_PARALLELISM_KEYjava.lang.String AKKA_DISPATCHER_THROUGHPUTjava.lang.String TASK_MANAGER_MEMORY_FRACTION_KEYjava.lang.String JOB_MANAGER_WEB_UPLOAD_DIR_KEYjava.lang.String SECURITY_SSL_KEY_PASSWORDint DEFAULT_BLOB_FETCH_RETRIESjava.lang.String MESOS_RESOURCEMANAGER_FRAMEWORK_USERjava.lang.String BLOB_SERVICE_SSL_ENABLEDjava.lang.String DEFAULT_YARN_APPLICATION_MASTER_PORTjava.lang.String METRICS_SCOPE_NAMING_TMjava.lang.String TASK_MANAGER_MAX_REGISTARTION_PAUSElong DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVALint DEFAULT_FS_STREAM_OPENING_TIMEOUTjava.lang.String YARN_VCORESjava.lang.String YARN_MAX_FAILED_CONTAINERSjava.lang.String METRICS_REPORTER_INTERVAL_SUFFIXjava.lang.String DEFAULT_HA_ZOOKEEPER_CLIENT_ACLfloat DEFAULT_MEMORY_MANAGER_MEMORY_FRACTIONjava.lang.String SAVEPOINT_DIRECTORY_KEYint DEFAULT_ZOOKEEPER_RETRY_WAITjava.lang.String ZOOKEEPER_LATCH_PATHjava.lang.String DEFAULT_RECOVERY_JOB_MANAGER_PORTboolean DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREADboolean DEFAULT_AKKA_SSL_ENABLED
- field removed:
org.apache.flink.configuration.ConfigOption- method removed:
java.lang.Iterable<java.lang.String> deprecatedKeys()boolean hasDeprecatedKeys()
- method removed:
org.apache.flink.configuration.ConfigOptions$OptionBuilder- method removed:
org.apache.flink.configuration.ConfigOption<T> defaultValue(java.lang.Object)org.apache.flink.configuration.ConfigOption<java.lang.String> noDefaultValue()
- method removed:
org.apache.flink.configuration.Configuration- method removed:
boolean getBoolean(java.lang.String, boolean)boolean getBoolean(org.apache.flink.configuration.ConfigOption<java.lang.Boolean>)boolean getBoolean(org.apache.flink.configuration.ConfigOption<java.lang.Boolean>, boolean)byte[] getBytes(java.lang.String, byte[])java.lang.Class<T> getClass(java.lang.String, java.lang.Class<? extends T>, java.lang.ClassLoader)double getDouble(java.lang.String, double)double getDouble(org.apache.flink.configuration.ConfigOption<java.lang.Double>)double getDouble(org.apache.flink.configuration.ConfigOption<java.lang.Double>, double)float getFloat(java.lang.String, float)float getFloat(org.apache.flink.configuration.ConfigOption<java.lang.Float>)float getFloat(org.apache.flink.configuration.ConfigOption<java.lang.Float>, float)int getInteger(java.lang.String, int)int getInteger(org.apache.flink.configuration.ConfigOption<java.lang.Integer>)int getInteger(org.apache.flink.configuration.ConfigOption<java.lang.Integer>, int)long getLong(java.lang.String, long)long getLong(org.apache.flink.configuration.ConfigOption<java.lang.Long>)long getLong(org.apache.flink.configuration.ConfigOption<java.lang.Long>, long)java.lang.String getString(org.apache.flink.configuration.ConfigOption<java.lang.String>)java.lang.String getString(org.apache.flink.configuration.ConfigOption<java.lang.String>, java.lang.String)void setBoolean(java.lang.String, boolean)void setBoolean(org.apache.flink.configuration.ConfigOption<java.lang.Boolean>, boolean)void setBytes(java.lang.String, byte[])void setClass(java.lang.String, java.lang.Class<?>)void setDouble(java.lang.String, double)void setDouble(org.apache.flink.configuration.ConfigOption<java.lang.Double>, double)void setFloat(java.lang.String, float)void setFloat(org.apache.flink.configuration.ConfigOption<java.lang.Float>, float)void setInteger(java.lang.String, int)void setInteger(org.apache.flink.configuration.ConfigOption<java.lang.Integer>, int)void setLong(java.lang.String, long)void setLong(org.apache.flink.configuration.ConfigOption<java.lang.Long>, long)void setString(org.apache.flink.configuration.ConfigOption<java.lang.String>, java.lang.String)
- method removed:
org.apache.flink.configuration.ExecutionOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Long> ASYNC_STATE_BUFFER_TIMEOUTorg.apache.flink.configuration.ConfigOption<java.lang.Integer> ASYNC_INFLIGHT_RECORDS_LIMITorg.apache.flink.configuration.ConfigOption<java.lang.Integer> ASYNC_STATE_BUFFER_SIZE
- field removed:
org.apache.flink.configuration.HighAvailabilityOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.String> HA_ZOOKEEPER_JOBGRAPHS_PATHorg.apache.flink.configuration.ConfigOption<java.lang.String> ZOOKEEPER_RUNNING_JOB_REGISTRY_PATHorg.apache.flink.configuration.ConfigOption<java.lang.String> HA_JOB_DELAY
- field removed:
org.apache.flink.configuration.JobManagerOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Integer> JOB_MANAGER_HEAP_MEMORY_MBorg.apache.flink.configuration.ConfigOption<java.time.Duration> BLOCK_SLOW_NODE_DURATIONorg.apache.flink.configuration.ConfigOption<org.apache.flink.configuration.MemorySize> ADAPTIVE_BATCH_SCHEDULER_AVG_DATA_VOLUME_PER_TASKorg.apache.flink.configuration.ConfigOption<java.lang.Integer> SPECULATIVE_MAX_CONCURRENT_EXECUTIONSorg.apache.flink.configuration.ConfigOption<java.lang.Integer> ADAPTIVE_BATCH_SCHEDULER_DEFAULT_SOURCE_PARALLELISMorg.apache.flink.configuration.ConfigOption<java.lang.Integer> ADAPTIVE_BATCH_SCHEDULER_MAX_PARALLELISMorg.apache.flink.configuration.ConfigOption<org.apache.flink.configuration.MemorySize> JOB_MANAGER_HEAP_MEMORYorg.apache.flink.configuration.ConfigOption<java.lang.Integer> ADAPTIVE_BATCH_SCHEDULER_MIN_PARALLELISMorg.apache.flink.configuration.ConfigOption<java.lang.Boolean> SPECULATIVE_ENABLED
- field removed:
org.apache.flink.configuration.JobManagerOptions$SchedulerType- field removed:
org.apache.flink.configuration.JobManagerOptions$SchedulerType Ng
- field removed:
org.apache.flink.configuration.MetricOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.String> REPORTER_CLASS
- field removed:
org.apache.flink.configuration.NettyShuffleEnvironmentOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Integer> NUM_THREADS_CLIENTorg.apache.flink.configuration.ConfigOption<java.lang.Integer> NETWORK_BUFFERS_PER_CHANNELorg.apache.flink.configuration.ConfigOption<java.lang.Integer> HYBRID_SHUFFLE_SPILLED_INDEX_REGION_GROUP_SIZEorg.apache.flink.configuration.ConfigOption<java.lang.Integer> CONNECT_BACKLOGorg.apache.flink.configuration.ConfigOption<java.lang.Boolean> NETWORK_HYBRID_SHUFFLE_ENABLE_NEW_MODEorg.apache.flink.configuration.ConfigOption<java.lang.Float> NETWORK_BUFFERS_MEMORY_FRACTIONorg.apache.flink.configuration.ConfigOption<java.lang.Integer> NUM_ARENASorg.apache.flink.configuration.ConfigOption<java.lang.Integer> NUM_THREADS_SERVERorg.apache.flink.configuration.ConfigOption<java.lang.Long> HYBRID_SHUFFLE_NUM_RETAINED_IN_MEMORY_REGIONS_MAXorg.apache.flink.configuration.ConfigOption<java.lang.String> NETWORK_BUFFERS_MEMORY_MINorg.apache.flink.configuration.ConfigOption<java.lang.String> TRANSPORT_TYPEorg.apache.flink.configuration.ConfigOption<java.lang.String> NETWORK_BLOCKING_SHUFFLE_TYPEorg.apache.flink.configuration.ConfigOption<java.lang.Integer> NETWORK_MAX_OVERDRAFT_BUFFERS_PER_GATEorg.apache.flink.configuration.ConfigOption<java.lang.Long> NETWORK_EXCLUSIVE_BUFFERS_REQUEST_TIMEOUT_MILLISECONDSorg.apache.flink.configuration.ConfigOption<java.lang.Boolean> BATCH_SHUFFLE_COMPRESSION_ENABLEDorg.apache.flink.configuration.ConfigOption<java.lang.Integer> SEND_RECEIVE_BUFFER_SIZEorg.apache.flink.configuration.ConfigOption<java.lang.String> NETWORK_BUFFERS_MEMORY_MAXorg.apache.flink.configuration.ConfigOption<java.lang.Integer> NETWORK_EXTRA_BUFFERS_PER_GATEorg.apache.flink.configuration.ConfigOption<java.lang.Integer> NETWORK_SORT_SHUFFLE_MIN_PARALLELISMorg.apache.flink.configuration.ConfigOption<java.lang.Integer> NETWORK_NUM_BUFFERSorg.apache.flink.configuration.ConfigOption<java.lang.Integer> NETWORK_MAX_BUFFERS_PER_CHANNELorg.apache.flink.configuration.ConfigOption<java.lang.Integer> MAX_NUM_TCP_CONNECTIONS
- field removed:
org.apache.flink.configuration.PipelineOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.util.List<java.lang.String>> KRYO_DEFAULT_SERIALIZERSorg.apache.flink.configuration.ConfigOption<java.util.List<java.lang.String>> POJO_REGISTERED_CLASSESorg.apache.flink.configuration.ConfigOption<java.lang.Boolean> AUTO_TYPE_REGISTRATIONorg.apache.flink.configuration.ConfigOption<java.util.List<java.lang.String>> KRYO_REGISTERED_CLASSES
- field removed:
org.apache.flink.configuration.ResourceManagerOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Integer> LOCAL_NUMBER_RESOURCE_MANAGERorg.apache.flink.configuration.ConfigOption<java.lang.Boolean> TASK_MANAGER_RELEASE_WHEN_RESULT_CONSUMEDorg.apache.flink.configuration.ConfigOption<java.lang.Long> SLOT_MANAGER_TASK_MANAGER_TIMEOUT
- field removed:
org.apache.flink.configuration.SecurityOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Double> KERBEROS_TOKENS_RENEWAL_TIME_RATIOorg.apache.flink.configuration.ConfigOption<java.time.Duration> KERBEROS_TOKENS_RENEWAL_RETRY_BACKOFForg.apache.flink.configuration.ConfigOption<java.lang.Boolean> KERBEROS_FETCH_DELEGATION_TOKENorg.apache.flink.configuration.ConfigOption<java.lang.Boolean> SSL_ENABLED
- field removed:
org.apache.flink.configuration.StateBackendOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Integer> LATENCY_TRACK_HISTORY_SIZEorg.apache.flink.configuration.ConfigOption<java.lang.Boolean> LATENCY_TRACK_STATE_NAME_AS_VARIABLEorg.apache.flink.configuration.ConfigOption<java.lang.Boolean> LATENCY_TRACK_ENABLEDorg.apache.flink.configuration.ConfigOption<java.lang.Integer> LATENCY_TRACK_SAMPLE_INTERVAL
- field removed:
org.apache.flink.configuration.TaskManagerOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.time.Duration> REGISTRATION_MAX_BACKOFForg.apache.flink.configuration.ConfigOption<java.time.Duration> INITIAL_REGISTRATION_BACKOFForg.apache.flink.configuration.ConfigOption<java.lang.Boolean> EXIT_ON_FATAL_AKKA_ERRORorg.apache.flink.configuration.ConfigOption<java.lang.Integer> TASK_MANAGER_HEAP_MEMORY_MBjava.lang.String MANAGED_MEMORY_CONSUMER_NAME_DATAPROCorg.apache.flink.configuration.ConfigOption<org.apache.flink.configuration.MemorySize> TASK_MANAGER_HEAP_MEMORYorg.apache.flink.configuration.ConfigOption<java.time.Duration> REFUSED_REGISTRATION_BACKOFF
- field removed:
org.apache.flink.configuration.TaskManagerOptions$TaskManagerLoadBalanceMode- method removed:
org.apache.flink.configuration.TaskManagerOptions$TaskManagerLoadBalanceMode loadFromConfiguration(org.apache.flink.configuration.Configuration)
- method removed:
org.apache.flink.configuration.WebOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Integer> BACKPRESSURE_DELAYorg.apache.flink.configuration.ConfigOption<java.lang.Integer> PORTorg.apache.flink.configuration.ConfigOption<java.lang.Integer> BACKPRESSURE_REFRESH_INTERVALorg.apache.flink.configuration.ConfigOption<java.lang.Integer> BACKPRESSURE_NUM_SAMPLESorg.apache.flink.configuration.ConfigOption<java.lang.String> ADDRESSorg.apache.flink.configuration.ConfigOption<java.lang.Integer> BACKPRESSURE_CLEANUP_INTERVALorg.apache.flink.configuration.ConfigOption<java.lang.Boolean> SSL_ENABLED
- field removed:
org.apache.flink.connector.base.sink.AsyncSinkBase- interface removed:
org.apache.flink.api.connector.sink2.StatefulSink
- interface removed:
org.apache.flink.connector.base.sink.writer.AsyncSinkWriter- constructor removed:
AsyncSinkWriter(org.apache.flink.connector.base.sink.writer.ElementConverter<InputT,RequestEntryT>, org.apache.flink.api.connector.sink2.Sink$InitContext, int, int, int, long, long, long)AsyncSinkWriter(org.apache.flink.connector.base.sink.writer.ElementConverter<InputT,RequestEntryT>, org.apache.flink.api.connector.sink2.Sink$InitContext, int, int, int, long, long, long, java.util.Collection<org.apache.flink.connector.base.sink.writer.BufferedRequestState<RequestEntryT>>)AsyncSinkWriter(org.apache.flink.connector.base.sink.writer.ElementConverter<InputT,RequestEntryT>, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration, java.util.Collection<org.apache.flink.connector.base.sink.writer.BufferedRequestState<RequestEntryT>>)
- constructor removed:
org.apache.flink.connector.base.sink.writer.ElementConverter- method removed:
void open(org.apache.flink.api.connector.sink2.Sink$InitContext)
- method removed:
org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager- constructor removed:
SingleThreadFetcherManager(org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue<org.apache.flink.connector.base.source.reader.RecordsWithSplitIds<E>>, java.util.function.Supplier<org.apache.flink.connector.base.source.reader.splitreader.SplitReader<E,SplitT>>)SingleThreadFetcherManager(org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue<org.apache.flink.connector.base.source.reader.RecordsWithSplitIds<E>>, java.util.function.Supplier<org.apache.flink.connector.base.source.reader.splitreader.SplitReader<E,SplitT>>, org.apache.flink.configuration.Configuration)SingleThreadFetcherManager(org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue<org.apache.flink.connector.base.source.reader.RecordsWithSplitIds<E>>, java.util.function.Supplier<org.apache.flink.connector.base.source.reader.splitreader.SplitReader<E,SplitT>>, org.apache.flink.configuration.Configuration, java.util.function.Consumer<java.util.Collection<java.lang.String>>)
- constructor removed:
org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager- constructor removed:
SplitFetcherManager(org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue<org.apache.flink.connector.base.source.reader.RecordsWithSplitIds<E>>, java.util.function.Supplier<org.apache.flink.connector.base.source.reader.splitreader.SplitReader<E,SplitT>>, org.apache.flink.configuration.Configuration, java.util.function.Consumer<java.util.Collection<java.lang.String>>)SplitFetcherManager(org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue<org.apache.flink.connector.base.source.reader.RecordsWithSplitIds<E>>, java.util.function.Supplier<org.apache.flink.connector.base.source.reader.splitreader.SplitReader<E,SplitT>>, org.apache.flink.configuration.Configuration)
- constructor removed:
org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase- constructor removed:
SingleThreadMultiplexSourceReaderBase(org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue<org.apache.flink.connector.base.source.reader.RecordsWithSplitIds<E>>, java.util.function.Supplier<org.apache.flink.connector.base.source.reader.splitreader.SplitReader<E,SplitT>>, org.apache.flink.connector.base.source.reader.RecordEmitter<E,T,SplitStateT>, org.apache.flink.configuration.Configuration, org.apache.flink.api.connector.source.SourceReaderContext)SingleThreadMultiplexSourceReaderBase(org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue<org.apache.flink.connector.base.source.reader.RecordsWithSplitIds<E>>, org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager<E,SplitT>, org.apache.flink.connector.base.source.reader.RecordEmitter<E,T,SplitStateT>, org.apache.flink.configuration.Configuration, org.apache.flink.api.connector.source.SourceReaderContext)SingleThreadMultiplexSourceReaderBase(org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue<org.apache.flink.connector.base.source.reader.RecordsWithSplitIds<E>>, org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager<E,SplitT>, org.apache.flink.connector.base.source.reader.RecordEmitter<E,T,SplitStateT>, org.apache.flink.connector.base.source.reader.RecordEvaluator<T>, org.apache.flink.configuration.Configuration, org.apache.flink.api.connector.source.SourceReaderContext)
- constructor removed:
org.apache.flink.connector.base.source.reader.SourceReaderBase- constructor removed:
SourceReaderBase(org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue<org.apache.flink.connector.base.source.reader.RecordsWithSplitIds<E>>, org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager<E,SplitT>, org.apache.flink.connector.base.source.reader.RecordEmitter<E,T,SplitStateT>, org.apache.flink.configuration.Configuration, org.apache.flink.api.connector.source.SourceReaderContext)SourceReaderBase(org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue<org.apache.flink.connector.base.source.reader.RecordsWithSplitIds<E>>, org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager<E,SplitT>, org.apache.flink.connector.base.source.reader.RecordEmitter<E,T,SplitStateT>, org.apache.flink.connector.base.source.reader.RecordEvaluator<T>, org.apache.flink.configuration.Configuration, org.apache.flink.api.connector.source.SourceReaderContext)
- constructor removed:
org.apache.flink.contrib.streaming.state.ConfigurableRocksDBOptionsFactory- interface removed:
org.apache.flink.contrib.streaming.state.RocksDBOptionsFactory
- method removed:
org.apache.flink.contrib.streaming.state.RocksDBOptionsFactory configure(org.apache.flink.configuration.ReadableConfig)
- interface removed:
org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend- method removed:
org.apache.flink.contrib.streaming.state.RocksDBMemoryConfiguration getMemoryConfiguration()org.apache.flink.contrib.streaming.state.PredefinedOptions getPredefinedOptions()org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend$PriorityQueueStateType getPriorityQueueStateType()org.apache.flink.contrib.streaming.state.RocksDBOptionsFactory getRocksDBOptions()void setPredefinedOptions(org.apache.flink.contrib.streaming.state.PredefinedOptions)void setPriorityQueueStateType(org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend$PriorityQueueStateType)void setRocksDBMemoryFactory(org.apache.flink.contrib.streaming.state.RocksDBMemoryControllerUtils$RocksDBMemoryFactory)void setRocksDBOptions(org.apache.flink.contrib.streaming.state.RocksDBOptionsFactory)
- method removed:
org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions- method removed:
org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions fromConfig(org.apache.flink.configuration.ReadableConfig)
- method removed:
org.apache.flink.contrib.streaming.state.RocksDBOptionsFactory- method removed:
org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions createNativeMetricsOptions(org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions)
- method removed:
org.apache.flink.core.execution.JobClient- method removed:
java.util.concurrent.CompletableFuture<java.lang.String> stopWithSavepoint(boolean, java.lang.String)java.util.concurrent.CompletableFuture<java.lang.String> triggerSavepoint(java.lang.String)
- method removed:
org.apache.flink.core.failure.FailureEnricher$Context- method removed:
org.apache.flink.api.common.JobID getJobId()java.lang.String getJobName()
- method removed:
org.apache.flink.core.fs.FileSystem- method removed:
org.apache.flink.core.fs.FileSystemKind getKind()
- method removed:
org.apache.flink.core.fs.Path- interface removed:
org.apache.flink.core.io.IOReadableWritable
- interface removed:
org.apache.flink.datastream.api.context.StateManager- method modified:
org.apache.flink.api.common.state.v2.ListState<T>(<- <org.apache.flink.api.common.state.ListState<T>>) (<-java.util.Optional<T>(<- <org.apache.flink.api.common.state.ListState<T>>)) getState(org.apache.flink.api.common.state.ListStateDeclaration<T><T>)org.apache.flink.api.common.state.v2.ValueState<T>(<- <org.apache.flink.api.common.state.ValueState<T>>) (<-java.util.Optional<T>(<- <org.apache.flink.api.common.state.ValueState<T>>)) getState(org.apache.flink.api.common.state.ValueStateDeclaration<T><T>)org.apache.flink.api.common.state.v2.MapState<K,V>(<- <org.apache.flink.api.common.state.MapState<K,V>>) (<-java.util.Optional<K,V>(<- <org.apache.flink.api.common.state.MapState<K,V>>)) getState(org.apache.flink.api.common.state.MapStateDeclaration<K,V><K,V>)org.apache.flink.api.common.state.v2.ReducingState<T>(<- <org.apache.flink.api.common.state.ReducingState<T>>) (<-java.util.Optional<T>(<- <org.apache.flink.api.common.state.ReducingState<T>>)) getState(org.apache.flink.api.common.state.ReducingStateDeclaration<T><T>)org.apache.flink.api.common.state.v2.AggregatingState<IN,OUT>(<- <org.apache.flink.api.common.state.AggregatingState<IN,OUT>>) (<-java.util.Optional<IN,OUT>(<- <org.apache.flink.api.common.state.AggregatingState<IN,OUT>>)) getState(org.apache.flink.api.common.state.AggregatingStateDeclaration<IN,ACC,OUT><IN,ACC,OUT>)org.apache.flink.api.common.state.BroadcastState<K,V>(<- <org.apache.flink.api.common.state.BroadcastState<K,V>>) (<-java.util.Optional<K,V>(<- <org.apache.flink.api.common.state.BroadcastState<K,V>>)) getState(org.apache.flink.api.common.state.BroadcastStateDeclaration<K,V><K,V>)
- method modified:
org.apache.flink.datastream.api.ExecutionEnvironment- method modified:
org.apache.flink.datastream.api.stream.NonKeyedPartitionStream$ProcessConfigurableAndNonKeyedPartitionStream<OUT><OUT> (<-org.apache.flink.datastream.api.stream.NonKeyedPartitionStream<OUT><OUT>) fromSource(org.apache.flink.api.connector.dsv2.Source<OUT><OUT>, java.lang.String)
- method modified:
org.apache.flink.datastream.api.function.ProcessFunction- method removed:
void open()
- method removed:
org.apache.flink.datastream.api.function.TwoOutputApplyPartitionFunction- method removed:
void apply(org.apache.flink.datastream.api.common.Collector<OUT1>, org.apache.flink.datastream.api.common.Collector<OUT2>, org.apache.flink.datastream.api.context.PartitionedContext)
- method removed:
org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction- method removed:
void onProcessingTimer(long, org.apache.flink.datastream.api.common.Collector<OUT1>, org.apache.flink.datastream.api.common.Collector<OUT2>, org.apache.flink.datastream.api.context.PartitionedContext)void processRecord(java.lang.Object, org.apache.flink.datastream.api.common.Collector<OUT1>, org.apache.flink.datastream.api.common.Collector<OUT2>, org.apache.flink.datastream.api.context.PartitionedContext)
- method removed:
org.apache.flink.datastream.api.stream.KeyedPartitionStream- method modified:
org.apache.flink.datastream.api.stream.KeyedPartitionStream$ProcessConfigurableAndTwoKeyedPartitionStreams<K,OUT1,OUT2><K,OUT1,OUT2> (<-org.apache.flink.datastream.api.stream.KeyedPartitionStream$TwoKeyedPartitionStreams<K,OUT1,OUT2><K,OUT1,OUT2>) process(org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction<T,OUT1,OUT2><T,OUT1,OUT2>, org.apache.flink.api.java.functions.KeySelector<OUT1,K><OUT1,K>, org.apache.flink.api.java.functions.KeySelector<OUT2,K><OUT2,K>)org.apache.flink.datastream.api.stream.NonKeyedPartitionStream$ProcessConfigurableAndTwoNonKeyedPartitionStream<OUT1,OUT2><OUT1,OUT2> (<-org.apache.flink.datastream.api.stream.NonKeyedPartitionStream$TwoNonKeyedPartitionStreams<OUT1,OUT2><OUT1,OUT2>) process(org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction<T,OUT1,OUT2><T,OUT1,OUT2>)
- method modified:
org.apache.flink.datastream.api.stream.NonKeyedPartitionStream- method modified:
org.apache.flink.datastream.api.stream.NonKeyedPartitionStream$ProcessConfigurableAndTwoNonKeyedPartitionStream<OUT1,OUT2><OUT1,OUT2> (<-org.apache.flink.datastream.api.stream.NonKeyedPartitionStream$TwoNonKeyedPartitionStreams<OUT1,OUT2><OUT1,OUT2>) process(org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction<T,OUT1,OUT2><T,OUT1,OUT2>)
- method modified:
org.apache.flink.streaming.api.connector.sink2.CommittableMessage- method removed:
java.util.OptionalLong getCheckpointId()
- method removed:
org.apache.flink.streaming.api.connector.sink2.CommittableSummary- constructor removed:
CommittableSummary(int, int, java.lang.Long, int, int, int)
- constructor removed:
org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage- constructor removed:
CommittableWithLineage(java.lang.Object, java.lang.Long, int)
- constructor removed:
org.apache.flink.streaming.api.datastream.AllWindowedStream- method removed:
org.apache.flink.streaming.api.datastream.AllWindowedStream<T,W> allowedLateness(org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> apply(org.apache.flink.api.common.functions.ReduceFunction<T>, org.apache.flink.streaming.api.functions.windowing.AllWindowFunction<T,R,W>)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> apply(org.apache.flink.api.common.functions.ReduceFunction<T>, org.apache.flink.streaming.api.functions.windowing.AllWindowFunction<T,R,W>, org.apache.flink.api.common.typeinfo.TypeInformation<R>)
- method removed:
org.apache.flink.streaming.api.datastream.CoGroupedStreams$WithWindow- method removed:
org.apache.flink.streaming.api.datastream.CoGroupedStreams$WithWindow<T1,T2,KEY,W> allowedLateness(org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> with(org.apache.flink.api.common.functions.CoGroupFunction<T1,T2,T>)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> with(org.apache.flink.api.common.functions.CoGroupFunction<T1,T2,T>, org.apache.flink.api.common.typeinfo.TypeInformation<T>)
- method modified:
org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T><T> (<-org.apache.flink.streaming.api.datastream.DataStream<T><T>) apply(org.apache.flink.api.common.functions.CoGroupFunction<T1,T2,T><T1,T2,T>)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T><T> (<-org.apache.flink.streaming.api.datastream.DataStream<T><T>) apply(org.apache.flink.api.common.functions.CoGroupFunction<T1,T2,T><T1,T2,T>, org.apache.flink.api.common.typeinfo.TypeInformation<T><T>)
- method removed:
org.apache.flink.streaming.api.datastream.DataStream- method removed:
org.apache.flink.streaming.api.datastream.DataStreamSink<T> addSink(org.apache.flink.streaming.api.functions.sink.SinkFunction<T>)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> assignTimestampsAndWatermarks(org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks<T>)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> assignTimestampsAndWatermarks(org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks<T>)org.apache.flink.streaming.api.datastream.IterativeStream<T> iterate()org.apache.flink.streaming.api.datastream.IterativeStream<T> iterate(long)org.apache.flink.streaming.api.datastream.KeyedStream<T,org.apache.flink.api.java.tuple.Tuple> keyBy(int[])org.apache.flink.streaming.api.datastream.KeyedStream<T,org.apache.flink.api.java.tuple.Tuple> keyBy(java.lang.String[])org.apache.flink.streaming.api.datastream.DataStream<T> partitionCustom(org.apache.flink.api.common.functions.Partitioner<K>, int)org.apache.flink.streaming.api.datastream.DataStream<T> partitionCustom(org.apache.flink.api.common.functions.Partitioner<K>, java.lang.String)org.apache.flink.streaming.api.datastream.DataStreamSink<T> sinkTo(org.apache.flink.api.connector.sink.Sink<T,?,?,?>)org.apache.flink.streaming.api.datastream.DataStreamSink<T> sinkTo(org.apache.flink.api.connector.sink.Sink<T,?,?,?>, org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes)org.apache.flink.streaming.api.datastream.AllWindowedStream<T,org.apache.flink.streaming.api.windowing.windows.TimeWindow> timeWindowAll(org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.datastream.AllWindowedStream<T,org.apache.flink.streaming.api.windowing.windows.TimeWindow> timeWindowAll(org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.datastream.DataStreamSink<T> writeAsCsv(java.lang.String)org.apache.flink.streaming.api.datastream.DataStreamSink<T> writeAsCsv(java.lang.String, org.apache.flink.core.fs.FileSystem$WriteMode)org.apache.flink.streaming.api.datastream.DataStreamSink<T> writeAsCsv(java.lang.String, org.apache.flink.core.fs.FileSystem$WriteMode, java.lang.String, java.lang.String)org.apache.flink.streaming.api.datastream.DataStreamSink<T> writeAsText(java.lang.String)org.apache.flink.streaming.api.datastream.DataStreamSink<T> writeAsText(java.lang.String, org.apache.flink.core.fs.FileSystem$WriteMode)
- method removed:
org.apache.flink.streaming.api.datastream.DataStreamUtils- method removed:
java.util.Iterator<OUT> collect(org.apache.flink.streaming.api.datastream.DataStream<OUT>)java.util.Iterator<OUT> collect(org.apache.flink.streaming.api.datastream.DataStream<OUT>, java.lang.String)java.util.List<E> collectBoundedStream(org.apache.flink.streaming.api.datastream.DataStream<E>, java.lang.String)java.util.List<E> collectRecordsFromUnboundedStream(org.apache.flink.streaming.api.operators.collect.ClientAndIterator<E>, int)java.util.List<E> collectUnboundedStream(org.apache.flink.streaming.api.datastream.DataStream<E>, int, java.lang.String)org.apache.flink.streaming.api.operators.collect.ClientAndIterator<OUT> collectWithClient(org.apache.flink.streaming.api.datastream.DataStream<OUT>, java.lang.String)
- method removed:
org.apache.flink.streaming.api.datastream.JoinedStreams$WithWindow- method removed:
org.apache.flink.streaming.api.datastream.JoinedStreams$WithWindow<T1,T2,KEY,W> allowedLateness(org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> with(org.apache.flink.api.common.functions.JoinFunction<T1,T2,T>)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> with(org.apache.flink.api.common.functions.FlatJoinFunction<T1,T2,T>, org.apache.flink.api.common.typeinfo.TypeInformation<T>)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> with(org.apache.flink.api.common.functions.FlatJoinFunction<T1,T2,T>)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> with(org.apache.flink.api.common.functions.JoinFunction<T1,T2,T>, org.apache.flink.api.common.typeinfo.TypeInformation<T>)
- method modified:
org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T><T> (<-org.apache.flink.streaming.api.datastream.DataStream<T><T>) apply(org.apache.flink.api.common.functions.JoinFunction<T1,T2,T><T1,T2,T>)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T><T> (<-org.apache.flink.streaming.api.datastream.DataStream<T><T>) apply(org.apache.flink.api.common.functions.FlatJoinFunction<T1,T2,T><T1,T2,T>, org.apache.flink.api.common.typeinfo.TypeInformation<T><T>)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T><T> (<-org.apache.flink.streaming.api.datastream.DataStream<T><T>) apply(org.apache.flink.api.common.functions.FlatJoinFunction<T1,T2,T><T1,T2,T>)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T><T> (<-org.apache.flink.streaming.api.datastream.DataStream<T><T>) apply(org.apache.flink.api.common.functions.JoinFunction<T1,T2,T><T1,T2,T>, org.apache.flink.api.common.typeinfo.TypeInformation<T><T>)
- method removed:
org.apache.flink.streaming.api.datastream.KeyedStream- method removed:
org.apache.flink.streaming.api.datastream.WindowedStream<T,KEY,org.apache.flink.streaming.api.windowing.windows.TimeWindow> timeWindow(org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.datastream.WindowedStream<T,KEY,org.apache.flink.streaming.api.windowing.windows.TimeWindow> timeWindow(org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time)
- method removed:
org.apache.flink.streaming.api.datastream.KeyedStream$IntervalJoin- method removed:
org.apache.flink.streaming.api.datastream.KeyedStream$IntervalJoined<T1,T2,KEY> between(org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time)
- method removed:
org.apache.flink.streaming.api.datastream.WindowedStream- method removed:
org.apache.flink.streaming.api.datastream.WindowedStream<T,K,W> allowedLateness(org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> apply(org.apache.flink.api.common.functions.ReduceFunction<T>, org.apache.flink.streaming.api.functions.windowing.WindowFunction<T,R,K,W>)org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> apply(org.apache.flink.api.common.functions.ReduceFunction<T>, org.apache.flink.streaming.api.functions.windowing.WindowFunction<T,R,K,W>, org.apache.flink.api.common.typeinfo.TypeInformation<R>)
- method removed:
org.apache.flink.streaming.api.environment.CheckpointConfig- field removed:
int UNDEFINED_TOLERABLE_CHECKPOINT_NUMBERlong DEFAULT_TIMEOUTlong DEFAULT_MIN_PAUSE_BETWEEN_CHECKPOINTSorg.apache.flink.streaming.api.CheckpointingMode DEFAULT_MODEint DEFAULT_MAX_CONCURRENT_CHECKPOINTSint DEFAULT_CHECKPOINT_ID_OF_IGNORED_IN_FLIGHT_DATA
- method removed:
void enableExternalizedCheckpoints(org.apache.flink.streaming.api.environment.CheckpointConfig$ExternalizedCheckpointCleanup)java.time.Duration getAlignmentTimeout()org.apache.flink.runtime.state.CheckpointStorage getCheckpointStorage()org.apache.flink.streaming.api.environment.CheckpointConfig$ExternalizedCheckpointCleanup getExternalizedCheckpointCleanup()boolean isFailOnCheckpointingErrors()boolean isForceCheckpointing()void setAlignmentTimeout(java.time.Duration)void setCheckpointStorage(org.apache.flink.runtime.state.CheckpointStorage)void setCheckpointStorage(java.lang.String)void setCheckpointStorage(java.net.URI)void setCheckpointStorage(org.apache.flink.core.fs.Path)void setExternalizedCheckpointCleanup(org.apache.flink.streaming.api.environment.CheckpointConfig$ExternalizedCheckpointCleanup)void setFailOnCheckpointingErrors(boolean)void setForceCheckpointing(boolean)
- field removed:
org.apache.flink.streaming.api.environment.RemoteStreamEnvironment- method removed:
org.apache.flink.configuration.Configuration getClientConfiguration()
- method removed:
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment- field removed:
java.lang.String DEFAULT_JOB_NAME
- method removed:
void addDefaultKryoSerializer(java.lang.Class<?>, com.esotericsoftware.kryo.Serializer<?>)void addDefaultKryoSerializer(java.lang.Class<?>, java.lang.Class<? extends com.esotericsoftware.kryo.Serializer<? extends ?>>)org.apache.flink.streaming.api.datastream.DataStreamSource<OUT> addSource(org.apache.flink.streaming.api.functions.source.SourceFunction<OUT>)org.apache.flink.streaming.api.datastream.DataStreamSource<OUT> addSource(org.apache.flink.streaming.api.functions.source.SourceFunction<OUT>, java.lang.String)org.apache.flink.streaming.api.datastream.DataStreamSource<OUT> addSource(org.apache.flink.streaming.api.functions.source.SourceFunction<OUT>, org.apache.flink.api.common.typeinfo.TypeInformation<OUT>)org.apache.flink.streaming.api.datastream.DataStreamSource<OUT> addSource(org.apache.flink.streaming.api.functions.source.SourceFunction<OUT>, java.lang.String, org.apache.flink.api.common.typeinfo.TypeInformation<OUT>)org.apache.flink.streaming.api.environment.StreamExecutionEnvironment enableCheckpointing(long, org.apache.flink.streaming.api.CheckpointingMode, boolean)org.apache.flink.streaming.api.environment.StreamExecutionEnvironment enableCheckpointing()int getNumberOfExecutionRetries()org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration getRestartStrategy()org.apache.flink.runtime.state.StateBackend getStateBackend()org.apache.flink.streaming.api.TimeCharacteristic getStreamTimeCharacteristic()boolean isForceCheckpointing()org.apache.flink.streaming.api.datastream.DataStream<java.lang.String> readFileStream(java.lang.String, long, org.apache.flink.streaming.api.functions.source.FileMonitoringFunction$WatchType)org.apache.flink.streaming.api.datastream.DataStreamSource<java.lang.String> readTextFile(java.lang.String)org.apache.flink.streaming.api.datastream.DataStreamSource<java.lang.String> readTextFile(java.lang.String, java.lang.String)void registerType(java.lang.Class<?>)void registerTypeWithKryoSerializer(java.lang.Class<?>, com.esotericsoftware.kryo.Serializer<?>)void registerTypeWithKryoSerializer(java.lang.Class<?>, java.lang.Class<? extends com.esotericsoftware.kryo.Serializer>)void setNumberOfExecutionRetries(int)void setRestartStrategy(org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration)org.apache.flink.streaming.api.environment.StreamExecutionEnvironment setStateBackend(org.apache.flink.runtime.state.StateBackend)void setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)
- field removed:
org.apache.flink.streaming.api.operators.AbstractStreamOperator- interface removed:
org.apache.flink.streaming.api.operators.SetupableStreamOperator
- method modified:
(<- NON_FINAL) void initializeState(org.apache.flink.streaming.api.operators.StreamTaskStateInitializer)PROTECTED (<- PUBLIC) void setProcessingTimeService(org.apache.flink.streaming.runtime.tasks.ProcessingTimeService)PROTECTED (<- PUBLIC) void setup(org.apache.flink.streaming.runtime.tasks.StreamTask<?,?><?,?>, org.apache.flink.streaming.api.graph.StreamConfig, org.apache.flink.streaming.api.operators.Output<org.apache.flink.streaming.runtime.streamrecord.StreamRecord<OUT>><org.apache.flink.streaming.runtime.streamrecord.StreamRecord<OUT>>)
- interface removed:
org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2- method modified:
(<- NON_FINAL) void initializeState(org.apache.flink.streaming.api.operators.StreamTaskStateInitializer)
- method modified:
org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator- method modified:
PROTECTED (<- PUBLIC) void setup(org.apache.flink.streaming.runtime.tasks.StreamTask<?,?><?,?>, org.apache.flink.streaming.api.graph.StreamConfig, org.apache.flink.streaming.api.operators.Output<org.apache.flink.streaming.runtime.streamrecord.StreamRecord<OUT>><org.apache.flink.streaming.runtime.streamrecord.StreamRecord<OUT>>)
- method modified:
org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows- method removed:
org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows withGap(org.apache.flink.streaming.api.windowing.time.Time)
- method removed:
org.apache.flink.streaming.api.windowing.assigners.ProcessingTimeSessionWindows- method removed:
org.apache.flink.streaming.api.windowing.assigners.ProcessingTimeSessionWindows withGap(org.apache.flink.streaming.api.windowing.time.Time)
- method removed:
org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows- method removed:
org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows of(org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows of(org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time)
- method removed:
org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows- method removed:
org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows of(org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows of(org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time)
- method removed:
org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows- method removed:
org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows of(org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows of(org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows of(org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.assigners.WindowStagger)
- method removed:
org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows- method removed:
org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows of(org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows of(org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows of(org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.time.Time, org.apache.flink.streaming.api.windowing.assigners.WindowStagger)
- method removed:
org.apache.flink.streaming.api.windowing.assigners.WindowAssigner- method modified:
(<- NON_ABSTRACT) org.apache.flink.streaming.api.windowing.triggers.Trigger<T,W><T,W> getDefaultTrigger()
- method removed:
org.apache.flink.streaming.api.windowing.triggers.Trigger<T,W> getDefaultTrigger(org.apache.flink.streaming.api.environment.StreamExecutionEnvironment)
- method modified:
org.apache.flink.streaming.api.windowing.evictors.TimeEvictor- method removed:
org.apache.flink.streaming.api.windowing.evictors.TimeEvictor<W> of(org.apache.flink.streaming.api.windowing.time.Time)org.apache.flink.streaming.api.windowing.evictors.TimeEvictor<W> of(org.apache.flink.streaming.api.windowing.time.Time, boolean)
- method removed:
org.apache.flink.streaming.api.windowing.triggers.ContinuousEventTimeTrigger- method removed:
org.apache.flink.streaming.api.windowing.triggers.ContinuousEventTimeTrigger<W> of(org.apache.flink.streaming.api.windowing.time.Time)
- method removed:
org.apache.flink.streaming.api.windowing.triggers.ContinuousProcessingTimeTrigger- method removed:
org.apache.flink.streaming.api.windowing.triggers.ContinuousProcessingTimeTrigger<W> of(org.apache.flink.streaming.api.windowing.time.Time)
- method removed:
org.apache.flink.streaming.api.windowing.triggers.Trigger$TriggerContext- method removed:
org.apache.flink.api.common.state.ValueState<S> getKeyValueState(java.lang.String, java.lang.Class<S>, java.io.Serializable)org.apache.flink.api.common.state.ValueState<S> getKeyValueState(java.lang.String, org.apache.flink.api.common.typeinfo.TypeInformation<S>, java.io.Serializable)
- method removed:
org.apache.flink.streaming.experimental.CollectSink- interface removed:
org.apache.flink.streaming.api.functions.sink.SinkFunction
- superclass modified:
org.apache.flink.streaming.api.functions.sink.legacy.RichSinkFunction (<- org.apache.flink.streaming.api.functions.sink.RichSinkFunction)
- interface removed:
org.apache.flink.types.DoubleValue- interface removed:
org.apache.flink.types.Key
- interface removed:
org.apache.flink.types.FloatValue- interface removed:
org.apache.flink.types.Key
- interface removed:
org.apache.flink.types.NormalizableKey- interface removed:
org.apache.flink.core.io.IOReadableWritableorg.apache.flink.types.Valueorg.apache.flink.types.Keyjava.io.Serializable
- interface removed:
org.apache.flink.test.junit5.MiniClusterExtension- method removed:
org.apache.flink.test.util.TestEnvironment getTestEnvironment()
- method removed:
org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Integer> PORTorg.apache.flink.configuration.ConfigOption<java.lang.String> HOST
- field removed:
org.apache.flink.formats.csv.CsvReaderFormat- method removed:
org.apache.flink.formats.csv.CsvReaderFormat<T> forSchema(org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper, org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema, org.apache.flink.api.common.typeinfo.TypeInformation<T>)
- method removed:
org.apache.flink.state.forst.ForStOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.String> REMOTE_DIRECTORY
- field removed:
org.apache.flink.state.forst.ForStOptionsFactory- method removed:
org.rocksdb.ColumnFamilyOptions createColumnOptions(org.rocksdb.ColumnFamilyOptions, java.util.Collection<java.lang.AutoCloseable>)org.rocksdb.DBOptions createDBOptions(org.rocksdb.DBOptions, java.util.Collection<java.lang.AutoCloseable>)org.rocksdb.ReadOptions createReadOptions(org.rocksdb.ReadOptions, java.util.Collection<java.lang.AutoCloseable>)org.rocksdb.WriteOptions createWriteOptions(org.rocksdb.WriteOptions, java.util.Collection<java.lang.AutoCloseable>)
- method removed:
org.apache.flink.table.client.config.SqlClientOptions- field removed:
org.apache.flink.configuration.ConfigOption<java.lang.Integer> DISPLAY_MAX_COLUMN_WIDTH
- field removed:
org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo- method removed:
org.apache.flink.api.common.typeutils.TypeSerializer<java.util.SortedMap<K,V>> createSerializer(org.apache.flink.api.common.ExecutionConfig)
- method removed:
org.apache.flink.connector.file.sink.FileSink- method removed:
org.apache.flink.api.connector.sink2.SinkWriter<IN> createWriter(org.apache.flink.api.connector.sink2.Sink$InitContext)
- method removed:
org.apache.flink.connector.file.src.FileSource- method removed:
org.apache.flink.connector.file.src.FileSource$FileSourceBuilder<T> forRecordFileFormat(org.apache.flink.connector.file.src.reader.FileRecordFormat<T>, org.apache.flink.core.fs.Path[])
- method removed:
org.apache.flink.connector.file.src.FileSourceSplit- constructor removed:
FileSourceSplit(java.lang.String, org.apache.flink.core.fs.Path, long, long)FileSourceSplit(java.lang.String, org.apache.flink.core.fs.Path, long, long, java.lang.String[])FileSourceSplit(java.lang.String, org.apache.flink.core.fs.Path, long, long, java.lang.String[], org.apache.flink.connector.file.src.util.CheckpointedPosition)
- constructor removed:
org.apache.flink.state.api.functions.KeyedStateReaderFunction- method removed:
void open(org.apache.flink.configuration.Configuration)
- method removed:
org.apache.flink.state.api.OperatorTransformation- method removed:
org.apache.flink.state.api.OneInputOperatorTransformation<T> bootstrapWith(org.apache.flink.api.java.DataSet<T>)
- method removed:
org.apache.flink.state.api.SavepointReader- method removed:
org.apache.flink.streaming.api.datastream.DataStream<org.apache.flink.api.java.tuple.Tuple2<K,V>> readBroadcastState(java.lang.String, java.lang.String, org.apache.flink.api.common.typeinfo.TypeInformation<K>, org.apache.flink.api.common.typeinfo.TypeInformation<V>)org.apache.flink.streaming.api.datastream.DataStream<org.apache.flink.api.java.tuple.Tuple2<K,V>> readBroadcastState(java.lang.String, java.lang.String, org.apache.flink.api.common.typeinfo.TypeInformation<K>, org.apache.flink.api.common.typeinfo.TypeInformation<V>, org.apache.flink.api.common.typeutils.TypeSerializer<K>, org.apache.flink.api.common.typeutils.TypeSerializer<V>)org.apache.flink.streaming.api.datastream.DataStream<OUT> readKeyedState(java.lang.String, org.apache.flink.state.api.functions.KeyedStateReaderFunction<K,OUT>)org.apache.flink.streaming.api.datastream.DataStream<OUT> readKeyedState(java.lang.String, org.apache.flink.state.api.functions.KeyedStateReaderFunction<K,OUT>, org.apache.flink.api.common.typeinfo.TypeInformation<K>, org.apache.flink.api.common.typeinfo.TypeInformation<OUT>)org.apache.flink.streaming.api.datastream.DataStream<T> readListState(java.lang.String, java.lang.String, org.apache.flink.api.common.typeinfo.TypeInformation<T>)org.apache.flink.streaming.api.datastream.DataStream<T> readListState(java.lang.String, java.lang.String, org.apache.flink.api.common.typeinfo.TypeInformation<T>, org.apache.flink.api.common.typeutils.TypeSerializer<T>)org.apache.flink.streaming.api.datastream.DataStream<T> readUnionState(java.lang.String, java.lang.String, org.apache.flink.api.common.typeinfo.TypeInformation<T>)org.apache.flink.streaming.api.datastream.DataStream<T> readUnionState(java.lang.String, java.lang.String, org.apache.flink.api.common.typeinfo.TypeInformation<T>, org.apache.flink.api.common.typeutils.TypeSerializer<T>)
- method removed:
org.apache.flink.state.api.SavepointWriter- method removed:
org.apache.flink.state.api.SavepointWriter fromExistingSavepoint(java.lang.String)org.apache.flink.state.api.SavepointWriter fromExistingSavepoint(java.lang.String, org.apache.flink.runtime.state.StateBackend)org.apache.flink.state.api.SavepointWriter newSavepoint(int)org.apache.flink.state.api.SavepointWriter newSavepoint(org.apache.flink.runtime.state.StateBackend, int)org.apache.flink.state.api.SavepointWriter removeOperator(java.lang.String)org.apache.flink.state.api.SavepointWriter withOperator(java.lang.String, org.apache.flink.state.api.StateBootstrapTransformation<T>)
- method removed:
org.apache.flink.state.api.SavepointWriterOperatorFactory- method modified:
org.apache.flink.streaming.api.operators.StreamOperatorFactory<org.apache.flink.state.api.output.TaggedOperatorSubtaskState><org.apache.flink.state.api.output.TaggedOperatorSubtaskState> (<-org.apache.flink.streaming.api.operators.StreamOperator<org.apache.flink.state.api.output.TaggedOperatorSubtaskState><org.apache.flink.state.api.output.TaggedOperatorSubtaskState>) createOperator(long, org.apache.flink.core.fs.Path)
- method modified:
List of removed configuration options #
- cluster.evenly-spread-out-slots
- execution.async-state.buffer-size
- execution.async-state.buffer-timeout
- execution.async-state.in-flight-records-limit
- fine-grained.shuffle-mode.all-blocking
- high-availability.job.delay
- high-availability.zookeeper.path.jobgraphs
- high-availability.zookeeper.path.running-registry
- jobmanager.adaptive-batch-scheduler.avg-data-volume-per-task
- jobmanager.adaptive-batch-scheduler.default-source-parallelism
- jobmanager.adaptive-batch-scheduler.max-parallelism
- jobmanager.adaptive-batch-scheduler.min-parallelism
- jobmanager.adaptive-batch-scheduler.speculative.block-slow-node-duration
- jobmanager.adaptive-batch-scheduler.speculative.enabled
- jobmanager.adaptive-batch-scheduler.speculative.max-concurrent-executions
- jobmanager.heap.mb
- jobmanager.heap.size
- jobmanager.web.address
- jobmanager.web.backpressure.cleanup-interval
- jobmanager.web.backpressure.delay-between-samples
- jobmanager.web.backpressure.num-samples
- jobmanager.web.backpressure.refresh-interval
- jobmanager.web.port;
- jobmanager.web.ssl.enabled
- local.number-resourcemanager
- pipeline.auto-type-registration
- pipeline.default-kryo-serializers
- pipeline.registered-kryo-types
- pipeline.registered-pojo-types
- recovery.job.delay
- resourcemanager.taskmanager-release.wait.result.consumed
- security.kerberos.fetch.delegation-token
- security.kerberos.tokens.renewal.retry.backoff
- security.kerberos.tokens.renewal.time-ratio
- security.ssl.enabled
- slotmanager.taskmanager-timeout
- sql-client.display.max-column-width
- state.backend.async
- state.backend.forst.remote-dir
- state.backend.latency-track.history-size
- state.backend.latency-track.keyed-state-enabled
- state.backend.latency-track.sample-interval
- state.backend.latency-track.state-name-as-variable
- state.backend.local-recovery
- state.backend.rocksdb.checkpointdir
- state.backend.type
- streaming-source.consume-order
- table.exec.deduplicate.insert-and-updateafter-sensitive.enabled
- table.exec.deduplicate.mini-batch.compact-changes.enabled
- table.exec.legacy-transformation-uids
- table.exec.shuffle-mode
- table.exec.topn-cache-size
- table.optimizer.source.aggregate-pushdown-enabled
- table.optimizer.source.predicate-pushdown-enabled
- table.optimizer.sql-to-rel.project.merge.enabled
- taskmanager.exit-on-fatal-akka-error
- taskmanager.heap.mb
- taskmanager.heap.size
- taskmanager.initial-registration-pause
- taskmanager.max-registration-pause
- taskmanager.net.client.numThreads
- taskmanager.net.num-arenas
- taskmanager.net.sendReceiveBufferSize
- taskmanager.net.server.backlog
- taskmanager.net.server.numThreads
- taskmanager.net.transport
- taskmanager.network.batch-shuffle.compression.enabled
- taskmanager.network.blocking-shuffle.compression.enabled
- taskmanager.network.blocking-shuffle.type
- taskmanager.network.hybrid-shuffle.enable-new-mode
- taskmanager.network.hybrid-shuffle.num-retained-in-memory-regions-max
- taskmanager.network.hybrid-shuffle.spill-index-region-group-size
- taskmanager.network.hybrid-shuffle.spill-index-segment-size
- taskmanager.network.max-num-tcp-connections
- taskmanager.network.memory.buffers-per-channel
- taskmanager.network.memory.exclusive-buffers-request-timeout-ms
- taskmanager.network.memory.floating-buffers-per-gate
- taskmanager.network.memory.fraction
- taskmanager.network.memory.max
- taskmanager.network.memory.max-buffers-per-channel
- taskmanager.network.memory.max-overdraft-buffers-per-gate
- taskmanager.network.memory.min
- taskmanager.network.netty.client.numThreads
- taskmanager.network.netty.num-arenas
- taskmanager.network.netty.sendReceiveBufferSize
- taskmanager.network.netty.server.backlog
- taskmanager.network.netty.server.numThreads
- taskmanager.network.netty.transport
- taskmanager.network.numberOfBuffers
- taskmanager.network.sort-shuffle.min-parallelism
- taskmanager.refused-registration-pause
- taskmanager.registration.initial-backoff
- taskmanager.registration.max-backoff
- taskmanager.registration.refused-backoff
- web.address
- web.backpressure.cleanup-interval
- web.backpressure.delay-between-samples
- web.backpressure.num-samples
- web.backpressure.refresh-interval
- web.port
- web.ssl.enabled
List of REST APIs changes #
| REST API | Changes |
|---|---|
| /taskmanagers/:taskmanagerid | In its response, “metrics.memorySegmentsAvailable” and “metrics.memorySegmentsTotal” are removed. |
| /jobs/:jobid/config | In its response, the “execution-mode” property is removed. |
| /jars/:jarid/run | In its request, the internal type of “claimMode” and “restoreMode” are changed from RestoreMode to RecoveryClaimMode, but their json structure is not affected. |
| /jobs/:jobid/vertices/:vertexid /jobs/:jobid/vertices/:vertexid/subtasks/accumulators /jobs/:jobid/vertices/:vertexid/subtasks/:subtaskindex /jobs/:jobid/vertices/:vertexid/subtasks/:subtaskindex/attempts/:attempt /jobs/:jobid/vertices/:vertexid/subtasktimes /jobs/:jobid/vertices/:vertexid/taskmanagers /jobs/:jobid/taskmanagers/:taskmanagerid/log-url |
In their responses, the “host”, “subtasks.host” or “taskmanagers.host” property is removed. |
List of removed CLI options #
- sql-client.sh:
-u,--update <SQL update statement>is removed
- flink-client:
run-applicationaction is removed: Please userun -t kubernetes-applicationto run Kubernetes Application mode