[
https://issues.apache.org/jira/browse/FLINK-9196?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457511#comment-16457511
]
ASF GitHub Bot commented on FLINK-9196:
---------------------------------------
Github user GJL commented on a diff in the pull request:
https://github.com/apache/flink/pull/5938#discussion_r184848354
--- Diff: flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTest.java ---
@@ -18,227 +18,35 @@
package org.apache.flink.yarn;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import
org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
-import
org.apache.flink.runtime.clusterframework.messages.NotifyResourceStarted;
-import
org.apache.flink.runtime.clusterframework.messages.RegisterResourceManager;
-import
org.apache.flink.runtime.clusterframework.messages.RegisterResourceManagerSuccessful;
-import org.apache.flink.runtime.instance.AkkaActorGateway;
-import
org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.util.TestLogger;
-import org.apache.flink.yarn.messages.NotifyWhenResourcesRegistered;
-import org.apache.flink.yarn.messages.RequestNumberOfRegisteredResources;
-
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.actor.PoisonPill;
-import akka.actor.Props;
-import akka.testkit.JavaTestKit;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.client.api.AMRMClient;
-import org.apache.hadoop.yarn.client.api.NMClient;
-import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.Rule;
import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
+import org.junit.rules.TemporaryFolder;
-import java.util.ArrayList;
+import java.nio.file.Files;
+import java.nio.file.Path;
import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.TimeUnit;
-
-import scala.Option;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertThat;
/**
* Tests for {@link Utils}.
*/
-public class UtilsTest extends TestLogger {
+public class UtilsTest {
--- End diff --
`TestLogger` missing.
> YARN: Flink binaries are not deleted from HDFS after cluster shutdown
> ---------------------------------------------------------------------
>
> Key: FLINK-9196
> URL: https://issues.apache.org/jira/browse/FLINK-9196
> Project: Flink
> Issue Type: Bug
> Components: YARN
> Affects Versions: 1.5.0
> Reporter: Gary Yao
> Assignee: Gary Yao
> Priority: Blocker
> Labels: flip-6
> Fix For: 1.5.0
>
> Attachments: 0001-xxx.patch
>
>
> When deploying on YARN in flip6 mode, the Flink binaries are not deleted from
> HDFS after the cluster shuts down.
> *Steps to reproduce*
> # Submit job in YARN job mode, non-detached:
> {noformat} HADOOP_CLASSPATH=`hadoop classpath` bin/flink run -m yarn-cluster
> -yjm 2048 -ytm 2048 ./examples/streaming/WordCount.jar {noformat}
> # Check contents of {{/user/hadoop/.flink/<application_id>}} on HDFS after
> job is finished:
> {noformat}
> [hadoop@ip-172-31-43-78 flink-1.5.0]$ hdfs dfs -ls
> /user/hadoop/.flink/application_1523966184826_0016
> Found 6 items
> -rw-r--r-- 1 hadoop hadoop 583 2018-04-17 14:54
> /user/hadoop/.flink/application_1523966184826_0016/90cf5b3a-039e-4d52-8266-4e9563d74827-taskmanager-conf.yaml
> -rw-r--r-- 1 hadoop hadoop 332 2018-04-17 14:54
> /user/hadoop/.flink/application_1523966184826_0016/application_1523966184826_0016-flink-conf.yaml3818971235442577934.tmp
> -rw-r--r-- 1 hadoop hadoop 89779342 2018-04-02 17:08
> /user/hadoop/.flink/application_1523966184826_0016/flink-dist_2.11-1.5.0.jar
> drwxrwxrwx - hadoop hadoop 0 2018-04-17 14:54
> /user/hadoop/.flink/application_1523966184826_0016/lib
> -rw-r--r-- 1 hadoop hadoop 1939 2018-04-02 15:37
> /user/hadoop/.flink/application_1523966184826_0016/log4j.properties
> -rw-r--r-- 1 hadoop hadoop 2331 2018-04-02 15:37
> /user/hadoop/.flink/application_1523966184826_0016/logback.xml
> {noformat}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)