You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/02/28 06:22:33 UTC

[GitHub] [flink] jinglining opened a new pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

jinglining opened a new pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250
 
 
   
   ## What is the purpose of the change
   
   This pull request makes rest api could get log list and get log by name for taskmanager
   
   
   ## Brief change log
     - get log list
     - get log by name
   
   ## Verifying this change
   
   
   
   This change added tests and can be verified as follows:
    -  Added TaskManagerLogsHandlerTest that verfied TaskManagerLogsHandler.
    -  Updated AbstractTaskManagerFileHandlerTest could verfied TaskManagerCustomFileHandler.
   
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (no)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes)
     - If yes, how is the feature documented? (docs)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394221144
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
+		log.debug("Request file name is {} upload.", fileName);
 
 Review comment:
   ok

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592334762
 
 
   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 912485a96d61febfa41b7c84631aeae19d819325 (Fri Feb 28 06:25:53 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
    * **This pull request references an unassigned [Jira ticket](https://issues.apache.org/jira/browse/FLINK-16302).** According to the [code contribution guide](https://flink.apache.org/contributing/contribute-code.html), tickets need to be assigned before starting with the implementation work.
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391809222
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogsInfo.java
 ##########
 @@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.messages.taskmanager;
+
+import org.apache.flink.runtime.rest.messages.ResponseBody;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Objects;
+
+/**
+ * Class containing a collection of {@link LogInfo}.
+ */
+public class LogsInfo implements ResponseBody {
+
+	public static final String FIELD_NAME_LOGS = "logs";
+
+	@JsonProperty(FIELD_NAME_LOGS)
+	private final Collection<LogInfo> logInfos;
+
+	@JsonCreator
+	public LogsInfo(
+			@JsonProperty(FIELD_NAME_LOGS) Collection<LogInfo> logInfos) {
 
 Review comment:
   The line break seems necessary. The line length would be the same even without the line break.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 54ac8f26c67344a0ed000c882afd66e68c8f6bd0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151328904) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812) 
   * c07d058a0f55764b319d64034b4f1bfdc26d99e6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 912485a96d61febfa41b7c84631aeae19d819325 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/150964833) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718) 
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * b8d51c94a0b93fdbfa4b167e0b4c630f791fba10 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150973416) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 912485a96d61febfa41b7c84631aeae19d819325 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/150964833) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718) 
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r390113102
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -900,7 +921,7 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
+	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, String fileName, Time timeout) {
 
 Review comment:
   Yes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394133406
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -309,6 +311,25 @@ public TaskExecutor(
 		return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty());
 	}
 
+	@Override
+	public CompletableFuture<Collection<Tuple2<String, Long>>> requestLogList(Time timeout) {
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null) {
+			final File[] logFiles = new File(logDir).listFiles();
+
+			if (logFiles == null) {
+				return FutureUtils.completedExceptionally(
+					new FlinkException("The specific log directory is not a valid directory."));
+			}
+
+			final List<Tuple2<String, Long>> logsWithLength = Arrays.stream(logFiles)
+				.map(logFile -> Tuple2.of(logFile.getName(), logFile.length()))
+				.collect(Collectors.toList());
+			return CompletableFuture.completedFuture(logsWithLength);
+		}
+		return FutureUtils.completedExceptionally(new FlinkException("There is no log file available on the TaskExecutor."));
 
 Review comment:
   Ok, that's reasonable.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/152623819) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130) 
   * 3984f26b34353ad5b2006c8768536fbaed5b512c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394971162
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogsHandlerTest.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException;
+import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.HandlerRequestException;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogsInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerLogsHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test for the {@link TaskManagerLogsHandler}.
+ */
+public class TaskManagerLogsHandlerTest extends TestLogger {
+
+	private static final ResourceID EXPECTED_TASK_MANAGER_ID = ResourceID.generate();
+
+	@Test
+	public void testGetTaskManagerLogsList() throws Exception {
+		final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway();
+		final TaskManagerLogsHandler taskManagerLogsHandler = new TaskManagerLogsHandler(
+			() -> CompletableFuture.completedFuture(null),
+			TestingUtils.TIMEOUT(),
+			Collections.emptyMap(),
+			TaskManagerLogsHeaders.getInstance(),
+			() -> CompletableFuture.completedFuture(resourceManagerGateway));
+		final HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> handlerRequest = createRequest(EXPECTED_TASK_MANAGER_ID);
+		List<LogInfo> logsList = new ArrayList<>();
+		logsList.add(new LogInfo("taskmanager.log", 1024L));
+		logsList.add(new LogInfo("taskmanager.out", 1024L));
+		logsList.add(new LogInfo("taskmanager-2.out", 1024L));
+		resourceManagerGateway.setRequestTaskManagerLogListFunction(EXPECTED_TASK_MANAGER_ID -> CompletableFuture.completedFuture(logsList));
+		LogsInfo logsInfo = taskManagerLogsHandler.handleRequest(handlerRequest, resourceManagerGateway).get();
+		assertEquals(logsInfo.getLogInfos().size(), resourceManagerGateway.requestTaskManagerLogList(EXPECTED_TASK_MANAGER_ID, TestingUtils.TIMEOUT()).get().size());
+	}
+
+	@Test
+	public void testGetTaskManagerLogsListForUnknownTaskExecutorException() throws Exception {
+		final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway();
 
 Review comment:
   There is a bit of redundancy here. I would initialize the handler in a setUp method:
   
   ```
   	@Before
   	public void setUp() {
   		resourceManagerGateway = new TestingResourceManagerGateway();
   		taskManagerLogsHandler = new TaskManagerLogsHandler(
   			() -> CompletableFuture.completedFuture(null),
   			TestingUtils.TIMEOUT(),
   			Collections.emptyMap(),
   			TaskManagerLogsHeaders.getInstance(),
   			() -> CompletableFuture.completedFuture(resourceManagerGateway));
   	}
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-596937306
 
 
   > to deprecate the previous handlers
   
   I don't think it's good to deprecate previous handlers.
   
   The reason is that we couldn't go to the task manager's log's page from the job's vertex's task manager directly, as we could only get the TaskManagerLocation without the task manager's log information. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r389849722
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java
 ##########
 @@ -181,7 +183,7 @@ public void disconnectResourceManager(Exception cause) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
+	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, String fileName, Time timeout) {
 
 Review comment:
   If `fileType` is `LOG` or `STDOUT`, it seems that the information in `fileName` won't be relevant. Is that right?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593952809
 
 
   @flinkbot run travis

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393744993
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerFileMessageParameters.java
 ##########
 @@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.messages.taskmanager;
+
+import org.apache.flink.runtime.rest.messages.MessagePathParameter;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Parameters for range read log REST handler.
 
 Review comment:
   {@TaskManagerCustomFileHandler}

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r390286564
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -900,7 +921,7 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
+	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, String fileName, Time timeout) {
 
 Review comment:
   Sounds good, I'll update it. Maybe one‘s name is requestFileUploadByType,other's name is requestFileUploadByName.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] Myasuka commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
Myasuka commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394273050
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogFileNamePathParameter.java
 ##########
 @@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.messages.taskmanager;
+
+import org.apache.flink.runtime.rest.messages.MessagePathParameter;
+
+/**
+ * TaskManager id path parameter used by TaskManager related handlers.
+ */
+public class LogFileNamePathParameter extends MessagePathParameter<String> {
+
+	public static final String KEY = "filename";
+
+	public LogFileNamePathParameter() {
+		super(KEY);
+	}
+
+	@Override
+	protected String convertFromString(String value) {
+		return value;
+	}
+
+	@Override
+	protected String convertToString(String value) {
+		return value;
+	}
+
+	@Override
+	public String getDescription() {
+		return "String value that specifies the file name which to read.";
 
 Review comment:
   ```suggestion
   		return "String value that identifies the file name from which to read.";
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r388031349
 
 

 ##########
 File path: docs/_includes/generated/rest_v1_dispatcher.html
 ##########
 @@ -4128,3 +4128,116 @@
     </tr>
   </tbody>
 </table>
+<table class="table table-bordered">
+    <tbody>
+    <tr>
+        <td class="text-left" colspan="2"><h5><strong>/taskmanagers/:taskmanagerid/logs</strong></h5></td>
+    </tr>
+    <tr>
+        <td class="text-left" style="width: 20%">Verb: <code>GET</code></td>
+        <td class="text-left">Response code: <code>200 OK</code></td>
+    </tr>
+    <tr>
+        <td colspan="2">Provides access to task manager logs list.</td>
+    </tr>
+    <tr>
+        <td colspan="2">Path parameters</td>
+    </tr>
+    <tr>
+        <td colspan="2">
+            <ul>
+                <li><code>taskmanagerid</code> - 32-character hexadecimal string that identifies a task manager.</li>
+            </ul>
+        </td>
+    </tr>
+    <tr>
+        <td colspan="2">
+            <button data-toggle="collapse" data-target="#1220778126">Request</button>
+            <div id="1220778127" class="collapse">
+          <pre>
+            <code>
+{}            </code>
+          </pre>
+            </div>
+        </td>
+    </tr>
+    <tr>
+        <td colspan="2">
+            <button data-toggle="collapse" data-target="#1531506487">Response</button>
+            <div id="1531506488" class="collapse">
+          <pre>
+            <code>
+{
+  "type" : "object",
+  "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:taskmanager:LogsInfo",
+  "properties" : {
+    "logs" : {
+      "type" : "array",
+      "items" : {
+        "type" : "object",
+        "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:taskmanager:LogInfo",
+        "properties" : {
+          "name" : {
+            "type" : "string"
+          },
+          "size" : {
+            "type" : "long"
+          }
+        }
+      }
+    }
+  }
+}            </code>
+          </pre>
+            </div>
+        </td>
+    </tr>
+    </tbody>
+</table>
+<table class="table table-bordered">
+    <tbody>
+    <tr>
+        <td class="text-left" colspan="2"><h5><strong>/taskmanagers/:taskmanagerid/log/:filename</strong></h5></td>
+    </tr>
 
 Review comment:
   As `/taskmanagers/:taskmanagerid/log` don't define in here, should this URL define in here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394117551
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -1647,6 +1660,30 @@ private boolean isJobManagerConnectionValid(JobID jobId, JobMasterId jobMasterId
 		return jmConnection != null && Objects.equals(jmConnection.getJobMasterId(), jobMasterId);
 	}
 
+	private CompletableFuture<TransientBlobKey> requestFileUploadByFilePath(String filePath, Time timeout, String fileTag) {
 
 Review comment:
   This parameter is from the method in TaskExecutorGateway, and it's RpcTimeout(**ps: Annotation for {@link RpcGateway} methods to specify an additional timeout parameter for the returned future to be completed. The rest of the provided parameters is passed to the remote rpc server for the rpc.**), so we need it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393752656
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
+		log.debug("Request file name is {} upload.", fileName);
 
 Review comment:
   Or  log.debug("Request file named {} upload.", fileName); ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391819491
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java
 ##########
 @@ -309,6 +317,26 @@ public void disconnectJobManager(JobID jobId, Exception cause) {
 		}
 	}
 
+	@Override
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByName(ResourceID taskManagerId, String fileName, Time timeout) {
+		final Function<Tuple2<ResourceID, String>, CompletableFuture<TransientBlobKey>> function = requestTaskManagerFileUploadByNameFunction;
+
+		if (function != null) {
+			return function.apply(Tuple2.of(taskManagerId, fileName));
+		} else {
+			return CompletableFuture.completedFuture(new TransientBlobKey());
+		}
+	}
+
+	@Override
+	public CompletableFuture<Collection<Tuple2<String, Long>>> requestTaskManagerLogList(ResourceID taskManagerId, Time timeout) {
+		List<Tuple2<String, Long>> logsList = new ArrayList<>();
 
 Review comment:
   This is not consistent with the other implementations. Moreover, we are not testing the handler's behavior if an exception is thrown by the gateway. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391812043
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogFileNamePathParameter.java
 ##########
 @@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.messages.taskmanager;
+
+import org.apache.flink.runtime.rest.messages.MessagePathParameter;
+
+/**
+ * TaskManager id path parameter used by TaskManager related handlers.
 
 Review comment:
   _"TaskManager related handlers"_ is a bit vague. Why not just link `TaskManagerCustomFileHandler`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394942388
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogFileHandler.java
 ##########
 @@ -53,7 +54,7 @@ public TaskManagerLogFileHandler(
 	}
 
 	@Override
-	protected CompletableFuture<TransientBlobKey> requestFileUpload(ResourceManagerGateway resourceManagerGateway, ResourceID taskManagerResourceId) {
-		return resourceManagerGateway.requestTaskManagerFileUpload(taskManagerResourceId, FileType.LOG, timeout);
+	protected CompletableFuture<TransientBlobKey> requestFileUpload(ResourceManagerGateway resourceManagerGateway, Tuple2<ResourceID, String> taskmanagerId2FileName) {
 
 Review comment:
   `taskmanagerIdAndFileName`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593313306
 
 
   @flinkbot run azure

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-595091021
 
 
   > I will review this in the next days.
   
   Thanks for reviewing.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-595547125
 
 
   @flinkbot run travis

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r388031349
 
 

 ##########
 File path: docs/_includes/generated/rest_v1_dispatcher.html
 ##########
 @@ -4128,3 +4128,116 @@
     </tr>
   </tbody>
 </table>
+<table class="table table-bordered">
+    <tbody>
+    <tr>
+        <td class="text-left" colspan="2"><h5><strong>/taskmanagers/:taskmanagerid/logs</strong></h5></td>
+    </tr>
+    <tr>
+        <td class="text-left" style="width: 20%">Verb: <code>GET</code></td>
+        <td class="text-left">Response code: <code>200 OK</code></td>
+    </tr>
+    <tr>
+        <td colspan="2">Provides access to task manager logs list.</td>
+    </tr>
+    <tr>
+        <td colspan="2">Path parameters</td>
+    </tr>
+    <tr>
+        <td colspan="2">
+            <ul>
+                <li><code>taskmanagerid</code> - 32-character hexadecimal string that identifies a task manager.</li>
+            </ul>
+        </td>
+    </tr>
+    <tr>
+        <td colspan="2">
+            <button data-toggle="collapse" data-target="#1220778126">Request</button>
+            <div id="1220778127" class="collapse">
+          <pre>
+            <code>
+{}            </code>
+          </pre>
+            </div>
+        </td>
+    </tr>
+    <tr>
+        <td colspan="2">
+            <button data-toggle="collapse" data-target="#1531506487">Response</button>
+            <div id="1531506488" class="collapse">
+          <pre>
+            <code>
+{
+  "type" : "object",
+  "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:taskmanager:LogsInfo",
+  "properties" : {
+    "logs" : {
+      "type" : "array",
+      "items" : {
+        "type" : "object",
+        "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:taskmanager:LogInfo",
+        "properties" : {
+          "name" : {
+            "type" : "string"
+          },
+          "size" : {
+            "type" : "long"
+          }
+        }
+      }
+    }
+  }
+}            </code>
+          </pre>
+            </div>
+        </td>
+    </tr>
+    </tbody>
+</table>
+<table class="table table-bordered">
+    <tbody>
+    <tr>
+        <td class="text-left" colspan="2"><h5><strong>/taskmanagers/:taskmanagerid/log/:filename</strong></h5></td>
+    </tr>
 
 Review comment:
   As `/taskmanagers/:taskmanagerid/log` don't define in here, should this URL define in here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593246362
 
 
   @flinkbot run travis

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391815230
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -582,16 +582,41 @@ public void notifySlotAvailable(
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) {
-		log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId);
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) {
+		log.debug("Request file which type is {}  upload from TaskExecutor {}.", fileType, taskManagerId);
 
 		final WorkerRegistration<WorkerType> taskExecutor = taskExecutors.get(taskManagerId);
 
 		if (taskExecutor == null) {
-			log.debug("Requested file {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
+			log.debug("Requested which type is {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
 
 Review comment:
   ```
   log.debug("Request upload of log file {} from TaskExecutor {}", fileType, taskManagerId);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391815230
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -582,16 +582,41 @@ public void notifySlotAvailable(
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) {
-		log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId);
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) {
+		log.debug("Request file which type is {}  upload from TaskExecutor {}.", fileType, taskManagerId);
 
 		final WorkerRegistration<WorkerType> taskExecutor = taskExecutors.get(taskManagerId);
 
 		if (taskExecutor == null) {
-			log.debug("Requested file {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
+			log.debug("Requested which type is {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
 
 Review comment:
   ```
   log.debug("Request upload of log file {} from TaskExecutor {}.", fileType, taskManagerId);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 436c1a75f3c98a2ccd7cc1c53445449d396bb916 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151891829) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947) 
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394971624
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogsHandlerTest.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException;
+import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.HandlerRequestException;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogsInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerLogsHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test for the {@link TaskManagerLogsHandler}.
+ */
+public class TaskManagerLogsHandlerTest extends TestLogger {
+
+	private static final ResourceID EXPECTED_TASK_MANAGER_ID = ResourceID.generate();
+
+	@Test
+	public void testGetTaskManagerLogsList() throws Exception {
+		final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway();
+		final TaskManagerLogsHandler taskManagerLogsHandler = new TaskManagerLogsHandler(
+			() -> CompletableFuture.completedFuture(null),
+			TestingUtils.TIMEOUT(),
+			Collections.emptyMap(),
+			TaskManagerLogsHeaders.getInstance(),
+			() -> CompletableFuture.completedFuture(resourceManagerGateway));
+		final HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> handlerRequest = createRequest(EXPECTED_TASK_MANAGER_ID);
+		List<LogInfo> logsList = new ArrayList<>();
+		logsList.add(new LogInfo("taskmanager.log", 1024L));
+		logsList.add(new LogInfo("taskmanager.out", 1024L));
+		logsList.add(new LogInfo("taskmanager-2.out", 1024L));
+		resourceManagerGateway.setRequestTaskManagerLogListFunction(EXPECTED_TASK_MANAGER_ID -> CompletableFuture.completedFuture(logsList));
+		LogsInfo logsInfo = taskManagerLogsHandler.handleRequest(handlerRequest, resourceManagerGateway).get();
+		assertEquals(logsInfo.getLogInfos().size(), resourceManagerGateway.requestTaskManagerLogList(EXPECTED_TASK_MANAGER_ID, TestingUtils.TIMEOUT()).get().size());
+	}
+
+	@Test
+	public void testGetTaskManagerLogsListForUnknownTaskExecutorException() throws Exception {
+		final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway();
+		final TaskManagerLogsHandler taskManagerLogsHandler = new TaskManagerLogsHandler(
+			() -> CompletableFuture.completedFuture(null),
+			TestingUtils.TIMEOUT(),
+			Collections.emptyMap(),
+			TaskManagerLogsHeaders.getInstance(),
+			() -> CompletableFuture.completedFuture(resourceManagerGateway));
+		final HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> handlerRequest = createRequest(EXPECTED_TASK_MANAGER_ID);
+		resourceManagerGateway.setRequestTaskManagerLogListFunction(EXPECTED_TASK_MANAGER_ID -> FutureUtils.completedExceptionally(new UnknownTaskExecutorException(EXPECTED_TASK_MANAGER_ID)));
+		try {
+			taskManagerLogsHandler.handleRequest(handlerRequest, resourceManagerGateway).get();
+		} catch (Exception exception){
+			assertEquals("org.apache.flink.runtime.rest.handler.RestHandlerException: Could not find TaskExecutor " + EXPECTED_TASK_MANAGER_ID + ".", exception.getMessage());
+		}
+	}
+
+	private static HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> createRequest(ResourceID taskManagerId) throws HandlerRequestException {
+		final Map<String, String> pathParameters = new HashMap<>();
+		pathParameters.put(TaskManagerIdPathParameter.KEY, taskManagerId.toString());
+		final Map<String, List<String>> queryParameters = new HashMap<>();
 
 Review comment:
   Does this have to be mutable? If not:
   ```
   Collections.emptyMap()
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r395459799
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java
 ##########
 @@ -263,4 +266,8 @@ private void transferFile(ChannelHandlerContext ctx, File file, HttpRequest http
 			throw new FlinkException("Could not transfer file " + file + " to the client.", ioe);
 		}
 	}
+
+	protected String getFileName(HandlerRequest<EmptyRequestBody, M> handlerRequest) {
+		return null;
 
 Review comment:
   Yes you are right. The cache is not shared.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 70e8ca9774fc4247657f5d6aecc43459229ba9bb Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151307447) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593954121
 
 
   @flinkbot run azure

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * c07d058a0f55764b319d64034b4f1bfdc26d99e6 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151502448) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394367376
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
+		log.debug("Request file name is {} upload.", fileName);
 
-				return CompletableFuture.completedFuture(transientBlobKey);
-			} else {
-				log.debug("The file {} does not exist on the TaskExecutor {}.", fileType, getResourceID());
-				return FutureUtils.completedExceptionally(new FlinkException("The file " + fileType + " does not exist on the TaskExecutor."));
-			}
+		final String filePath;
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null && !logDir.isEmpty() && fileName != null && !fileName.isEmpty()) {
+			filePath = logDir + "/" + fileName;
 
 Review comment:
   maybe below code is better:
   ```java
   filePath = Paths.get(logDir, fileName).toFile().getAbsolutePath();
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r390286564
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -900,7 +921,7 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
+	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, String fileName, Time timeout) {
 
 Review comment:
   Sounds good,  but also update methods' name may be better. WDYT?
   ```java
   CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, @RpcTimeout Time timeout);
   
   CompletableFuture<TransientBlobKey> requestFileUploadByType(FileType fileType, @RpcTimeout Time timeout);
   ```
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 54ac8f26c67344a0ed000c882afd66e68c8f6bd0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151328904) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812) 
   * c07d058a0f55764b319d64034b4f1bfdc26d99e6 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151502448) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/152623819) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r390183535
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -900,7 +921,7 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
+	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, String fileName, Time timeout) {
 
 Review comment:
   I think this method signature is hard to understand. Let me summarize:
   - If `fileType` is `LOG` or `STDOUT`,  `fileName` will be `taskmanager.log` or `taskmanager.out`, respectively. However, `fileName` can be ignored because the file to be uploaded will be determined by `taskmanagerConfiguration`).
   - If `fileType` is `CUSTOM`, the value of `fileName` should be used.
   
   That is, the task fulfilled by of this method changes depending on the method arguments. Moreover, just by looking at the signature it is unclear what this method does. This is normally a hint to split the methods:
   
   ```
   CompletableFuture<TransientBlobKey> requestFileUpload(String fileName, @RpcTimeout Time timeout);
   
   CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, @RpcTimeout Time timeout);
   ```
   I think the same applies for `ResourceManagerGateway`. WDYT?
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393751222
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -309,6 +311,25 @@ public TaskExecutor(
 		return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty());
 	}
 
+	@Override
+	public CompletableFuture<Collection<Tuple2<String, Long>>> requestLogList(Time timeout) {
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null) {
+			final File[] logFiles = new File(logDir).listFiles();
+
+			if (logFiles == null) {
+				return FutureUtils.completedExceptionally(
+					new FlinkException("The specific log directory is not a valid directory."));
+			}
+
+			final List<Tuple2<String, Long>> logsWithLength = Arrays.stream(logFiles)
+				.map(logFile -> Tuple2.of(logFile.getName(), logFile.length()))
+				.collect(Collectors.toList());
+			return CompletableFuture.completedFuture(logsWithLength);
+		}
+		return FutureUtils.completedExceptionally(new FlinkException("There is no log file available on the TaskExecutor."));
 
 Review comment:
   It needs to confirm. It depends on what will the frontend handle the exception.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r395418363
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogsHandlerTest.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException;
+import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.HandlerRequestException;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogsInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerLogsHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test for the {@link TaskManagerLogsHandler}.
+ */
+public class TaskManagerLogsHandlerTest extends TestLogger {
+
+	private static final ResourceID EXPECTED_TASK_MANAGER_ID = ResourceID.generate();
+
+	@Test
+	public void testGetTaskManagerLogsList() throws Exception {
+		final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway();
+		final TaskManagerLogsHandler taskManagerLogsHandler = new TaskManagerLogsHandler(
+			() -> CompletableFuture.completedFuture(null),
+			TestingUtils.TIMEOUT(),
+			Collections.emptyMap(),
+			TaskManagerLogsHeaders.getInstance(),
+			() -> CompletableFuture.completedFuture(resourceManagerGateway));
+		final HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> handlerRequest = createRequest(EXPECTED_TASK_MANAGER_ID);
+		List<LogInfo> logsList = new ArrayList<>();
+		logsList.add(new LogInfo("taskmanager.log", 1024L));
+		logsList.add(new LogInfo("taskmanager.out", 1024L));
+		logsList.add(new LogInfo("taskmanager-2.out", 1024L));
+		resourceManagerGateway.setRequestTaskManagerLogListFunction(EXPECTED_TASK_MANAGER_ID -> CompletableFuture.completedFuture(logsList));
+		LogsInfo logsInfo = taskManagerLogsHandler.handleRequest(handlerRequest, resourceManagerGateway).get();
+		assertEquals(logsInfo.getLogInfos().size(), resourceManagerGateway.requestTaskManagerLogList(EXPECTED_TASK_MANAGER_ID, TestingUtils.TIMEOUT()).get().size());
+	}
+
+	@Test
+	public void testGetTaskManagerLogsListForUnknownTaskExecutorException() throws Exception {
+		final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway();
+		final TaskManagerLogsHandler taskManagerLogsHandler = new TaskManagerLogsHandler(
+			() -> CompletableFuture.completedFuture(null),
+			TestingUtils.TIMEOUT(),
+			Collections.emptyMap(),
+			TaskManagerLogsHeaders.getInstance(),
+			() -> CompletableFuture.completedFuture(resourceManagerGateway));
+		final HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> handlerRequest = createRequest(EXPECTED_TASK_MANAGER_ID);
+		resourceManagerGateway.setRequestTaskManagerLogListFunction(EXPECTED_TASK_MANAGER_ID -> FutureUtils.completedExceptionally(new UnknownTaskExecutorException(EXPECTED_TASK_MANAGER_ID)));
+		try {
+			taskManagerLogsHandler.handleRequest(handlerRequest, resourceManagerGateway).get();
+		} catch (Exception exception){
+			assertEquals("org.apache.flink.runtime.rest.handler.RestHandlerException: Could not find TaskExecutor " + EXPECTED_TASK_MANAGER_ID + ".", exception.getMessage());
+		}
+	}
+
+	private static HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> createRequest(ResourceID taskManagerId) throws HandlerRequestException {
+		final Map<String, String> pathParameters = new HashMap<>();
+		pathParameters.put(TaskManagerIdPathParameter.KEY, taskManagerId.toString());
+		final Map<String, List<String>> queryParameters = new HashMap<>();
 
 Review comment:
   It needn't be mutable, so `Collections.emptyMap()` is ok.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393754916
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
+		log.debug("Request file name is {} upload.", fileName);
 
-				return CompletableFuture.completedFuture(transientBlobKey);
-			} else {
-				log.debug("The file {} does not exist on the TaskExecutor {}.", fileType, getResourceID());
-				return FutureUtils.completedExceptionally(new FlinkException("The file " + fileType + " does not exist on the TaskExecutor."));
-			}
+		final String filePath;
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null && !logDir.isEmpty() && fileName != null && !fileName.isEmpty()) {
 
 Review comment:
   !StringUtils.isNullOrWhitespaceOnly(logDir)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * b8d51c94a0b93fdbfa4b167e0b4c630f791fba10 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150973416) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r389850874
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -900,7 +921,7 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
+	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, String fileName, Time timeout) {
 
 Review comment:
   If `fileType` is `LOG` or `STDOUT`, it seems that the information in `fileName` won't be relevant. Is that right?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * c07d058a0f55764b319d64034b4f1bfdc26d99e6 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151502448) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854) 
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 436c1a75f3c98a2ccd7cc1c53445449d396bb916 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151891829) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947) 
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394977194
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerCustomFileHandler.java
 ##########
 @@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.blob.TransientBlobKey;
+import org.apache.flink.runtime.blob.TransientBlobService;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogFileNamePathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerFileMessageParameters;
+import org.apache.flink.runtime.taskexecutor.TaskExecutor;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+
+import javax.annotation.Nonnull;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Rest handler which serves the custom file of the {@link TaskExecutor}.
+ */
+public class TaskManagerCustomFileHandler extends AbstractTaskManagerFileHandler<TaskManagerFileMessageParameters> {
+
+	public TaskManagerCustomFileHandler(
 
 Review comment:
   I would rename this to `TaskManagerCustomLogFileHandler` so that we are consistent with the other handlers.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 436c1a75f3c98a2ccd7cc1c53445449d396bb916 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151891829) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947) 
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 912485a96d61febfa41b7c84631aeae19d819325 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/150964833) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718) 
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391855203
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogsHandler.java
 ##########
 @@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogsInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+/**
+ * Handler which serves detailed TaskManager log list information.
+ */
+public class TaskManagerLogsHandler extends AbstractTaskManagerHandler<RestfulGateway, EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> {
+
+	private final GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever;
+
+	public TaskManagerLogsHandler(
+			GatewayRetriever<? extends RestfulGateway> leaderRetriever,
+			Time timeout,
+			Map<String, String> responseHeaders,
+			MessageHeaders<EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> messageHeaders,
+			GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever) {
+		super(leaderRetriever, timeout, responseHeaders, messageHeaders, resourceManagerGatewayRetriever);
+
+		this.resourceManagerGatewayRetriever = Preconditions.checkNotNull(resourceManagerGatewayRetriever);
+	}
+
+	@Override
+	protected CompletableFuture<LogsInfo> handleRequest(
+			@Nonnull HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> request,
+			@Nonnull ResourceManagerGateway gateway) throws RestHandlerException {
+		final ResourceID taskManagerId = request.getPathParameter(TaskManagerIdPathParameter.class);
+		final ResourceManagerGateway resourceManagerGateway = getResourceManagerGateway(resourceManagerGatewayRetriever);
+		final CompletableFuture<Collection<Tuple2<String, Long>>> logsWithLengthFuture = resourceManagerGateway.requestTaskManagerLogList(taskManagerId, timeout);
+
+		return logsWithLengthFuture.thenApply(logName2Sizes -> {
+			if (null != logName2Sizes) {
+				Collection<LogInfo> logs = logName2Sizes.stream().map(logName2Size -> new LogInfo(logName2Size.f0, logName2Size.f1)).collect(Collectors.toSet());
+				return new LogsInfo(logs);
+			} else {
+				return LogsInfo.empty();
+			}
+		}).exceptionally(
+				(Throwable throwable) -> {
+					final Throwable strippedThrowable = ExceptionUtils.stripExecutionException(throwable);
+
+					if (strippedThrowable instanceof UnknownTaskExecutorException) {
+						throw new CompletionException(
+							new RestHandlerException(
+								"Could not find TaskExecutor " + taskManagerId + '.',
+								HttpResponseStatus.NOT_FOUND,
+								strippedThrowable));
+					} else {
+						throw new CompletionException(strippedThrowable);
 
 Review comment:
   It doesn't seem right that we return internal server error if the file does not exist.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * c07d058a0f55764b319d64034b4f1bfdc26d99e6 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151502448) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393724492
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java
 ##########
 @@ -263,4 +264,10 @@ private void transferFile(ChannelHandlerContext ctx, File file, HttpRequest http
 			throw new FlinkException("Could not transfer file " + file + " to the client.", ioe);
 		}
 	}
+
+	protected String getFileName(HandlerRequest<EmptyRequestBody, M> handlerRequest) {
+		return "undefined";
 
 Review comment:
   What is the fileName used for?  default value?
   It should be considered here whether a special approach is necessary.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] Myasuka commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
Myasuka commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394263540
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -1647,6 +1660,30 @@ private boolean isJobManagerConnectionValid(JobID jobId, JobMasterId jobMasterId
 		return jmConnection != null && Objects.equals(jmConnection.getJobMasterId(), jobMasterId);
 	}
 
+	private CompletableFuture<TransientBlobKey> requestFileUploadByFilePath(String filePath, Time timeout, String fileTag) {
+		if (filePath != null && !filePath.isEmpty()) {
+			final File file = new File(filePath);
+			if (file.exists()) {
+				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
+				final TransientBlobKey transientBlobKey;
+				try (FileInputStream fileInputStream = new FileInputStream(file)) {
+					transientBlobKey = transientBlobService.putTransient(fileInputStream);
+				} catch (IOException e) {
+					log.debug("Could not upload file {}.", fileTag, e);
+					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileTag + '.', e));
+				}
+
+				return CompletableFuture.completedFuture(transientBlobKey);
+			} else {
+				log.debug("The file {} does not exist on the TaskExecutor {}.", fileTag, getResourceID());
+				return FutureUtils.completedExceptionally(new FlinkException("The file " + fileTag + " does not exist on the TaskExecutor."));
+			}
+		} else {
+			log.debug("The file {} unavailable on the TaskExecutor {}.", fileTag, getResourceID());
 
 Review comment:
   "The file {} is unavailable on the TaskExecutor"

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] Myasuka commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
Myasuka commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394261661
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -1647,6 +1660,30 @@ private boolean isJobManagerConnectionValid(JobID jobId, JobMasterId jobMasterId
 		return jmConnection != null && Objects.equals(jmConnection.getJobMasterId(), jobMasterId);
 	}
 
+	private CompletableFuture<TransientBlobKey> requestFileUploadByFilePath(String filePath, Time timeout, String fileTag) {
+		if (filePath != null && !filePath.isEmpty()) {
 
 Review comment:
   Can be simplified as `!StringUtils.isNullOrWhitespaceOnly(filePath)`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593952478
 
 
   @flinkbot run travis

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394221144
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
+		log.debug("Request file name is {} upload.", fileName);
 
 Review comment:
   ok

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r390823198
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -900,7 +921,7 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
+	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, String fileName, Time timeout) {
 
 Review comment:
   Ok, the new method names make sense.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r395621108
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -309,6 +312,25 @@ public TaskExecutor(
 		return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty());
 	}
 
+	@Override
+	public CompletableFuture<Collection<LogInfo>> requestLogList(Time timeout) {
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null) {
+			final File[] logFiles = new File(logDir).listFiles();
 
 Review comment:
   Unfortunately this is a blocking IO operation in the main thread which is discouraged. But the code was already broken before as we are uploading the files in the main thread. I'll have to think about whether this needs to be addressed immediately.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/152951654",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * 3984f26b34353ad5b2006c8768536fbaed5b512c Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/152951654) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/152951654",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6364",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153925757",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * 0d0a96f5919000033e3b43d4ce009793d1ed0544 Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/153925757) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6364) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393712257
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java
 ##########
 @@ -118,9 +119,11 @@ protected AbstractTaskManagerFileHandler(
 	protected CompletableFuture<Void> respondToRequest(ChannelHandlerContext ctx, HttpRequest httpRequest, HandlerRequest<EmptyRequestBody, M> handlerRequest, RestfulGateway gateway) throws RestHandlerException {
 		final ResourceID taskManagerId = handlerRequest.getPathParameter(TaskManagerIdPathParameter.class);
 
+		String filename = getFileName(handlerRequest);
+		final Tuple2<ResourceID, String> taskManagerId2FileName = new Tuple2<>(taskManagerId, filename);
 
 Review comment:
   I personally prefer to use `taskManagerIdAndFileName` instead of `taskManagerId2FileName`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391851087
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -309,6 +311,25 @@ public TaskExecutor(
 		return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty());
 	}
 
+	@Override
+	public CompletableFuture<Collection<Tuple2<String, Long>>> requestLogList(Time timeout) {
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null) {
+			final File[] logFiles = new File(logDir).listFiles();
 
 Review comment:
   This will also list directories which cannot be uploaded/displayed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394976832
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogsHandler.java
 ##########
 @@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogsInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+
+/**
+ * Handler which serves detailed TaskManager log list information.
+ */
+public class TaskManagerLogsHandler extends AbstractTaskManagerHandler<RestfulGateway, EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> {
 
 Review comment:
   I would rename this to `TaskManagerLogListHandler`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391803582
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
+		log.debug("Request file name is {} upload.", fileName);
 
-				return CompletableFuture.completedFuture(transientBlobKey);
-			} else {
-				log.debug("The file {} does not exist on the TaskExecutor {}.", fileType, getResourceID());
-				return FutureUtils.completedExceptionally(new FlinkException("The file " + fileType + " does not exist on the TaskExecutor."));
-			}
+		final String filePath;
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null && !logDir.isEmpty() && fileName != null && !fileName.isEmpty()) {
+			filePath = logDir + "/" + fileName;
 
 Review comment:
   This won't work on windows. It's better to use `new File(logDir, fileName).getPath()`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/152951654",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6364",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153925757",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * 0d0a96f5919000033e3b43d4ce009793d1ed0544 Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/153925757) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6364) 
   * 1dc6d0c40bedc119afa4b1101b86a2f5238eff31 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/152951654",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6364",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153925757",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6433",
       "triggerID" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154208064",
       "triggerID" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * 1dc6d0c40bedc119afa4b1101b86a2f5238eff31 Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/154208064) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6433) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/152951654",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/152623819) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130) 
   * 3984f26b34353ad5b2006c8768536fbaed5b512c Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/152951654) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r390113102
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -900,7 +921,7 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
+	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, String fileName, Time timeout) {
 
 Review comment:
   yes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394129123
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogsHandler.java
 ##########
 @@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogsInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+/**
+ * Handler which serves detailed TaskManager log list information.
+ */
+public class TaskManagerLogsHandler extends AbstractTaskManagerHandler<RestfulGateway, EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> {
+
+	private final GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever;
+
+	public TaskManagerLogsHandler(
+			GatewayRetriever<? extends RestfulGateway> leaderRetriever,
+			Time timeout,
+			Map<String, String> responseHeaders,
+			MessageHeaders<EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> messageHeaders,
+			GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever) {
+		super(leaderRetriever, timeout, responseHeaders, messageHeaders, resourceManagerGatewayRetriever);
+
+		this.resourceManagerGatewayRetriever = Preconditions.checkNotNull(resourceManagerGatewayRetriever);
+	}
+
+	@Override
+	protected CompletableFuture<LogsInfo> handleRequest(
+			@Nonnull HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> request,
+			@Nonnull ResourceManagerGateway gateway) throws RestHandlerException {
+		final ResourceID taskManagerId = request.getPathParameter(TaskManagerIdPathParameter.class);
+		final ResourceManagerGateway resourceManagerGateway = getResourceManagerGateway(resourceManagerGatewayRetriever);
+		final CompletableFuture<Collection<Tuple2<String, Long>>> logsWithLengthFuture = resourceManagerGateway.requestTaskManagerLogList(taskManagerId, timeout);
+
+		return logsWithLengthFuture.thenApply(logName2Sizes -> {
+			if (null != logName2Sizes) {
+				Collection<LogInfo> logs = logName2Sizes.stream().map(logName2Size -> new LogInfo(logName2Size.f0, logName2Size.f1)).collect(Collectors.toSet());
+				return new LogsInfo(logs);
+			} else {
+				return LogsInfo.empty();
+			}
+		}).exceptionally(
+				(Throwable throwable) -> {
+					final Throwable strippedThrowable = ExceptionUtils.stripExecutionException(throwable);
+
+					if (strippedThrowable instanceof UnknownTaskExecutorException) {
+						throw new CompletionException(
+							new RestHandlerException(
+								"Could not find TaskExecutor " + taskManagerId + '.',
+								HttpResponseStatus.NOT_FOUND,
+								strippedThrowable));
+					} else {
+						throw new CompletionException(strippedThrowable);
 
 Review comment:
   What about 204 which means not content
   ```java
   new RestHandlerException("Could not find any file.", HttpResponseStatus.NO_CONTENT,strippedThrowable));
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393692076
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -582,16 +582,41 @@ public void notifySlotAvailable(
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) {
-		log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId);
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) {
+		log.debug("Request file which type is {}  upload from TaskExecutor {}.", fileType, taskManagerId);
 
 Review comment:
   ```suggestion
   		log.debug("Request file of type {} upload from TaskExecutor {}.", fileType, taskManagerId);
   ```
   Note: there are more than one space between `}` and `upload` in your log.
   
   I'm not a native speaker, so this is just a suggestion IMO.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * c07d058a0f55764b319d64034b4f1bfdc26d99e6 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151502448) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393726637
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java
 ##########
 @@ -263,4 +264,10 @@ private void transferFile(ChannelHandlerContext ctx, File file, HttpRequest http
 			throw new FlinkException("Could not transfer file " + file + " to the client.", ioe);
 		}
 	}
+
+	protected String getFileName(HandlerRequest<EmptyRequestBody, M> handlerRequest) {
+		return "undefined";
+	}
+
+	protected abstract CompletableFuture<TransientBlobKey> requestFileUpload(ResourceManagerGateway resourceManagerGateway, Tuple2<ResourceID, String> taskManagerId2FileName);
 
 Review comment:
   It is not necessary to move the position of this method. It used to be near `loadTaskManagerFile` which calls it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394133133
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -1647,6 +1660,30 @@ private boolean isJobManagerConnectionValid(JobID jobId, JobMasterId jobMasterId
 		return jmConnection != null && Objects.equals(jmConnection.getJobMasterId(), jobMasterId);
 	}
 
+	private CompletableFuture<TransientBlobKey> requestFileUploadByFilePath(String filePath, Time timeout, String fileTag) {
 
 Review comment:
   I don't think we need it. This is a `private` method. The timeout argument extraction happens before this method is invoked: https://github.com/apache/flink/blob/ddec07323701a9998e854c378468617bafffa1f5/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java#L306-L324

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391812518
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java
 ##########
 @@ -263,4 +264,10 @@ private void transferFile(ChannelHandlerContext ctx, File file, HttpRequest http
 			throw new FlinkException("Could not transfer file " + file + " to the client.", ioe);
 		}
 	}
+
+	protected String getFileName(HandlerRequest<EmptyRequestBody, M> handlerRequest) {
+		return "undefined";
 
 Review comment:
   Why not just return `null`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393732815
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerCustomFileHandler.java
 ##########
 @@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.blob.TransientBlobKey;
+import org.apache.flink.runtime.blob.TransientBlobService;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogFileNamePathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerFileMessageParameters;
+import org.apache.flink.runtime.taskexecutor.TaskExecutor;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+
+import javax.annotation.Nonnull;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+
 
 Review comment:
   One more line

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394065435
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java
 ##########
 @@ -263,4 +264,10 @@ private void transferFile(ChannelHandlerContext ctx, File file, HttpRequest http
 			throw new FlinkException("Could not transfer file " + file + " to the client.", ioe);
 		}
 	}
+
+	protected String getFileName(HandlerRequest<EmptyRequestBody, M> handlerRequest) {
+		return "undefined";
 
 Review comment:
   Here use the task manager id and file name as the key to LoadingCache.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 912485a96d61febfa41b7c84631aeae19d819325 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/150964833) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718) 
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * b8d51c94a0b93fdbfa4b167e0b4c630f791fba10 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593313306
 
 
   @flinkbot run azure

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593828424
 
 
   @flinkbot run azure

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r390157774
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
 ##########
 @@ -211,9 +211,18 @@ void notifySlotAvailable(
 	 *
 	 * @param taskManagerId identifying the {@link TaskExecutor} to upload the specified file
 	 * @param fileType type of the file to upload
+	 * @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}.
 	 */
-	CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, @RpcTimeout Time timeout);
+	CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, String fileName, @RpcTimeout Time timeout);
 
 Review comment:
   Imo a method signature like this is hard to understand and confusing. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r389849722
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java
 ##########
 @@ -181,7 +183,7 @@ public void disconnectResourceManager(Exception cause) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
+	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, String fileName, Time timeout) {
 
 Review comment:
   If `fileType` is `LOG` or `STDOUT`, it seems that the information in `fileName` won't be relevant. Is that right?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394221309
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
+		log.debug("Request file name is {} upload.", fileName);
 
 Review comment:
   > We could add it in `requestFileUploadByFilePath().`
   > 
   > ```
   > log.debug("Received file upload request for file {}", fileTag);
   > ```
   
   ok

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r390286564
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -900,7 +921,7 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
+	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, String fileName, Time timeout) {
 
 Review comment:
   Sounds good, but we also update name may be better.
   ```java
   CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, @RpcTimeout Time timeout);
   
   CompletableFuture<TransientBlobKey> requestFileUploadByType(FileType fileType, @RpcTimeout Time timeout);
   ```
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391807323
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -309,6 +311,25 @@ public TaskExecutor(
 		return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty());
 	}
 
+	@Override
+	public CompletableFuture<Collection<Tuple2<String, Long>>> requestLogList(Time timeout) {
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null) {
+			final File[] logFiles = new File(logDir).listFiles();
+
+			if (logFiles == null) {
+				return FutureUtils.completedExceptionally(
+					new FlinkException("The specific log directory is not a valid directory."));
 
 Review comment:
   Maybe include the `logDir` in the error message.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393690451
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -582,16 +582,41 @@ public void notifySlotAvailable(
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) {
-		log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId);
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) {
+		log.debug("Request file which type is {}  upload from TaskExecutor {}.", fileType, taskManagerId);
 
 		final WorkerRegistration<WorkerType> taskExecutor = taskExecutors.get(taskManagerId);
 
 		if (taskExecutor == null) {
-			log.debug("Requested file {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
+			log.debug("Requested which type is {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
 			return FutureUtils.completedExceptionally(new UnknownTaskExecutorException(taskManagerId));
 		} else {
-			return taskExecutor.getTaskExecutorGateway().requestFileUpload(fileType, timeout);
+			return taskExecutor.getTaskExecutorGateway().requestFileUploadByType(fileType, timeout);
+		}
+	}
+
+	@Override
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByName(ResourceID taskManagerId, String fileName, Time timeout) {
+		log.debug("Request file which name is {}  upload from TaskExecutor {}.", fileName, taskManagerId);
 
 Review comment:
   ```suggestion
   		log.debug("Request file named {} upload from TaskExecutor {}.", fileName, taskManagerId);
   ```
   
   Note: there are more than one space between `}` and `upload` in your log.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393742705
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerCustomFileHeaders.java
 ##########
 @@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.messages.taskmanager;
+
+import org.apache.flink.runtime.rest.HttpMethodWrapper;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders;
+
+/**
+ * Headers for the {@link TaskManagerCustomFileHeaders}.
 
 Review comment:
   ```suggestion
    * Headers for the {@link TaskManagerCustomFileHandler}.
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/152951654",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6364",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153925757",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * 3984f26b34353ad5b2006c8768536fbaed5b512c Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/152951654) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226) 
   * 0d0a96f5919000033e3b43d4ce009793d1ed0544 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/153925757) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6364) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391844236
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogsHandler.java
 ##########
 @@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogsInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+/**
+ * Handler which serves detailed TaskManager log list information.
+ */
+public class TaskManagerLogsHandler extends AbstractTaskManagerHandler<RestfulGateway, EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> {
+
+	private final GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever;
+
+	public TaskManagerLogsHandler(
+			GatewayRetriever<? extends RestfulGateway> leaderRetriever,
+			Time timeout,
+			Map<String, String> responseHeaders,
+			MessageHeaders<EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> messageHeaders,
+			GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever) {
+		super(leaderRetriever, timeout, responseHeaders, messageHeaders, resourceManagerGatewayRetriever);
+
+		this.resourceManagerGatewayRetriever = Preconditions.checkNotNull(resourceManagerGatewayRetriever);
+	}
+
+	@Override
+	protected CompletableFuture<LogsInfo> handleRequest(
+			@Nonnull HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> request,
+			@Nonnull ResourceManagerGateway gateway) throws RestHandlerException {
+		final ResourceID taskManagerId = request.getPathParameter(TaskManagerIdPathParameter.class);
+		final ResourceManagerGateway resourceManagerGateway = getResourceManagerGateway(resourceManagerGatewayRetriever);
+		final CompletableFuture<Collection<Tuple2<String, Long>>> logsWithLengthFuture = resourceManagerGateway.requestTaskManagerLogList(taskManagerId, timeout);
+
+		return logsWithLengthFuture.thenApply(logName2Sizes -> {
+			if (null != logName2Sizes) {
+				Collection<LogInfo> logs = logName2Sizes.stream().map(logName2Size -> new LogInfo(logName2Size.f0, logName2Size.f1)).collect(Collectors.toSet());
+				return new LogsInfo(logs);
+			} else {
+				return LogsInfo.empty();
 
 Review comment:
   I don't think this is a valid code path. When is `logName2Sizes` null?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394969273
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogsHandlerTest.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException;
+import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.HandlerRequestException;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogsInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerLogsHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test for the {@link TaskManagerLogsHandler}.
+ */
+public class TaskManagerLogsHandlerTest extends TestLogger {
+
+	private static final ResourceID EXPECTED_TASK_MANAGER_ID = ResourceID.generate();
+
+	@Test
+	public void testGetTaskManagerLogsList() throws Exception {
+		final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway();
+		final TaskManagerLogsHandler taskManagerLogsHandler = new TaskManagerLogsHandler(
+			() -> CompletableFuture.completedFuture(null),
+			TestingUtils.TIMEOUT(),
+			Collections.emptyMap(),
+			TaskManagerLogsHeaders.getInstance(),
+			() -> CompletableFuture.completedFuture(resourceManagerGateway));
+		final HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> handlerRequest = createRequest(EXPECTED_TASK_MANAGER_ID);
+		List<LogInfo> logsList = new ArrayList<>();
+		logsList.add(new LogInfo("taskmanager.log", 1024L));
+		logsList.add(new LogInfo("taskmanager.out", 1024L));
+		logsList.add(new LogInfo("taskmanager-2.out", 1024L));
+		resourceManagerGateway.setRequestTaskManagerLogListFunction(EXPECTED_TASK_MANAGER_ID -> CompletableFuture.completedFuture(logsList));
+		LogsInfo logsInfo = taskManagerLogsHandler.handleRequest(handlerRequest, resourceManagerGateway).get();
+		assertEquals(logsInfo.getLogInfos().size(), resourceManagerGateway.requestTaskManagerLogList(EXPECTED_TASK_MANAGER_ID, TestingUtils.TIMEOUT()).get().size());
+	}
+
+	@Test
+	public void testGetTaskManagerLogsListForUnknownTaskExecutorException() throws Exception {
+		final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway();
+		final TaskManagerLogsHandler taskManagerLogsHandler = new TaskManagerLogsHandler(
+			() -> CompletableFuture.completedFuture(null),
+			TestingUtils.TIMEOUT(),
+			Collections.emptyMap(),
+			TaskManagerLogsHeaders.getInstance(),
+			() -> CompletableFuture.completedFuture(resourceManagerGateway));
+		final HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> handlerRequest = createRequest(EXPECTED_TASK_MANAGER_ID);
+		resourceManagerGateway.setRequestTaskManagerLogListFunction(EXPECTED_TASK_MANAGER_ID -> FutureUtils.completedExceptionally(new UnknownTaskExecutorException(EXPECTED_TASK_MANAGER_ID)));
+		try {
+			taskManagerLogsHandler.handleRequest(handlerRequest, resourceManagerGateway).get();
+		} catch (Exception exception){
+			assertEquals("org.apache.flink.runtime.rest.handler.RestHandlerException: Could not find TaskExecutor " + EXPECTED_TASK_MANAGER_ID + ".", exception.getMessage());
 
 Review comment:
   I think what matters here is that the response code is 404.
   ```
   import static org.hamcrest.Matchers.containsString;
   import static org.hamcrest.Matchers.equalTo;
   import static org.hamcrest.Matchers.instanceOf;
   import static org.hamcrest.Matchers.is;
   import static org.junit.Assert.assertEquals;
   import static org.junit.Assert.assertThat;
   [...]
   
   		} catch (ExecutionException e) {
   			final Throwable cause = e.getCause();
   			assertThat(cause, is(instanceOf(RestHandlerException.class)));
   			final RestHandlerException restHandlerException = (RestHandlerException) cause;
   			assertThat(restHandlerException.getHttpResponseStatus(), is(equalTo(HttpResponseStatus.NOT_FOUND)));
   			assertThat(restHandlerException.getMessage(), containsString("Could not find TaskExecutor " + EXPECTED_TASK_MANAGER_ID));
   		}
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 54ac8f26c67344a0ed000c882afd66e68c8f6bd0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151328904) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 436c1a75f3c98a2ccd7cc1c53445449d396bb916 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151891829) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947) 
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r390286564
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -900,7 +921,7 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
+	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, String fileName, Time timeout) {
 
 Review comment:
   Sounds good, I'll update it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394119984
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
+		log.debug("Request file name is {} upload.", fileName);
 
 Review comment:
   We could add it in `requestFileUploadByFilePath().`
   ```code
   log.debug("Received file upload request for file {}", fileTag);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394976527
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java
 ##########
 @@ -263,4 +266,8 @@ private void transferFile(ChannelHandlerContext ctx, File file, HttpRequest http
 			throw new FlinkException("Could not transfer file " + file + " to the client.", ioe);
 		}
 	}
+
+	protected String getFileName(HandlerRequest<EmptyRequestBody, M> handlerRequest) {
+		return null;
 
 Review comment:
   One way to resolve this could be to use `Tuple3<ResourceID, String, FileType>` as a key and let the handler implementation chose its key.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/152951654",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6364",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153925757",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * 0d0a96f5919000033e3b43d4ce009793d1ed0544 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/153925757) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6364) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393736205
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogsHandler.java
 ##########
 @@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogsInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+/**
+ * Handler which serves detailed TaskManager log list information.
+ */
+public class TaskManagerLogsHandler extends AbstractTaskManagerHandler<RestfulGateway, EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> {
+
+	private final GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever;
+
+	public TaskManagerLogsHandler(
+			GatewayRetriever<? extends RestfulGateway> leaderRetriever,
+			Time timeout,
+			Map<String, String> responseHeaders,
+			MessageHeaders<EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> messageHeaders,
+			GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever) {
+		super(leaderRetriever, timeout, responseHeaders, messageHeaders, resourceManagerGatewayRetriever);
+
+		this.resourceManagerGatewayRetriever = Preconditions.checkNotNull(resourceManagerGatewayRetriever);
+	}
+
+	@Override
+	protected CompletableFuture<LogsInfo> handleRequest(
+			@Nonnull HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> request,
+			@Nonnull ResourceManagerGateway gateway) throws RestHandlerException {
+		final ResourceID taskManagerId = request.getPathParameter(TaskManagerIdPathParameter.class);
+		final ResourceManagerGateway resourceManagerGateway = getResourceManagerGateway(resourceManagerGatewayRetriever);
+		final CompletableFuture<Collection<Tuple2<String, Long>>> logsWithLengthFuture = resourceManagerGateway.requestTaskManagerLogList(taskManagerId, timeout);
+
+		return logsWithLengthFuture.thenApply(logName2Sizes -> {
 
 Review comment:
   `logName2Sizes` -> `logNameAndSize` ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391800050
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -1647,6 +1660,30 @@ private boolean isJobManagerConnectionValid(JobID jobId, JobMasterId jobMasterId
 		return jmConnection != null && Objects.equals(jmConnection.getJobMasterId(), jobMasterId);
 	}
 
+	private CompletableFuture<TransientBlobKey> requestFileUploadByFilePath(String filePath, Time timeout, String fileTag) {
 
 Review comment:
   `timeout` parameter is unused.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391872861
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
 
 Review comment:
   This method could be vulnerable against a path traversal attack: https://wiki.owasp.org/index.php/Path_Traversal
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 70e8ca9774fc4247657f5d6aecc43459229ba9bb Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151307447) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801) 
   * 54ac8f26c67344a0ed000c882afd66e68c8f6bd0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 912485a96d61febfa41b7c84631aeae19d819325 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593246398
 
 
   @flinkbot run azure

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 436c1a75f3c98a2ccd7cc1c53445449d396bb916 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151891829) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947) 
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394133992
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogsHandler.java
 ##########
 @@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogsInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+/**
+ * Handler which serves detailed TaskManager log list information.
+ */
+public class TaskManagerLogsHandler extends AbstractTaskManagerHandler<RestfulGateway, EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> {
+
+	private final GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever;
+
+	public TaskManagerLogsHandler(
+			GatewayRetriever<? extends RestfulGateway> leaderRetriever,
+			Time timeout,
+			Map<String, String> responseHeaders,
+			MessageHeaders<EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> messageHeaders,
+			GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever) {
+		super(leaderRetriever, timeout, responseHeaders, messageHeaders, resourceManagerGatewayRetriever);
+
+		this.resourceManagerGatewayRetriever = Preconditions.checkNotNull(resourceManagerGatewayRetriever);
+	}
+
+	@Override
+	protected CompletableFuture<LogsInfo> handleRequest(
+			@Nonnull HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> request,
+			@Nonnull ResourceManagerGateway gateway) throws RestHandlerException {
+		final ResourceID taskManagerId = request.getPathParameter(TaskManagerIdPathParameter.class);
+		final ResourceManagerGateway resourceManagerGateway = getResourceManagerGateway(resourceManagerGatewayRetriever);
+		final CompletableFuture<Collection<Tuple2<String, Long>>> logsWithLengthFuture = resourceManagerGateway.requestTaskManagerLogList(taskManagerId, timeout);
+
+		return logsWithLengthFuture.thenApply(logName2Sizes -> {
+			if (null != logName2Sizes) {
+				Collection<LogInfo> logs = logName2Sizes.stream().map(logName2Size -> new LogInfo(logName2Size.f0, logName2Size.f1)).collect(Collectors.toSet());
+				return new LogsInfo(logs);
+			} else {
+				return LogsInfo.empty();
+			}
+		}).exceptionally(
+				(Throwable throwable) -> {
+					final Throwable strippedThrowable = ExceptionUtils.stripExecutionException(throwable);
+
+					if (strippedThrowable instanceof UnknownTaskExecutorException) {
+						throw new CompletionException(
+							new RestHandlerException(
+								"Could not find TaskExecutor " + taskManagerId + '.',
+								HttpResponseStatus.NOT_FOUND,
+								strippedThrowable));
+					} else {
+						throw new CompletionException(strippedThrowable);
 
 Review comment:
   Why not 404? Requesting an invalid file should be a client error, i.e., 4xx.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-600463751
 
 
   > Thank you for work. I left some comments.
   > 
   > For the test cases, does new added `TaskManagerCustomFileHandler` be covered ?
   
   As AbstractTaskManagerFileHandlerTest has tested, so we needn't test for  TaskManagerCustomFileHandler.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] Myasuka commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
Myasuka commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394281662
 
 

 ##########
 File path: docs/_includes/generated/rest_v1_dispatcher.html
 ##########
 @@ -4128,3 +4128,116 @@
     </tr>
   </tbody>
 </table>
+<table class="table table-bordered">
+    <tbody>
+    <tr>
+        <td class="text-left" colspan="2"><h5><strong>/taskmanagers/:taskmanagerid/logs</strong></h5></td>
+    </tr>
+    <tr>
+        <td class="text-left" style="width: 20%">Verb: <code>GET</code></td>
+        <td class="text-left">Response code: <code>200 OK</code></td>
+    </tr>
+    <tr>
+        <td colspan="2">Returns the list of log files on a TaskManager.</td>
+    </tr>
+    <tr>
+        <td colspan="2">Path parameters</td>
+    </tr>
+    <tr>
+        <td colspan="2">
+            <ul>
+                <li><code>taskmanagerid</code> - 32-character hexadecimal string that identifies a task manager.</li>
+            </ul>
+        </td>
+    </tr>
+    <tr>
+        <td colspan="2">
+            <button data-toggle="collapse" data-target="#1220778126">Request</button>
+            <div id="1220778127" class="collapse">
+          <pre>
+            <code>
+{}            </code>
+          </pre>
+            </div>
+        </td>
+    </tr>
+    <tr>
+        <td colspan="2">
+            <button data-toggle="collapse" data-target="#1531506487">Response</button>
+            <div id="1531506488" class="collapse">
+          <pre>
+            <code>
+{
+  "type" : "object",
+  "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:taskmanager:LogsInfo",
+  "properties" : {
+    "logs" : {
+      "type" : "array",
+      "items" : {
+        "type" : "object",
+        "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:taskmanager:LogInfo",
+        "properties" : {
+          "name" : {
+            "type" : "string"
+          },
+          "size" : {
+            "type" : "long"
+          }
+        }
+      }
+    }
+  }
+}            </code>
+          </pre>
+            </div>
+        </td>
+    </tr>
+    </tbody>
+</table>
+<table class="table table-bordered">
+    <tbody>
+    <tr>
+        <td class="text-left" colspan="2"><h5><strong>/taskmanagers/:taskmanagerid/logs/:filename</strong></h5></td>
+    </tr>
+    <tr>
+        <td class="text-left" style="width: 20%">Verb: <code>GET</code></td>
+        <td class="text-left">Response code: <code>200 OK</code></td>
+    </tr>
+    <tr>
+        <td colspan="2">Provides access to get task manager file.</td>
+    </tr>
+    <tr>
+        <td colspan="2">Path parameters</td>
+    </tr>
+    <tr>
+        <td colspan="2">
+            <ul>
+                <li><code>taskmanagerid</code> - 32-character hexadecimal string that identifies a task manager.</li>
+                <li><code>filename</code> - string that identifies a file's name in task manager's log dir.</li>
 
 Review comment:
   These descriptions should be generated by java code of `getDescription`. However, they seems not the same as current descriptions.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393695494
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -582,16 +582,41 @@ public void notifySlotAvailable(
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) {
-		log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId);
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) {
+		log.debug("Request file which type is {}  upload from TaskExecutor {}.", fileType, taskManagerId);
 
 		final WorkerRegistration<WorkerType> taskExecutor = taskExecutors.get(taskManagerId);
 
 		if (taskExecutor == null) {
-			log.debug("Requested file {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
+			log.debug("Requested which type is {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
 			return FutureUtils.completedExceptionally(new UnknownTaskExecutorException(taskManagerId));
 		} else {
-			return taskExecutor.getTaskExecutorGateway().requestFileUpload(fileType, timeout);
+			return taskExecutor.getTaskExecutorGateway().requestFileUploadByType(fileType, timeout);
+		}
+	}
+
+	@Override
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByName(ResourceID taskManagerId, String fileName, Time timeout) {
+		log.debug("Request file which name is {}  upload from TaskExecutor {}.", fileName, taskManagerId);
+
+		final WorkerRegistration<WorkerType> taskExecutor = taskExecutors.get(taskManagerId);
+
+		if (taskExecutor == null) {
+			log.debug("Requested file which name is {} upload from unregistered TaskExecutor {}.", fileName, taskManagerId);
+			return FutureUtils.completedExceptionally(new UnknownTaskExecutorException(taskManagerId));
+		} else {
+			return taskExecutor.getTaskExecutorGateway().requestFileUploadByName(fileName, timeout);
+		}
+	}
+
+	@Override
+	public CompletableFuture<Collection<Tuple2<String, Long>>> requestTaskManagerLogList(ResourceID taskManagerId, Time timeout) {
+		final WorkerRegistration<WorkerType> taskExecutor = taskExecutors.get(taskManagerId);
+		if (taskExecutor == null) {
+			log.debug("Requested historical loglist from unregistered TaskExecutor {}.", taskManagerId);
 
 Review comment:
   ```suggestion
   			log.debug("Requested historical log list from unregistered TaskExecutor {}.", taskManagerId);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/152951654",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * 3984f26b34353ad5b2006c8768536fbaed5b512c Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/152951654) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226) 
   * 0d0a96f5919000033e3b43d4ce009793d1ed0544 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393693052
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -582,16 +582,41 @@ public void notifySlotAvailable(
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) {
-		log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId);
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) {
+		log.debug("Request file which type is {}  upload from TaskExecutor {}.", fileType, taskManagerId);
 
 		final WorkerRegistration<WorkerType> taskExecutor = taskExecutors.get(taskManagerId);
 
 		if (taskExecutor == null) {
-			log.debug("Requested file {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
+			log.debug("Requested which type is {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
 
 Review comment:
   ```suggestion
   			log.debug("Requested file of type {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/152951654",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6364",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153925757",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6433",
       "triggerID" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154208064",
       "triggerID" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * 0d0a96f5919000033e3b43d4ce009793d1ed0544 Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/153925757) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6364) 
   * 1dc6d0c40bedc119afa4b1101b86a2f5238eff31 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/154208064) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6433) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593952478
 
 
   @flinkbot run travis

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-594484307
 
 
   I will review this in the next days. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-594254491
 
 
   @flinkbot run azure

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394367376
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
+		log.debug("Request file name is {} upload.", fileName);
 
-				return CompletableFuture.completedFuture(transientBlobKey);
-			} else {
-				log.debug("The file {} does not exist on the TaskExecutor {}.", fileType, getResourceID());
-				return FutureUtils.completedExceptionally(new FlinkException("The file " + fileType + " does not exist on the TaskExecutor."));
-			}
+		final String filePath;
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null && !logDir.isEmpty() && fileName != null && !fileName.isEmpty()) {
+			filePath = logDir + "/" + fileName;
 
 Review comment:
   maybe below code is better:
   ```java
   filePath = Paths.get(logDir, fileName).toFile().getAbsolutePath();
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391814623
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -582,16 +582,41 @@ public void notifySlotAvailable(
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) {
-		log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId);
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) {
+		log.debug("Request file which type is {}  upload from TaskExecutor {}.", fileType, taskManagerId);
 
 Review comment:
   Unfortunately this log message and the previous one are not valid English. My suggestion is:
   ```
   log.debug("Request {} file upload from TaskExecutor {}.", fileType, taskManagerId);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394133406
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -309,6 +311,25 @@ public TaskExecutor(
 		return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty());
 	}
 
+	@Override
+	public CompletableFuture<Collection<Tuple2<String, Long>>> requestLogList(Time timeout) {
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null) {
+			final File[] logFiles = new File(logDir).listFiles();
+
+			if (logFiles == null) {
+				return FutureUtils.completedExceptionally(
+					new FlinkException("The specific log directory is not a valid directory."));
+			}
+
+			final List<Tuple2<String, Long>> logsWithLength = Arrays.stream(logFiles)
+				.map(logFile -> Tuple2.of(logFile.getName(), logFile.length()))
+				.collect(Collectors.toList());
+			return CompletableFuture.completedFuture(logsWithLength);
+		}
+		return FutureUtils.completedExceptionally(new FlinkException("There is no log file available on the TaskExecutor."));
 
 Review comment:
   Ok, that makes sense.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * c07d058a0f55764b319d64034b4f1bfdc26d99e6 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151502448) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854) 
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391808587
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -309,6 +311,25 @@ public TaskExecutor(
 		return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty());
 	}
 
+	@Override
+	public CompletableFuture<Collection<Tuple2<String, Long>>> requestLogList(Time timeout) {
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null) {
+			final File[] logFiles = new File(logDir).listFiles();
+
+			if (logFiles == null) {
+				return FutureUtils.completedExceptionally(
+					new FlinkException("The specific log directory is not a valid directory."));
+			}
+
+			final List<Tuple2<String, Long>> logsWithLength = Arrays.stream(logFiles)
+				.map(logFile -> Tuple2.of(logFile.getName(), logFile.length()))
+				.collect(Collectors.toList());
+			return CompletableFuture.completedFuture(logsWithLength);
+		}
+		return FutureUtils.completedExceptionally(new FlinkException("There is no log file available on the TaskExecutor."));
 
 Review comment:
   Another reasonable behavior would be to return an empty collection. Not sure what is better here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394064136
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -309,6 +311,25 @@ public TaskExecutor(
 		return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty());
 	}
 
+	@Override
+	public CompletableFuture<Collection<Tuple2<String, Long>>> requestLogList(Time timeout) {
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null) {
+			final File[] logFiles = new File(logDir).listFiles();
+
+			if (logFiles == null) {
+				return FutureUtils.completedExceptionally(
+					new FlinkException("The specific log directory is not a valid directory."));
+			}
+
+			final List<Tuple2<String, Long>> logsWithLength = Arrays.stream(logFiles)
+				.map(logFile -> Tuple2.of(logFile.getName(), logFile.length()))
+				.collect(Collectors.toList());
+			return CompletableFuture.completedFuture(logsWithLength);
+		}
+		return FutureUtils.completedExceptionally(new FlinkException("There is no log file available on the TaskExecutor."));
 
 Review comment:
   Because there must be a log file currently, if there is no log file, it should be an exception, not empty.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 436c1a75f3c98a2ccd7cc1c53445449d396bb916 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151891829) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947) 
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * b8d51c94a0b93fdbfa4b167e0b4c630f791fba10 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150973416) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394940530
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java
 ##########
 @@ -263,4 +266,8 @@ private void transferFile(ChannelHandlerContext ctx, File file, HttpRequest http
 			throw new FlinkException("Could not transfer file " + file + " to the client.", ioe);
 		}
 	}
+
+	protected String getFileName(HandlerRequest<EmptyRequestBody, M> handlerRequest) {
+		return null;
 
 Review comment:
   I missed it before but I think putting a constant won't work here. If a user requests `STDOUT` and then `LOG`, he will get `STDOUT` twice?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 912485a96d61febfa41b7c84631aeae19d819325 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150964833) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718) 
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * b8d51c94a0b93fdbfa4b167e0b4c630f791fba10 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/150973416) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720) 
   * 70e8ca9774fc4247657f5d6aecc43459229ba9bb Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151307447) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 70e8ca9774fc4247657f5d6aecc43459229ba9bb Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151307447) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801) 
   * 54ac8f26c67344a0ed000c882afd66e68c8f6bd0 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151328904) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592334762
 
 
   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit b8d51c94a0b93fdbfa4b167e0b4c630f791fba10 (Fri Feb 28 21:49:49 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
    * **This pull request references an unassigned [Jira ticket](https://issues.apache.org/jira/browse/FLINK-16302).** According to the [code contribution guide](https://flink.apache.org/contributing/contribute-code.html), tickets need to be assigned before starting with the implementation work.
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593828424
 
 
   @flinkbot run azure

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-595091021
 
 
   > I will review this in the next days.
   Thanks for reviewing.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394942814
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerStdoutFileHandler.java
 ##########
 @@ -53,7 +54,7 @@ public TaskManagerStdoutFileHandler(
 	}
 
 	@Override
-	protected CompletableFuture<TransientBlobKey> requestFileUpload(ResourceManagerGateway resourceManagerGateway, ResourceID taskManagerResourceId) {
-		return resourceManagerGateway.requestTaskManagerFileUpload(taskManagerResourceId, FileType.STDOUT, timeout);
+	protected CompletableFuture<TransientBlobKey> requestFileUpload(ResourceManagerGateway resourceManagerGateway, Tuple2<ResourceID, String> taskmanagerId2FileName) {
 
 Review comment:
   `taskmanagerIdAndFileName`, to be consistent

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 436c1a75f3c98a2ccd7cc1c53445449d396bb916 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151891829) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947) 
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/152623819) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 54ac8f26c67344a0ed000c882afd66e68c8f6bd0 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151328904) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592334929
 
 
   @flinkbot run travis

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r395440082
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java
 ##########
 @@ -263,4 +266,8 @@ private void transferFile(ChannelHandlerContext ctx, File file, HttpRequest http
 			throw new FlinkException("Could not transfer file " + file + " to the client.", ioe);
 		}
 	}
+
+	protected String getFileName(HandlerRequest<EmptyRequestBody, M> handlerRequest) {
+		return null;
 
 Review comment:
   If a user requests STDOUT and then LOG, he will not get STDOUT twice. 
   As `fileBlobKeys` is not static, so it's belonging to its object. And I have tested it by:
   ```java
   log.info(String.format("fileBlobKeys [%s] cached file for TaskExecutor [%s] taskManagerId [%s], blobKey [%s]", fileBlobKeys.toString(), taskManagerIdAndFileName, taskManagerId, blobKey.toString()));
   ```
   
   The result is:
   ```text
   2020-03-20 11:32:18,582 INFO  org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerLogFileHandler [] - fileBlobKeys [org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalLoadingCache@6cfede43] cached file for TaskExecutor [(c65f774c537555a1fb64c64f9b1ff18b,null)] taskManagerId [c65f774c537555a1fb64c64f9b1ff18b], blobKey [t-a8637ce83ae75216ce99399eb59f07c63c53603b-f018c857f158c354d3018f6d03ae2ad6]
   2020-03-20 11:32:24,202 INFO  org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerStdoutFileHandler [] - fileBlobKeys [org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalLoadingCache@76dd5d49] cached file for TaskExecutor [(c65f774c537555a1fb64c64f9b1ff18b,null)] taskManagerId [c65f774c537555a1fb64c64f9b1ff18b], blobKey [t-da39a3ee5e6b4b0d3255bfef95601890afd80709-89a50af389ae582c598680476d832e0d]
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r390286564
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -900,7 +921,7 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
+	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, String fileName, Time timeout) {
 
 Review comment:
   Sounds good,  also update methods' name may be better.
   ```java
   CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, @RpcTimeout Time timeout);
   
   CompletableFuture<TransientBlobKey> requestFileUploadByType(FileType fileType, @RpcTimeout Time timeout);
   ```
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r390191813
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerLogsHeaders.java
 ##########
 @@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.messages.taskmanager;
+
+import org.apache.flink.runtime.rest.HttpMethodWrapper;
+import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerLogsHandler;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+/**
+ * Headers for the {@link TaskManagerLogsHandler}.
+ */
+public class TaskManagerLogsHeaders implements MessageHeaders<EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> {
+
+	private static final TaskManagerLogsHeaders INSTANCE = new TaskManagerLogsHeaders();
+
+	private static final String URL = String.format("/taskmanagers/:%s/logs", TaskManagerIdPathParameter.KEY);
+
+	private TaskManagerLogsHeaders() {}
+
+	public static TaskManagerLogsHeaders getInstance() {
+		return INSTANCE;
+	}
+
+	@Override
+	public Class<LogsInfo> getResponseClass() {
+		return LogsInfo.class;
+	}
+
+	@Override
+	public HttpResponseStatus getResponseStatusCode() {
+		return HttpResponseStatus.OK;
+	}
+
+	@Override
+	public String getDescription() {
+		return "Provides access to task manager logs list.";
 
 Review comment:
   _Returns the list of log files on a TaskManager_

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] Myasuka commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
Myasuka commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394259005
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
 ##########
 @@ -216,9 +227,18 @@
 	 */
 	CompletableFuture<Boolean> canBeReleased();
 
+
 
 Review comment:
   No necessary one more line here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 436c1a75f3c98a2ccd7cc1c53445449d396bb916 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151891829) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947) 
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394944291
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -309,6 +312,25 @@ public TaskExecutor(
 		return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty());
 	}
 
+	@Override
+	public CompletableFuture<Collection<LogInfo>> requestLogList(Time timeout) {
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null) {
+			final File[] logFiles = new File(logDir).listFiles();
+
+			if (logFiles == null) {
+				return FutureUtils.completedExceptionally(new FlinkException(String.format("There isn't a log file in TaskExecutor’s log dir %s.", logDir)));
+			}
+
+			final List<LogInfo> logsWithLength = Arrays.stream(logFiles).filter(File::isFile)
 
 Review comment:
   `.filter` should be on a new line. See https://flink.apache.org/contributing/code-style-and-quality-formatting.html

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394124255
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -582,16 +582,41 @@ public void notifySlotAvailable(
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) {
-		log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId);
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) {
+		log.debug("Request file which type is {}  upload from TaskExecutor {}.", fileType, taskManagerId);
 
 		final WorkerRegistration<WorkerType> taskExecutor = taskExecutors.get(taskManagerId);
 
 		if (taskExecutor == null) {
-			log.debug("Requested file {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
+			log.debug("Requested which type is {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
 
 Review comment:
   As here taskExecutor is null, so we need add unregistered for TaskExecutor.
   ```code
   log.debug("Request upload of file {} from unregistered TaskExecutor {}.", fileType, taskManagerId);
   ```
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r390157774
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
 ##########
 @@ -211,9 +211,18 @@ void notifySlotAvailable(
 	 *
 	 * @param taskManagerId identifying the {@link TaskExecutor} to upload the specified file
 	 * @param fileType type of the file to upload
+	 * @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}.
 	 */
-	CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, @RpcTimeout Time timeout);
+	CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, String fileName, @RpcTimeout Time timeout);
 
 Review comment:
   Imo a method signature like this is hard to understand and confusing. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/152623819) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391798715
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
 ##########
 @@ -216,9 +227,18 @@
 	 */
 	CompletableFuture<Boolean> canBeReleased();
 
+
+	/**
+	 * Requests for the historical log file names on the TaskManager.
+	 *
+	 * @return A Tuple2 Array with all log file names with its length.
+	 */
+	CompletableFuture<Collection<Tuple2<String, Long>>> requestLogList(@RpcTimeout Time timeout);
 
 Review comment:
   I would prefer not to use `Tuple2<String, Long>`. IMO tuple types should be used sparingly because its semantics are often not clear. I would prefer to introduce a new type or just use `org.apache.flink.runtime.rest.messages.taskmanager.LogInfo`. There are some downsides with reusing types but it has been done like that before (for example, see `ResourceManager#requestTaskManagerInfo()`)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * b8d51c94a0b93fdbfa4b167e0b4c630f791fba10 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150973416) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720) 
   * 70e8ca9774fc4247657f5d6aecc43459229ba9bb UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391872861
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
 
 Review comment:
   This method could be vulnerable against a path traversal attack: https://wiki.owasp.org/index.php/Path_Traversal
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593954121
 
 
   @flinkbot run azure

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391803582
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
+		log.debug("Request file name is {} upload.", fileName);
 
-				return CompletableFuture.completedFuture(transientBlobKey);
-			} else {
-				log.debug("The file {} does not exist on the TaskExecutor {}.", fileType, getResourceID());
-				return FutureUtils.completedExceptionally(new FlinkException("The file " + fileType + " does not exist on the TaskExecutor."));
-			}
+		final String filePath;
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null && !logDir.isEmpty() && fileName != null && !fileName.isEmpty()) {
+			filePath = logDir + "/" + fileName;
 
 Review comment:
   This won't work on windows. It's better to use `new File(logDir, fileName).getPath()`
   
   _Edit:_
   This method could be vulnerable against a path traversal attack: https://wiki.owasp.org/index.php/Path_Traversal
   
   So maybe:
   
   `new File(logDir, new File(fileName).getName()).getPath()`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/152951654",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6364",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153925757",
       "triggerID" : "0d0a96f5919000033e3b43d4ce009793d1ed0544",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6433",
       "triggerID" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154208064",
       "triggerID" : "1dc6d0c40bedc119afa4b1101b86a2f5238eff31",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * 1dc6d0c40bedc119afa4b1101b86a2f5238eff31 Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/154208064) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6433) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-596675835
 
 
   What is your opinion about the future of the `TaskManagerLogFileHandler` and `TaskManagerStdoutFileHandler`? It seems to me that after this PR there will be two ways to retrieve the TM logs and stdout file, respectively. Does it make sense to deprecate the previous handlers?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393732242
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerHandler.java
 ##########
 @@ -65,13 +64,24 @@ protected AbstractTaskManagerHandler(
 
 	@Override
 	protected CompletableFuture<P> handleRequest(@Nonnull HandlerRequest<R, M> request, @Nonnull T gateway) throws RestHandlerException {
-		Optional<ResourceManagerGateway> resourceManagerGatewayOptional = resourceManagerGatewayRetriever.getNow();
-
-		ResourceManagerGateway resourceManagerGateway = resourceManagerGatewayOptional.orElseThrow(
-			() -> new RestHandlerException("Cannot connect to ResourceManager right now. Please try to refresh.", HttpResponseStatus.NOT_FOUND));
+		ResourceManagerGateway resourceManagerGateway = getResourceManagerGateway(resourceManagerGatewayRetriever);
 
 		return handleRequest(request, resourceManagerGateway);
 	}
 
 	protected abstract CompletableFuture<P> handleRequest(@Nonnull HandlerRequest<R, M> request, @Nonnull ResourceManagerGateway gateway) throws RestHandlerException;
+
+	/**
+	 * Get ResourceManagerGateway from GatewayRetriever ResourceManagerGateway.
+	 * @param resourceManagerGatewayRetriever
+	 * @return ResourceManagerGateway
+	 * @throws RestHandlerException
 
 Review comment:
   `resourceManagerGatewayRetriever` and `RestHandlerException`  tag desc is missing

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391803582
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
+		log.debug("Request file name is {} upload.", fileName);
 
-				return CompletableFuture.completedFuture(transientBlobKey);
-			} else {
-				log.debug("The file {} does not exist on the TaskExecutor {}.", fileType, getResourceID());
-				return FutureUtils.completedExceptionally(new FlinkException("The file " + fileType + " does not exist on the TaskExecutor."));
-			}
+		final String filePath;
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null && !logDir.isEmpty() && fileName != null && !fileName.isEmpty()) {
+			filePath = logDir + "/" + fileName;
 
 Review comment:
   This won't work on windows. It's better to use new File(logDir, fileName).getPath()

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r396087467
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -309,6 +312,25 @@ public TaskExecutor(
 		return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty());
 	}
 
+	@Override
+	public CompletableFuture<Collection<LogInfo>> requestLogList(Time timeout) {
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null) {
+			final File[] logFiles = new File(logDir).listFiles();
 
 Review comment:
   We can continue for now but FLINK-16710 should be fixed next.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394945209
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -309,6 +312,25 @@ public TaskExecutor(
 		return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty());
 	}
 
+	@Override
+	public CompletableFuture<Collection<LogInfo>> requestLogList(Time timeout) {
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null) {
+			final File[] logFiles = new File(logDir).listFiles();
+
+			if (logFiles == null) {
+				return FutureUtils.completedExceptionally(new FlinkException(String.format("There isn't a log file in TaskExecutor’s log dir %s.", logDir)));
+			}
+
+			final List<LogInfo> logsWithLength = Arrays.stream(logFiles).filter(File::isFile)
+				.map(logFile -> new LogInfo(logFile.getName(), logFile.length()))
+				.collect(Collectors.toList());
+			return CompletableFuture.completedFuture(logsWithLength);
+		}
+		//There isn't log file available on the TaskExecutor.
 
 Review comment:
   This comment is superfluous. I would remove it. The method isn't very long and also from the exception text it should be obvious.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393751695
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -900,8 +921,8 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestFileUpload(FileType fileType, Time timeout) {
-		log.debug("Request file {} upload.", fileType);
+	public CompletableFuture<TransientBlobKey> requestFileUploadByType(FileType fileType, Time timeout) {
+		log.debug("Request file type is {} upload.", fileType);
 
 Review comment:
   ```suggestion
   		log.debug("Request file of type {} upload.", fileType);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393559002
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
 ##########
 @@ -216,9 +227,18 @@
 	 */
 	CompletableFuture<Boolean> canBeReleased();
 
+
+	/**
+	 * Requests for the historical log file names on the TaskManager.
+	 *
+	 * @return A Tuple2 Array with all log file names with its length.
+	 */
+	CompletableFuture<Collection<Tuple2<String, Long>>> requestLogList(@RpcTimeout Time timeout);
 
 Review comment:
   If the reuse type is ok, I'll use it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391805526
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -915,29 +936,21 @@ public void heartbeatFromResourceManager(ResourceID resourceID) {
 			default:
 				filePath = null;
 		}
+		return requestFileUploadByFilePath(filePath, timeout, fileType.toString());
+	}
 
-		if (filePath != null && !filePath.isEmpty()) {
-			final File file = new File(filePath);
-
-			if (file.exists()) {
-				final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService();
-				final TransientBlobKey transientBlobKey;
-				try (FileInputStream fileInputStream = new FileInputStream(file)) {
-					transientBlobKey = transientBlobService.putTransient(fileInputStream);
-				} catch (IOException e) {
-					log.debug("Could not upload file {}.", fileType, e);
-					return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e));
-				}
+	@Override
+	public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout) {
+		log.debug("Request file name is {} upload.", fileName);
 
 Review comment:
   Log message could be clearer. How about
   
   ```
   log.debug("Received file upload request for file {}", fileName);
   ```
   Also, I think it's enough to log once in `requestFileUploadByFilePath()`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593246398
 
 
   @flinkbot run azure

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391814623
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -582,16 +582,41 @@ public void notifySlotAvailable(
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) {
-		log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId);
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) {
+		log.debug("Request file which type is {}  upload from TaskExecutor {}.", fileType, taskManagerId);
 
 Review comment:
   Unfortunately this log message and the previous one are not valid English. My suggestion is:
   ```
   log.debug("Request {} file upload from TaskExecutor {}", fileType, taskManagerId);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 70e8ca9774fc4247657f5d6aecc43459229ba9bb Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151307447) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r391814623
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -582,16 +582,41 @@ public void notifySlotAvailable(
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) {
-		log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId);
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) {
+		log.debug("Request file which type is {}  upload from TaskExecutor {}.", fileType, taskManagerId);
 
 Review comment:
   Unfortunately this log message and the previous one is not valid English. My suggestion is:
   ```
   log.debug("Request {} file upload from TaskExecutor {}.", fileType, taskManagerId);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL closed pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL closed pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining removed a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593246362
 
 
   @flinkbot run travis

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * b8d51c94a0b93fdbfa4b167e0b4c630f791fba10 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150973416) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * c07d058a0f55764b319d64034b4f1bfdc26d99e6 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151502448) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a612b90d60e58bd59164575f3eef28fdfa5eb01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151891829",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5947",
       "triggerID" : "595547125",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6130",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/152623819",
       "triggerID" : "d5b64bcfcc07c4d5ec5545ae8f068f4e63e0086a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/152951654",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226",
       "triggerID" : "3984f26b34353ad5b2006c8768536fbaed5b512c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 7a612b90d60e58bd59164575f3eef28fdfa5eb01 UNKNOWN
   * 3984f26b34353ad5b2006c8768536fbaed5b512c Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/152951654) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6226) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-593952809
 
 
   @flinkbot run travis

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394971804
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogsHandlerTest.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException;
+import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.HandlerRequestException;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogsInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerLogsHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test for the {@link TaskManagerLogsHandler}.
+ */
+public class TaskManagerLogsHandlerTest extends TestLogger {
+
+	private static final ResourceID EXPECTED_TASK_MANAGER_ID = ResourceID.generate();
+
+	@Test
+	public void testGetTaskManagerLogsList() throws Exception {
+		final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway();
+		final TaskManagerLogsHandler taskManagerLogsHandler = new TaskManagerLogsHandler(
+			() -> CompletableFuture.completedFuture(null),
+			TestingUtils.TIMEOUT(),
+			Collections.emptyMap(),
+			TaskManagerLogsHeaders.getInstance(),
+			() -> CompletableFuture.completedFuture(resourceManagerGateway));
+		final HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> handlerRequest = createRequest(EXPECTED_TASK_MANAGER_ID);
+		List<LogInfo> logsList = new ArrayList<>();
+		logsList.add(new LogInfo("taskmanager.log", 1024L));
+		logsList.add(new LogInfo("taskmanager.out", 1024L));
+		logsList.add(new LogInfo("taskmanager-2.out", 1024L));
+		resourceManagerGateway.setRequestTaskManagerLogListFunction(EXPECTED_TASK_MANAGER_ID -> CompletableFuture.completedFuture(logsList));
+		LogsInfo logsInfo = taskManagerLogsHandler.handleRequest(handlerRequest, resourceManagerGateway).get();
+		assertEquals(logsInfo.getLogInfos().size(), resourceManagerGateway.requestTaskManagerLogList(EXPECTED_TASK_MANAGER_ID, TestingUtils.TIMEOUT()).get().size());
+	}
+
+	@Test
+	public void testGetTaskManagerLogsListForUnknownTaskExecutorException() throws Exception {
+		final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway();
+		final TaskManagerLogsHandler taskManagerLogsHandler = new TaskManagerLogsHandler(
+			() -> CompletableFuture.completedFuture(null),
+			TestingUtils.TIMEOUT(),
+			Collections.emptyMap(),
+			TaskManagerLogsHeaders.getInstance(),
+			() -> CompletableFuture.completedFuture(resourceManagerGateway));
+		final HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> handlerRequest = createRequest(EXPECTED_TASK_MANAGER_ID);
+		resourceManagerGateway.setRequestTaskManagerLogListFunction(EXPECTED_TASK_MANAGER_ID -> FutureUtils.completedExceptionally(new UnknownTaskExecutorException(EXPECTED_TASK_MANAGER_ID)));
+		try {
+			taskManagerLogsHandler.handleRequest(handlerRequest, resourceManagerGateway).get();
+		} catch (Exception exception){
+			assertEquals("org.apache.flink.runtime.rest.handler.RestHandlerException: Could not find TaskExecutor " + EXPECTED_TASK_MANAGER_ID + ".", exception.getMessage());
+		}
+	}
+
+	private static HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> createRequest(ResourceID taskManagerId) throws HandlerRequestException {
+		final Map<String, String> pathParameters = new HashMap<>();
 
 Review comment:
   Maybe `Collections.singletonMap()`;

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
GJL commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r394133992
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogsHandler.java
 ##########
 @@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.taskmanager;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.LogsInfo;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
+import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+/**
+ * Handler which serves detailed TaskManager log list information.
+ */
+public class TaskManagerLogsHandler extends AbstractTaskManagerHandler<RestfulGateway, EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> {
+
+	private final GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever;
+
+	public TaskManagerLogsHandler(
+			GatewayRetriever<? extends RestfulGateway> leaderRetriever,
+			Time timeout,
+			Map<String, String> responseHeaders,
+			MessageHeaders<EmptyRequestBody, LogsInfo, TaskManagerMessageParameters> messageHeaders,
+			GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever) {
+		super(leaderRetriever, timeout, responseHeaders, messageHeaders, resourceManagerGatewayRetriever);
+
+		this.resourceManagerGatewayRetriever = Preconditions.checkNotNull(resourceManagerGatewayRetriever);
+	}
+
+	@Override
+	protected CompletableFuture<LogsInfo> handleRequest(
+			@Nonnull HandlerRequest<EmptyRequestBody, TaskManagerMessageParameters> request,
+			@Nonnull ResourceManagerGateway gateway) throws RestHandlerException {
+		final ResourceID taskManagerId = request.getPathParameter(TaskManagerIdPathParameter.class);
+		final ResourceManagerGateway resourceManagerGateway = getResourceManagerGateway(resourceManagerGatewayRetriever);
+		final CompletableFuture<Collection<Tuple2<String, Long>>> logsWithLengthFuture = resourceManagerGateway.requestTaskManagerLogList(taskManagerId, timeout);
+
+		return logsWithLengthFuture.thenApply(logName2Sizes -> {
+			if (null != logName2Sizes) {
+				Collection<LogInfo> logs = logName2Sizes.stream().map(logName2Size -> new LogInfo(logName2Size.f0, logName2Size.f1)).collect(Collectors.toSet());
+				return new LogsInfo(logs);
+			} else {
+				return LogsInfo.empty();
+			}
+		}).exceptionally(
+				(Throwable throwable) -> {
+					final Throwable strippedThrowable = ExceptionUtils.stripExecutionException(throwable);
+
+					if (strippedThrowable instanceof UnknownTaskExecutorException) {
+						throw new CompletionException(
+							new RestHandlerException(
+								"Could not find TaskExecutor " + taskManagerId + '.',
+								HttpResponseStatus.NOT_FOUND,
+								strippedThrowable));
+					} else {
+						throw new CompletionException(strippedThrowable);
 
 Review comment:
   Why not 404? Requesting an invalid file should be a client error, i.e., 4xx. 2xx still signals success.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393690922
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -582,16 +582,41 @@ public void notifySlotAvailable(
 	}
 
 	@Override
-	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) {
-		log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId);
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) {
+		log.debug("Request file which type is {}  upload from TaskExecutor {}.", fileType, taskManagerId);
 
 		final WorkerRegistration<WorkerType> taskExecutor = taskExecutors.get(taskManagerId);
 
 		if (taskExecutor == null) {
-			log.debug("Requested file {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
+			log.debug("Requested which type is {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId);
 			return FutureUtils.completedExceptionally(new UnknownTaskExecutorException(taskManagerId));
 		} else {
-			return taskExecutor.getTaskExecutorGateway().requestFileUpload(fileType, timeout);
+			return taskExecutor.getTaskExecutorGateway().requestFileUploadByType(fileType, timeout);
+		}
+	}
+
+	@Override
+	public CompletableFuture<TransientBlobKey> requestTaskManagerFileUploadByName(ResourceID taskManagerId, String fileName, Time timeout) {
+		log.debug("Request file which name is {}  upload from TaskExecutor {}.", fileName, taskManagerId);
+
+		final WorkerRegistration<WorkerType> taskExecutor = taskExecutors.get(taskManagerId);
+
+		if (taskExecutor == null) {
+			log.debug("Requested file which name is {} upload from unregistered TaskExecutor {}.", fileName, taskManagerId);
 
 Review comment:
   ```suggestion
   			log.debug("Requested file named {} upload from unregistered TaskExecutor {}.", fileName, taskManagerId);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
kaibozhou commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r393749291
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -309,6 +311,25 @@ public TaskExecutor(
 		return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty());
 	}
 
+	@Override
+	public CompletableFuture<Collection<Tuple2<String, Long>>> requestLogList(Time timeout) {
+		final String logDir = taskManagerConfiguration.getTaskManagerLogDir();
+		if (logDir != null) {
+			final File[] logFiles = new File(logDir).listFiles();
 
 Review comment:
   +1 it's better to ignore directories.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
jinglining commented on a change in pull request #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#discussion_r395440082
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java
 ##########
 @@ -263,4 +266,8 @@ private void transferFile(ChannelHandlerContext ctx, File file, HttpRequest http
 			throw new FlinkException("Could not transfer file " + file + " to the client.", ioe);
 		}
 	}
+
+	protected String getFileName(HandlerRequest<EmptyRequestBody, M> handlerRequest) {
+		return null;
 
 Review comment:
   If a user requests STDOUT and then LOG, he will not get STDOUT twice. As `fileBlobKeys` is not static, so it's belonging to its object. And I hava test it by 
   ```java
   log.info(String.format("fileBlobKeys [%s] cached file for TaskExecutor [%s] taskManagerId [%s], blobKey [%s]", fileBlobKeys.toString(), taskManagerIdAndFileName, taskManagerId, blobKey.toString()));
   ```
   
   The result is
   ```text
   2020-03-20 11:32:18,582 INFO  org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerLogFileHandler [] - fileBlobKeys [org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalLoadingCache@6cfede43] cached file for TaskExecutor [(c65f774c537555a1fb64c64f9b1ff18b,null)] taskManagerId [c65f774c537555a1fb64c64f9b1ff18b], blobKey [t-a8637ce83ae75216ce99399eb59f07c63c53603b-f018c857f158c354d3018f6d03ae2ad6]
   2020-03-20 11:32:24,202 INFO  org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerStdoutFileHandler [] - fileBlobKeys [org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalLoadingCache@76dd5d49] cached file for TaskExecutor [(c65f774c537555a1fb64c64f9b1ff18b,null)] taskManagerId [c65f774c537555a1fb64c64f9b1ff18b], blobKey [t-da39a3ee5e6b4b0d3255bfef95601890afd80709-89a50af389ae582c598680476d832e0d]
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * c07d058a0f55764b319d64034b4f1bfdc26d99e6 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151502448) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11250: [FLINK-16302][rest]add log list and read log by name for taskmanager
URL: https://github.com/apache/flink/pull/11250#issuecomment-592336212
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5718",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "912485a96d61febfa41b7c84631aeae19d819325",
       "triggerType" : "PUSH"
     }, {
       "hash" : "912485a96d61febfa41b7c84631aeae19d819325",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150964833",
       "triggerID" : "592334929",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e474761e49927bb30bac6a297e992dc2ec98e01a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150973416",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8d51c94a0b93fdbfa4b167e0b4c630f791fba10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5720",
       "triggerID" : "593246362",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151307447",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5801",
       "triggerID" : "70e8ca9774fc4247657f5d6aecc43459229ba9bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5812",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151328904",
       "triggerID" : "54ac8f26c67344a0ed000c882afd66e68c8f6bd0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151502448",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c07d058a0f55764b319d64034b4f1bfdc26d99e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854",
       "triggerID" : "593952809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "306b1e6622445245bdc59517d23790bac9d5ea52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "436c1a75f3c98a2ccd7cc1c53445449d396bb916",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e474761e49927bb30bac6a297e992dc2ec98e01a UNKNOWN
   * c07d058a0f55764b319d64034b4f1bfdc26d99e6 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151502448) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5854) 
   * 306b1e6622445245bdc59517d23790bac9d5ea52 UNKNOWN
   * 436c1a75f3c98a2ccd7cc1c53445449d396bb916 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services