浏览代码

YARN-3391. Clearly define flow ID/ flow run / flow version in API and storage. Contributed by Zhijie Shen

(cherry picked from commit 68c6232f8423e55b4d152ef3d1d66aeb2d6a555e)
Junping Du 10 年之前
父节点
当前提交
47f35a30bb
共有 26 个文件被更改,包括 258 次插入129 次删除
  1. 25 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
  2. 7 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
  3. 30 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
  4. 11 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetTimelineCollectorContextResponse.java
  5. 25 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetTimelineCollectorContextResponsePBImpl.java
  6. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
  7. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
  8. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/NMCollectorService.java
  9. 13 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  10. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/Application.java
  11. 22 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
  12. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java
  13. 18 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockApp.java
  14. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
  15. 21 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  16. 14 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
  17. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
  18. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
  19. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
  20. 21 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorContext.java
  21. 9 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
  22. 7 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java
  23. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.java
  24. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
  25. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollectorManager.java
  26. 5 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java

+ 25 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java

@@ -189,8 +189,9 @@ public class Client {
   // Timeline domain writer access control
   // Timeline domain writer access control
   private String modifyACLs = null;
   private String modifyACLs = null;
 
 
-  private String flowId = null;
-  private String flowRunId = null;
+  private String flowName = null;
+  private String flowVersion = null;
+  private long flowRunId = 0L;
 
 
   // Command line options
   // Command line options
   private Options opts;
   private Options opts;
@@ -293,9 +294,11 @@ public class Client {
         + "modify the timeline entities in the given domain");
         + "modify the timeline entities in the given domain");
     opts.addOption("create", false, "Flag to indicate whether to create the "
     opts.addOption("create", false, "Flag to indicate whether to create the "
         + "domain specified with -domain.");
         + "domain specified with -domain.");
-    opts.addOption("flow", true, "ID of the flow which the distributed shell "
+    opts.addOption("flow_name", true, "Flow name which the distributed shell "
         + "app belongs to");
         + "app belongs to");
-    opts.addOption("flow_run", true, "ID of the flowrun which the distributed "
+    opts.addOption("flow_version", true, "Flow version which the distributed "
+        + "shell app belongs to");
+    opts.addOption("flow_run_id", true, "Flow run ID which the distributed "
         + "shell app belongs to");
         + "shell app belongs to");
     opts.addOption("help", false, "Print usage");
     opts.addOption("help", false, "Print usage");
     opts.addOption("node_label_expression", true,
     opts.addOption("node_label_expression", true,
@@ -486,11 +489,19 @@ public class Client {
           + cliParser.getOptionValue("container_retry_interval"));
           + cliParser.getOptionValue("container_retry_interval"));
     }
     }
 
 
-    if (cliParser.hasOption("flow")) {
-      flowId = cliParser.getOptionValue("flow");
+    if (cliParser.hasOption("flow_name")) {
+      flowName = cliParser.getOptionValue("flow_name");
+    }
+    if (cliParser.hasOption("flow_version")) {
+      flowVersion = cliParser.getOptionValue("flow_version");
     }
     }
-    if (cliParser.hasOption("flow_run")) {
-      flowRunId = cliParser.getOptionValue("flow_run");
+    if (cliParser.hasOption("flow_run_id")) {
+      try {
+        flowRunId = Long.valueOf(cliParser.getOptionValue("flow_run_id"));
+      } catch (NumberFormatException e) {
+        throw new IllegalArgumentException(
+            "Flow run is not a valid long value", e);
+      }
     }
     }
     return true;
     return true;
   }
   }
@@ -584,10 +595,13 @@ public class Client {
     }
     }
 
 
     Set<String> tags = new HashSet<String>();
     Set<String> tags = new HashSet<String>();
-    if (flowId != null) {
-      tags.add(TimelineUtils.generateFlowIdTag(flowId));
+    if (flowName != null) {
+      tags.add(TimelineUtils.generateFlowNameTag(flowName));
+    }
+    if (flowVersion != null) {
+      tags.add(TimelineUtils.generateFlowVersionTag(flowVersion));
     }
     }
-    if (flowRunId != null) {
+    if (flowRunId != 0) {
       tags.add(TimelineUtils.generateFlowRunIdTag(flowRunId));
       tags.add(TimelineUtils.generateFlowRunIdTag(flowRunId));
     }
     }
     appContext.setApplicationTags(tags);
     appContext.setApplicationTags(tags);

+ 7 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -332,9 +332,11 @@ public class TestDistributedShell {
       args = mergeArgs(args, timelineArgs);
       args = mergeArgs(args, timelineArgs);
       if (!defaultFlow) {
       if (!defaultFlow) {
         String[] flowArgs = {
         String[] flowArgs = {
-            "--flow",
-            "test_flow_id",
-            "--flow_run",
+            "--flow_name",
+            "test_flow_name",
+            "--flow_version",
+            "test_flow_version",
+            "--flow_run_id",
             "12345678"
             "12345678"
         };
         };
         args = mergeArgs(args, flowArgs);
         args = mergeArgs(args, flowArgs);
@@ -489,7 +491,8 @@ public class TestDistributedShell {
           UserGroupInformation.getCurrentUser().getShortUserName() +
           UserGroupInformation.getCurrentUser().getShortUserName() +
           (defaultFlow ? "/" +
           (defaultFlow ? "/" +
               TimelineUtils.generateDefaultFlowIdBasedOnAppId(appId) +
               TimelineUtils.generateDefaultFlowIdBasedOnAppId(appId) +
-              "/0/" : "/test_flow_id/12345678/") + appId.toString();
+              "/1/1/" : "/test_flow_name/test_flow_version/12345678/") +
+              appId.toString();
       // for this test, we expect DS_APP_ATTEMPT AND DS_CONTAINER dirs
       // for this test, we expect DS_APP_ATTEMPT AND DS_CONTAINER dirs
       String outputDirApp = basePath + "/DS_APP_ATTEMPT/";
       String outputDirApp = basePath + "/DS_APP_ATTEMPT/";
 
 
@@ -514,8 +517,6 @@ public class TestDistributedShell {
       String containerFileName = outputDirContainer + containerTimestampFileName;
       String containerFileName = outputDirContainer + containerTimestampFileName;
       File containerFile = new File(containerFileName);
       File containerFile = new File(containerFileName);
       Assert.assertTrue(containerFile.exists());
       Assert.assertTrue(containerFile.exists());
-      String appTimeStamp = appId.getClusterTimestamp() + "_" + appId.getId()
-          + "_";
 
 
       // Verify NM posting container metrics info.
       // Verify NM posting container metrics info.
       String outputDirContainerMetrics = basePath + "/" + 
       String outputDirContainerMetrics = basePath + "/" + 

+ 30 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java

@@ -44,7 +44,8 @@ import org.codehaus.jackson.map.ObjectMapper;
 @Evolving
 @Evolving
 public class TimelineUtils {
 public class TimelineUtils {
 
 
-  public static final String FLOW_ID_TAG_PREFIX = "TIMELINE_FLOW_ID_TAG";
+  public static final String FLOW_NAME_TAG_PREFIX = "TIMELINE_FLOW_NAME_TAG";
+  public static final String FLOW_VERSION_TAG_PREFIX = "TIMELINE_FLOW_VERSION_TAG";
   public static final String FLOW_RUN_ID_TAG_PREFIX = "TIMELINE_FLOW_RUN_ID_TAG";
   public static final String FLOW_RUN_ID_TAG_PREFIX = "TIMELINE_FLOW_RUN_ID_TAG";
 
 
   private static ObjectMapper mapper;
   private static ObjectMapper mapper;
@@ -163,11 +164,36 @@ public class TimelineUtils {
     return "flow_" + appId.getClusterTimestamp() + "_" + appId.getId();
     return "flow_" + appId.getClusterTimestamp() + "_" + appId.getId();
   }
   }
 
 
-  public static String generateFlowIdTag(String flowId) {
-    return FLOW_ID_TAG_PREFIX + ":" + flowId;
+  /**
+   * Generate flow name tag
+   *
+   * @param flowName flow name that identifies a distinct flow application which
+   *                 can be run repeatedly over time
+   * @return
+   */
+  public static String generateFlowNameTag(String flowName) {
+    return FLOW_NAME_TAG_PREFIX + ":" + flowName;
   }
   }
 
 
-  public static String generateFlowRunIdTag(String flowRunId) {
+  /**
+   * Generate flow version tag
+   *
+   * @param flowVersion flow version that keeps track of the changes made to the
+   *                    flow
+   * @return
+   */
+  public static String generateFlowVersionTag(String flowVersion) {
+    return FLOW_VERSION_TAG_PREFIX + ":" + flowVersion;
+  }
+
+  /**
+   * Generate flow run ID tag
+   *
+   * @param flowRunId flow run ID that identifies one instance (or specific
+   *                  execution) of that flow
+   * @return
+   */
+  public static String generateFlowRunIdTag(long flowRunId) {
     return FLOW_RUN_ID_TAG_PREFIX + ":" + flowRunId;
     return FLOW_RUN_ID_TAG_PREFIX + ":" + flowRunId;
   }
   }
 }
 }

+ 11 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetTimelineCollectorContextResponse.java

@@ -23,11 +23,12 @@ import org.apache.hadoop.yarn.util.Records;
 public abstract class GetTimelineCollectorContextResponse {
 public abstract class GetTimelineCollectorContextResponse {
 
 
   public static GetTimelineCollectorContextResponse newInstance(
   public static GetTimelineCollectorContextResponse newInstance(
-      String userId, String flowId, String flowRunId) {
+      String userId, String flowName, String flowVersion, long flowRunId) {
     GetTimelineCollectorContextResponse response =
     GetTimelineCollectorContextResponse response =
         Records.newRecord(GetTimelineCollectorContextResponse.class);
         Records.newRecord(GetTimelineCollectorContextResponse.class);
     response.setUserId(userId);
     response.setUserId(userId);
-    response.setFlowId(flowId);
+    response.setFlowName(flowName);
+    response.setFlowVersion(flowVersion);
     response.setFlowRunId(flowRunId);
     response.setFlowRunId(flowRunId);
     return response;
     return response;
   }
   }
@@ -36,11 +37,15 @@ public abstract class GetTimelineCollectorContextResponse {
 
 
   public abstract void setUserId(String userId);
   public abstract void setUserId(String userId);
 
 
-  public abstract String getFlowId();
+  public abstract String getFlowName();
 
 
-  public abstract void setFlowId(String flowId);
+  public abstract void setFlowName(String flowName);
 
 
-  public abstract String getFlowRunId();
+  public abstract String getFlowVersion();
 
 
-  public abstract void setFlowRunId(String flowRunId);
+  public abstract void setFlowVersion(String flowVersion);
+
+  public abstract long getFlowRunId();
+
+  public abstract void setFlowRunId(long flowRunId);
 }
 }

+ 25 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetTimelineCollectorContextResponsePBImpl.java

@@ -102,40 +102,52 @@ public class GetTimelineCollectorContextResponsePBImpl extends
   }
   }
 
 
   @Override
   @Override
-  public String getFlowId() {
+  public String getFlowName() {
     GetTimelineCollectorContextResponseProtoOrBuilder p = viaProto ? proto : builder;
     GetTimelineCollectorContextResponseProtoOrBuilder p = viaProto ? proto : builder;
-    if (!p.hasFlowId()) {
+    if (!p.hasFlowName()) {
       return null;
       return null;
     }
     }
-    return p.getFlowId();
+    return p.getFlowName();
   }
   }
 
 
   @Override
   @Override
-  public void setFlowId(String flowId) {
+  public void setFlowName(String flowName) {
     maybeInitBuilder();
     maybeInitBuilder();
-    if (flowId == null) {
-      builder.clearFlowId();
+    if (flowName == null) {
+      builder.clearFlowName();
       return;
       return;
     }
     }
-    builder.setFlowId(flowId);
+    builder.setFlowName(flowName);
   }
   }
 
 
   @Override
   @Override
-  public String getFlowRunId() {
+  public String getFlowVersion() {
     GetTimelineCollectorContextResponseProtoOrBuilder p = viaProto ? proto : builder;
     GetTimelineCollectorContextResponseProtoOrBuilder p = viaProto ? proto : builder;
-    if (!p.hasFlowRunId()) {
+    if (!p.hasFlowVersion()) {
       return null;
       return null;
     }
     }
-    return p.getFlowRunId();
+    return p.getFlowVersion();
   }
   }
 
 
   @Override
   @Override
-  public void setFlowRunId(String flowRunId) {
+  public void setFlowVersion(String flowVersion) {
     maybeInitBuilder();
     maybeInitBuilder();
-    if (flowRunId == null) {
-      builder.clearFlowRunId();
+    if (flowVersion == null) {
+      builder.clearFlowVersion();
       return;
       return;
     }
     }
+    builder.setFlowVersion(flowVersion);
+  }
+
+  @Override
+  public long getFlowRunId() {
+    GetTimelineCollectorContextResponseProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getFlowRunId();
+  }
+
+  @Override
+  public void setFlowRunId(long flowRunId) {
+    maybeInitBuilder();
     builder.setFlowRunId(flowRunId);
     builder.setFlowRunId(flowRunId);
   }
   }
 }
 }

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto

@@ -146,8 +146,9 @@ message GetTimelineCollectorContextRequestProto {
 
 
 message GetTimelineCollectorContextResponseProto {
 message GetTimelineCollectorContextResponseProto {
   optional string user_id = 1;
   optional string user_id = 1;
-  optional string flow_id = 2;
-  optional string flow_run_id = 3;
+  optional string flow_name = 2;
+  optional string flow_version = 3;
+  optional int64 flow_run_id = 4;
 }
 }
 
 
 message NMContainerStatusProto {
 message NMContainerStatusProto {

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java

@@ -180,8 +180,9 @@ public class TestRPC {
       GetTimelineCollectorContextResponse response =
       GetTimelineCollectorContextResponse response =
           proxy.getTimelineCollectorContext(request);
           proxy.getTimelineCollectorContext(request);
       Assert.assertEquals("test_user_id", response.getUserId());
       Assert.assertEquals("test_user_id", response.getUserId());
-      Assert.assertEquals("test_flow_id", response.getFlowId());
-      Assert.assertEquals("test_flow_run_id", response.getFlowRunId());
+      Assert.assertEquals("test_flow_name", response.getFlowName());
+      Assert.assertEquals("test_flow_version", response.getFlowVersion());
+      Assert.assertEquals(12345678L, response.getFlowRunId());
     } catch (YarnException | IOException e) {
     } catch (YarnException | IOException e) {
       Assert.fail("RPC call failured is not expected here.");
       Assert.fail("RPC call failured is not expected here.");
     }
     }
@@ -392,7 +393,7 @@ public class TestRPC {
         throws  YarnException, IOException {
         throws  YarnException, IOException {
       if (request.getApplicationId().getId() == 1) {
       if (request.getApplicationId().getId() == 1) {
          return GetTimelineCollectorContextResponse.newInstance(
          return GetTimelineCollectorContextResponse.newInstance(
-                "test_user_id", "test_flow_id", "test_flow_run_id");
+                "test_user_id", "test_flow_name", "test_flow_version", 12345678L);
       } else {
       } else {
         throw new YarnException("The application is not found.");
         throw new YarnException("The application is not found.");
       }
       }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/NMCollectorService.java

@@ -130,6 +130,6 @@ public class NMCollectorService extends CompositeService implements
           " doesn't exist on NM.");
           " doesn't exist on NM.");
     }
     }
     return GetTimelineCollectorContextResponse.newInstance(
     return GetTimelineCollectorContextResponse.newInstance(
-        app.getUser(), app.getFlowId(), app.getFlowRunId());
+        app.getUser(), app.getFlowName(), app.getFlowVersion(), app.getFlowRunId());
   }
   }
 }
 }

+ 13 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java

@@ -334,8 +334,8 @@ public class ContainerManagerImpl extends CompositeService implements
     LOG.info("Recovering application " + appId);
     LOG.info("Recovering application " + appId);
     //TODO: Recover flow and flow run ID
     //TODO: Recover flow and flow run ID
     ApplicationImpl app = new ApplicationImpl(
     ApplicationImpl app = new ApplicationImpl(
-        dispatcher, p.getUser(), null, null, appId, creds, context, p.getAppLogAggregationInitedTime());
-
+        dispatcher, p.getUser(), null, null, 0L, appId, creds, context, 
+        p.getAppLogAggregationInitedTime());
     context.getApplications().put(appId, app);
     context.getApplications().put(appId, app);
     app.handle(new ApplicationInitEvent(appId, acls, logAggregationContext));
     app.handle(new ApplicationInitEvent(appId, acls, logAggregationContext));
   }
   }
@@ -954,12 +954,18 @@ public class ContainerManagerImpl extends CompositeService implements
     try {
     try {
       if (!isServiceStopped()) {
       if (!isServiceStopped()) {
         // Create the application
         // Create the application
-        String flowId = launchContext.getEnvironment().get(
-            TimelineUtils.FLOW_ID_TAG_PREFIX);
-        String flowRunId = launchContext.getEnvironment().get(
+        String flowName = launchContext.getEnvironment().get(
+            TimelineUtils.FLOW_NAME_TAG_PREFIX);
+        String flowVersion = launchContext.getEnvironment().get(
+            TimelineUtils.FLOW_VERSION_TAG_PREFIX);
+        String flowRunIdStr = launchContext.getEnvironment().get(
             TimelineUtils.FLOW_RUN_ID_TAG_PREFIX);
             TimelineUtils.FLOW_RUN_ID_TAG_PREFIX);
-        Application application = new ApplicationImpl(
-            dispatcher, user, flowId, flowRunId, applicationID, credentials, context);
+        long flowRunId = 0L;
+        if (flowRunIdStr != null && !flowRunIdStr.isEmpty()) {
+          flowRunId = Long.valueOf(flowRunIdStr);
+        }
+        Application application = new ApplicationImpl(dispatcher, user,
+            flowName, flowVersion, flowRunId, applicationID, credentials, context);
         if (null == context.getApplications().putIfAbsent(applicationID,
         if (null == context.getApplications().putIfAbsent(applicationID,
           application)) {
           application)) {
           LOG.info("Creating a new application reference for app "
           LOG.info("Creating a new application reference for app "

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/Application.java

@@ -36,9 +36,11 @@ public interface Application extends EventHandler<ApplicationEvent> {
 
 
   ApplicationState getApplicationState();
   ApplicationState getApplicationState();
 
 
-  String getFlowId();
+  String getFlowName();
 
 
-  String getFlowRunId();
+  String getFlowVersion();
+
+  long getFlowRunId();
   
   
   TimelineClient getTimelineClient();
   TimelineClient getTimelineClient();
 
 

+ 22 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java

@@ -74,8 +74,9 @@ public class ApplicationImpl implements Application {
 
 
   final Dispatcher dispatcher;
   final Dispatcher dispatcher;
   final String user;
   final String user;
-  final String flowId;
-  final String flowRunId;
+  final String flowName;
+  final String flowVersion;
+  final long flowRunId;
   final ApplicationId appId;
   final ApplicationId appId;
   final Credentials credentials;
   final Credentials credentials;
   Map<ApplicationAccessType, String> applicationACLs;
   Map<ApplicationAccessType, String> applicationACLs;
@@ -101,13 +102,14 @@ public class ApplicationImpl implements Application {
   private long applicationLogInitedTimestamp = -1;
   private long applicationLogInitedTimestamp = -1;
   private final NMStateStoreService appStateStore;
   private final NMStateStoreService appStateStore;
 
 
-  public ApplicationImpl(Dispatcher dispatcher, String user, String flowId,
-      String flowRunId, ApplicationId appId, Credentials credentials,
-      Context context, long recoveredLogInitedTime)  {
-
+  public ApplicationImpl(Dispatcher dispatcher, String user, String flowName,
+      String flowVersion, long flowRunId, ApplicationId appId,
+      Credentials credentials, Context context,
+      long recoveredLogInitedTime) {
     this.dispatcher = dispatcher;
     this.dispatcher = dispatcher;
     this.user = user;
     this.user = user;
-    this.flowId = flowId;
+    this.flowName = flowName;
+    this.flowVersion = flowVersion;
     this.flowRunId = flowRunId;
     this.flowRunId = flowRunId;
     this.appId = appId;
     this.appId = appId;
     this.credentials = credentials;
     this.credentials = credentials;
@@ -122,9 +124,9 @@ public class ApplicationImpl implements Application {
   }
   }
 
 
   public ApplicationImpl(Dispatcher dispatcher, String user, String flowId,
   public ApplicationImpl(Dispatcher dispatcher, String user, String flowId,
-      String flowRunId, ApplicationId appId, Credentials credentials,
-      Context context) {
-    this(dispatcher, user, flowId, flowRunId, appId, credentials,
+      String flowVersion, long flowRunId, ApplicationId appId,
+      Credentials credentials, Context context) {
+    this(dispatcher, user, flowId, flowVersion, flowRunId, appId, credentials,
       context, -1);
       context, -1);
     Configuration conf = context.getConf();
     Configuration conf = context.getConf();
     if (YarnConfiguration.systemMetricsPublisherEnabled(conf)) {
     if (YarnConfiguration.systemMetricsPublisherEnabled(conf)) {
@@ -591,11 +593,18 @@ public class ApplicationImpl implements Application {
     }
     }
   }
   }
 
 
-  public String getFlowId() {
-    return flowId;
+  @Override
+  public String getFlowName() {
+    return flowName;
   }
   }
 
 
-  public String getFlowRunId() {
+  @Override
+  public String getFlowVersion() {
+    return flowVersion;
+  }
+
+  @Override
+  public long getFlowRunId() {
     return flowRunId;
     return flowRunId;
   }
   }
 }
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java

@@ -551,7 +551,7 @@ public class TestApplication {
       this.appId = BuilderUtils.newApplicationId(timestamp, id);
       this.appId = BuilderUtils.newApplicationId(timestamp, id);
 
 
       app = new ApplicationImpl(
       app = new ApplicationImpl(
-          dispatcher, this.user, null, null, appId, null, context);
+          dispatcher, this.user, null, null, 0, appId, null, context);
       containers = new ArrayList<Container>();
       containers = new ArrayList<Container>();
       for (int i = 0; i < numContainers; i++) {
       for (int i = 0; i < numContainers; i++) {
         Container container = createMockedContainer(this.appId, i);
         Container container = createMockedContainer(this.appId, i);

+ 18 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockApp.java

@@ -40,8 +40,9 @@ public class MockApp implements Application {
   Map<ContainerId, Container> containers = new HashMap<ContainerId, Container>();
   Map<ContainerId, Container> containers = new HashMap<ContainerId, Container>();
   ApplicationState appState;
   ApplicationState appState;
   Application app;
   Application app;
-  String flowId;
-  String flowRunId;
+  String flowName;
+  String flowVersion;
+  long flowRunId;
   TimelineClient timelineClient = null;
   TimelineClient timelineClient = null;
 
 
   public MockApp(int uniqId) {
   public MockApp(int uniqId) {
@@ -59,6 +60,14 @@ public class MockApp implements Application {
     appState = ApplicationState.NEW;
     appState = ApplicationState.NEW;
   }
   }
 
 
+  public MockApp(String user, long clusterTimeStamp, int uniqId,
+      String flowName, String flowVersion, long flowRunId) {
+    this(user, clusterTimeStamp, uniqId);
+    this.flowName = flowName;
+    this.flowVersion = flowVersion;
+    this.flowRunId = flowRunId;
+  }
+
   public void setState(ApplicationState state) {
   public void setState(ApplicationState state) {
     this.appState = state;
     this.appState = state;
   }
   }
@@ -81,11 +90,15 @@ public class MockApp implements Application {
 
 
   public void handle(ApplicationEvent event) {}
   public void handle(ApplicationEvent event) {}
 
 
-  public String getFlowId() {
-    return flowId;
+  public String getFlowName() {
+    return flowName;
+  }
+
+  public String getFlowVersion() {
+    return flowVersion;
   }
   }
 
 
-  public String getFlowRunId() {
+  public long getFlowRunId() {
     return flowRunId;
     return flowRunId;
   }
   }
   
   

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java

@@ -339,7 +339,7 @@ public class TestNMWebServices extends JerseyTestBase {
     final String filename = "logfile1";
     final String filename = "logfile1";
     final String logMessage = "log message\n";
     final String logMessage = "log message\n";
     nmContext.getApplications().put(appId, new ApplicationImpl(null, "user",
     nmContext.getApplications().put(appId, new ApplicationImpl(null, "user",
-        null, null, appId, null, nmContext));
+        null, null, 0, appId, null, nmContext));
     
     
     MockContainer container = new MockContainer(appAttemptId,
     MockContainer container = new MockContainer(appAttemptId,
         new AsyncDispatcher(), new Configuration(), "user", appId, 1);
         new AsyncDispatcher(), new Configuration(), "user", appId, 1);

+ 21 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -173,6 +173,7 @@ import org.apache.hadoop.yarn.util.UTCClock;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.SettableFuture;
 import com.google.common.util.concurrent.SettableFuture;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 
 
 
 
 /**
 /**
@@ -583,6 +584,26 @@ public class ClientRMService extends AbstractService implements
       throw RPCUtil.getRemoteException(ie);
       throw RPCUtil.getRemoteException(ie);
     }
     }
 
 
+    // Sanity check for flow run
+    String value = null;
+    try {
+      for (String tag : submissionContext.getApplicationTags()) {
+        if (tag.startsWith(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX + ":") ||
+            tag.startsWith(
+                TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.toLowerCase() + ":")) {
+          value = tag.substring(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.length() + 1);
+          Long.valueOf(value);
+        }
+      }
+    } catch (NumberFormatException e) {
+      LOG.warn("Invalid to flow run: " + value +
+          ". Flow run should be a long integer", e);
+      RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_APP_REQUEST,
+          e.getMessage(), "ClientRMService",
+          "Exception in submitting application", applicationId);
+      throw RPCUtil.getRemoteException(e);
+    }
+
     // Check whether app has already been put into rmContext,
     // Check whether app has already been put into rmContext,
     // If it is, simply return the response
     // If it is, simply return the response
     if (rmContext.getRMApps().get(applicationId) != null) {
     if (rmContext.getRMApps().get(applicationId) != null) {

+ 14 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java

@@ -232,22 +232,9 @@ public class AMLauncher implements Runnable {
     // Set flow context info
     // Set flow context info
     for (String tag :
     for (String tag :
         rmContext.getRMApps().get(applicationId).getApplicationTags()) {
         rmContext.getRMApps().get(applicationId).getApplicationTags()) {
-      if (tag.startsWith(TimelineUtils.FLOW_ID_TAG_PREFIX  + ":") ||
-          tag.startsWith(TimelineUtils.FLOW_ID_TAG_PREFIX.toLowerCase() + ":")) {
-        String value = tag.substring(
-            TimelineUtils.FLOW_ID_TAG_PREFIX.length() + 1);
-        if (!value.isEmpty()) {
-          environment.put(TimelineUtils.FLOW_ID_TAG_PREFIX, value);
-        }
-      }
-      if (tag.startsWith(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX  + ":") ||
-          tag.startsWith(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.toLowerCase() + ":")) {
-        String value = tag.substring(
-            TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.length() + 1);
-        if (!value.isEmpty()) {
-          environment.put(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX, value);
-        }
-      }
+      setFlowTags(environment, TimelineUtils.FLOW_NAME_TAG_PREFIX, tag);
+      setFlowTags(environment, TimelineUtils.FLOW_VERSION_TAG_PREFIX, tag);
+      setFlowTags(environment, TimelineUtils.FLOW_RUN_ID_TAG_PREFIX, tag);
     }
     }
     Credentials credentials = new Credentials();
     Credentials credentials = new Credentials();
     DataInputByteBuffer dibb = new DataInputByteBuffer();
     DataInputByteBuffer dibb = new DataInputByteBuffer();
@@ -269,6 +256,17 @@ public class AMLauncher implements Runnable {
     container.setTokens(ByteBuffer.wrap(dob.getData(), 0, dob.getLength()));
     container.setTokens(ByteBuffer.wrap(dob.getData(), 0, dob.getLength()));
   }
   }
 
 
+  private static void setFlowTags(
+      Map<String, String> environment, String tagPrefix, String tag) {
+    if (tag.startsWith(tagPrefix + ":") ||
+        tag.startsWith(tagPrefix.toLowerCase() + ":")) {
+      String value = tag.substring(tagPrefix.length() + 1);
+      if (!value.isEmpty()) {
+        environment.put(tagPrefix, value);
+      }
+    }
+  }
+
   @VisibleForTesting
   @VisibleForTesting
   protected Token<AMRMTokenIdentifier> createAndSetAMRMToken() {
   protected Token<AMRMTokenIdentifier> createAndSetAMRMToken() {
     Token<AMRMTokenIdentifier> amrmToken =
     Token<AMRMTokenIdentifier> amrmToken =

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java

@@ -97,7 +97,7 @@ public class TestTimelineServiceClientIntegration {
           mock(CollectorNodemanagerProtocol.class);
           mock(CollectorNodemanagerProtocol.class);
       try {
       try {
         GetTimelineCollectorContextResponse response =
         GetTimelineCollectorContextResponse response =
-            GetTimelineCollectorContextResponse.newInstance(null, null, null);
+            GetTimelineCollectorContextResponse.newInstance(null, null, null, 0L);
         when(protocol.getTimelineCollectorContext(any(
         when(protocol.getTimelineCollectorContext(any(
             GetTimelineCollectorContextRequest.class))).thenReturn(response);
             GetTimelineCollectorContextRequest.class))).thenReturn(response);
       } catch (YarnException | IOException e) {
       } catch (YarnException | IOException e) {

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java

@@ -54,10 +54,12 @@ public class AppLevelTimelineCollector extends TimelineCollector {
     // context info from NM.
     // context info from NM.
     // Current user usually is not the app user, but keep this field non-null
     // Current user usually is not the app user, but keep this field non-null
     context.setUserId(UserGroupInformation.getCurrentUser().getShortUserName());
     context.setUserId(UserGroupInformation.getCurrentUser().getShortUserName());
-    // Use app ID to generate a default flow ID for orphan app
-    context.setFlowId(TimelineUtils.generateDefaultFlowIdBasedOnAppId(appId));
-    // Set the flow run ID to 0 if it's an orphan app
-    context.setFlowRunId("0");
+    // Use app ID to generate a default flow name for orphan app
+    context.setFlowName(TimelineUtils.generateDefaultFlowIdBasedOnAppId(appId));
+    // Set the flow version to string 1 if it's an orphan app
+    context.setFlowVersion("1");
+    // Set the flow run ID to 1 if it's an orphan app
+    context.setFlowRunId(1L);
     context.setAppId(appId.toString());
     context.setAppId(appId.toString());
     super.serviceInit(conf);
     super.serviceInit(conf);
   }
   }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java

@@ -100,8 +100,8 @@ public abstract class TimelineCollector extends CompositeService {
 
 
     TimelineCollectorContext context = getTimelineEntityContext();
     TimelineCollectorContext context = getTimelineEntityContext();
     return writer.write(context.getClusterId(), context.getUserId(),
     return writer.write(context.getClusterId(), context.getUserId(),
-        context.getFlowId(), context.getFlowRunId(), context.getAppId(),
-        entities);
+        context.getFlowName(), context.getFlowVersion(), context.getFlowRunId(),
+        context.getAppId(), entities);
   }
   }
 
 
   /**
   /**

+ 21 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorContext.java

@@ -22,19 +22,21 @@ public class TimelineCollectorContext {
 
 
   private String clusterId;
   private String clusterId;
   private String userId;
   private String userId;
-  private String flowId;
-  private String flowRunId;
+  private String flowName;
+  private String flowVersion;
+  private long flowRunId;
   private String appId;
   private String appId;
 
 
   public TimelineCollectorContext() {
   public TimelineCollectorContext() {
-    this(null, null, null, null, null);
+    this(null, null, null, null, 0L, null);
   }
   }
 
 
   public TimelineCollectorContext(String clusterId, String userId,
   public TimelineCollectorContext(String clusterId, String userId,
-      String flowId, String flowRunId, String appId) {
+      String flowName, String flowVersion, long flowRunId, String appId) {
     this.clusterId = clusterId;
     this.clusterId = clusterId;
     this.userId = userId;
     this.userId = userId;
-    this.flowId = flowId;
+    this.flowName = flowName;
+    this.flowVersion = flowVersion;
     this.flowRunId = flowRunId;
     this.flowRunId = flowRunId;
     this.appId = appId;
     this.appId = appId;
   }
   }
@@ -55,19 +57,27 @@ public class TimelineCollectorContext {
     this.userId = userId;
     this.userId = userId;
   }
   }
 
 
-  public String getFlowId() {
-    return flowId;
+  public String getFlowName() {
+    return flowName;
   }
   }
 
 
-  public void setFlowId(String flowId) {
-    this.flowId = flowId;
+  public void setFlowName(String flowName) {
+    this.flowName = flowName;
   }
   }
 
 
-  public String getFlowRunId() {
+  public String getFlowVersion() {
+    return flowVersion;
+  }
+
+  public void setFlowVersion(String flowVersion) {
+    this.flowVersion = flowVersion;
+  }
+
+  public long getFlowRunId() {
     return flowRunId;
     return flowRunId;
   }
   }
 
 
-  public void setFlowRunId(String flowRunId) {
+  public void setFlowRunId(long flowRunId) {
     this.flowRunId = flowRunId;
     this.flowRunId = flowRunId;
   }
   }
 
 

+ 9 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java

@@ -35,7 +35,6 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.http.lib.StaticUserWebFilter;
 import org.apache.hadoop.http.lib.StaticUserWebFilter;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -273,12 +272,16 @@ public class TimelineCollectorManager extends CompositeService {
     if (userId != null && !userId.isEmpty()) {
     if (userId != null && !userId.isEmpty()) {
       collector.getTimelineEntityContext().setUserId(userId);
       collector.getTimelineEntityContext().setUserId(userId);
     }
     }
-    String flowId = response.getFlowId();
-    if (flowId != null && !flowId.isEmpty()) {
-      collector.getTimelineEntityContext().setFlowId(flowId);
+    String flowName = response.getFlowName();
+    if (flowName != null && !flowName.isEmpty()) {
+      collector.getTimelineEntityContext().setFlowName(flowName);
     }
     }
-    String flowRunId = response.getFlowRunId();
-    if (flowRunId != null && !flowRunId.isEmpty()) {
+    String flowVersion = response.getFlowVersion();
+    if (flowVersion != null && !flowVersion.isEmpty()) {
+      collector.getTimelineEntityContext().setFlowVersion(flowVersion);
+    }
+    long flowRunId = response.getFlowRunId();
+    if (flowRunId != 0L) {
       collector.getTimelineEntityContext().setFlowRunId(flowRunId);
       collector.getTimelineEntityContext().setFlowRunId(flowRunId);
     }
     }
   }
   }

+ 7 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java

@@ -65,22 +65,23 @@ public class FileSystemTimelineWriterImpl extends AbstractService
 
 
   @Override
   @Override
   public TimelineWriteResponse write(String clusterId, String userId,
   public TimelineWriteResponse write(String clusterId, String userId,
-      String flowId, String flowRunId, String appId,
+      String flowName, String flowVersion, long flowRunId, String appId,
       TimelineEntities entities) throws IOException {
       TimelineEntities entities) throws IOException {
     TimelineWriteResponse response = new TimelineWriteResponse();
     TimelineWriteResponse response = new TimelineWriteResponse();
     for (TimelineEntity entity : entities.getEntities()) {
     for (TimelineEntity entity : entities.getEntities()) {
-      write(clusterId, userId, flowId, flowRunId, appId, entity, response);
+      write(clusterId, userId, flowName, flowVersion, flowRunId, appId, entity,
+          response);
     }
     }
     return response;
     return response;
   }
   }
 
 
-  private void write(String clusterId, String userId,
-      String flowId, String flowRunId, String appId, TimelineEntity entity,
+  private void write(String clusterId, String userId, String flowName,
+      String flowVersion, long flowRun, String appId, TimelineEntity entity,
       TimelineWriteResponse response) throws IOException {
       TimelineWriteResponse response) throws IOException {
     PrintWriter out = null;
     PrintWriter out = null;
     try {
     try {
-      String dir = mkdirs(outputRoot, ENTITIES_DIR, clusterId, userId,flowId,
-          flowRunId, appId, entity.getType());
+      String dir = mkdirs(outputRoot, ENTITIES_DIR, clusterId, userId,flowName,
+          flowVersion, String.valueOf(flowRun), appId, entity.getType());
       String fileName = dir + entity.getId() + TIMELINE_SERVICE_STORAGE_EXTENSION;
       String fileName = dir + entity.getId() + TIMELINE_SERVICE_STORAGE_EXTENSION;
       out = new PrintWriter(new BufferedWriter(new FileWriter(fileName, true)));
       out = new PrintWriter(new BufferedWriter(new FileWriter(fileName, true)));
       out.println(TimelineUtils.dumpTimelineRecordtoJSON(entity));
       out.println(TimelineUtils.dumpTimelineRecordtoJSON(entity));

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.java

@@ -41,8 +41,9 @@ public interface TimelineWriter extends Service {
    *
    *
    * @param clusterId context cluster ID
    * @param clusterId context cluster ID
    * @param userId context user ID
    * @param userId context user ID
-   * @param flowId context flow ID
-   * @param flowRunId context flow run ID
+   * @param flowName context flow name
+   * @param flowVersion context flow version
+   * @param flowRunId
    * @param appId context app ID
    * @param appId context app ID
    * @param data
    * @param data
    *          a {@link TimelineEntities} object.
    *          a {@link TimelineEntities} object.
@@ -50,7 +51,7 @@ public interface TimelineWriter extends Service {
    * @throws IOException
    * @throws IOException
    */
    */
   TimelineWriteResponse write(String clusterId, String userId,
   TimelineWriteResponse write(String clusterId, String userId,
-      String flowId, String flowRunId, String appId,
+      String flowName, String flowVersion, long flowRunId, String appId,
       TimelineEntities data) throws IOException;
       TimelineEntities data) throws IOException;
 
 
   /**
   /**

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java

@@ -162,7 +162,7 @@ public class TestPerNodeTimelineCollectorsAuxService {
     CollectorNodemanagerProtocol nmCollectorService =
     CollectorNodemanagerProtocol nmCollectorService =
         mock(CollectorNodemanagerProtocol.class);
         mock(CollectorNodemanagerProtocol.class);
     GetTimelineCollectorContextResponse response =
     GetTimelineCollectorContextResponse response =
-        GetTimelineCollectorContextResponse.newInstance(null, null, null);
+        GetTimelineCollectorContextResponse.newInstance(null, null, null, 0L);
     try {
     try {
       when(nmCollectorService.getTimelineCollectorContext(any(
       when(nmCollectorService.getTimelineCollectorContext(any(
           GetTimelineCollectorContextRequest.class))).thenReturn(response);
           GetTimelineCollectorContextRequest.class))).thenReturn(response);

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollectorManager.java

@@ -1,3 +1,4 @@
+
 /**
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * or more contributor license agreements.  See the NOTICE file
@@ -146,7 +147,7 @@ public class TestTimelineCollectorManager {
     CollectorNodemanagerProtocol nmCollectorService =
     CollectorNodemanagerProtocol nmCollectorService =
         mock(CollectorNodemanagerProtocol.class);
         mock(CollectorNodemanagerProtocol.class);
     GetTimelineCollectorContextResponse response =
     GetTimelineCollectorContextResponse response =
-        GetTimelineCollectorContextResponse.newInstance(null, null, null);
+        GetTimelineCollectorContextResponse.newInstance(null, null, null, 0L);
     try {
     try {
       when(nmCollectorService.getTimelineCollectorContext(any(
       when(nmCollectorService.getTimelineCollectorContext(any(
           GetTimelineCollectorContextRequest.class))).thenReturn(response);
           GetTimelineCollectorContextRequest.class))).thenReturn(response);

+ 5 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java

@@ -57,11 +57,13 @@ public class TestFileSystemTimelineWriterImpl {
       fsi = new FileSystemTimelineWriterImpl();
       fsi = new FileSystemTimelineWriterImpl();
       fsi.init(new YarnConfiguration());
       fsi.init(new YarnConfiguration());
       fsi.start();
       fsi.start();
-      fsi.write("cluster_id", "user_id", "flow_id", "flow_run_id", "app_id", te);
+      fsi.write("cluster_id", "user_id", "flow_name", "flow_version", 12345678L,
+          "app_id", te);
 
 
       String fileName = fsi.getOutputRoot() +
       String fileName = fsi.getOutputRoot() +
-          "/entities/cluster_id/user_id/flow_id/flow_run_id/app_id/" + type +
-          "/" + id + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
+          "/entities/cluster_id/user_id/flow_name/flow_version/12345678/app_id/" +
+          type + "/" + id +
+          FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
       Path path = Paths.get(fileName);
       Path path = Paths.get(fileName);
       File f = new File(fileName);
       File f = new File(fileName);
       assertTrue(f.exists() && !f.isDirectory());
       assertTrue(f.exists() && !f.isDirectory());