From f88f7506834a006368ee41962d97209197e84e43 Mon Sep 17 00:00:00 2001 From: Rui Fan <1996fanrui@gmail.com> Date: Fri, 1 Mar 2024 16:46:44 +0900 Subject: [PATCH] [hotfix][runtime] Remove unneeded requestTaskManagerFileUploadByName (#24386) ResourceManagerGateway is internal class, so the deprecated method can be removed directly. --- .../resourcemanager/ResourceManager.java | 7 ------ .../ResourceManagerGateway.java | 18 --------------- .../utils/TestingResourceManagerGateway.java | 23 ------------------- 3 files changed, 48 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index efb0d55f517..45aa27eca69 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -810,13 +810,6 @@ public CompletableFuture requestTaskManagerFileUploadByType( } } - @Override - public CompletableFuture requestTaskManagerFileUploadByName( - ResourceID taskManagerId, String fileName, Time timeout) { - return requestTaskManagerFileUploadByNameAndType( - taskManagerId, fileName, FileType.LOG, Duration.ofMillis(timeout.toMilliseconds())); - } - @Override public CompletableFuture requestTaskManagerFileUploadByNameAndType( ResourceID taskManagerId, String fileName, FileType fileType, Duration timeout) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index 9faf8847493..bb42c529d4f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -226,24 +226,6 @@ CompletableFuture requestTaskManagerDetailsInfo( CompletableFuture requestTaskManagerFileUploadByType( ResourceID taskManagerId, FileType fileType, @RpcTimeout Time timeout); - /** - * Request the file upload from the given {@link TaskExecutor} to the cluster's {@link - * BlobServer}. The corresponding {@link TransientBlobKey} is returned. To support different - * type file upload with name, using {@link - * ResourceManager#requestTaskManagerFileUploadByNameAndType} as instead. - * - * @param taskManagerId identifying the {@link TaskExecutor} to upload the specified file - * @param fileName name of the file to upload - * @param timeout for the asynchronous operation - * @return Future which is completed with the {@link TransientBlobKey} after uploading the file - * to the {@link BlobServer}. - * @deprecated use {@link #requestTaskManagerFileUploadByNameAndType(ResourceID, String, - * FileType, Duration)} as instead. - */ - @Deprecated - CompletableFuture requestTaskManagerFileUploadByName( - ResourceID taskManagerId, String fileName, @RpcTimeout Time timeout); - /** * Request the file upload from the given {@link TaskExecutor} to the cluster's {@link * BlobServer}. The corresponding {@link TransientBlobKey} is returned. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java index 088fed278bb..3c7e3991e83 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java @@ -94,9 +94,6 @@ public class TestingResourceManagerGateway implements ResourceManagerGateway { private volatile Function, CompletableFuture> requestTaskManagerFileUploadByTypeFunction; - private volatile Function, CompletableFuture> - requestTaskManagerFileUploadByNameFunction; - private volatile Function< Tuple3, CompletableFuture> requestTaskManagerFileUploadByNameAndTypeFunction; @@ -192,13 +189,6 @@ public void setRequestTaskManagerFileUploadByTypeFunction( requestTaskManagerFileUploadByTypeFunction; } - public void setRequestTaskManagerFileUploadByNameFunction( - Function, CompletableFuture> - requestTaskManagerFileUploadByNameFunction) { - this.requestTaskManagerFileUploadByNameFunction = - requestTaskManagerFileUploadByNameFunction; - } - public void setRequestTaskManagerFileUploadByNameAndTypeFunction( Function, CompletableFuture> requestTaskManagerFileUploadByNameAndTypeFunction) { @@ -458,19 +448,6 @@ public CompletableFuture requestTaskManagerFileUploadByType( } } - @Override - public CompletableFuture requestTaskManagerFileUploadByName( - ResourceID taskManagerId, String fileName, Time timeout) { - final Function, CompletableFuture> function = - requestTaskManagerFileUploadByNameFunction; - - if (function != null) { - return function.apply(Tuple2.of(taskManagerId, fileName)); - } else { - return CompletableFuture.completedFuture(new TransientBlobKey()); - } - } - @Override public CompletableFuture requestTaskManagerFileUploadByNameAndType( ResourceID taskManagerId, String fileName, FileType fileType, Duration timeout) {