diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/endpoint/hive/util/OperationExecutorFactory.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/endpoint/hive/util/OperationExecutorFactory.java index d3d31f3923027f..1329f2a5ddd270 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/endpoint/hive/util/OperationExecutorFactory.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/endpoint/hive/util/OperationExecutorFactory.java @@ -508,10 +508,10 @@ private static Pattern convertNamePattern(@Nullable String pattern) { String wStr = ".*"; return Pattern.compile( pattern.replaceAll("([^\\\\])%", "$1" + wStr) - .replaceAll("\\\\%", "%") + .replace("\\\\%", "%") .replaceAll("^%", wStr) .replaceAll("([^\\\\])_", "$1.") - .replaceAll("\\\\_", "_") + .replace("\\\\_", "_") .replaceAll("^_", ".")); } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTLexer.g b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTLexer.g index e0d3a8dd0db9a0..0f40a3e1d8143d 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTLexer.g +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTLexer.g @@ -487,7 +487,7 @@ Identifier fragment QuotedIdentifier : - '`' ( '``' | ~('`') )* '`' { setText(getText().substring(1, getText().length() -1 ).replaceAll("``", "`")); } + '`' ( '``' | ~('`') )* '`' { setText(getText().substring(1, getText().length() -1 ).replace("``", "`")); } ; CharSetName diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java index d733637e83cc5a..50994aa5b69d82 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java @@ -763,7 +763,7 @@ private static void verifyWrittenData(List expected, List results) Set expectedSet = new HashSet<>(); for (int i = 0; i < results.size(); i++) { final String rowString = expected.get(i).toString(); - expectedSet.add(rowString.substring(3, rowString.length() - 1).replaceAll(", ", "\t")); + expectedSet.add(rowString.substring(3, rowString.length() - 1).replace(", ", "\t")); } assertThat(new HashSet<>(results)).isEqualTo(expectedSet); } diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index fd7978d3235421..c231957874d4e6 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -2013,7 +2013,7 @@ private boolean isValidPojoField(Field f, Class clazz, List typeHierarc return true; } else { boolean hasGetter = false, hasSetter = false; - final String fieldNameLow = f.getName().toLowerCase().replaceAll("_", ""); + final String fieldNameLow = f.getName().toLowerCase().replace("_", ""); Type fieldType = f.getGenericType(); Class fieldTypeWrapper = ClassUtils.primitiveToWrapper(f.getType()); @@ -2028,9 +2028,9 @@ private boolean isValidPojoField(Field f, Class clazz, List typeHierarc m.getName().endsWith("_$eq") ? m.getName() .toLowerCase() - .replaceAll("_", "") + .replace("_", "") .replaceFirst("\\$eq$", "_\\$eq") - : m.getName().toLowerCase().replaceAll("_", ""); + : m.getName().toLowerCase().replace("_", ""); // check for getter if ( // The name should be "get" or "" (for scala) or diff --git a/flink-core/src/main/java/org/apache/flink/configuration/StructuredOptionsSplitter.java b/flink-core/src/main/java/org/apache/flink/configuration/StructuredOptionsSplitter.java index 9373eec596b57b..e45bbc88966a31 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/StructuredOptionsSplitter.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/StructuredOptionsSplitter.java @@ -81,7 +81,7 @@ static String escapeWithSingleQuote(String string, String... charsToEscape) { || string.contains("'"); if (escape) { - return "'" + string.replaceAll("'", "''") + "'"; + return "'" + string.replace("'", "''") + "'"; } return string; diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/Formatter.java b/flink-core/src/main/java/org/apache/flink/configuration/description/Formatter.java index 3dcd89f8883bbe..660d20e32f6b85 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/description/Formatter.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/description/Formatter.java @@ -105,8 +105,8 @@ protected abstract void formatText( private static final String TEMPORARY_PLACEHOLDER = "randomPlaceholderForStringFormat"; private static String escapeFormatPlaceholder(String value) { - return value.replaceAll("%s", TEMPORARY_PLACEHOLDER) - .replaceAll("%", "%%") - .replaceAll(TEMPORARY_PLACEHOLDER, "%s"); + return value.replace("%s", TEMPORARY_PLACEHOLDER) + .replace("%", "%%") + .replace(TEMPORARY_PLACEHOLDER, "%s"); } } diff --git a/flink-docs/src/main/java/org/apache/flink/docs/util/ConfigurationOptionLocator.java b/flink-docs/src/main/java/org/apache/flink/docs/util/ConfigurationOptionLocator.java index 81f5713d0552d0..cc2bf2c2a2cb77 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/util/ConfigurationOptionLocator.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/util/ConfigurationOptionLocator.java @@ -150,7 +150,7 @@ public static List> discoverOptionsAndApply( throws IOException, ClassNotFoundException { Path configDir = - rootDir.resolve(Paths.get(module, pathPrefix, packageName.replaceAll("\\.", "/"))); + rootDir.resolve(Paths.get(module, pathPrefix, packageName.replace(".", "/"))); List> optionClasses = new ArrayList<>(); try (DirectoryStream stream = Files.newDirectoryStream(configDir)) { diff --git a/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java b/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java index 43e2a44b8dd647..35f2f89c7b3df9 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java @@ -29,9 +29,9 @@ public enum Utils { private static final String TEMPORARY_PLACEHOLDER = "superRandomTemporaryPlaceholder"; public static String escapeCharacters(String value) { - return value.replaceAll("", TEMPORARY_PLACEHOLDER) - .replaceAll("<", "<") - .replaceAll(">", ">") - .replaceAll(TEMPORARY_PLACEHOLDER, ""); + return value.replace("", TEMPORARY_PLACEHOLDER) + .replace("<", "<") + .replace(">", ">") + .replace(TEMPORARY_PLACEHOLDER, ""); } } diff --git a/flink-end-to-end-tests/flink-tpch-test/src/main/java/org/apache/flink/table/tpch/TpchResultComparator.java b/flink-end-to-end-tests/flink-tpch-test/src/main/java/org/apache/flink/table/tpch/TpchResultComparator.java index 2cdb7d3b6bbdfc..cbd30307c85a2e 100644 --- a/flink-end-to-end-tests/flink-tpch-test/src/main/java/org/apache/flink/table/tpch/TpchResultComparator.java +++ b/flink-end-to-end-tests/flink-tpch-test/src/main/java/org/apache/flink/table/tpch/TpchResultComparator.java @@ -96,7 +96,7 @@ public static void main(String[] args) throws IOException { failed = !expected[i] .trim() - .equals(actual[i].replaceAll("\"", "").trim()); + .equals(actual[i].replace("\"", "").trim()); } } if (failed) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RescaleMappings.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RescaleMappings.java index a6b96c4ad1ef51..4990ad59e1bd48 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RescaleMappings.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RescaleMappings.java @@ -37,7 +37,7 @@ * #invert()}, the direction may change accordingly. To generalize, the left side is called source * and the right side is called target(s) in this class. * - *

{@ImplNote This class omits trailing empty targets.} + *

{@implNote This class omits trailing empty targets.} */ public class RescaleMappings implements Serializable { public static final RescaleMappings SYMMETRIC_IDENTITY = diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/filter/DefaultMetricFilter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/filter/DefaultMetricFilter.java index b31a30efa430ce..08f6ebeb0f0ad7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/filter/DefaultMetricFilter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/filter/DefaultMetricFilter.java @@ -94,8 +94,8 @@ static Pattern convertToPattern(String scopeOrNameComponent) { final String rawPattern = Arrays.stream(split) - .map(s -> s.replaceAll("\\.", "\\.")) - .map(s -> s.replaceAll("\\*", ".*")) + .map(s -> s.replace(".", "\\.")) + .map(s -> s.replace("*", ".*")) .collect(Collectors.joining("|", "(", ")")); return Pattern.compile(rawPattern); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/GeneratedLogUrlHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/GeneratedLogUrlHandler.java index f4d5af81dbfda0..ae194098d4c6cb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/GeneratedLogUrlHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/GeneratedLogUrlHandler.java @@ -79,9 +79,9 @@ protected void channelRead0(ChannelHandlerContext ctx, RoutedRequest rou @VisibleForTesting static String generateLogUrl(String pattern, String jobId, String taskManagerId) { - String generatedUrl = pattern.replaceAll("", jobId); + String generatedUrl = pattern.replace("", jobId); if (null != taskManagerId) { - generatedUrl = generatedUrl.replaceAll("", taskManagerId); + generatedUrl = generatedUrl.replace("", taskManagerId); } return generatedUrl; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/Hardware.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/Hardware.java index 28e7a4614ad08f..e72ddb2878f92e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/Hardware.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/Hardware.java @@ -258,7 +258,7 @@ private static long getSizeOfPhysicalMemoryForWindows() { continue; } - line = line.replaceAll(" ", ""); + line = line.replace(" ", ""); sizeOfPhyiscalMemory += Long.parseLong(line); } return sizeOfPhyiscalMemory; diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksDBSnapshotStrategyBase.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksDBSnapshotStrategyBase.java index 9cb73d01dc9623..ee2ec6fdeb3b63 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksDBSnapshotStrategyBase.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksDBSnapshotStrategyBase.java @@ -135,7 +135,7 @@ public RocksDBSnapshotStrategyBase( this.localRecoveryConfig = localRecoveryConfig; this.description = description; this.instanceBasePath = instanceBasePath; - this.localDirectoryName = backendUID.toString().replaceAll("[\\-]", ""); + this.localDirectoryName = backendUID.toString().replace("-", ""); this.backendUID = backendUID; } diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/SqlClientSyntaxHighlighter.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/SqlClientSyntaxHighlighter.java index 75e773b2ba6bdc..6b21569df07c51 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/SqlClientSyntaxHighlighter.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/SqlClientSyntaxHighlighter.java @@ -44,7 +44,7 @@ public class SqlClientSyntaxHighlighter extends DefaultHighlighter { private static final Set KEYWORDS = Collections.unmodifiableSet( Arrays.stream(FlinkSqlParserImplConstants.tokenImage) - .map(t -> t.replaceAll("\"", "")) + .map(t -> t.replace("\"", "")) .collect(Collectors.toSet())); private final Executor executor; diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/JoinQueryOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/JoinQueryOperation.java index bd139ca3d10e3d..7b8552b66e504c 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/JoinQueryOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/JoinQueryOperation.java @@ -112,7 +112,7 @@ public String asSerializableString() { "SELECT %s FROM (%s\n) %s JOIN %s ON %s", OperationUtils.formatSelectColumns(resolvedSchema), OperationUtils.indent(left.asSerializableString()), - joinType.toString().replaceAll("_", " "), + joinType.toString().replace("_", " "), rightToSerializable(), condition.asSerializableString()); } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/OperationUtils.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/OperationUtils.java index 7bf9f2c54f7971..2b8cea27714c1a 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/OperationUtils.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/OperationUtils.java @@ -63,7 +63,7 @@ static String indent(String item) { if (group.startsWith("'")) { matcher.appendReplacement(output, Matcher.quoteReplacement(group)); } else { - String replaced = group.replaceAll("\n", "\n" + OPERATION_INDENT); + String replaced = group.replace("\n", "\n" + OPERATION_INDENT); matcher.appendReplacement(output, Matcher.quoteReplacement(replaced)); } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/JsonFunctionsCallSyntax.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/JsonFunctionsCallSyntax.java index f60b9cddcdd4b8..dd0da63a86af30 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/JsonFunctionsCallSyntax.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/JsonFunctionsCallSyntax.java @@ -110,8 +110,8 @@ class JsonFunctionsCallSyntax { operands.get(0).asSerializableString(), operands.get(1).asSerializableString(), toString(wrapper), - onEmpty.toString().replaceAll("_", " "), - onError.toString().replaceAll("_", " ")); + onEmpty.toString().replace("_", " "), + onError.toString().replace("_", " ")); }; static final SqlCallSyntax JSON_OBJECT = diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/Indenter.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/Indenter.scala index 626abcec4e5f73..ef43515481c07c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/Indenter.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/Indenter.scala @@ -25,7 +25,7 @@ class IndentStringContext(sc: StringContext) { val ind = getindent(s) if (ind.nonEmpty) { - sb.append(a.toString.replaceAll("\n", "\n" + ind)) + sb.append(a.toString.replace("\n", "\n" + ind)) } else { sb.append(a.toString) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/LogicalPlanFormatUtils.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/LogicalPlanFormatUtils.scala index 89962cca4332e1..40ee9cb89ba4a7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/LogicalPlanFormatUtils.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/LogicalPlanFormatUtils.scala @@ -21,7 +21,7 @@ object LogicalPlanFormatUtils { private val tempPattern = """TMP_\d+""".r def formatTempTableId(preStr: String): String = { - val str = preStr.replaceAll("ArrayBuffer\\(", "List\\(") + val str = preStr.replace("ArrayBuffer(", "List(") val minId = getMinTempTableId(str) tempPattern.replaceAllIn(str, s => "TMP_" + (s.matched.substring(4).toInt - minId)) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index e5b418365be688..8a9128800a2048 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -849,7 +849,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) // add the postfix to the path to avoid conflicts // between the test class name and the result file name val clazz = test.getClass - val testClassDirPath = clazz.getName.replaceAll("\\.", "/") + "_jsonplan" + val testClassDirPath = clazz.getName.replace(".", "/") + "_jsonplan" val testMethodFileName = test.methodName + ".out" val resourceTestFilePath = s"/$testClassDirPath/$testMethodFileName" val plannerDirPath = clazz.getResource("/").getFile.replace("/target/test-classes/", "") @@ -1165,7 +1165,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) /** Replace the estimated costs for the given plan, because it may be unstable. */ protected def replaceEstimatedCost(s: String): String = { - var str = s.replaceAll("\\r\\n", "\n") + var str = s.replace("\r\n", "\n") val scientificFormRegExpr = "[+-]?[\\d]+([\\.][\\d]*)?([Ee][+-]?[0-9]{0,2})?" str = str.replaceAll(s"rowcount = $scientificFormRegExpr", "rowcount = ") str = str.replaceAll(s"$scientificFormRegExpr rows", "rows") @@ -1813,7 +1813,7 @@ object TableTestUtil { * StreamExecutionEnvironment is up */ def replaceStageId(s: String): String = { - s.replaceAll("\\r\\n", "\n").replaceAll("Stage \\d+", "") + s.replace("\r\n", "\n").replaceAll("Stage \\d+", "") } /** diff --git a/tools/ci/flink-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java b/tools/ci/flink-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java index cb27b998ed9570..e4d9fc7c5dd9c1 100644 --- a/tools/ci/flink-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java +++ b/tools/ci/flink-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java @@ -188,7 +188,7 @@ private static Collection asPatterns(String... texts) { private static Pattern asPatternWithPotentialLineBreaks(String text) { // allows word sequences to be separated by whitespace, line-breaks and comments(//, #) - return Pattern.compile(text.toLowerCase(Locale.ROOT).replaceAll(" ", " ?\\\\R?[\\\\s/#]*")); + return Pattern.compile(text.toLowerCase(Locale.ROOT).replace(" ", " ?\\\\R?[\\\\s/#]*")); } private static int findNonBinaryFilesContainingText(