| |
| <!DOCTYPE html> |
| <html lang="en" dir=ZgotmplZ> |
| |
| <head> |
| |
| |
| |
| <link rel="stylesheet" href="/bootstrap/css/bootstrap.min.css"> |
| <script src="/bootstrap/js/bootstrap.bundle.min.js"></script> |
| <link rel="stylesheet" type="text/css" href="/font-awesome/css/font-awesome.min.css"> |
| <script src="/js/anchor.min.js"></script> |
| <script src="/js/flink.js"></script> |
| <link rel="canonical" href="https://flink.apache.org/2025/03/24/apache-flink-2.0.0-a-new-era-of-real-time-data-processing/"> |
| |
| <meta charset="UTF-8"> |
| <meta name="viewport" content="width=device-width, initial-scale=1.0"> |
| <meta name="description" content="Today, 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."> |
| <meta name="theme-color" content="#FFFFFF"><meta property="og:title" content="Apache Flink 2.0.0: A new Era of Real-Time Data Processing" /> |
| <meta property="og:description" content="Today, 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." /> |
| <meta property="og:type" content="article" /> |
| <meta property="og:url" content="https://flink.apache.org/2025/03/24/apache-flink-2.0.0-a-new-era-of-real-time-data-processing/" /><meta property="article:section" content="posts" /> |
| <meta property="article:published_time" content="2025-03-24T08:00:00+00:00" /> |
| <meta property="article:modified_time" content="2025-03-24T08:00:00+00:00" /> |
| <title>Apache Flink 2.0.0: A new Era of Real-Time Data Processing | Apache Flink</title> |
| <link rel="manifest" href="/manifest.json"> |
| <link rel="icon" href="/favicon.png" type="image/x-icon"> |
| <link rel="stylesheet" href="/book.min.22eceb4d17baa9cdc0f57345edd6f215a40474022dfee39b63befb5fb3c596b5.css" integrity="sha256-IuzrTRe6qc3A9XNF7dbyFaQEdAIt/uObY777X7PFlrU="> |
| <script defer src="/en.search.min.2dcb18b1dc51a58cf008db2e374638721a2e778c982365ff222f54319caecd83.js" integrity="sha256-LcsYsdxRpYzwCNsuN0Y4choud4yYI2X/Ii9UMZyuzYM="></script> |
| <!-- |
| Made with Book Theme |
| https://github.com/alex-shpak/hugo-book |
| --> |
| |
| <meta name="generator" content="Hugo 0.124.1"> |
| |
| |
| <script> |
| var _paq = window._paq = window._paq || []; |
| |
| |
| _paq.push(['disableCookies']); |
| |
| _paq.push(["setDomains", ["*.flink.apache.org","*.nightlies.apache.org/flink"]]); |
| _paq.push(['trackPageView']); |
| _paq.push(['enableLinkTracking']); |
| (function() { |
| var u="//analytics.apache.org/"; |
| _paq.push(['setTrackerUrl', u+'matomo.php']); |
| _paq.push(['setSiteId', '1']); |
| var d=document, g=d.createElement('script'), s=d.getElementsByTagName('script')[0]; |
| g.async=true; g.src=u+'matomo.js'; s.parentNode.insertBefore(g,s); |
| })(); |
| </script> |
| |
| </head> |
| |
| <body dir=ZgotmplZ> |
| |
| |
| |
| <header> |
| <nav class="navbar navbar-expand-xl"> |
| <div class="container-fluid"> |
| <a class="navbar-brand" href="/"> |
| <img src="/img/logo/png/100/flink_squirrel_100_color.png" alt="Apache Flink" height="47" width="47" class="d-inline-block align-text-middle"> |
| <span>Apache Flink</span> |
| </a> |
| <button class="navbar-toggler" type="button" data-bs-toggle="collapse" data-bs-target="#navbarSupportedContent" aria-controls="navbarSupportedContent" aria-expanded="false" aria-label="Toggle navigation"> |
| <i class="fa fa-bars navbar-toggler-icon"></i> |
| </button> |
| <div class="collapse navbar-collapse" id="navbarSupportedContent"> |
| <ul class="navbar-nav"> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| <li class="nav-item dropdown"> |
| <a class="nav-link dropdown-toggle" href="#" role="button" data-bs-toggle="dropdown" aria-expanded="false">About</a> |
| <ul class="dropdown-menu"> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/what-is-flink/flink-architecture/">Architecture</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/what-is-flink/flink-applications/">Applications</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/what-is-flink/flink-operations/">Operations</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/what-is-flink/use-cases/">Use Cases</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/what-is-flink/powered-by/">Powered By</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/what-is-flink/roadmap/">Roadmap</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/what-is-flink/community/">Community & Project Info</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/what-is-flink/security/">Security</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/what-is-flink/special-thanks/">Special Thanks</a> |
| |
| |
| </li> |
| |
| </ul> |
| </li> |
| |
| |
| |
| |
| |
| <li class="nav-item dropdown"> |
| <a class="nav-link dropdown-toggle" href="#" role="button" data-bs-toggle="dropdown" aria-expanded="false">Getting Started</a> |
| <ul class="dropdown-menu"> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-docs-stable/docs/try-flink/local_installation/">With Flink<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-stable/docs/try-flink-kubernetes-operator/quick-start/">With Flink Kubernetes Operator<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-cdc-docs-stable/docs/get-started/introduction/">With Flink CDC<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-ml-docs-stable/docs/try-flink-ml/quick-start/">With Flink ML<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-statefun-docs-stable/getting-started/project-setup.html">With Flink Stateful Functions<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-docs-stable/docs/learn-flink/overview/">Training Course<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| </ul> |
| </li> |
| |
| |
| |
| |
| |
| <li class="nav-item dropdown"> |
| <a class="nav-link dropdown-toggle" href="#" role="button" data-bs-toggle="dropdown" aria-expanded="false">Documentation</a> |
| <ul class="dropdown-menu"> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-docs-stable/">Flink 2.1 (stable)<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-docs-lts/">Flink 1.20 (LTS)<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-docs-master/">Flink Master (snapshot)<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-stable/">Kubernetes Operator 1.13 (latest)<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-main">Kubernetes Operator Main (snapshot)<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-cdc-docs-stable">CDC 3.5 (stable)<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-cdc-docs-master">CDC Master (snapshot)<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-ml-docs-stable/">ML 2.3 (stable)<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-ml-docs-master">ML Master (snapshot)<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-statefun-docs-stable/">Stateful Functions 3.3 (stable)<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-statefun-docs-master">Stateful Functions Master (snapshot)<i class="link fa fa-external-link title" aria-hidden="true"></i> |
| </a> |
| |
| |
| </li> |
| |
| </ul> |
| </li> |
| |
| |
| |
| |
| |
| <li class="nav-item dropdown"> |
| <a class="nav-link dropdown-toggle" href="#" role="button" data-bs-toggle="dropdown" aria-expanded="false">How to Contribute</a> |
| <ul class="dropdown-menu"> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/how-to-contribute/overview/">Overview</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/how-to-contribute/contribute-code/">Contribute Code</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/how-to-contribute/reviewing-prs/">Review Pull Requests</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/how-to-contribute/code-style-and-quality-preamble/">Code Style and Quality Guide</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/how-to-contribute/contribute-documentation/">Contribute Documentation</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/how-to-contribute/documentation-style-guide/">Documentation Style Guide</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/how-to-contribute/improve-website/">Contribute to the Website</a> |
| |
| |
| </li> |
| |
| <li> |
| |
| |
| <a class="dropdown-item" href="/how-to-contribute/getting-help/">Getting Help</a> |
| |
| |
| </li> |
| |
| </ul> |
| </li> |
| |
| |
| |
| |
| |
| |
| |
| |
| <li class="nav-item"> |
| |
| |
| <a class="nav-link" href="/posts/">Flink Blog</a> |
| |
| |
| </li> |
| |
| |
| |
| |
| |
| <li class="nav-item"> |
| |
| |
| <a class="nav-link" href="/downloads/">Downloads</a> |
| |
| |
| </li> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </ul> |
| <div class="book-search"> |
| <div class="book-search-spinner hidden"> |
| <i class="fa fa-refresh fa-spin"></i> |
| </div> |
| <form class="search-bar d-flex" onsubmit="return false;"su> |
| <input type="text" id="book-search-input" placeholder="Search" aria-label="Search" maxlength="64" data-hotkeys="s/"> |
| <i class="fa fa-search search"></i> |
| <i class="fa fa-circle-o-notch fa-spin spinner"></i> |
| </form> |
| <div class="book-search-spinner hidden"></div> |
| <ul id="book-search-results"></ul> |
| </div> |
| </div> |
| </div> |
| </nav> |
| <div class="navbar-clearfix"></div> |
| </header> |
| |
| |
| <main class="flex"> |
| <section class="container book-page"> |
| |
| <article class="markdown"> |
| <h1> |
| <a href="/2025/03/24/apache-flink-2.0.0-a-new-era-of-real-time-data-processing/">Apache Flink 2.0.0: A new Era of Real-Time Data Processing</a> |
| </h1> |
| |
| |
| |
| March 24, 2025 - |
| |
| |
| |
| Xintong Song |
| |
| |
| |
| |
| <p><p>Today, 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.</p> |
| <p>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!</p> |
| <p>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.</p> |
| <p>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.</p> |
| <ul> |
| <li>The <strong>Disaggregated State Management</strong> architecture enables more efficient resource utilization in cloud-native environments, ensuring high-performance real-time processing while minimizing resource overhead.</li> |
| <li>The introduction and refinement of <strong>Materialized Tables</strong> 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 <strong>Batch Execution</strong> 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.</li> |
| <li>Additionally, the deep integration with Apache Paimon strengthens the <strong>Streaming Lakehouse</strong> architecture, making Flink a leading solution for real-time data lake use cases.</li> |
| <li>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 <strong>AI workflows</strong>, ensuring that Flink remains at the forefront of real-time data processing innovations.</li> |
| </ul> |
| <p>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.</p> |
| <p>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.</p> |
| <h1 id="highlights-of-new-features"> |
| Highlights of New Features |
| <a class="anchor" href="#highlights-of-new-features">#</a> |
| </h1> |
| <h2 id="disaggregated-state-management"> |
| Disaggregated State Management |
| <a class="anchor" href="#disaggregated-state-management">#</a> |
| </h2> |
| <p>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.</p> |
| <p>This new architecture solves the following challenges brought in the cloud-native era for Flink.</p> |
| <ol> |
| <li>Local Disk Constraints in containerization</li> |
| <li>Spiky Resource Usage caused by compaction in the current state model</li> |
| <li>Fast Rescaling for jobs with large states (hundreds of Terabytes)</li> |
| <li>Light and Fast Checkpoint in a native way</li> |
| </ol> |
| <p>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.</p> |
| <p>Flink 2.0 delivers a comprehensive end-to-end experience for disaggregated state management, encompassing both the runtime and SQL operator layers:</p> |
| <h3 id="asynchronous-execution-model"> |
| Asynchronous Execution Model |
| <a class="anchor" href="#asynchronous-execution-model">#</a> |
| </h3> |
| <ul> |
| <li>Out-of-Order Record Processing: Decouples state access from computation to enable parallel execution.</li> |
| <li>Asynchronous State APIs: Full support for non-blocking state operations during checkpointing, reducing latency and improving resource utilization.</li> |
| <li>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.</li> |
| </ul> |
| <h3 id="enhanced-sql-operators"> |
| Enhanced SQL Operators |
| <a class="anchor" href="#enhanced-sql-operators">#</a> |
| </h3> |
| <ul> |
| <li>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.</li> |
| <li>Users can enable this feature by setting the configuration parameter <code>table.exec.async-state.enabled</code>. Once activated, all supported SQL operators within a job automatically switch to asynchronous state access mode without requiring code changes.</li> |
| <li>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.</li> |
| </ul> |
| <h3 id="forst---a-disaggregated-state-backend"> |
| ForSt - A Disaggregated State Backend |
| <a class="anchor" href="#forst---a-disaggregated-state-backend">#</a> |
| </h3> |
| <ul> |
| <li>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.</li> |
| <li>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.</li> |
| </ul> |
| <h3 id="performance-evaluation-on-nexmark"> |
| Performance Evaluation (on Nexmark) |
| <a class="anchor" href="#performance-evaluation-on-nexmark">#</a> |
| </h3> |
| <p><a href="https://github.com/nexmark/nexmark">Nexmark</a> 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.</p> |
| <div style="text-align: center;"> |
| <img src="/img/blog/2025-03-24-release-2.0.0/nexmark.png" style="width:70%;margin:15px"> |
| </div> |
| <ul> |
| <li>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.</li> |
| <li>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.</li> |
| <li>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.</li> |
| </ul> |
| <p>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.</p> |
| <h2 id="stream-batch-unification"> |
| Stream-Batch Unification |
| <a class="anchor" href="#stream-batch-unification">#</a> |
| </h2> |
| <h3 id="materialized-table"> |
| Materialized Table |
| <a class="anchor" href="#materialized-table">#</a> |
| </h3> |
| <p>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.</p> |
| <p>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:</p> |
| <p><strong>Query Modifications</strong> - 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.</p> |
| <p><strong>Kubernetes/Yarn Submission</strong> - 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.</p> |
| <p><strong>Ecosystem Integration</strong> - 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.</p> |
| <p>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.</p> |
| <h3 id="adaptive-batch-execution"> |
| Adaptive Batch Execution |
| <a class="anchor" href="#adaptive-batch-execution">#</a> |
| </h3> |
| <p>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:</p> |
| <p><strong>Adaptive Broadcast Join</strong> - 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.</p> |
| <p><strong>Automatic Join Skew Optimization</strong> - 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.</p> |
| <p>See more details about the capabilities and usages of Flink’s <a href="https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/deployment/adaptive_batch/">Adaptive Batch Execution</a>.</p> |
| <h3 id="performance"> |
| Performance |
| <a class="anchor" href="#performance">#</a> |
| </h3> |
| <p>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 <code>ANALYZE TABLE</code> statement, Flink 2.0 achieves an 8% performance improvement compared to Flink 1.20; without additional statistical information, it achieves a 16% performance improvement.</p> |
| <h2 id="streaming-lakehouse"> |
| Streaming Lakehouse |
| <a class="anchor" href="#streaming-lakehouse">#</a> |
| </h2> |
| <p>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.</p> |
| <ul> |
| <li>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.</li> |
| <li>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.</li> |
| <li>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.</li> |
| <li>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.</li> |
| <li>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.</li> |
| </ul> |
| <h2 id="ai"> |
| AI |
| <a class="anchor" href="#ai">#</a> |
| </h2> |
| <p>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.</p> |
| <p>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.</p> |
| <p>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.</p> |
| <h2 id="misc"> |
| Misc |
| <a class="anchor" href="#misc">#</a> |
| </h2> |
| <h3 id="datastream-v2-api"> |
| DataStream V2 API |
| <a class="anchor" href="#datastream-v2-api">#</a> |
| </h3> |
| <p>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.</p> |
| <p>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.</p> |
| <p>See <a href="https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/dev/datastream-v2/overview/">documentations</a> for more details</p> |
| <p><strong>NOTICE:</strong> The new DataStream API is currently in the experimental stage and is not yet stable, thus not recommended for production usage at the moment.</p> |
| <h3 id="sql-gateway-supports-application-mode"> |
| SQL gateway supports application mode |
| <a class="anchor" href="#sql-gateway-supports-application-mode">#</a> |
| </h3> |
| <p>SQL gateway now supports executing SQL jobs in application mode, serving as a replacement of the removed per-job deployment mode.</p> |
| <h3 id="sql-syntax-enhancements"> |
| SQL Syntax Enhancements |
| <a class="anchor" href="#sql-syntax-enhancements">#</a> |
| </h3> |
| <p>Flink SQL now supports C-style escape strings. See the <a href="https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/dev/table/sql/queries/overview/#syntax">documentation</a> for more details.</p> |
| <p>A new <code>QUALIFY</code> clause has been added as a more concise syntax for filtering outputs of window functions. Demonstrations on this can be found in the <a href="https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/dev/table/sql/queries/topn/">Top-N</a> and <a href="https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/dev/table/sql/queries/deduplication/">Deduplication</a> examples.</p> |
| <p>For table function calls it is now possible to use them without <code>TABLE()</code> wrapper in <code>FROM</code>. Examples could be found in updated documentation for <a href="https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/dev/table/sql/queries/window-tvf">Windowing table-valued functions</a>. Queries with such wrapper will continue working as before.</p> |
| <h3 id="java-supports"> |
| Java Supports |
| <a class="anchor" href="#java-supports">#</a> |
| </h3> |
| <p>Starting the 2.0 version, Flink officially supports Java 21.</p> |
| <p>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.</p> |
| <p>Meanwhile, Java 8 is no longer supported.</p> |
| <h3 id="serialization-improvements"> |
| Serialization Improvements |
| <a class="anchor" href="#serialization-improvements">#</a> |
| </h3> |
| <p>Flink 2.0 introduces much more efficient built-in serializers for collection types (i.e., Map / List / Set), which is enabled by default.</p> |
| <p>We have also upgraded Kryo to version 5.6, which is faster and more memory efficient, and has better supports for newer Java versions.</p> |
| <h1 id="breaking-changes"> |
| Breaking Changes |
| <a class="anchor" href="#breaking-changes">#</a> |
| </h1> |
| <h2 id="api"> |
| API |
| <a class="anchor" href="#api">#</a> |
| </h2> |
| <p>The following sets of APIs have been completely removed.</p> |
| <ul> |
| <li><strong>DataSet API.</strong> Please migrate to <a href="https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/dev/datastream/overview/">DataStream API</a>, or <a href="https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/dev/table/overview/">Table API/SQL</a> if applicable. See also <a href="https://nightlies.apache.org/flink/flink-docs-release-1.20/docs/dev/datastream/dataset_migration">How to Migrate from DataSet to DataStream</a>.</li> |
| <li><strong>Scala DataStream and DataSet API.</strong> Please migrate to the Java <a href="https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/dev/datastream/overview/">DataStream API</a>.</li> |
| <li><strong>SourceFuction, SinkFunction and Sink V1.</strong> Please migrate to <a href="https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/api/connector/source/Source.java">Source</a> and <a href="https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/api/connector/sink2/Sink.java">Sink V2</a>.</li> |
| <li><strong>TableSoure and TableSink.</strong> Please migrate to <a href="https://github.com/apache/flink/blob/master/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/DynamicTableSource.java">DynamicTableSource</a> and <a href="https://github.com/apache/flink/blob/master/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/DynamicTableSink.java">DynamicTableSink</a>. See also <a href="https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/dev/table/sourcessinks/">User-defined Sources & Sinks</a>.</li> |
| <li><strong>TableSchema, TableColumn and Types.</strong> Please migrate to <a href="https://github.com/apache/flink/blob/master/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java">Schema</a>, <a href="https://github.com/apache/flink/blob/master/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Column.java">Column</a> and <a href="https://github.com/apache/flink/blob/master/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/DataTypes.java">DataTypes</a> respectively.</li> |
| </ul> |
| <p>Some deprecated methods have been removed from <strong>DataStream API</strong>. See also the list of <a href="#breaking_programming_apis">breaking programming APIs</a>.</p> |
| <p>Some deprecated fields have been removed from <strong>REST API</strong>. See also the list of <a href="#breaking_rest_apis">breaking REST APIs</a>.</p> |
| <p><strong>NOTICE:</strong> 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 <strong>DO NOT USE</strong> them.</p> |
| <h3 id="connector-adaption-plan"> |
| Connector Adaption Plan |
| <a class="anchor" href="#connector-adaption-plan">#</a> |
| </h3> |
| <p>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.</p> |
| <ul> |
| <li>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.</li> |
| <li>We plan to gradually migrate the remaining first-party connectors within 3 subsequent minor releases (i.e., by Flink 2.3).</li> |
| </ul> |
| <h2 id="configuration"> |
| Configuration |
| <a class="anchor" href="#configuration">#</a> |
| </h2> |
| <p>Configuration options meet the following criteria are removed. See also the list of <a href="#removed_configs">removed configuration options</a>.</p> |
| <ul> |
| <li>Annotated as <code>@Public</code> and have been deprecated for at least 2 minor releases.</li> |
| <li>Annotated as <code>@PublicEvolving</code> and have been deprecated for at least 1 minor releases.</li> |
| </ul> |
| <p>The legacy configuration file <code>flink-conf.yaml</code> is no longer supported. Please use <code>config.yaml</code> with standard YAML format instead. A migration tool is provided to convert a legacy <code>flink-conf.yaml</code> into a new <code>config.yaml</code>. See <a href="https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/deployment/config/#migrate-from-flink-confyaml-to-configyaml">Migrate from flink-conf.yaml to config.yaml</a> for more details.</p> |
| <p>Configuration APIs that takes java objects as arguments are removed from <code>StreamExecutionEnvironment</code> and <code>ExecutionConfig</code>. They should now be set via <code>Configuration</code> and <code>ConfigOption</code>. See also the list of <a href="#breaking_programming_apis">breaking programming APIs</a>.</p> |
| <p>To avoid exposing internal interfaces, User-Defined Functions no longer have full access to <code>ExecutionConfig</code>. Instead, necessary functions such as <code>createSerializer()</code>, <code>getGlobalJobParameters()</code> and <code>isObjectReuseEnabled()</code> can now be accessed from <code>RuntimeContext</code> directly.</p> |
| <h2 id="misc-1"> |
| Misc |
| <a class="anchor" href="#misc-1">#</a> |
| </h2> |
| <ul> |
| <li>State Compatibility is not guaranteed between 1.x and 2.x.</li> |
| <li>Java 8 is no longer supported. The minimum Java version supported by Flink now is Java 11.</li> |
| <li>The Per-job deployment mode is no longer supported. Please use the Application mode instead.</li> |
| <li>Legacy Mode of Hybrid Shuffle is removed.</li> |
| </ul> |
| <h1 id="release-notes"> |
| Release notes |
| <a class="anchor" href="#release-notes">#</a> |
| </h1> |
| <p>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 <a href="https://nightlies.apache.org/flink/flink-docs-release-2.0/release-notes/flink-2.0/">release notes</a>.</p> |
| <h1 id="list-of-contributors"> |
| List of Contributors |
| <a class="anchor" href="#list-of-contributors">#</a> |
| </h1> |
| <p>The Apache Flink community would like to thank each one of the contributors that have made this release possible:</p> |
| <p>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, 余良, 皆非, 马越, 林尚泉</p> |
| <h1 id="appendix"> |
| Appendix |
| <a class="anchor" href="#appendix">#</a> |
| </h1> |
| <h2 id="list-of-breaking-change-programming-apis-a-namebreaking_programming_apis-"> |
| List of breaking change programming APIs <a name="breaking_programming_apis" /> |
| <a class="anchor" href="#list-of-breaking-change-programming-apis-a-namebreaking_programming_apis-">#</a> |
| </h2> |
| <h3 id="removed-classes"> |
| Removed Classes |
| <a class="anchor" href="#removed-classes">#</a> |
| </h3> |
| <ul> |
| <li><code>org.apache.flink.api.common.ExecutionConfig$SerializableSerializer</code></li> |
| <li><code>org.apache.flink.api.common.ExecutionMode</code></li> |
| <li><code>org.apache.flink.api.common.InputDependencyConstraint</code></li> |
| <li><code>org.apache.flink.api.common.restartstrategy.RestartStrategies$ExponentialDelayRestartStrategyConfiguration</code></li> |
| <li><code>org.apache.flink.api.common.restartstrategy.RestartStrategies$FailureRateRestartStrategyConfiguration</code></li> |
| <li><code>org.apache.flink.api.common.restartstrategy.RestartStrategies$FallbackRestartStrategyConfiguration</code></li> |
| <li><code>org.apache.flink.api.common.restartstrategy.RestartStrategies$FixedDelayRestartStrategyConfiguration</code></li> |
| <li><code>org.apache.flink.api.common.restartstrategy.RestartStrategies$NoRestartStrategyConfiguration</code></li> |
| <li><code>org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration</code></li> |
| <li><code>org.apache.flink.api.common.restartstrategy.RestartStrategies</code></li> |
| <li><code>org.apache.flink.api.common.time.Time</code></li> |
| <li><code>org.apache.flink.api.connector.sink.Committer</code></li> |
| <li><code>org.apache.flink.api.connector.sink.GlobalCommitter</code></li> |
| <li><code>org.apache.flink.api.connector.sink.Sink$InitContext</code></li> |
| <li><code>org.apache.flink.api.connector.sink.Sink$ProcessingTimeService$ProcessingTimeCallback</code></li> |
| <li><code>org.apache.flink.api.connector.sink.Sink$ProcessingTimeService</code></li> |
| <li><code>org.apache.flink.api.connector.sink.SinkWriter$Context</code></li> |
| <li><code>org.apache.flink.api.connector.sink.SinkWriter</code></li> |
| <li><code>org.apache.flink.api.connector.sink.Sink</code></li> |
| <li><code>org.apache.flink.api.connector.sink2.Sink$InitContextWrapper</code></li> |
| <li><code>org.apache.flink.api.connector.sink2.Sink$InitContext</code></li> |
| <li><code>org.apache.flink.api.connector.sink2.StatefulSink$StatefulSinkWriter</code></li> |
| <li><code>org.apache.flink.api.connector.sink2.StatefulSink$WithCompatibleState</code></li> |
| <li><code>org.apache.flink.api.connector.sink2.StatefulSink</code></li> |
| <li><code>org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink$PrecommittingSinkWriter</code></li> |
| <li><code>org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink</code></li> |
| <li><code>org.apache.flink.api.java.CollectionEnvironment</code></li> |
| <li><code>org.apache.flink.api.java.DataSet</code></li> |
| <li><code>org.apache.flink.api.java.ExecutionEnvironmentFactory</code></li> |
| <li><code>org.apache.flink.api.java.ExecutionEnvironment</code></li> |
| <li><code>org.apache.flink.api.java.LocalEnvironment</code></li> |
| <li><code>org.apache.flink.api.java.RemoteEnvironment</code></li> |
| <li><code>org.apache.flink.api.java.aggregation.Aggregations</code></li> |
| <li><code>org.apache.flink.api.java.aggregation.UnsupportedAggregationTypeException</code></li> |
| <li><code>org.apache.flink.api.java.functions.FlatMapIterator</code></li> |
| <li><code>org.apache.flink.api.java.functions.FunctionAnnotation$ForwardedFieldsFirst</code></li> |
| <li><code>org.apache.flink.api.java.functions.FunctionAnnotation$ForwardedFieldsSecond</code></li> |
| <li><code>org.apache.flink.api.java.functions.FunctionAnnotation$ForwardedFields</code></li> |
| <li><code>org.apache.flink.api.java.functions.FunctionAnnotation$NonForwardedFieldsFirst</code></li> |
| <li><code>org.apache.flink.api.java.functions.FunctionAnnotation$NonForwardedFieldsSecond</code></li> |
| <li><code>org.apache.flink.api.java.functions.FunctionAnnotation$NonForwardedFields</code></li> |
| <li><code>org.apache.flink.api.java.functions.FunctionAnnotation$ReadFieldsFirst</code></li> |
| <li><code>org.apache.flink.api.java.functions.FunctionAnnotation$ReadFieldsSecond</code></li> |
| <li><code>org.apache.flink.api.java.functions.FunctionAnnotation$ReadFields</code></li> |
| <li><code>org.apache.flink.api.java.functions.FunctionAnnotation</code></li> |
| <li><code>org.apache.flink.api.java.functions.GroupReduceIterator</code></li> |
| <li><code>org.apache.flink.api.java.io.CollectionInputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.CsvOutputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.CsvReader</code></li> |
| <li><code>org.apache.flink.api.java.io.DiscardingOutputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.IteratorInputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.LocalCollectionOutputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.ParallelIteratorInputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.PrimitiveInputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.PrintingOutputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.RowCsvInputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.SplitDataProperties$SourcePartitionerMarker</code></li> |
| <li><code>org.apache.flink.api.java.io.SplitDataProperties</code></li> |
| <li><code>org.apache.flink.api.java.io.TextInputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.TextOutputFormat$TextFormatter</code></li> |
| <li><code>org.apache.flink.api.java.io.TextOutputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.TextValueInputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.TypeSerializerInputFormat</code></li> |
| <li><code>org.apache.flink.api.java.io.TypeSerializerOutputFormat</code></li> |
| <li><code>org.apache.flink.api.java.operators.AggregateOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.CoGroupOperator$CoGroupOperatorSets</code></li> |
| <li><code>org.apache.flink.api.java.operators.CoGroupOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.CrossOperator$DefaultCross</code></li> |
| <li><code>org.apache.flink.api.java.operators.CrossOperator$ProjectCross</code></li> |
| <li><code>org.apache.flink.api.java.operators.CrossOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.CustomUnaryOperation</code></li> |
| <li><code>org.apache.flink.api.java.operators.DataSink</code></li> |
| <li><code>org.apache.flink.api.java.operators.DataSource</code></li> |
| <li><code>org.apache.flink.api.java.operators.DeltaIteration$SolutionSetPlaceHolder</code></li> |
| <li><code>org.apache.flink.api.java.operators.DeltaIteration$WorksetPlaceHolder</code></li> |
| <li><code>org.apache.flink.api.java.operators.DeltaIterationResultSet</code></li> |
| <li><code>org.apache.flink.api.java.operators.DeltaIteration</code></li> |
| <li><code>org.apache.flink.api.java.operators.DistinctOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.FilterOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.FlatMapOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.GroupCombineOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.GroupReduceOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.Grouping</code></li> |
| <li><code>org.apache.flink.api.java.operators.IterativeDataSet</code></li> |
| <li><code>org.apache.flink.api.java.operators.JoinOperator$DefaultJoin</code></li> |
| <li><code>org.apache.flink.api.java.operators.JoinOperator$EquiJoin</code></li> |
| <li><code>org.apache.flink.api.java.operators.JoinOperator$JoinOperatorSets$JoinOperatorSetsPredicate</code></li> |
| <li><code>org.apache.flink.api.java.operators.JoinOperator$JoinOperatorSets</code></li> |
| <li><code>org.apache.flink.api.java.operators.JoinOperator$ProjectJoin</code></li> |
| <li><code>org.apache.flink.api.java.operators.JoinOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.MapOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.MapPartitionOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.Operator</code></li> |
| <li><code>org.apache.flink.api.java.operators.PartitionOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.ProjectOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.ReduceOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.SingleInputOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.SingleInputUdfOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.SortPartitionOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.SortedGrouping</code></li> |
| <li><code>org.apache.flink.api.java.operators.TwoInputOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.TwoInputUdfOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.UdfOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.UnionOperator</code></li> |
| <li><code>org.apache.flink.api.java.operators.UnsortedGrouping</code></li> |
| <li><code>org.apache.flink.api.java.operators.join.JoinFunctionAssigner</code></li> |
| <li><code>org.apache.flink.api.java.operators.join.JoinOperatorSetsBase$JoinOperatorSetsPredicateBase</code></li> |
| <li><code>org.apache.flink.api.java.operators.join.JoinOperatorSetsBase</code></li> |
| <li><code>org.apache.flink.api.java.operators.join.JoinType</code></li> |
| <li><code>org.apache.flink.api.java.summarize.BooleanColumnSummary</code></li> |
| <li><code>org.apache.flink.api.java.summarize.ColumnSummary</code></li> |
| <li><code>org.apache.flink.api.java.summarize.NumericColumnSummary</code></li> |
| <li><code>org.apache.flink.api.java.summarize.ObjectColumnSummary</code></li> |
| <li><code>org.apache.flink.api.java.summarize.StringColumnSummary</code></li> |
| <li><code>org.apache.flink.api.java.utils.AbstractParameterTool</code></li> |
| <li><code>org.apache.flink.api.java.utils.DataSetUtils</code></li> |
| <li><code>org.apache.flink.api.java.utils.MultipleParameterTool</code></li> |
| <li><code>org.apache.flink.api.java.utils.ParameterTool</code></li> |
| <li><code>org.apache.flink.configuration.AkkaOptions</code></li> |
| <li><code>org.apache.flink.connector.file.src.reader.FileRecordFormat$Reader</code></li> |
| <li><code>org.apache.flink.connector.file.src.reader.FileRecordFormat</code></li> |
| <li><code>org.apache.flink.connector.testframe.external.sink.DataStreamSinkV1ExternalContext</code></li> |
| <li><code>org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend$PriorityQueueStateType</code></li> |
| <li><code>org.apache.flink.core.execution.RestoreMode</code></li> |
| <li><code>org.apache.flink.datastream.api.stream.KeyedPartitionStream$TwoKeyedPartitionStreams</code></li> |
| <li><code>org.apache.flink.datastream.api.stream.NonKeyedPartitionStream$TwoNonKeyedPartitionStreams</code></li> |
| <li><code>org.apache.flink.formats.avro.AvroRowDeserializationSchema</code></li> |
| <li><code>org.apache.flink.formats.csv.CsvRowDeserializationSchema$Builder</code></li> |
| <li><code>org.apache.flink.formats.csv.CsvRowDeserializationSchema</code></li> |
| <li><code>org.apache.flink.formats.csv.CsvRowSerializationSchema$Builder</code></li> |
| <li><code>org.apache.flink.formats.csv.CsvRowSerializationSchema</code></li> |
| <li><code>org.apache.flink.formats.json.JsonRowDeserializationSchema$Builder</code></li> |
| <li><code>org.apache.flink.formats.json.JsonRowDeserializationSchema</code></li> |
| <li><code>org.apache.flink.formats.json.JsonRowSerializationSchema$Builder</code></li> |
| <li><code>org.apache.flink.formats.json.JsonRowSerializationSchema</code></li> |
| <li><code>org.apache.flink.metrics.reporter.InstantiateViaFactory</code></li> |
| <li><code>org.apache.flink.metrics.reporter.InterceptInstantiationViaReflection</code></li> |
| <li><code>org.apache.flink.runtime.jobgraph.SavepointConfigOptions</code></li> |
| <li><code>org.apache.flink.runtime.state.CheckpointListener</code></li> |
| <li><code>org.apache.flink.runtime.state.filesystem.FsStateBackendFactory</code></li> |
| <li><code>org.apache.flink.runtime.state.filesystem.FsStateBackend</code></li> |
| <li><code>org.apache.flink.runtime.state.memory.MemoryStateBackendFactory</code></li> |
| <li><code>org.apache.flink.runtime.state.memory.MemoryStateBackend</code></li> |
| <li><code>org.apache.flink.state.api.BootstrapTransformation</code></li> |
| <li><code>org.apache.flink.state.api.EvictingWindowReader</code></li> |
| <li><code>org.apache.flink.state.api.ExistingSavepoint</code></li> |
| <li><code>org.apache.flink.state.api.KeyedOperatorTransformation</code></li> |
| <li><code>org.apache.flink.state.api.NewSavepoint</code></li> |
| <li><code>org.apache.flink.state.api.OneInputOperatorTransformation</code></li> |
| <li><code>org.apache.flink.state.api.Savepoint</code></li> |
| <li><code>org.apache.flink.state.api.WindowReader</code></li> |
| <li><code>org.apache.flink.state.api.WindowedOperatorTransformation</code></li> |
| <li><code>org.apache.flink.state.api.WritableSavepoint</code></li> |
| <li><code>org.apache.flink.state.forst.fs.ByteBufferReadableFSDataInputStream</code></li> |
| <li><code>org.apache.flink.state.forst.fs.ByteBufferWritableFSDataOutputStream</code></li> |
| <li><code>org.apache.flink.state.forst.fs.ForStFlinkFileSystem</code></li> |
| <li><code>org.apache.flink.streaming.api.TimeCharacteristic</code></li> |
| <li><code>org.apache.flink.streaming.api.checkpoint.ExternallyInducedSource$CheckpointTrigger</code></li> |
| <li><code>org.apache.flink.streaming.api.checkpoint.ExternallyInducedSource</code></li> |
| <li><code>org.apache.flink.streaming.api.connector.sink2.WithPostCommitTopology</code></li> |
| <li><code>org.apache.flink.streaming.api.connector.sink2.WithPreCommitTopology</code></li> |
| <li><code>org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.IterativeStream$ConnectedIterativeStreams</code></li> |
| <li><code>org.apache.flink.streaming.api.environment.CheckpointConfig$ExternalizedCheckpointCleanup</code></li> |
| <li><code>org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions</code></li> |
| <li><code>org.apache.flink.streaming.api.environment.StreamPipelineOptions</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.AscendingTimestampExtractor</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.DiscardingSink</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.PrintSinkFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.RichSinkFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.SinkFunction$Context</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.SinkFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.SocketClientSink</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.WriteFormatAsCsv</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.WriteFormatAsText</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.WriteFormat</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.WriteSinkFunctionByMillis</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.WriteSinkFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink$BulkFormatBuilder</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink$DefaultBulkFormatBuilder</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink$DefaultRowFormatBuilder</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink$RowFormatBuilder</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.FromElementsFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.FromIteratorFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.FromSplittableIteratorFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.MessageAcknowledgingSourceBase</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.MultipleIdsMessageAcknowledgingSourceBase</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.ParallelSourceFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.RichSourceFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.SourceFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.StatefulSequenceSource</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.source.datagen.DataGeneratorSource</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.windowing.RichProcessAllWindowFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.functions.windowing.RichProcessWindowFunction</code></li> |
| <li><code>org.apache.flink.streaming.api.operators.SetupableStreamOperator</code></li> |
| <li><code>org.apache.flink.streaming.api.operators.YieldingOperatorFactory</code></li> |
| <li><code>org.apache.flink.streaming.api.windowing.time.Time</code></li> |
| <li><code>org.apache.flink.streaming.util.serialization.AbstractDeserializationSchema</code></li> |
| <li><code>org.apache.flink.streaming.util.serialization.DeserializationSchema</code></li> |
| <li><code>org.apache.flink.streaming.util.serialization.SerializationSchema</code></li> |
| <li><code>org.apache.flink.streaming.util.serialization.SimpleStringSchema</code></li> |
| <li><code>org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema</code></li> |
| <li><code>org.apache.flink.table.api.TableColumn$ComputedColumn</code></li> |
| <li><code>org.apache.flink.table.api.TableColumn$MetadataColumn</code></li> |
| <li><code>org.apache.flink.table.api.TableColumn$PhysicalColumn</code></li> |
| <li><code>org.apache.flink.table.api.TableColumn</code></li> |
| <li><code>org.apache.flink.table.api.TableSchema$Builder</code></li> |
| <li><code>org.apache.flink.table.api.TableSchema</code></li> |
| <li><code>org.apache.flink.table.api.constraints.Constraint$ConstraintType</code></li> |
| <li><code>org.apache.flink.table.api.constraints.Constraint</code></li> |
| <li><code>org.apache.flink.table.api.constraints.UniqueConstraint</code></li> |
| <li><code>org.apache.flink.table.connector.sink.SinkFunctionProvider</code></li> |
| <li><code>org.apache.flink.table.connector.sink.SinkProvider</code></li> |
| <li><code>org.apache.flink.table.connector.source.AsyncTableFunctionProvider</code></li> |
| <li><code>org.apache.flink.table.connector.source.SourceFunctionProvider</code></li> |
| <li><code>org.apache.flink.table.connector.source.TableFunctionProvider</code></li> |
| <li><code>org.apache.flink.table.descriptors.Descriptor</code></li> |
| <li><code>org.apache.flink.table.descriptors.RowtimeValidator</code></li> |
| <li><code>org.apache.flink.table.descriptors.Rowtime</code></li> |
| <li><code>org.apache.flink.table.descriptors.SchemaValidator</code></li> |
| <li><code>org.apache.flink.table.descriptors.Schema</code></li> |
| <li><code>org.apache.flink.table.factories.StreamTableSinkFactory</code></li> |
| <li><code>org.apache.flink.table.factories.StreamTableSourceFactory</code></li> |
| <li><code>org.apache.flink.table.factories.TableFactory</code></li> |
| <li><code>org.apache.flink.table.factories.TableSinkFactory$Context</code></li> |
| <li><code>org.apache.flink.table.factories.TableSinkFactory</code></li> |
| <li><code>org.apache.flink.table.factories.TableSourceFactory$Context</code></li> |
| <li><code>org.apache.flink.table.factories.TableSourceFactory</code></li> |
| <li><code>org.apache.flink.table.planner.codegen.agg.batch.HashAggCodeGenerator$</code></li> |
| <li><code>org.apache.flink.table.planner.plan.metadata.FlinkRelMdRowCount$</code></li> |
| <li><code>org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder$</code></li> |
| <li><code>org.apache.flink.table.planner.plan.rules.logical.JoinDeriveNullFilterRule$</code></li> |
| <li><code>org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalJoinRuleBase$</code></li> |
| <li><code>org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalSortMergeJoinRule$</code></li> |
| <li><code>org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalSortRule$</code></li> |
| <li><code>org.apache.flink.table.planner.plan.rules.physical.stream.IncrementalAggregateRule$</code></li> |
| <li><code>org.apache.flink.table.planner.plan.utils.FlinkRexUtil$</code></li> |
| <li><code>org.apache.flink.table.sinks.AppendStreamTableSink</code></li> |
| <li><code>org.apache.flink.table.sinks.OutputFormatTableSink</code></li> |
| <li><code>org.apache.flink.table.sinks.OverwritableTableSink</code></li> |
| <li><code>org.apache.flink.table.sinks.PartitionableTableSink</code></li> |
| <li><code>org.apache.flink.table.sinks.RetractStreamTableSink</code></li> |
| <li><code>org.apache.flink.table.sinks.TableSink</code></li> |
| <li><code>org.apache.flink.table.sinks.UpsertStreamTableSink</code></li> |
| <li><code>org.apache.flink.table.sources.DefinedFieldMapping</code></li> |
| <li><code>org.apache.flink.table.sources.DefinedProctimeAttribute</code></li> |
| <li><code>org.apache.flink.table.sources.DefinedRowtimeAttributes</code></li> |
| <li><code>org.apache.flink.table.sources.FieldComputer</code></li> |
| <li><code>org.apache.flink.table.sources.InputFormatTableSource</code></li> |
| <li><code>org.apache.flink.table.sources.LimitableTableSource</code></li> |
| <li><code>org.apache.flink.table.sources.LookupableTableSource</code></li> |
| <li><code>org.apache.flink.table.sources.NestedFieldsProjectableTableSource</code></li> |
| <li><code>org.apache.flink.table.sources.PartitionableTableSource</code></li> |
| <li><code>org.apache.flink.table.sources.ProjectableTableSource</code></li> |
| <li><code>org.apache.flink.table.sources.TableSource</code></li> |
| <li><code>org.apache.flink.table.sources.tsextractors.ExistingField</code></li> |
| <li><code>org.apache.flink.table.sources.tsextractors.StreamRecordTimestamp</code></li> |
| <li><code>org.apache.flink.table.sources.tsextractors.TimestampExtractor</code></li> |
| <li><code>org.apache.flink.table.types.logical.TypeInformationRawType</code></li> |
| <li><code>org.apache.flink.table.utils.TypeStringUtils</code></li> |
| <li><code>org.apache.flink.walkthrough.common.sink.AlertSink</code></li> |
| <li><code>org.apache.flink.walkthrough.common.source.TransactionSource</code></li> |
| </ul> |
| <h3 id="modified-classes"> |
| Modified Classes |
| <a class="anchor" href="#modified-classes">#</a> |
| </h3> |
| <ul> |
| <li><code>org.apache.flink.table.api.bridge.java.StreamTableEnvironment</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>void registerDataStream(java.lang.String, org.apache.flink.streaming.api.datastream.DataStream<T>)</code></li> |
| <li><code>void registerFunction(java.lang.String, org.apache.flink.table.functions.TableFunction<T>)</code></li> |
| <li><code>void registerFunction(java.lang.String, org.apache.flink.table.functions.AggregateFunction<T,ACC>)</code></li> |
| <li><code>void registerFunction(java.lang.String, org.apache.flink.table.functions.TableAggregateFunction<T,ACC>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.config.ExecutionConfigOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> TABLE_EXEC_LEGACY_TRANSFORMATION_UIDS</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> TABLE_EXEC_SHUFFLE_MODE</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.config.LookupJoinHintOptions</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>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()</code></li> |
| <li><code>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()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.config.OptimizerConfigOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> TABLE_OPTIMIZER_SOURCE_PREDICATE_PUSHDOWN_ENABLED</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> TABLE_OPTIMIZER_SOURCE_AGGREGATE_PUSHDOWN_ENABLED</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.dataview.ListView</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>TRANSIENT(-) org.apache.flink.api.common.typeinfo.TypeInformation<?> elementType</code></li> |
| </ul> |
| </li> |
| <li>constructor removed: |
| <ul> |
| <li><code>ListView(org.apache.flink.api.common.typeinfo.TypeInformation<?>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.dataview.MapView</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>TRANSIENT(-) org.apache.flink.api.common.typeinfo.TypeInformation<?> valueType</code></li> |
| <li><code>TRANSIENT(-) org.apache.flink.api.common.typeinfo.TypeInformation<?> keyType</code></li> |
| </ul> |
| </li> |
| <li>constructor removed: |
| <ul> |
| <li><code>MapView(org.apache.flink.api.common.typeinfo.TypeInformation<?>, org.apache.flink.api.common.typeinfo.TypeInformation<?>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.EnvironmentSettings</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.table.api.EnvironmentSettings fromConfiguration(org.apache.flink.configuration.ReadableConfig)</code></li> |
| <li><code>org.apache.flink.configuration.Configuration toConfiguration()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.internal.BaseExpressions</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>java.lang.Object cast(org.apache.flink.api.common.typeinfo.TypeInformation<?>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.OverWindow</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>java.util.Optional<org.apache.flink.table.expressions.Expression> (<-org.apache.flink.table.expressions.Expression<org.apache.flink.table.expressions.Expression>) getPreceding()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.Table</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>org.apache.flink.table.legacy.api.TableSchema (<-org.apache.flink.table.api.TableSchema) getSchema()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.TableConfig</code> |
| <ul> |
| <li>constructor modified: |
| <ul> |
| <li><code>PRIVATE (<- PUBLIC) TableConfig()</code></li> |
| </ul> |
| </li> |
| <li>method removed: |
| <ul> |
| <li><code>long getMaxIdleStateRetentionTime()</code></li> |
| <li><code>long getMinIdleStateRetentionTime()</code></li> |
| <li><code>void setIdleStateRetentionTime(org.apache.flink.api.common.time.Time, org.apache.flink.api.common.time.Time)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.TableDescriptor</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.table.api.TableDescriptor$Builder forManaged()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.TableResult</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.table.api.TableSchema getTableSchema()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.catalog.Catalog</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>java.util.Optional<org.apache.flink.table.factories.TableFactory> getTableFactory()</code></li> |
| <li><code>boolean supportsManagedTable()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.catalog.CatalogBaseTable</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>org.apache.flink.table.legacy.api.TableSchema (<-org.apache.flink.table.api.TableSchema) getSchema()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.catalog.CatalogFunction</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>boolean isGeneric()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.catalog.CatalogTable</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>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>)</code></li> |
| <li><code>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)</code></li> |
| <li><code>java.util.Map<java.lang.String,java.lang.String> toProperties()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.catalog.ResolvedCatalogBaseTable</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>org.apache.flink.table.legacy.api.TableSchema (<-org.apache.flink.table.api.TableSchema) getSchema()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.connector.sink.DataStreamSinkProvider</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>(<- 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>)</code></li> |
| </ul> |
| </li> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamSink<?> consumeDataStream(org.apache.flink.streaming.api.datastream.DataStream<org.apache.flink.table.data.RowData>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>void applyProjection(int[][])</code></li> |
| </ul> |
| </li> |
| <li>method modified: |
| <ul> |
| <li><code>(<- NON_ABSTRACT) void applyProjection(int[][], org.apache.flink.table.types.DataType)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.connector.source.DataStreamScanProvider</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>(<- 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)</code></li> |
| </ul> |
| </li> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStream<org.apache.flink.table.data.RowData> produceDataStream(org.apache.flink.streaming.api.environment.StreamExecutionEnvironment)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.expressions.CallExpression</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>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)</code></li> |
| <li><code>CallExpression(org.apache.flink.table.functions.FunctionDefinition, java.util.List<org.apache.flink.table.expressions.ResolvedExpression>, org.apache.flink.table.types.DataType)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.factories.FactoryUtil</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.factories.FunctionDefinitionFactory</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.table.functions.FunctionDefinition createFunctionDefinition(java.lang.String, org.apache.flink.table.catalog.CatalogFunction)</code></li> |
| </ul> |
| </li> |
| <li>method modified: |
| <ul> |
| <li><code>(<- 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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.functions.FunctionContext</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>FunctionContext(org.apache.flink.api.common.functions.RuntimeContext, java.lang.ClassLoader, org.apache.flink.configuration.Configuration)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.plan.stats.ColumnStats</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>ColumnStats(java.lang.Long, java.lang.Long, java.lang.Double, java.lang.Integer, java.lang.Number, java.lang.Number)</code></li> |
| </ul> |
| </li> |
| <li>method removed: |
| <ul> |
| <li><code>java.lang.Number getMaxValue()</code></li> |
| <li><code>java.lang.Number getMinValue()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.types.logical.SymbolType</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>SymbolType(boolean, java.lang.Class<T>)</code></li> |
| <li><code>SymbolType(java.lang.Class<T>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.types.logical.utils.LogicalTypeParser</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.table.types.logical.LogicalType parse(java.lang.String)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.state.v2.StateIterator</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>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>>)</code></li> |
| <li><code>org.apache.flink.api.common.state.v2.StateFuture<java.lang.Void> onNext(java.util.function.Consumer<T>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.api.ImplicitExpressionConversions</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.table.expressions.Expression toTimestampLtz(org.apache.flink.table.expressions.Expression, org.apache.flink.table.expressions.Expression)</code></li> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.eventtime.WatermarksWithIdleness</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>WatermarksWithIdleness(org.apache.flink.api.common.eventtime.WatermarkGenerator<T>, java.time.Duration)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.ExecutionConfig</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>int PARALLELISM_AUTO_MAX</code></li> |
| </ul> |
| </li> |
| <li>method removed: |
| <ul> |
| <li><code>void addDefaultKryoSerializer(java.lang.Class<?>, com.esotericsoftware.kryo.Serializer<?>)</code></li> |
| <li><code>void addDefaultKryoSerializer(java.lang.Class<?>, java.lang.Class<? extends com.esotericsoftware.kryo.Serializer<? extends ?>>)</code></li> |
| <li><code>boolean canEqual(java.lang.Object)</code></li> |
| <li><code>void disableAutoTypeRegistration()</code></li> |
| <li><code>void disableForceAvro()</code></li> |
| <li><code>void disableForceKryo()</code></li> |
| <li><code>void disableGenericTypes()</code></li> |
| <li><code>void enableForceAvro()</code></li> |
| <li><code>void enableForceKryo()</code></li> |
| <li><code>void enableGenericTypes()</code></li> |
| <li><code>int getAsyncInflightRecordsLimit()</code></li> |
| <li><code>int getAsyncStateBufferSize()</code></li> |
| <li><code>long getAsyncStateBufferTimeout()</code></li> |
| <li><code>org.apache.flink.api.common.InputDependencyConstraint getDefaultInputDependencyConstraint()</code></li> |
| <li><code>java.util.LinkedHashMap<java.lang.Class<?>,java.lang.Class<com.esotericsoftware.kryo.Serializer<? extends ?>>> getDefaultKryoSerializerClasses()</code></li> |
| <li><code>java.util.LinkedHashMap<java.lang.Class<?>,org.apache.flink.api.common.ExecutionConfig$SerializableSerializer<?>> getDefaultKryoSerializers()</code></li> |
| <li><code>org.apache.flink.api.common.ExecutionMode getExecutionMode()</code></li> |
| <li><code>long getExecutionRetryDelay()</code></li> |
| <li><code>int getNumberOfExecutionRetries()</code></li> |
| <li><code>java.util.LinkedHashSet<java.lang.Class<?>> getRegisteredKryoTypes()</code></li> |
| <li><code>java.util.LinkedHashSet<java.lang.Class<?>> getRegisteredPojoTypes()</code></li> |
| <li><code>java.util.LinkedHashMap<java.lang.Class<?>,java.lang.Class<com.esotericsoftware.kryo.Serializer<? extends ?>>> getRegisteredTypesWithKryoSerializerClasses()</code></li> |
| <li><code>java.util.LinkedHashMap<java.lang.Class<?>,org.apache.flink.api.common.ExecutionConfig$SerializableSerializer<?>> getRegisteredTypesWithKryoSerializers()</code></li> |
| <li><code>org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration getRestartStrategy()</code></li> |
| <li><code>boolean hasGenericTypesDisabled()</code></li> |
| <li><code>boolean isAutoTypeRegistrationDisabled()</code></li> |
| <li><code>boolean isForceAvroEnabled()</code></li> |
| <li><code>boolean isForceKryoEnabled()</code></li> |
| <li><code>void registerKryoType(java.lang.Class<?>)</code></li> |
| <li><code>void registerPojoType(java.lang.Class<?>)</code></li> |
| <li><code>void registerTypeWithKryoSerializer(java.lang.Class<?>, com.esotericsoftware.kryo.Serializer<?>)</code></li> |
| <li><code>void registerTypeWithKryoSerializer(java.lang.Class<?>, java.lang.Class<? extends com.esotericsoftware.kryo.Serializer>)</code></li> |
| <li><code>org.apache.flink.api.common.ExecutionConfig setAsyncInflightRecordsLimit(int)</code></li> |
| <li><code>org.apache.flink.api.common.ExecutionConfig setAsyncStateBufferSize(int)</code></li> |
| <li><code>org.apache.flink.api.common.ExecutionConfig setAsyncStateBufferTimeout(long)</code></li> |
| <li><code>void setDefaultInputDependencyConstraint(org.apache.flink.api.common.InputDependencyConstraint)</code></li> |
| <li><code>void setExecutionMode(org.apache.flink.api.common.ExecutionMode)</code></li> |
| <li><code>org.apache.flink.api.common.ExecutionConfig setExecutionRetryDelay(long)</code></li> |
| <li><code>org.apache.flink.api.common.ExecutionConfig setNumberOfExecutionRetries(int)</code></li> |
| <li><code>void setRestartStrategy(org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.functions.RichFunction</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>void open(org.apache.flink.configuration.Configuration)</code></li> |
| </ul> |
| </li> |
| <li>method modified: |
| <ul> |
| <li><code>(<- NON_ABSTRACT) void open(org.apache.flink.api.common.functions.OpenContext)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.functions.RuntimeContext</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>int getAttemptNumber()</code></li> |
| <li><code>org.apache.flink.api.common.ExecutionConfig getExecutionConfig()</code></li> |
| <li><code>int getIndexOfThisSubtask()</code></li> |
| <li><code>org.apache.flink.api.common.JobID getJobId()</code></li> |
| <li><code>int getMaxNumberOfParallelSubtasks()</code></li> |
| <li><code>int getNumberOfParallelSubtasks()</code></li> |
| <li><code>java.lang.String getTaskName()</code></li> |
| <li><code>java.lang.String getTaskNameWithSubtasks()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.io.BinaryInputFormat</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>java.lang.String BLOCK_SIZE_PARAMETER_KEY</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.io.BinaryOutputFormat</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>java.lang.String BLOCK_SIZE_PARAMETER_KEY</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.io.FileInputFormat</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>java.lang.String ENUMERATE_NESTED_FILES_FLAG</code></li> |
| </ul> |
| </li> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.core.fs.Path getFilePath()</code></li> |
| <li><code>boolean supportsMultiPaths()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.io.FileOutputFormat</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>java.lang.String FILE_PARAMETER_KEY</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.io.FinalizeOnMaster</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>void finalizeGlobal(int)</code></li> |
| </ul> |
| </li> |
| <li>method modified: |
| <ul> |
| <li><code>(<- NON_ABSTRACT) void finalizeGlobal(org.apache.flink.api.common.io.FinalizeOnMaster$FinalizationContext)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.io.OutputFormat</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>void open(int, int)</code></li> |
| </ul> |
| </li> |
| <li>method modified: |
| <ul> |
| <li><code>(<- NON_ABSTRACT) void open(org.apache.flink.api.common.io.OutputFormat$InitializationContext)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.JobExecutionResult</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.api.common.JobExecutionResult fromJobSubmissionResult(org.apache.flink.api.common.JobSubmissionResult)</code></li> |
| <li><code>java.lang.Integer getIntCounterResult(java.lang.String)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.serialization.SerializerConfig</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>java.util.LinkedHashMap<java.lang.Class<?>,org.apache.flink.api.common.ExecutionConfig$SerializableSerializer<?>> getDefaultKryoSerializers()</code></li> |
| <li><code>java.util.LinkedHashMap<java.lang.Class<?>,org.apache.flink.api.common.ExecutionConfig$SerializableSerializer<?>> getRegisteredTypesWithKryoSerializers()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.state.StateTtlConfig</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.api.common.time.Time getTtl()</code></li> |
| <li><code>org.apache.flink.api.common.state.StateTtlConfig$Builder newBuilder(org.apache.flink.api.common.time.Time)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.state.StateTtlConfig$Builder</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>StateTtlConfig$Builder(org.apache.flink.api.common.time.Time)</code></li> |
| </ul> |
| </li> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.api.common.state.StateTtlConfig$Builder setTtl(org.apache.flink.api.common.time.Time)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.typeinfo.TypeInformation</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>(<- NON_ABSTRACT) org.apache.flink.api.common.typeutils.TypeSerializer<T><T> createSerializer(org.apache.flink.api.common.serialization.SerializerConfig)</code></li> |
| </ul> |
| </li> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.api.common.typeutils.TypeSerializer<T> createSerializer(org.apache.flink.api.common.ExecutionConfig)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.common.typeutils.TypeSerializerSnapshot</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility(org.apache.flink.api.common.typeutils.TypeSerializer<T>)</code></li> |
| </ul> |
| </li> |
| <li>method modified: |
| <ul> |
| <li><code>(<- NON_ABSTRACT) org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility<T><T> resolveSchemaCompatibility(org.apache.flink.api.common.typeutils.TypeSerializerSnapshot<T><T>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.connector.sink2.Sink</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.api.connector.sink2.SinkWriter<InputT> createWriter(org.apache.flink.api.connector.sink2.Sink$InitContext)</code></li> |
| </ul> |
| </li> |
| <li>method modified: |
| <ul> |
| <li><code>(<- NON_ABSTRACT) org.apache.flink.api.connector.sink2.SinkWriter<InputT><InputT> createWriter(org.apache.flink.api.connector.sink2.WriterInitContext)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.java.typeutils.PojoTypeInfo</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.api.java.typeutils.runtime.PojoSerializer<T> createPojoSerializer(org.apache.flink.api.common.ExecutionConfig)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.api.java.typeutils.RowTypeInfo</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.api.common.typeutils.TypeSerializer<org.apache.flink.types.Row> createLegacySerializer(org.apache.flink.api.common.serialization.SerializerConfig)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.CheckpointingOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> LOCAL_RECOVERY</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> STATE_BACKEND</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> ASYNC_SNAPSHOTS</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.ClusterOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> FINE_GRAINED_SHUFFLE_MODE_ALL_BLOCKING</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> EVENLY_SPREAD_OUT_SLOTS_STRATEGY</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.ConfigConstants</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>java.lang.String HA_ZOOKEEPER_LEADER_PATH</code></li> |
| <li><code>double DEFAULT_AKKA_WATCH_THRESHOLD</code></li> |
| <li><code>int DEFAULT_JOB_MANAGER_IPC_PORT</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_TMPDIR_KEY</code></li> |
| <li><code>int DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE</code></li> |
| <li><code>java.lang.String METRICS_SCOPE_NAMING_TASK</code></li> |
| <li><code>java.lang.String ZOOKEEPER_NAMESPACE_KEY</code></li> |
| <li><code>int DEFAULT_AKKA_DISPATCHER_THROUGHPUT</code></li> |
| <li><code>java.lang.String RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS</code></li> |
| <li><code>java.lang.String MESOS_MASTER_URL</code></li> |
| <li><code>java.lang.String FLINK_BASE_DIR_PATH_KEY</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_SSL_ENABLED</code></li> |
| <li><code>java.lang.String YARN_APPLICATION_TAGS</code></li> |
| <li><code>java.lang.String HDFS_SITE_CONFIG</code></li> |
| <li><code>java.lang.String EXECUTION_RETRY_DELAY_KEY</code></li> |
| <li><code>int DEFAULT_MESOS_ARTIFACT_SERVER_PORT</code></li> |
| <li><code>boolean DEFAULT_SECURITY_SSL_VERIFY_HOSTNAME</code></li> |
| <li><code>java.lang.String CONTAINERIZED_HEAP_CUTOFF_MIN</code></li> |
| <li><code>java.lang.String YARN_HEARTBEAT_DELAY_SECONDS</code></li> |
| <li><code>java.lang.String AKKA_SSL_ENABLED</code></li> |
| <li><code>java.lang.String HA_MODE</code></li> |
| <li><code>java.lang.String ZOOKEEPER_MESOS_WORKERS_PATH</code></li> |
| <li><code>boolean DEFAULT_ZOOKEEPER_SASL_DISABLE</code></li> |
| <li><code>java.lang.String METRICS_SCOPE_DELIMITER</code></li> |
| <li><code>java.lang.String LOCAL_NUMBER_RESOURCE_MANAGER</code></li> |
| <li><code>java.lang.String AKKA_TCP_TIMEOUT</code></li> |
| <li><code>java.lang.String METRICS_SCOPE_NAMING_OPERATOR</code></li> |
| <li><code>java.lang.String ZOOKEEPER_RECOVERY_PATH</code></li> |
| <li><code>int DEFAULT_ZOOKEEPER_LEADER_PORT</code></li> |
| <li><code>java.lang.String DEFAULT_ZOOKEEPER_LATCH_PATH</code></li> |
| <li><code>int DEFAULT_ZOOKEEPER_PEER_PORT</code></li> |
| <li><code>java.lang.String METRICS_SCOPE_NAMING_TM_JOB</code></li> |
| <li><code>int DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_NUM_SAMPLES</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_SESSION_TIMEOUT</code></li> |
| <li><code>java.lang.String FLINK_JVM_OPTIONS</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_CHECKPOINT_COUNTER_PATH</code></li> |
| <li><code>java.lang.String METRICS_SCOPE_NAMING_JM</code></li> |
| <li><code>java.lang.String DEFAULT_YARN_JOB_MANAGER_PORT</code></li> |
| <li><code>boolean DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLE</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_QUORUM_KEY</code></li> |
| <li><code>boolean DEFAULT_JOB_MANAGER_WEB_SUBMIT_ENABLED</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE</code></li> |
| <li><code>java.lang.String ZOOKEEPER_JOBGRAPHS_PATH</code></li> |
| <li><code>java.lang.String ZOOKEEPER_SASL_SERVICE_NAME</code></li> |
| <li><code>java.lang.String DEFAULT_AKKA_LOOKUP_TIMEOUT</code></li> |
| <li><code>java.lang.String RESTART_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_PORT_KEY</code></li> |
| <li><code>java.lang.String METRICS_LATENCY_HISTORY_SIZE</code></li> |
| <li><code>int DEFAULT_BLOB_FETCH_BACKLOG</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_BACK_PRESSURE_REFRESH_INTERVAL</code></li> |
| <li><code>float DEFAULT_SORT_SPILLING_THRESHOLD</code></li> |
| <li><code>java.lang.String DEFAULT_AKKA_TRANSPORT_HEARTBEAT_INTERVAL</code></li> |
| <li><code>java.lang.String CONTAINERIZED_MASTER_ENV_PREFIX</code></li> |
| <li><code>int DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT</code></li> |
| <li><code>java.lang.String TASK_MANAGER_HOSTNAME_KEY</code></li> |
| <li><code>java.lang.String AKKA_WATCH_HEARTBEAT_INTERVAL</code></li> |
| <li><code>java.lang.String DEFAULT_TASK_MANAGER_TMP_PATH</code></li> |
| <li><code>int DEFAULT_EXECUTION_RETRIES</code></li> |
| <li><code>int DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_LOG_PATH_KEY</code></li> |
| <li><code>java.lang.String TASK_MANAGER_MEMORY_SIZE_KEY</code></li> |
| <li><code>java.lang.String DEFAULT_MESOS_RESOURCEMANAGER_FRAMEWORK_NAME</code></li> |
| <li><code>java.lang.String TASK_MANAGER_DATA_PORT_KEY</code></li> |
| <li><code>java.lang.String ZOOKEEPER_CHECKPOINTS_PATH</code></li> |
| <li><code>java.lang.String HA_JOB_MANAGER_PORT</code></li> |
| <li><code>java.lang.String TASK_MANAGER_REFUSED_REGISTRATION_PAUSE</code></li> |
| <li><code>java.lang.String CONTAINERIZED_HEAP_CUTOFF_RATIO</code></li> |
| <li><code>java.lang.String DEFAULT_SORT_SPILLING_THRESHOLD_KEY</code></li> |
| <li><code>java.lang.String YARN_CONTAINER_START_COMMAND_TEMPLATE</code></li> |
| <li><code>boolean DEFAULT_JOB_MANAGER_WEB_SSL_ENABLED</code></li> |
| <li><code>java.lang.String LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_CHECKPOINTS_DISABLE</code></li> |
| <li><code>java.lang.String DEFAULT_ZOOKEEPER_LEADER_PATH</code></li> |
| <li><code>int DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_DELAY</code></li> |
| <li><code>java.lang.String DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE</code></li> |
| <li><code>java.lang.String METRICS_REPORTERS_LIST</code></li> |
| <li><code>java.lang.String DEFAULT_RECOVERY_MODE</code></li> |
| <li><code>int DEFAULT_METRICS_LATENCY_HISTORY_SIZE</code></li> |
| <li><code>java.lang.String TASK_MANAGER_INITIAL_REGISTRATION_PAUSE</code></li> |
| <li><code>java.lang.String DEFAULT_MESOS_RESOURCEMANAGER_FRAMEWORK_ROLE</code></li> |
| <li><code>int DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE</code></li> |
| <li><code>java.lang.String YARN_PROPERTIES_FILE_LOCATION</code></li> |
| <li><code>java.lang.String RECOVERY_JOB_MANAGER_PORT</code></li> |
| <li><code>boolean DEFAULT_SECURITY_SSL_ENABLED</code></li> |
| <li><code>java.lang.String MESOS_FAILOVER_TIMEOUT_SECONDS</code></li> |
| <li><code>java.lang.String RUNTIME_HASH_JOIN_BLOOM_FILTERS_KEY</code></li> |
| <li><code>java.lang.String ZOOKEEPER_LEADER_PATH</code></li> |
| <li><code>java.lang.String ZOOKEEPER_MAX_RETRY_ATTEMPTS</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_CHECKPOINTS_PATH</code></li> |
| <li><code>java.lang.String MESOS_RESOURCEMANAGER_FRAMEWORK_ROLE</code></li> |
| <li><code>int DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_REFRESH_INTERVAL</code></li> |
| <li><code>java.lang.String DEFAULT_ZOOKEEPER_MESOS_WORKERS_PATH</code></li> |
| <li><code>java.lang.String JOB_MANAGER_IPC_PORT_KEY</code></li> |
| <li><code>java.lang.String AKKA_WATCH_HEARTBEAT_PAUSE</code></li> |
| <li><code>java.lang.String MESOS_RESOURCEMANAGER_FRAMEWORK_NAME</code></li> |
| <li><code>java.lang.String DELIMITED_FORMAT_MAX_SAMPLE_LENGTH_KEY</code></li> |
| <li><code>java.lang.String STATE_BACKEND</code></li> |
| <li><code>java.lang.String MESOS_RESOURCEMANAGER_FRAMEWORK_PRINCIPAL</code></li> |
| <li><code>long DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS</code></li> |
| <li><code>java.lang.String DEFAULT_AKKA_CLIENT_TIMEOUT</code></li> |
| <li><code>int DEFAULT_SPILLING_MAX_FAN</code></li> |
| <li><code>java.lang.String TASK_MANAGER_IPC_PORT_KEY</code></li> |
| <li><code>java.lang.String TASK_MANAGER_MEMORY_OFF_HEAP_KEY</code></li> |
| <li><code>boolean DEFAULT_FILESYSTEM_OVERWRITE</code></li> |
| <li><code>boolean DEFAULT_USE_LARGE_RECORD_HANDLER</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_JOBGRAPHS_PATH</code></li> |
| <li><code>boolean DEFAULT_BLOB_SERVICE_SSL_ENABLED</code></li> |
| <li><code>java.lang.String ZOOKEEPER_SESSION_TIMEOUT</code></li> |
| <li><code>java.lang.String TASK_MANAGER_NETWORK_DEFAULT_IO_MODE</code></li> |
| <li><code>java.lang.String SECURITY_SSL_TRUSTSTORE_PASSWORD</code></li> |
| <li><code>int DEFAULT_ZOOKEEPER_MAX_RETRY_ATTEMPTS</code></li> |
| <li><code>java.lang.String AKKA_STARTUP_TIMEOUT</code></li> |
| <li><code>java.lang.String TASK_MANAGER_TMP_DIR_KEY</code></li> |
| <li><code>java.lang.String USE_LARGE_RECORD_HANDLER_KEY</code></li> |
| <li><code>java.lang.String DEFAULT_ZOOKEEPER_DIR_KEY</code></li> |
| <li><code>int DEFAULT_YARN_MIN_HEAP_CUTOFF</code></li> |
| <li><code>java.lang.String TASK_MANAGER_DATA_SSL_ENABLED</code></li> |
| <li><code>java.lang.String HDFS_DEFAULT_CONFIG</code></li> |
| <li><code>boolean DEFAULT_TASK_MANAGER_DATA_SSL_ENABLED</code></li> |
| <li><code>java.lang.String DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_MESOS_WORKERS_PATH</code></li> |
| <li><code>java.lang.String BLOB_STORAGE_DIRECTORY_KEY</code></li> |
| <li><code>java.lang.String DEFAULT_STATE_BACKEND</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_RETRY_WAIT</code></li> |
| <li><code>java.lang.String AKKA_ASK_TIMEOUT</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_SUBMIT_ENABLED_KEY</code></li> |
| <li><code>java.lang.String DEFAULT_ZOOKEEPER_NAMESPACE_KEY</code></li> |
| <li><code>java.lang.String DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH</code></li> |
| <li><code>int DEFAULT_LOCAL_NUMBER_JOB_MANAGER</code></li> |
| <li><code>java.lang.String AKKA_TRANSPORT_HEARTBEAT_INTERVAL</code></li> |
| <li><code>java.lang.String DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH</code></li> |
| <li><code>java.lang.String FS_STREAM_OPENING_TIMEOUT_KEY</code></li> |
| <li><code>java.lang.String SECURITY_SSL_TRUSTSTORE</code></li> |
| <li><code>java.lang.String METRICS_SCOPE_NAMING_JM_JOB</code></li> |
| <li><code>java.lang.String MESOS_INITIAL_TASKS</code></li> |
| <li><code>java.lang.String AKKA_FRAMESIZE</code></li> |
| <li><code>int DEFAULT_ZOOKEEPER_INIT_LIMIT</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_BACK_PRESSURE_CLEAN_UP_INTERVAL</code></li> |
| <li><code>java.lang.String SECURITY_SSL_KEYSTORE</code></li> |
| <li><code>boolean DEFAULT_MESOS_ARTIFACT_SERVER_SSL_ENABLED</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_MAX_RETRY_ATTEMPTS</code></li> |
| <li><code>int DEFAULT_PARALLELISM</code></li> |
| <li><code>java.lang.String RECOVERY_MODE</code></li> |
| <li><code>java.lang.String EXECUTION_RETRIES_KEY</code></li> |
| <li><code>java.lang.String METRICS_REPORTER_SCOPE_DELIMITER</code></li> |
| <li><code>java.lang.String LOCAL_START_WEBSERVER</code></li> |
| <li><code>java.lang.String LOCAL_NUMBER_JOB_MANAGER</code></li> |
| <li><code>java.lang.String RESTART_STRATEGY</code></li> |
| <li><code>java.lang.String ZOOKEEPER_QUORUM_KEY</code></li> |
| <li><code>int DEFAULT_MESOS_FAILOVER_TIMEOUT_SECS</code></li> |
| <li><code>boolean DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE</code></li> |
| <li><code>int DEFAULT_LOCAL_NUMBER_RESOURCE_MANAGER</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_CLIENT_ACL</code></li> |
| <li><code>java.lang.String METRICS_REPORTER_FACTORY_CLASS_SUFFIX</code></li> |
| <li><code>boolean DEFAULT_FILESYSTEM_ALWAYS_CREATE_DIRECTORY</code></li> |
| <li><code>java.lang.String BLOB_FETCH_CONCURRENT_KEY</code></li> |
| <li><code>java.lang.String FILESYSTEM_DEFAULT_OVERWRITE_KEY</code></li> |
| <li><code>java.lang.String RESOURCE_MANAGER_IPC_PORT_KEY</code></li> |
| <li><code>java.lang.String DEFAULT_AKKA_ASK_TIMEOUT</code></li> |
| <li><code>int DEFAULT_ZOOKEEPER_CLIENT_PORT</code></li> |
| <li><code>double DEFAULT_AKKA_TRANSPORT_THRESHOLD</code></li> |
| <li><code>java.lang.String DEFAULT_AKKA_FRAMESIZE</code></li> |
| <li><code>java.lang.String TASK_MANAGER_NUM_TASK_SLOTS</code></li> |
| <li><code>java.lang.String YARN_APPLICATION_MASTER_ENV_PREFIX</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_BACK_PRESSURE_DELAY</code></li> |
| <li><code>long DEFAULT_TASK_CANCELLATION_INTERVAL_MILLIS</code></li> |
| <li><code>java.lang.String TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY</code></li> |
| <li><code>java.lang.String FILESYSTEM_SCHEME</code></li> |
| <li><code>java.lang.String TASK_MANAGER_MAX_REGISTRATION_DURATION</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_DIR_KEY</code></li> |
| <li><code>java.lang.String DEFAULT_MESOS_RESOURCEMANAGER_FRAMEWORK_USER</code></li> |
| <li><code>java.lang.String DEFAULT_FILESYSTEM_SCHEME</code></li> |
| <li><code>java.lang.String MESOS_RESOURCEMANAGER_FRAMEWORK_SECRET</code></li> |
| <li><code>int DEFAULT_DELIMITED_FORMAT_MAX_SAMPLE_LEN</code></li> |
| <li><code>java.lang.String ENV_FLINK_BIN_DIR</code></li> |
| <li><code>float DEFAULT_YARN_HEAP_CUTOFF_RATIO</code></li> |
| <li><code>java.lang.String SAVEPOINT_FS_DIRECTORY_KEY</code></li> |
| <li><code>java.lang.String AKKA_JVM_EXIT_ON_FATAL_ERROR</code></li> |
| <li><code>java.lang.String ZOOKEEPER_RETRY_WAIT</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_NAMESPACE_KEY</code></li> |
| <li><code>java.lang.String ZOOKEEPER_CONNECTION_TIMEOUT</code></li> |
| <li><code>java.lang.String TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_ARCHIVE_COUNT</code></li> |
| <li><code>int DEFAULT_RESOURCE_MANAGER_IPC_PORT</code></li> |
| <li><code>int DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_CLEAN_UP_INTERVAL</code></li> |
| <li><code>java.lang.String YARN_REALLOCATE_FAILED_CONTAINERS</code></li> |
| <li><code>java.lang.String SECURITY_SSL_KEYSTORE_PASSWORD</code></li> |
| <li><code>java.lang.String DEFAULT_HA_JOB_MANAGER_PORT</code></li> |
| <li><code>java.lang.String BLOB_FETCH_RETRIES_KEY</code></li> |
| <li><code>java.lang.String METRICS_REPORTER_EXCLUDED_VARIABLES</code></li> |
| <li><code>java.lang.String DEFAULT_SECURITY_SSL_PROTOCOL</code></li> |
| <li><code>java.lang.String RECOVERY_JOB_DELAY</code></li> |
| <li><code>java.lang.String TASK_CANCELLATION_INTERVAL_MILLIS</code></li> |
| <li><code>java.lang.String YARN_APPLICATION_MASTER_PORT</code></li> |
| <li><code>int DEFAULT_TASK_MANAGER_DATA_PORT</code></li> |
| <li><code>java.lang.String RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL</code></li> |
| <li><code>java.lang.String YARN_TASK_MANAGER_ENV_PREFIX</code></li> |
| <li><code>int DEFAULT_DELIMITED_FORMAT_MIN_LINE_SAMPLES</code></li> |
| <li><code>java.lang.String AKKA_LOG_LIFECYCLE_EVENTS</code></li> |
| <li><code>boolean DEFAULT_AKKA_LOG_LIFECYCLE_EVENTS</code></li> |
| <li><code>java.lang.String SECURITY_SSL_ENABLED</code></li> |
| <li><code>int DEFAULT_DELIMITED_FORMAT_MAX_LINE_SAMPLES</code></li> |
| <li><code>java.lang.String LOCAL_NUMBER_TASK_MANAGER</code></li> |
| <li><code>java.lang.String DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE</code></li> |
| <li><code>java.lang.String DEFAULT_SECURITY_SSL_ALGORITHMS</code></li> |
| <li><code>java.lang.String MESOS_MAX_FAILED_TASKS</code></li> |
| <li><code>int DEFAULT_TASK_MANAGER_IPC_PORT</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> DEFAULT_JOB_MANAGER_WEB_FRONTEND_ADDRESS</code></li> |
| <li><code>java.lang.String SECURITY_SSL_ALGORITHMS</code></li> |
| <li><code>int DEFAULT_ZOOKEEPER_CONNECTION_TIMEOUT</code></li> |
| <li><code>java.lang.String YARN_HEAP_CUTOFF_RATIO</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_LATCH_PATH</code></li> |
| <li><code>int DEFAULT_ZOOKEEPER_SESSION_TIMEOUT</code></li> |
| <li><code>java.lang.String DEFAULT_SPILLING_MAX_FAN_KEY</code></li> |
| <li><code>java.lang.String AKKA_WATCH_THRESHOLD</code></li> |
| <li><code>java.lang.String TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_STORAGE_PATH</code></li> |
| <li><code>java.lang.String DEFAULT_BLOB_SERVER_PORT</code></li> |
| <li><code>java.lang.String AKKA_TRANSPORT_THRESHOLD</code></li> |
| <li><code>java.lang.String ZOOKEEPER_CHECKPOINT_COUNTER_PATH</code></li> |
| <li><code>boolean DEFAULT_RUNTIME_HASH_JOIN_BLOOM_FILTERS</code></li> |
| <li><code>int DEFAULT_BLOB_FETCH_CONCURRENT</code></li> |
| <li><code>java.lang.String BLOB_SERVER_PORT</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> RESTART_STRATEGY_FIXED_DELAY_DELAY</code></li> |
| <li><code>java.lang.String METRICS_REPORTER_CLASS_SUFFIX</code></li> |
| <li><code>java.lang.String ZOOKEEPER_DIR_KEY</code></li> |
| <li><code>java.lang.String JOB_MANAGER_IPC_ADDRESS_KEY</code></li> |
| <li><code>int DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS</code></li> |
| <li><code>java.lang.String DEFAULT_AKKA_TRANSPORT_HEARTBEAT_PAUSE</code></li> |
| <li><code>java.lang.String MESOS_ARTIFACT_SERVER_SSL_ENABLED</code></li> |
| <li><code>java.lang.String RESTART_STRATEGY_FAILURE_RATE_DELAY</code></li> |
| <li><code>java.lang.String DELIMITED_FORMAT_MIN_LINE_SAMPLES_KEY</code></li> |
| <li><code>java.lang.String BLOB_FETCH_BACKLOG_KEY</code></li> |
| <li><code>java.lang.String FILESYSTEM_OUTPUT_ALWAYS_CREATE_DIRECTORY_KEY</code></li> |
| <li><code>java.lang.String DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION</code></li> |
| <li><code>java.lang.String TASK_MANAGER_LOG_PATH_KEY</code></li> |
| <li><code>java.lang.String DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE</code></li> |
| <li><code>int DEFAULT_YARN_HEAP_CUTOFF</code></li> |
| <li><code>java.lang.String SECURITY_SSL_PROTOCOL</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_BACK_PRESSURE_NUM_SAMPLES</code></li> |
| <li><code>java.lang.String CHECKPOINTS_DIRECTORY_KEY</code></li> |
| <li><code>java.lang.String DELIMITED_FORMAT_MAX_LINE_SAMPLES_KEY</code></li> |
| <li><code>java.lang.String PATH_HADOOP_CONFIG</code></li> |
| <li><code>java.lang.String ZOOKEEPER_SASL_DISABLE</code></li> |
| <li><code>java.lang.String AKKA_LOOKUP_TIMEOUT</code></li> |
| <li><code>java.lang.String YARN_HEAP_CUTOFF_MIN</code></li> |
| <li><code>java.lang.String AKKA_CLIENT_TIMEOUT</code></li> |
| <li><code>int DEFAULT_ZOOKEEPER_SYNC_LIMIT</code></li> |
| <li><code>java.lang.String DEFAULT_HA_MODE</code></li> |
| <li><code>java.lang.String CONTAINERIZED_TASK_MANAGER_ENV_PREFIX</code></li> |
| <li><code>java.lang.String HA_ZOOKEEPER_CONNECTION_TIMEOUT</code></li> |
| <li><code>java.lang.String METRICS_REPORTER_ADDITIONAL_VARIABLES</code></li> |
| <li><code>java.lang.String MESOS_ARTIFACT_SERVER_PORT_KEY</code></li> |
| <li><code>java.lang.String TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD</code></li> |
| <li><code>java.lang.String TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY</code></li> |
| <li><code>java.lang.String YARN_APPLICATION_ATTEMPTS</code></li> |
| <li><code>java.lang.String AKKA_TRANSPORT_HEARTBEAT_PAUSE</code></li> |
| <li><code>java.lang.String DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE</code></li> |
| <li><code>java.lang.String SECURITY_SSL_VERIFY_HOSTNAME</code></li> |
| <li><code>java.lang.String DEFAULT_PARALLELISM_KEY</code></li> |
| <li><code>java.lang.String AKKA_DISPATCHER_THROUGHPUT</code></li> |
| <li><code>java.lang.String TASK_MANAGER_MEMORY_FRACTION_KEY</code></li> |
| <li><code>java.lang.String JOB_MANAGER_WEB_UPLOAD_DIR_KEY</code></li> |
| <li><code>java.lang.String SECURITY_SSL_KEY_PASSWORD</code></li> |
| <li><code>int DEFAULT_BLOB_FETCH_RETRIES</code></li> |
| <li><code>java.lang.String MESOS_RESOURCEMANAGER_FRAMEWORK_USER</code></li> |
| <li><code>java.lang.String BLOB_SERVICE_SSL_ENABLED</code></li> |
| <li><code>java.lang.String DEFAULT_YARN_APPLICATION_MASTER_PORT</code></li> |
| <li><code>java.lang.String METRICS_SCOPE_NAMING_TM</code></li> |
| <li><code>java.lang.String TASK_MANAGER_MAX_REGISTARTION_PAUSE</code></li> |
| <li><code>long DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL</code></li> |
| <li><code>int DEFAULT_FS_STREAM_OPENING_TIMEOUT</code></li> |
| <li><code>java.lang.String YARN_VCORES</code></li> |
| <li><code>java.lang.String YARN_MAX_FAILED_CONTAINERS</code></li> |
| <li><code>java.lang.String METRICS_REPORTER_INTERVAL_SUFFIX</code></li> |
| <li><code>java.lang.String DEFAULT_HA_ZOOKEEPER_CLIENT_ACL</code></li> |
| <li><code>float DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION</code></li> |
| <li><code>java.lang.String SAVEPOINT_DIRECTORY_KEY</code></li> |
| <li><code>int DEFAULT_ZOOKEEPER_RETRY_WAIT</code></li> |
| <li><code>java.lang.String ZOOKEEPER_LATCH_PATH</code></li> |
| <li><code>java.lang.String DEFAULT_RECOVERY_JOB_MANAGER_PORT</code></li> |
| <li><code>boolean DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD</code></li> |
| <li><code>boolean DEFAULT_AKKA_SSL_ENABLED</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.ConfigOption</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>java.lang.Iterable<java.lang.String> deprecatedKeys()</code></li> |
| <li><code>boolean hasDeprecatedKeys()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.ConfigOptions$OptionBuilder</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<T> defaultValue(java.lang.Object)</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> noDefaultValue()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.Configuration</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>boolean getBoolean(java.lang.String, boolean)</code></li> |
| <li><code>boolean getBoolean(org.apache.flink.configuration.ConfigOption<java.lang.Boolean>)</code></li> |
| <li><code>boolean getBoolean(org.apache.flink.configuration.ConfigOption<java.lang.Boolean>, boolean)</code></li> |
| <li><code>byte[] getBytes(java.lang.String, byte[])</code></li> |
| <li><code>java.lang.Class<T> getClass(java.lang.String, java.lang.Class<? extends T>, java.lang.ClassLoader)</code></li> |
| <li><code>double getDouble(java.lang.String, double)</code></li> |
| <li><code>double getDouble(org.apache.flink.configuration.ConfigOption<java.lang.Double>)</code></li> |
| <li><code>double getDouble(org.apache.flink.configuration.ConfigOption<java.lang.Double>, double)</code></li> |
| <li><code>float getFloat(java.lang.String, float)</code></li> |
| <li><code>float getFloat(org.apache.flink.configuration.ConfigOption<java.lang.Float>)</code></li> |
| <li><code>float getFloat(org.apache.flink.configuration.ConfigOption<java.lang.Float>, float)</code></li> |
| <li><code>int getInteger(java.lang.String, int)</code></li> |
| <li><code>int getInteger(org.apache.flink.configuration.ConfigOption<java.lang.Integer>)</code></li> |
| <li><code>int getInteger(org.apache.flink.configuration.ConfigOption<java.lang.Integer>, int)</code></li> |
| <li><code>long getLong(java.lang.String, long)</code></li> |
| <li><code>long getLong(org.apache.flink.configuration.ConfigOption<java.lang.Long>)</code></li> |
| <li><code>long getLong(org.apache.flink.configuration.ConfigOption<java.lang.Long>, long)</code></li> |
| <li><code>java.lang.String getString(org.apache.flink.configuration.ConfigOption<java.lang.String>)</code></li> |
| <li><code>java.lang.String getString(org.apache.flink.configuration.ConfigOption<java.lang.String>, java.lang.String)</code></li> |
| <li><code>void setBoolean(java.lang.String, boolean)</code></li> |
| <li><code>void setBoolean(org.apache.flink.configuration.ConfigOption<java.lang.Boolean>, boolean)</code></li> |
| <li><code>void setBytes(java.lang.String, byte[])</code></li> |
| <li><code>void setClass(java.lang.String, java.lang.Class<?>)</code></li> |
| <li><code>void setDouble(java.lang.String, double)</code></li> |
| <li><code>void setDouble(org.apache.flink.configuration.ConfigOption<java.lang.Double>, double)</code></li> |
| <li><code>void setFloat(java.lang.String, float)</code></li> |
| <li><code>void setFloat(org.apache.flink.configuration.ConfigOption<java.lang.Float>, float)</code></li> |
| <li><code>void setInteger(java.lang.String, int)</code></li> |
| <li><code>void setInteger(org.apache.flink.configuration.ConfigOption<java.lang.Integer>, int)</code></li> |
| <li><code>void setLong(java.lang.String, long)</code></li> |
| <li><code>void setLong(org.apache.flink.configuration.ConfigOption<java.lang.Long>, long)</code></li> |
| <li><code>void setString(org.apache.flink.configuration.ConfigOption<java.lang.String>, java.lang.String)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.ExecutionOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Long> ASYNC_STATE_BUFFER_TIMEOUT</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> ASYNC_INFLIGHT_RECORDS_LIMIT</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> ASYNC_STATE_BUFFER_SIZE</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.HighAvailabilityOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> HA_ZOOKEEPER_JOBGRAPHS_PATH</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> ZOOKEEPER_RUNNING_JOB_REGISTRY_PATH</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> HA_JOB_DELAY</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.JobManagerOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> JOB_MANAGER_HEAP_MEMORY_MB</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.time.Duration> BLOCK_SLOW_NODE_DURATION</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<org.apache.flink.configuration.MemorySize> ADAPTIVE_BATCH_SCHEDULER_AVG_DATA_VOLUME_PER_TASK</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> SPECULATIVE_MAX_CONCURRENT_EXECUTIONS</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> ADAPTIVE_BATCH_SCHEDULER_DEFAULT_SOURCE_PARALLELISM</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> ADAPTIVE_BATCH_SCHEDULER_MAX_PARALLELISM</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<org.apache.flink.configuration.MemorySize> JOB_MANAGER_HEAP_MEMORY</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> ADAPTIVE_BATCH_SCHEDULER_MIN_PARALLELISM</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> SPECULATIVE_ENABLED</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.JobManagerOptions$SchedulerType</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.JobManagerOptions$SchedulerType Ng</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.MetricOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> REPORTER_CLASS</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.NettyShuffleEnvironmentOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> NUM_THREADS_CLIENT</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> NETWORK_BUFFERS_PER_CHANNEL</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> HYBRID_SHUFFLE_SPILLED_INDEX_REGION_GROUP_SIZE</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> CONNECT_BACKLOG</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> NETWORK_HYBRID_SHUFFLE_ENABLE_NEW_MODE</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Float> NETWORK_BUFFERS_MEMORY_FRACTION</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> NUM_ARENAS</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> NUM_THREADS_SERVER</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Long> HYBRID_SHUFFLE_NUM_RETAINED_IN_MEMORY_REGIONS_MAX</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> NETWORK_BUFFERS_MEMORY_MIN</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> TRANSPORT_TYPE</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> NETWORK_BLOCKING_SHUFFLE_TYPE</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> NETWORK_MAX_OVERDRAFT_BUFFERS_PER_GATE</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Long> NETWORK_EXCLUSIVE_BUFFERS_REQUEST_TIMEOUT_MILLISECONDS</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> BATCH_SHUFFLE_COMPRESSION_ENABLED</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> SEND_RECEIVE_BUFFER_SIZE</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> NETWORK_BUFFERS_MEMORY_MAX</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> NETWORK_EXTRA_BUFFERS_PER_GATE</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> NETWORK_SORT_SHUFFLE_MIN_PARALLELISM</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> NETWORK_NUM_BUFFERS</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> NETWORK_MAX_BUFFERS_PER_CHANNEL</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> MAX_NUM_TCP_CONNECTIONS</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.PipelineOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.util.List<java.lang.String>> KRYO_DEFAULT_SERIALIZERS</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.util.List<java.lang.String>> POJO_REGISTERED_CLASSES</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> AUTO_TYPE_REGISTRATION</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.util.List<java.lang.String>> KRYO_REGISTERED_CLASSES</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.ResourceManagerOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> LOCAL_NUMBER_RESOURCE_MANAGER</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> TASK_MANAGER_RELEASE_WHEN_RESULT_CONSUMED</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Long> SLOT_MANAGER_TASK_MANAGER_TIMEOUT</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.SecurityOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Double> KERBEROS_TOKENS_RENEWAL_TIME_RATIO</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.time.Duration> KERBEROS_TOKENS_RENEWAL_RETRY_BACKOFF</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> KERBEROS_FETCH_DELEGATION_TOKEN</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> SSL_ENABLED</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.StateBackendOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> LATENCY_TRACK_HISTORY_SIZE</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> LATENCY_TRACK_STATE_NAME_AS_VARIABLE</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> LATENCY_TRACK_ENABLED</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> LATENCY_TRACK_SAMPLE_INTERVAL</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.TaskManagerOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.time.Duration> REGISTRATION_MAX_BACKOFF</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.time.Duration> INITIAL_REGISTRATION_BACKOFF</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> EXIT_ON_FATAL_AKKA_ERROR</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> TASK_MANAGER_HEAP_MEMORY_MB</code></li> |
| <li><code>java.lang.String MANAGED_MEMORY_CONSUMER_NAME_DATAPROC</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<org.apache.flink.configuration.MemorySize> TASK_MANAGER_HEAP_MEMORY</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.time.Duration> REFUSED_REGISTRATION_BACKOFF</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.TaskManagerOptions$TaskManagerLoadBalanceMode</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.TaskManagerOptions$TaskManagerLoadBalanceMode loadFromConfiguration(org.apache.flink.configuration.Configuration)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.configuration.WebOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> BACKPRESSURE_DELAY</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> PORT</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> BACKPRESSURE_REFRESH_INTERVAL</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> BACKPRESSURE_NUM_SAMPLES</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> ADDRESS</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> BACKPRESSURE_CLEANUP_INTERVAL</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Boolean> SSL_ENABLED</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.connector.base.sink.AsyncSinkBase</code> |
| <ul> |
| <li>interface removed: |
| <ul> |
| <li><code>org.apache.flink.api.connector.sink2.StatefulSink</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.connector.base.sink.writer.AsyncSinkWriter</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>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)</code></li> |
| <li><code>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>>)</code></li> |
| <li><code>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>>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.connector.base.sink.writer.ElementConverter</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>void open(org.apache.flink.api.connector.sink2.Sink$InitContext)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>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>>)</code></li> |
| <li><code>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)</code></li> |
| <li><code>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>>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>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>>)</code></li> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.connector.base.source.reader.SourceReaderBase</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.contrib.streaming.state.ConfigurableRocksDBOptionsFactory</code> |
| <ul> |
| <li>interface removed: |
| <ul> |
| <li><code>org.apache.flink.contrib.streaming.state.RocksDBOptionsFactory</code></li> |
| </ul> |
| </li> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.contrib.streaming.state.RocksDBOptionsFactory configure(org.apache.flink.configuration.ReadableConfig)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.contrib.streaming.state.RocksDBMemoryConfiguration getMemoryConfiguration()</code></li> |
| <li><code>org.apache.flink.contrib.streaming.state.PredefinedOptions getPredefinedOptions()</code></li> |
| <li><code>org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend$PriorityQueueStateType getPriorityQueueStateType()</code></li> |
| <li><code>org.apache.flink.contrib.streaming.state.RocksDBOptionsFactory getRocksDBOptions()</code></li> |
| <li><code>void setPredefinedOptions(org.apache.flink.contrib.streaming.state.PredefinedOptions)</code></li> |
| <li><code>void setPriorityQueueStateType(org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend$PriorityQueueStateType)</code></li> |
| <li><code>void setRocksDBMemoryFactory(org.apache.flink.contrib.streaming.state.RocksDBMemoryControllerUtils$RocksDBMemoryFactory)</code></li> |
| <li><code>void setRocksDBOptions(org.apache.flink.contrib.streaming.state.RocksDBOptionsFactory)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions fromConfig(org.apache.flink.configuration.ReadableConfig)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.contrib.streaming.state.RocksDBOptionsFactory</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions createNativeMetricsOptions(org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.core.execution.JobClient</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>java.util.concurrent.CompletableFuture<java.lang.String> stopWithSavepoint(boolean, java.lang.String)</code></li> |
| <li><code>java.util.concurrent.CompletableFuture<java.lang.String> triggerSavepoint(java.lang.String)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.core.failure.FailureEnricher$Context</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.api.common.JobID getJobId()</code></li> |
| <li><code>java.lang.String getJobName()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.core.fs.FileSystem</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.core.fs.FileSystemKind getKind()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.core.fs.Path</code> |
| <ul> |
| <li>interface removed: |
| <ul> |
| <li><code>org.apache.flink.core.io.IOReadableWritable</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.datastream.api.context.StateManager</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.datastream.api.ExecutionEnvironment</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.datastream.api.function.ProcessFunction</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>void open()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.datastream.api.function.TwoOutputApplyPartitionFunction</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.datastream.api.stream.KeyedPartitionStream</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.datastream.api.stream.NonKeyedPartitionStream</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>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>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.connector.sink2.CommittableMessage</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>java.util.OptionalLong getCheckpointId()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.connector.sink2.CommittableSummary</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>CommittableSummary(int, int, java.lang.Long, int, int, int)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>CommittableWithLineage(java.lang.Object, java.lang.Long, int)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.datastream.AllWindowedStream</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.datastream.AllWindowedStream<T,W> allowedLateness(org.apache.flink.streaming.api.windowing.time.Time)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.datastream.CoGroupedStreams$WithWindow</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.datastream.CoGroupedStreams$WithWindow<T1,T2,KEY,W> allowedLateness(org.apache.flink.streaming.api.windowing.time.Time)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> with(org.apache.flink.api.common.functions.CoGroupFunction<T1,T2,T>)</code></li> |
| <li><code>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>)</code></li> |
| </ul> |
| </li> |
| <li>method modified: |
| <ul> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStream</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamSink<T> addSink(org.apache.flink.streaming.api.functions.sink.SinkFunction<T>)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> assignTimestampsAndWatermarks(org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks<T>)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> assignTimestampsAndWatermarks(org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks<T>)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.IterativeStream<T> iterate()</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.IterativeStream<T> iterate(long)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.KeyedStream<T,org.apache.flink.api.java.tuple.Tuple> keyBy(int[])</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.KeyedStream<T,org.apache.flink.api.java.tuple.Tuple> keyBy(java.lang.String[])</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStream<T> partitionCustom(org.apache.flink.api.common.functions.Partitioner<K>, int)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStream<T> partitionCustom(org.apache.flink.api.common.functions.Partitioner<K>, java.lang.String)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamSink<T> sinkTo(org.apache.flink.api.connector.sink.Sink<T,?,?,?>)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamSink<T> sinkTo(org.apache.flink.api.connector.sink.Sink<T,?,?,?>, org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes)</code></li> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamSink<T> writeAsCsv(java.lang.String)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamSink<T> writeAsCsv(java.lang.String, org.apache.flink.core.fs.FileSystem$WriteMode)</code></li> |
| <li><code>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)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamSink<T> writeAsText(java.lang.String)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamSink<T> writeAsText(java.lang.String, org.apache.flink.core.fs.FileSystem$WriteMode)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamUtils</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>java.util.Iterator<OUT> collect(org.apache.flink.streaming.api.datastream.DataStream<OUT>)</code></li> |
| <li><code>java.util.Iterator<OUT> collect(org.apache.flink.streaming.api.datastream.DataStream<OUT>, java.lang.String)</code></li> |
| <li><code>java.util.List<E> collectBoundedStream(org.apache.flink.streaming.api.datastream.DataStream<E>, java.lang.String)</code></li> |
| <li><code>java.util.List<E> collectRecordsFromUnboundedStream(org.apache.flink.streaming.api.operators.collect.ClientAndIterator<E>, int)</code></li> |
| <li><code>java.util.List<E> collectUnboundedStream(org.apache.flink.streaming.api.datastream.DataStream<E>, int, java.lang.String)</code></li> |
| <li><code>org.apache.flink.streaming.api.operators.collect.ClientAndIterator<OUT> collectWithClient(org.apache.flink.streaming.api.datastream.DataStream<OUT>, java.lang.String)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.datastream.JoinedStreams$WithWindow</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.datastream.JoinedStreams$WithWindow<T1,T2,KEY,W> allowedLateness(org.apache.flink.streaming.api.windowing.time.Time)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> with(org.apache.flink.api.common.functions.JoinFunction<T1,T2,T>)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<T> with(org.apache.flink.api.common.functions.FlatJoinFunction<T1,T2,T>)</code></li> |
| <li><code>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>)</code></li> |
| </ul> |
| </li> |
| <li>method modified: |
| <ul> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.datastream.KeyedStream</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.datastream.KeyedStream$IntervalJoin</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.datastream.WindowedStream</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.datastream.WindowedStream<T,K,W> allowedLateness(org.apache.flink.streaming.api.windowing.time.Time)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.environment.CheckpointConfig</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>int UNDEFINED_TOLERABLE_CHECKPOINT_NUMBER</code></li> |
| <li><code>long DEFAULT_TIMEOUT</code></li> |
| <li><code>long DEFAULT_MIN_PAUSE_BETWEEN_CHECKPOINTS</code></li> |
| <li><code>org.apache.flink.streaming.api.CheckpointingMode DEFAULT_MODE</code></li> |
| <li><code>int DEFAULT_MAX_CONCURRENT_CHECKPOINTS</code></li> |
| <li><code>int DEFAULT_CHECKPOINT_ID_OF_IGNORED_IN_FLIGHT_DATA</code></li> |
| </ul> |
| </li> |
| <li>method removed: |
| <ul> |
| <li><code>void enableExternalizedCheckpoints(org.apache.flink.streaming.api.environment.CheckpointConfig$ExternalizedCheckpointCleanup)</code></li> |
| <li><code>java.time.Duration getAlignmentTimeout()</code></li> |
| <li><code>org.apache.flink.runtime.state.CheckpointStorage getCheckpointStorage()</code></li> |
| <li><code>org.apache.flink.streaming.api.environment.CheckpointConfig$ExternalizedCheckpointCleanup getExternalizedCheckpointCleanup()</code></li> |
| <li><code>boolean isFailOnCheckpointingErrors()</code></li> |
| <li><code>boolean isForceCheckpointing()</code></li> |
| <li><code>void setAlignmentTimeout(java.time.Duration)</code></li> |
| <li><code>void setCheckpointStorage(org.apache.flink.runtime.state.CheckpointStorage)</code></li> |
| <li><code>void setCheckpointStorage(java.lang.String)</code></li> |
| <li><code>void setCheckpointStorage(java.net.URI)</code></li> |
| <li><code>void setCheckpointStorage(org.apache.flink.core.fs.Path)</code></li> |
| <li><code>void setExternalizedCheckpointCleanup(org.apache.flink.streaming.api.environment.CheckpointConfig$ExternalizedCheckpointCleanup)</code></li> |
| <li><code>void setFailOnCheckpointingErrors(boolean)</code></li> |
| <li><code>void setForceCheckpointing(boolean)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.environment.RemoteStreamEnvironment</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.Configuration getClientConfiguration()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.environment.StreamExecutionEnvironment</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>java.lang.String DEFAULT_JOB_NAME</code></li> |
| </ul> |
| </li> |
| <li>method removed: |
| <ul> |
| <li><code>void addDefaultKryoSerializer(java.lang.Class<?>, com.esotericsoftware.kryo.Serializer<?>)</code></li> |
| <li><code>void addDefaultKryoSerializer(java.lang.Class<?>, java.lang.Class<? extends com.esotericsoftware.kryo.Serializer<? extends ?>>)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamSource<OUT> addSource(org.apache.flink.streaming.api.functions.source.SourceFunction<OUT>)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamSource<OUT> addSource(org.apache.flink.streaming.api.functions.source.SourceFunction<OUT>, java.lang.String)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>org.apache.flink.streaming.api.environment.StreamExecutionEnvironment enableCheckpointing(long, org.apache.flink.streaming.api.CheckpointingMode, boolean)</code></li> |
| <li><code>org.apache.flink.streaming.api.environment.StreamExecutionEnvironment enableCheckpointing()</code></li> |
| <li><code>int getNumberOfExecutionRetries()</code></li> |
| <li><code>org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration getRestartStrategy()</code></li> |
| <li><code>org.apache.flink.runtime.state.StateBackend getStateBackend()</code></li> |
| <li><code>org.apache.flink.streaming.api.TimeCharacteristic getStreamTimeCharacteristic()</code></li> |
| <li><code>boolean isForceCheckpointing()</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStream<java.lang.String> readFileStream(java.lang.String, long, org.apache.flink.streaming.api.functions.source.FileMonitoringFunction$WatchType)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamSource<java.lang.String> readTextFile(java.lang.String)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStreamSource<java.lang.String> readTextFile(java.lang.String, java.lang.String)</code></li> |
| <li><code>void registerType(java.lang.Class<?>)</code></li> |
| <li><code>void registerTypeWithKryoSerializer(java.lang.Class<?>, com.esotericsoftware.kryo.Serializer<?>)</code></li> |
| <li><code>void registerTypeWithKryoSerializer(java.lang.Class<?>, java.lang.Class<? extends com.esotericsoftware.kryo.Serializer>)</code></li> |
| <li><code>void setNumberOfExecutionRetries(int)</code></li> |
| <li><code>void setRestartStrategy(org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration)</code></li> |
| <li><code>org.apache.flink.streaming.api.environment.StreamExecutionEnvironment setStateBackend(org.apache.flink.runtime.state.StateBackend)</code></li> |
| <li><code>void setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.operators.AbstractStreamOperator</code> |
| <ul> |
| <li>interface removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.operators.SetupableStreamOperator</code></li> |
| </ul> |
| </li> |
| <li>method modified: |
| <ul> |
| <li><code>(<- NON_FINAL) void initializeState(org.apache.flink.streaming.api.operators.StreamTaskStateInitializer)</code></li> |
| <li><code>PROTECTED (<- PUBLIC) void setProcessingTimeService(org.apache.flink.streaming.runtime.tasks.ProcessingTimeService)</code></li> |
| <li><code>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>>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>(<- NON_FINAL) void initializeState(org.apache.flink.streaming.api.operators.StreamTaskStateInitializer)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>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>>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows withGap(org.apache.flink.streaming.api.windowing.time.Time)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.windowing.assigners.ProcessingTimeSessionWindows</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.windowing.assigners.ProcessingTimeSessionWindows withGap(org.apache.flink.streaming.api.windowing.time.Time)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows of(org.apache.flink.streaming.api.windowing.time.Time)</code></li> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows of(org.apache.flink.streaming.api.windowing.time.Time)</code></li> |
| <li><code>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)</code></li> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.windowing.assigners.WindowAssigner</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>(<- NON_ABSTRACT) org.apache.flink.streaming.api.windowing.triggers.Trigger<T,W><T,W> getDefaultTrigger()</code></li> |
| </ul> |
| </li> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.windowing.triggers.Trigger<T,W> getDefaultTrigger(org.apache.flink.streaming.api.environment.StreamExecutionEnvironment)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.windowing.evictors.TimeEvictor</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.windowing.evictors.TimeEvictor<W> of(org.apache.flink.streaming.api.windowing.time.Time)</code></li> |
| <li><code>org.apache.flink.streaming.api.windowing.evictors.TimeEvictor<W> of(org.apache.flink.streaming.api.windowing.time.Time, boolean)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.windowing.triggers.ContinuousEventTimeTrigger</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.windowing.triggers.ContinuousEventTimeTrigger<W> of(org.apache.flink.streaming.api.windowing.time.Time)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.windowing.triggers.ContinuousProcessingTimeTrigger</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.windowing.triggers.ContinuousProcessingTimeTrigger<W> of(org.apache.flink.streaming.api.windowing.time.Time)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.api.windowing.triggers.Trigger$TriggerContext</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.api.common.state.ValueState<S> getKeyValueState(java.lang.String, java.lang.Class<S>, java.io.Serializable)</code></li> |
| <li><code>org.apache.flink.api.common.state.ValueState<S> getKeyValueState(java.lang.String, org.apache.flink.api.common.typeinfo.TypeInformation<S>, java.io.Serializable)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.streaming.experimental.CollectSink</code> |
| <ul> |
| <li>interface removed: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.functions.sink.SinkFunction</code></li> |
| </ul> |
| </li> |
| <li>superclass modified: |
| <ul> |
| <li><code>org.apache.flink.streaming.api.functions.sink.legacy.RichSinkFunction (<- org.apache.flink.streaming.api.functions.sink.RichSinkFunction)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.types.DoubleValue</code> |
| <ul> |
| <li>interface removed: |
| <ul> |
| <li><code>org.apache.flink.types.Key</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.types.FloatValue</code> |
| <ul> |
| <li>interface removed: |
| <ul> |
| <li><code>org.apache.flink.types.Key</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.types.NormalizableKey</code> |
| <ul> |
| <li>interface removed: |
| <ul> |
| <li><code>org.apache.flink.core.io.IOReadableWritable</code></li> |
| <li><code>org.apache.flink.types.Value</code></li> |
| <li><code>org.apache.flink.types.Key</code></li> |
| <li><code>java.io.Serializable</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.test.junit5.MiniClusterExtension</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.test.util.TestEnvironment getTestEnvironment()</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> PORT</code></li> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> HOST</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.formats.csv.CsvReaderFormat</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>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>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.state.forst.ForStOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.String> REMOTE_DIRECTORY</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.state.forst.ForStOptionsFactory</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.rocksdb.ColumnFamilyOptions createColumnOptions(org.rocksdb.ColumnFamilyOptions, java.util.Collection<java.lang.AutoCloseable>)</code></li> |
| <li><code>org.rocksdb.DBOptions createDBOptions(org.rocksdb.DBOptions, java.util.Collection<java.lang.AutoCloseable>)</code></li> |
| <li><code>org.rocksdb.ReadOptions createReadOptions(org.rocksdb.ReadOptions, java.util.Collection<java.lang.AutoCloseable>)</code></li> |
| <li><code>org.rocksdb.WriteOptions createWriteOptions(org.rocksdb.WriteOptions, java.util.Collection<java.lang.AutoCloseable>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.client.config.SqlClientOptions</code> |
| <ul> |
| <li>field removed: |
| <ul> |
| <li><code>org.apache.flink.configuration.ConfigOption<java.lang.Integer> DISPLAY_MAX_COLUMN_WIDTH</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.api.common.typeutils.TypeSerializer<java.util.SortedMap<K,V>> createSerializer(org.apache.flink.api.common.ExecutionConfig)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.connector.file.sink.FileSink</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.api.connector.sink2.SinkWriter<IN> createWriter(org.apache.flink.api.connector.sink2.Sink$InitContext)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.connector.file.src.FileSource</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>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[])</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.connector.file.src.FileSourceSplit</code> |
| <ul> |
| <li>constructor removed: |
| <ul> |
| <li><code>FileSourceSplit(java.lang.String, org.apache.flink.core.fs.Path, long, long)</code></li> |
| <li><code>FileSourceSplit(java.lang.String, org.apache.flink.core.fs.Path, long, long, java.lang.String[])</code></li> |
| <li><code>FileSourceSplit(java.lang.String, org.apache.flink.core.fs.Path, long, long, java.lang.String[], org.apache.flink.connector.file.src.util.CheckpointedPosition)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.state.api.functions.KeyedStateReaderFunction</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>void open(org.apache.flink.configuration.Configuration)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.state.api.OperatorTransformation</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.state.api.OneInputOperatorTransformation<T> bootstrapWith(org.apache.flink.api.java.DataSet<T>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.state.api.SavepointReader</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>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>)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStream<OUT> readKeyedState(java.lang.String, org.apache.flink.state.api.functions.KeyedStateReaderFunction<K,OUT>)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStream<T> readListState(java.lang.String, java.lang.String, org.apache.flink.api.common.typeinfo.TypeInformation<T>)</code></li> |
| <li><code>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>)</code></li> |
| <li><code>org.apache.flink.streaming.api.datastream.DataStream<T> readUnionState(java.lang.String, java.lang.String, org.apache.flink.api.common.typeinfo.TypeInformation<T>)</code></li> |
| <li><code>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>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.state.api.SavepointWriter</code> |
| <ul> |
| <li>method removed: |
| <ul> |
| <li><code>org.apache.flink.state.api.SavepointWriter fromExistingSavepoint(java.lang.String)</code></li> |
| <li><code>org.apache.flink.state.api.SavepointWriter fromExistingSavepoint(java.lang.String, org.apache.flink.runtime.state.StateBackend)</code></li> |
| <li><code>org.apache.flink.state.api.SavepointWriter newSavepoint(int)</code></li> |
| <li><code>org.apache.flink.state.api.SavepointWriter newSavepoint(org.apache.flink.runtime.state.StateBackend, int)</code></li> |
| <li><code>org.apache.flink.state.api.SavepointWriter removeOperator(java.lang.String)</code></li> |
| <li><code>org.apache.flink.state.api.SavepointWriter withOperator(java.lang.String, org.apache.flink.state.api.StateBootstrapTransformation<T>)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code>org.apache.flink.state.api.SavepointWriterOperatorFactory</code> |
| <ul> |
| <li>method modified: |
| <ul> |
| <li><code>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)</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| </ul> |
| <h2 id="list-of-removed-configuration-options-a-nameremoved_configs-"> |
| List of removed configuration options <a name="removed_configs" /> |
| <a class="anchor" href="#list-of-removed-configuration-options-a-nameremoved_configs-">#</a> |
| </h2> |
| <ul> |
| <li>cluster.evenly-spread-out-slots</li> |
| <li>execution.async-state.buffer-size</li> |
| <li>execution.async-state.buffer-timeout</li> |
| <li>execution.async-state.in-flight-records-limit</li> |
| <li>fine-grained.shuffle-mode.all-blocking</li> |
| <li>high-availability.job.delay</li> |
| <li>high-availability.zookeeper.path.jobgraphs</li> |
| <li>high-availability.zookeeper.path.running-registry</li> |
| <li>jobmanager.adaptive-batch-scheduler.avg-data-volume-per-task</li> |
| <li>jobmanager.adaptive-batch-scheduler.default-source-parallelism</li> |
| <li>jobmanager.adaptive-batch-scheduler.max-parallelism</li> |
| <li>jobmanager.adaptive-batch-scheduler.min-parallelism</li> |
| <li>jobmanager.adaptive-batch-scheduler.speculative.block-slow-node-duration</li> |
| <li>jobmanager.adaptive-batch-scheduler.speculative.enabled</li> |
| <li>jobmanager.adaptive-batch-scheduler.speculative.max-concurrent-executions</li> |
| <li>jobmanager.heap.mb</li> |
| <li>jobmanager.heap.size</li> |
| <li>jobmanager.web.address</li> |
| <li>jobmanager.web.backpressure.cleanup-interval</li> |
| <li>jobmanager.web.backpressure.delay-between-samples</li> |
| <li>jobmanager.web.backpressure.num-samples</li> |
| <li>jobmanager.web.backpressure.refresh-interval</li> |
| <li>jobmanager.web.port;</li> |
| <li>jobmanager.web.ssl.enabled</li> |
| <li>local.number-resourcemanager</li> |
| <li>pipeline.auto-type-registration</li> |
| <li>pipeline.default-kryo-serializers</li> |
| <li>pipeline.registered-kryo-types</li> |
| <li>pipeline.registered-pojo-types</li> |
| <li>recovery.job.delay</li> |
| <li>resourcemanager.taskmanager-release.wait.result.consumed</li> |
| <li>security.kerberos.fetch.delegation-token</li> |
| <li>security.kerberos.tokens.renewal.retry.backoff</li> |
| <li>security.kerberos.tokens.renewal.time-ratio</li> |
| <li>security.ssl.enabled</li> |
| <li>slotmanager.taskmanager-timeout</li> |
| <li>sql-client.display.max-column-width</li> |
| <li>state.backend.async</li> |
| <li>state.backend.forst.remote-dir</li> |
| <li>state.backend.latency-track.history-size</li> |
| <li>state.backend.latency-track.keyed-state-enabled</li> |
| <li>state.backend.latency-track.sample-interval</li> |
| <li>state.backend.latency-track.state-name-as-variable</li> |
| <li>state.backend.local-recovery</li> |
| <li>state.backend.rocksdb.checkpointdir</li> |
| <li>state.backend.type</li> |
| <li>streaming-source.consume-order</li> |
| <li>table.exec.deduplicate.insert-and-updateafter-sensitive.enabled</li> |
| <li>table.exec.deduplicate.mini-batch.compact-changes.enabled</li> |
| <li>table.exec.legacy-transformation-uids</li> |
| <li>table.exec.shuffle-mode</li> |
| <li>table.exec.topn-cache-size</li> |
| <li>table.optimizer.source.aggregate-pushdown-enabled</li> |
| <li>table.optimizer.source.predicate-pushdown-enabled</li> |
| <li>table.optimizer.sql-to-rel.project.merge.enabled</li> |
| <li>taskmanager.exit-on-fatal-akka-error</li> |
| <li>taskmanager.heap.mb</li> |
| <li>taskmanager.heap.size</li> |
| <li>taskmanager.initial-registration-pause</li> |
| <li>taskmanager.max-registration-pause</li> |
| <li>taskmanager.net.client.numThreads</li> |
| <li>taskmanager.net.num-arenas</li> |
| <li>taskmanager.net.sendReceiveBufferSize</li> |
| <li>taskmanager.net.server.backlog</li> |
| <li>taskmanager.net.server.numThreads</li> |
| <li>taskmanager.net.transport</li> |
| <li>taskmanager.network.batch-shuffle.compression.enabled</li> |
| <li>taskmanager.network.blocking-shuffle.compression.enabled</li> |
| <li>taskmanager.network.blocking-shuffle.type</li> |
| <li>taskmanager.network.hybrid-shuffle.enable-new-mode</li> |
| <li>taskmanager.network.hybrid-shuffle.num-retained-in-memory-regions-max</li> |
| <li>taskmanager.network.hybrid-shuffle.spill-index-region-group-size</li> |
| <li>taskmanager.network.hybrid-shuffle.spill-index-segment-size</li> |
| <li>taskmanager.network.max-num-tcp-connections</li> |
| <li>taskmanager.network.memory.buffers-per-channel</li> |
| <li>taskmanager.network.memory.exclusive-buffers-request-timeout-ms</li> |
| <li>taskmanager.network.memory.floating-buffers-per-gate</li> |
| <li>taskmanager.network.memory.fraction</li> |
| <li>taskmanager.network.memory.max</li> |
| <li>taskmanager.network.memory.max-buffers-per-channel</li> |
| <li>taskmanager.network.memory.max-overdraft-buffers-per-gate</li> |
| <li>taskmanager.network.memory.min</li> |
| <li>taskmanager.network.netty.client.numThreads</li> |
| <li>taskmanager.network.netty.num-arenas</li> |
| <li>taskmanager.network.netty.sendReceiveBufferSize</li> |
| <li>taskmanager.network.netty.server.backlog</li> |
| <li>taskmanager.network.netty.server.numThreads</li> |
| <li>taskmanager.network.netty.transport</li> |
| <li>taskmanager.network.numberOfBuffers</li> |
| <li>taskmanager.network.sort-shuffle.min-parallelism</li> |
| <li>taskmanager.refused-registration-pause</li> |
| <li>taskmanager.registration.initial-backoff</li> |
| <li>taskmanager.registration.max-backoff</li> |
| <li>taskmanager.registration.refused-backoff</li> |
| <li>web.address</li> |
| <li>web.backpressure.cleanup-interval</li> |
| <li>web.backpressure.delay-between-samples</li> |
| <li>web.backpressure.num-samples</li> |
| <li>web.backpressure.refresh-interval</li> |
| <li>web.port</li> |
| <li>web.ssl.enabled</li> |
| </ul> |
| <h2 id="list-of-rest-apis-changes-a-namebreaking_rest_apis"> |
| List of REST APIs changes <a name="breaking_rest_apis"> |
| <a class="anchor" href="#list-of-rest-apis-changes-a-namebreaking_rest_apis">#</a> |
| </h2> |
| <table> |
| <thead> |
| <tr> |
| <th>REST API</th> |
| <th>Changes</th> |
| </tr> |
| </thead> |
| <tbody> |
| <tr> |
| <td>/taskmanagers/:taskmanagerid</td> |
| <td>In its response, “metrics.memorySegmentsAvailable” and “metrics.memorySegmentsTotal” are removed.</td> |
| </tr> |
| <tr> |
| <td>/jobs/:jobid/config</td> |
| <td>In its response, the “execution-mode” property is removed.</td> |
| </tr> |
| <tr> |
| <td>/jars/:jarid/run</td> |
| <td>In its request, the internal type of “claimMode” and “restoreMode” are changed from RestoreMode to RecoveryClaimMode, but their json structure is not affected.</td> |
| </tr> |
| <tr> |
| <td>/jobs/:jobid/vertices/:vertexid<br />/jobs/:jobid/vertices/:vertexid/subtasks/accumulators<br />/jobs/:jobid/vertices/:vertexid/subtasks/:subtaskindex<br />/jobs/:jobid/vertices/:vertexid/subtasks/:subtaskindex/attempts/:attempt<br />/jobs/:jobid/vertices/:vertexid/subtasktimes<br />/jobs/:jobid/vertices/:vertexid/taskmanagers<br />/jobs/:jobid/taskmanagers/:taskmanagerid/log-url</td> |
| <td>In their responses, the “host”, “subtasks.host” or “taskmanagers.host” property is removed.</td> |
| </tr> |
| </tbody> |
| </table> |
| <h2 id="list-of-removed-cli-options-a-nameremoved_cli_options-"> |
| List of removed CLI options <a name="removed_cli_options" /> |
| <a class="anchor" href="#list-of-removed-cli-options-a-nameremoved_cli_options-">#</a> |
| </h2> |
| <ul> |
| <li>sql-client.sh: |
| <ul> |
| <li><code>-u,--update <SQL update statement></code> is removed</li> |
| </ul> |
| </li> |
| <li>flink-client: |
| <ul> |
| <li><code>run-application</code> action is removed: Please use <code>run -t kubernetes-application</code> to run Kubernetes Application mode</li> |
| </ul> |
| </li> |
| </ul> |
| </p> |
| </article> |
| |
| |
| |
| |
| |
| |
| |
| <div class="edit-this-page"> |
| <p> |
| <a href="https://cwiki.apache.org/confluence/display/FLINK/Flink+Translation+Specifications">Want to contribute translation?</a> |
| </p> |
| <p> |
| <a href="//github.com/apache/flink-web/edit/asf-site/docs/content/posts/2025-03-24-release-2.0.0.md"> |
| Edit This Page<i class="fa fa-edit fa-fw"></i> |
| </a> |
| </p> |
| </div> |
| |
| </section> |
| |
| <aside class="book-toc"> |
| |
| |
| |
| <nav id="TableOfContents"><h3>On This Page <a href="javascript:void(0)" class="toc" onclick="collapseToc()"><i class="fa fa-times" aria-hidden="true"></i></a></h3> |
| <ul> |
| <li><a href="#highlights-of-new-features">Highlights of New Features</a> |
| <ul> |
| <li><a href="#disaggregated-state-management">Disaggregated State Management</a> |
| <ul> |
| <li><a href="#asynchronous-execution-model">Asynchronous Execution Model</a></li> |
| <li><a href="#enhanced-sql-operators">Enhanced SQL Operators</a></li> |
| <li><a href="#forst---a-disaggregated-state-backend">ForSt - A Disaggregated State Backend</a></li> |
| <li><a href="#performance-evaluation-on-nexmark">Performance Evaluation (on Nexmark)</a></li> |
| </ul> |
| </li> |
| <li><a href="#stream-batch-unification">Stream-Batch Unification</a> |
| <ul> |
| <li><a href="#materialized-table">Materialized Table</a></li> |
| <li><a href="#adaptive-batch-execution">Adaptive Batch Execution</a></li> |
| <li><a href="#performance">Performance</a></li> |
| </ul> |
| </li> |
| <li><a href="#streaming-lakehouse">Streaming Lakehouse</a></li> |
| <li><a href="#ai">AI</a></li> |
| <li><a href="#misc">Misc</a> |
| <ul> |
| <li><a href="#datastream-v2-api">DataStream V2 API</a></li> |
| <li><a href="#sql-gateway-supports-application-mode">SQL gateway supports application mode</a></li> |
| <li><a href="#sql-syntax-enhancements">SQL Syntax Enhancements</a></li> |
| <li><a href="#java-supports">Java Supports</a></li> |
| <li><a href="#serialization-improvements">Serialization Improvements</a></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><a href="#breaking-changes">Breaking Changes</a> |
| <ul> |
| <li><a href="#api">API</a> |
| <ul> |
| <li><a href="#connector-adaption-plan">Connector Adaption Plan</a></li> |
| </ul> |
| </li> |
| <li><a href="#configuration">Configuration</a></li> |
| <li><a href="#misc-1">Misc</a></li> |
| </ul> |
| </li> |
| <li><a href="#release-notes">Release notes</a></li> |
| <li><a href="#list-of-contributors">List of Contributors</a></li> |
| <li><a href="#appendix">Appendix</a> |
| <ul> |
| <li><a href="#list-of-breaking-change-programming-apis-a-namebreaking_programming_apis-">List of breaking change programming APIs <a name="breaking_programming_apis" /></a> |
| <ul> |
| <li><a href="#removed-classes">Removed Classes</a></li> |
| <li><a href="#modified-classes">Modified Classes</a></li> |
| </ul> |
| </li> |
| <li><a href="#list-of-removed-configuration-options-a-nameremoved_configs-">List of removed configuration options <a name="removed_configs" /></a></li> |
| <li><a href="#list-of-rest-apis-changes-a-namebreaking_rest_apis">List of REST APIs changes <a name="breaking_rest_apis"></a></li> |
| <li><a href="#list-of-removed-cli-options-a-nameremoved_cli_options-">List of removed CLI options <a name="removed_cli_options" /></a></li> |
| </ul> |
| </li> |
| </ul> |
| </nav> |
| |
| |
| </aside> |
| <aside class="expand-toc hidden"> |
| <a class="toc" onclick="expandToc()" href="javascript:void(0)"> |
| <i class="fa fa-bars" aria-hidden="true"></i> |
| </a> |
| </aside> |
| |
| </main> |
| |
| <footer> |
| |
| |
| |
| <div class="separator"></div> |
| <div class="panels"> |
| <div class="wrapper"> |
| <div class="panel"> |
| <ul> |
| <li> |
| <a href="https://flink-packages.org/">flink-packages.org</a> |
| </li> |
| <li> |
| <a href="https://www.apache.org/">Apache Software Foundation</a> |
| </li> |
| <li> |
| <a href="https://www.apache.org/licenses/">License</a> |
| </li> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| <li> |
| <a href="/zh/"> |
| <i class="fa fa-globe" aria-hidden="true"></i> 中文版 |
| </a> |
| </li> |
| |
| |
| |
| </ul> |
| </div> |
| <div class="panel"> |
| <ul> |
| <li> |
| <a href="/what-is-flink/security">Security</a--> |
| </li> |
| <li> |
| <a href="https://www.apache.org/foundation/sponsorship.html">Donate</a> |
| </li> |
| <li> |
| <a href="https://www.apache.org/foundation/thanks.html">Thanks</a> |
| </li> |
| </ul> |
| </div> |
| <div class="panel icons"> |
| <div> |
| <a href="/posts"> |
| <div class="icon flink-blog-icon"></div> |
| <span>Flink blog</span> |
| </a> |
| </div> |
| <div> |
| <a href="https://github.com/apache/flink"> |
| <div class="icon flink-github-icon"></div> |
| <span>Github</span> |
| </a> |
| </div> |
| <div> |
| <a href="https://twitter.com/apacheflink"> |
| <div class="icon flink-twitter-icon"></div> |
| <span>Twitter</span> |
| </a> |
| </div> |
| </div> |
| </div> |
| </div> |
| |
| <hr/> |
| |
| <div class="container disclaimer"> |
| <p>The contents of this website are © 2024 Apache Software Foundation under the terms of the Apache License v2. Apache Flink, Flink, and the Flink logo are either registered trademarks or trademarks of The Apache Software Foundation in the United States and other countries.</p> |
| </div> |
| |
| |
| |
| </footer> |
| |
| </body> |
| </html> |
| |
| |
| |
| |
| |
| |