This is an automated email from the ASF dual-hosted git repository.
gaborgsomogyi pushed a commit to branch release-2.1
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/release-2.1 by this push:
new abab9195bd8 [FLINK-39022][security] Set security.ssl.algorithms
default value to modern cipher suite
abab9195bd8 is described below
commit abab9195bd8cc90f4c84ca209a0ffcb2b4597bc6
Author: balassai <[email protected]>
AuthorDate: Mon Mar 2 08:41:06 2026 +0100
[FLINK-39022][security] Set security.ssl.algorithms default value to modern
cipher suite
---
.../docs/deployment/security/security-ssl.md | 16 ++++++----------
docs/content/docs/deployment/security/security-ssl.md | 16 ++++++----------
docs/content/release-notes/flink-2.1.md | 19 ++++++++++++++++++-
.../shortcodes/generated/security_configuration.html | 4 ++--
.../shortcodes/generated/security_ssl_section.html | 4 ++--
.../apache/flink/configuration/SecurityOptions.java | 7 ++++---
.../flink/configuration/SecurityOptionsTest.java | 7 +++++++
.../org/apache/flink/runtime/net/SSLUtilsTest.java | 5 -----
8 files changed, 45 insertions(+), 33 deletions(-)
diff --git a/docs/content.zh/docs/deployment/security/security-ssl.md
b/docs/content.zh/docs/deployment/security/security-ssl.md
index baa3fc2091e..36179b3b786 100644
--- a/docs/content.zh/docs/deployment/security/security-ssl.md
+++ b/docs/content.zh/docs/deployment/security/security-ssl.md
@@ -150,20 +150,16 @@ security.ssl.rest.authentication-enabled: false
### Cipher suites
-{{< hint warning >}}
-The [IETF RFC 7525](https://tools.ietf.org/html/rfc7525) recommends to use a
specific set of cipher suites for strong security.
-Because these cipher suites were not available on many setups out of the box,
Flink's default value is set to a slightly
-weaker but more compatible cipher suite.
-We recommend that SSL setups update to the stronger cipher suites, if
possible, by adding the below entry to the Flink configuration:
+For strong security, it is crucial to use modern and robust cipher suites.
[IETF RFC 9325](https://www.rfc-editor.org/info/rfc9325), which supersedes the
older RFC 7525, provides current recommendations for the secure use of TLS.
+In response to evolving security standards and to ensure compatibility with
modern Java versions, Flink has updated its default cipher suites. Recent JDK
updates (affecting versions like 11.0.30+, 17.0.18+, etc.) have disabled older
`TLS_RSA_*` cipher suites that lack forward secrecy.
-```yaml
-security.ssl.algorithms:
TLS_DHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_DHE_RSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384
-```
+To support these secure-by-default JDK versions and align with best practices,
Flink's default value for `security.ssl.algorithms` is now:
-If these cipher suites are not supported on your setup, you will see that
Flink processes will not be able to connect to each other.
+`TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384`
-{{< /hint >}}
+This default provides strong security and wide compatibility. You can
customize the cipher suites using the `security.ssl.algorithms` configuration
option if your environment has different requirements.
+If these cipher suites are not supported on your setup, you will see that
Flink processes will not be able to connect to each other.
### Complete List of SSL Options
diff --git a/docs/content/docs/deployment/security/security-ssl.md
b/docs/content/docs/deployment/security/security-ssl.md
index a13d663c2ac..dca744ee3f2 100644
--- a/docs/content/docs/deployment/security/security-ssl.md
+++ b/docs/content/docs/deployment/security/security-ssl.md
@@ -150,20 +150,16 @@ security.ssl.rest.authentication-enabled: false
### Cipher suites
-{{< hint warning >}}
-The [IETF RFC 7525](https://tools.ietf.org/html/rfc7525) recommends to use a
specific set of cipher suites for strong security.
-Because these cipher suites were not available on many setups out of the box,
Flink's default value is set to a slightly
-weaker but more compatible cipher suite.
-We recommend that SSL setups update to the stronger cipher suites, if
possible, by adding the below entry to the Flink configuration:
+For strong security, it is crucial to use modern and robust cipher suites.
[IETF RFC 9325](https://www.rfc-editor.org/info/rfc9325), which supersedes the
older RFC 7525, provides current recommendations for the secure use of TLS.
+In response to evolving security standards and to ensure compatibility with
modern Java versions, Flink has updated its default cipher suites. Recent JDK
updates (affecting versions like 11.0.30+, 17.0.18+, etc.) have disabled older
`TLS_RSA_*` cipher suites that lack forward secrecy.
-```yaml
-security.ssl.algorithms:
TLS_DHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_DHE_RSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384
-```
+To support these secure-by-default JDK versions and align with best practices,
Flink's default value for `security.ssl.algorithms` is now:
-If these cipher suites are not supported on your setup, you will see that
Flink processes will not be able to connect to each other.
+`TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384`
-{{< /hint >}}
+This default provides strong security and wide compatibility. You can
customize the cipher suites using the `security.ssl.algorithms` configuration
option if your environment has different requirements.
+If these cipher suites are not supported on your setup, you will see that
Flink processes will not be able to connect to each other.
### Complete List of SSL Options
diff --git a/docs/content/release-notes/flink-2.1.md
b/docs/content/release-notes/flink-2.1.md
index b5e1b4f2f80..3130cdd7526 100644
--- a/docs/content/release-notes/flink-2.1.md
+++ b/docs/content/release-notes/flink-2.1.md
@@ -182,4 +182,21 @@ Bump flink-shaded version to 20.0 to support Smile format.
##### [FLINK-37760](https://issues.apache.org/jira/browse/FLINK-37760)
Bump parquet version to 1.15.3 to resolve parquet-avro module
-vulnerability found in
[CVE-2025-30065](https://nvd.nist.gov/vuln/detail/CVE-2025-30065).
\ No newline at end of file
+vulnerability found in
[CVE-2025-30065](https://nvd.nist.gov/vuln/detail/CVE-2025-30065).
+
+
+### Core
+
+#### Set security.ssl.algorithms default value to modern cipher suite
+
+### [FLINK-39022](https://issues.apache.org/jira/browse/FLINK-39022)
+
+A JDK update (affecting JDK 11.0.30+, 17.0.18+, 21.0.10+, and 24+) disabled
`TLS_RSA_*` cipher suites.
+This was done to support forward-secrecy (RFC 9325) and comply with the IETF
Draft on *Deprecating Obsolete Key Exchange Methods in TLS*.
+
+To support these and future JDK versions, the default value for the Flink
configuration option `security.ssl.algorithms` has been changed to a modern,
widely available cipher suite:
+
+`TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384`
+
+This default provides strong security and wide compatibility. You can
customize the cipher suites using the `security.ssl.algorithms` configuration
option if your environment has different requirements.
+If these cipher suites are not supported on your setup, you will see that
Flink processes will not be able to connect to each other.
diff --git a/docs/layouts/shortcodes/generated/security_configuration.html
b/docs/layouts/shortcodes/generated/security_configuration.html
index ff19479042e..37f6db0b589 100644
--- a/docs/layouts/shortcodes/generated/security_configuration.html
+++ b/docs/layouts/shortcodes/generated/security_configuration.html
@@ -88,9 +88,9 @@
</tr>
<tr>
<td><h5>security.ssl.algorithms</h5></td>
- <td style="word-wrap:
break-word;">"TLS_RSA_WITH_AES_128_CBC_SHA"</td>
+ <td style="word-wrap:
break-word;">"TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384"</td>
<td>String</td>
- <td>The comma separated list of standard SSL algorithms to be
supported. Read more <a
href="http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites">here</a></td>
+ <td>The comma separated list of standard SSL algorithms to be
supported. Read more <a
href="https://docs.oracle.com/en/java/javase/11/docs/specs/security/standard-names.html#jsse-cipher-suite-names">here</a></td>
</tr>
<tr>
<td><h5>security.ssl.internal.cert.fingerprint</h5></td>
diff --git a/docs/layouts/shortcodes/generated/security_ssl_section.html
b/docs/layouts/shortcodes/generated/security_ssl_section.html
index ad5c72b3cf2..9417f900825 100644
--- a/docs/layouts/shortcodes/generated/security_ssl_section.html
+++ b/docs/layouts/shortcodes/generated/security_ssl_section.html
@@ -10,9 +10,9 @@
<tbody>
<tr>
<td><h5>security.ssl.algorithms</h5></td>
- <td style="word-wrap:
break-word;">"TLS_RSA_WITH_AES_128_CBC_SHA"</td>
+ <td style="word-wrap:
break-word;">"TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384"</td>
<td>String</td>
- <td>The comma separated list of standard SSL algorithms to be
supported. Read more <a
href="http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites">here</a></td>
+ <td>The comma separated list of standard SSL algorithms to be
supported. Read more <a
href="https://docs.oracle.com/en/java/javase/11/docs/specs/security/standard-names.html#jsse-cipher-suite-names">here</a></td>
</tr>
<tr>
<td><h5>security.ssl.internal.cert.fingerprint</h5></td>
diff --git
a/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java
b/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java
index 24ecf2f6eaf..97fa7a17174 100644
---
a/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java
+++
b/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java
@@ -498,19 +498,20 @@ public class SecurityOptions {
* The standard SSL algorithms to be supported.
*
* <p>More options here -
- *
http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites
+ *
https://docs.oracle.com/en/java/javase/11/docs/specs/security/standard-names.html#jsse-cipher-suite-names
*/
@Documentation.Section(Documentation.Sections.SECURITY_SSL)
public static final ConfigOption<String> SSL_ALGORITHMS =
key("security.ssl.algorithms")
.stringType()
- .defaultValue("TLS_RSA_WITH_AES_128_CBC_SHA")
+ .defaultValue(
+
"TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384")
.withDescription(
Description.builder()
.text(
"The comma separated list of
standard SSL algorithms to be supported. Read more %s",
link(
-
"http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites",
+
"https://docs.oracle.com/en/java/javase/11/docs/specs/security/standard-names.html#jsse-cipher-suite-names",
"here"))
.build());
diff --git
a/flink-core/src/test/java/org/apache/flink/configuration/SecurityOptionsTest.java
b/flink-core/src/test/java/org/apache/flink/configuration/SecurityOptionsTest.java
index 7edf67ab8ea..34ac0160bf6 100644
---
a/flink-core/src/test/java/org/apache/flink/configuration/SecurityOptionsTest.java
+++
b/flink-core/src/test/java/org/apache/flink/configuration/SecurityOptionsTest.java
@@ -58,4 +58,11 @@ class SecurityOptionsTest {
options.set(SecurityOptions.SSL_REST_AUTHENTICATION_ENABLED, true);
assertThat(SecurityOptions.isRestSSLAuthenticationEnabled(options)).isTrue();
}
+
+ @Test
+ void checkDefaultCipherSuite() {
+ assertThat(SecurityOptions.SSL_ALGORITHMS.defaultValue())
+ .isEqualTo(
+
"TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384");
+ }
}
diff --git
a/flink-runtime/src/test/java/org/apache/flink/runtime/net/SSLUtilsTest.java
b/flink-runtime/src/test/java/org/apache/flink/runtime/net/SSLUtilsTest.java
index de7a5445639..43e7de101ee 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/net/SSLUtilsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/net/SSLUtilsTest.java
@@ -523,11 +523,6 @@ public class SSLUtilsTest {
private static void addSslProviderConfig(Configuration config, String
sslProvider) {
if (sslProvider.equalsIgnoreCase("OPENSSL")) {
OpenSsl.ensureAvailability();
-
- // Flink's default algorithm set is not available for openSSL -
choose a different one:
- config.set(
- SecurityOptions.SSL_ALGORITHMS,
-
"TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384");
}
config.set(SecurityOptions.SSL_PROVIDER, sslProvider);
}