You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by mo...@apache.org on 2017/09/25 21:41:03 UTC

reef git commit: [REEF-1878] Allow multiple tokens to pass in from client

Repository: reef
Updated Branches:
  refs/heads/master 9fb6cc41a -> 85bae60ab


[REEF-1878]  Allow multiple tokens to pass in from client

   * Added SecurityTokenWriter at .Net side
   * Added SecurityTOkenWriter at Java side
   * Added named parameters to allow .Net clients to pass tokens
   * Modified Java side to allow multiple tokens to be pass in
   * Support backward compitability
   * Added tests.

JIRA: [REEF-1878](https://issues.apache.org/jira/browse/REEF-1878)

This closes #1372


Project: http://git-wip-us.apache.org/repos/asf/reef/repo
Commit: http://git-wip-us.apache.org/repos/asf/reef/commit/85bae60a
Tree: http://git-wip-us.apache.org/repos/asf/reef/tree/85bae60a
Diff: http://git-wip-us.apache.org/repos/asf/reef/diff/85bae60a

Branch: refs/heads/master
Commit: 85bae60ab0b988b143ddd49f8f4295475ebdf49c
Parents: 9fb6cc4
Author: Julia Wang <ju...@microsoft.com>
Authored: Fri Sep 1 19:05:53 2017 -0700
Committer: Sergiy Matusevych <mo...@apache.com>
Committed: Mon Sep 25 12:14:48 2017 -0700

----------------------------------------------------------------------
 lang/common/token/avro/SecurityToken.avsc       |  52 ++++
 .../Avro/YARN/SecurityToken.cs                  |  87 +++++++
 .../Org.Apache.REEF.Client.csproj               |   3 +
 .../YARN/Parameters/SecurityTokenParameters.cs  |   8 +-
 .../YARN/Parameters/SecurityTokenStrings.cs     |  30 +++
 .../YARN/SecurityTokenWriter.cs                 |  81 +++++++
 .../YARN/YARNClientConfiguration.cs             |   3 +
 .../YARN/YARNREEFClient.cs                      |   2 +
 .../YARN/YarnREEFParamSerializer.cs             |  53 +++-
 .../Files/REEFFileNames.cs                      |  15 ++
 .../HelloREEFYarn.cs                            |  24 +-
 .../Org.Apache.REEF.Examples.HelloREEF.csproj   |   4 +
 .../packages.config                             |   1 +
 .../Functional/Bridge/TestSecurityToken.cs      | 240 +++++++++++++++++++
 .../Org.Apache.REEF.Tests.csproj                |   9 +-
 lang/java/reef-bridge-client/pom.xml            |  12 +
 .../bridge/client/SecurityTokensReader.java     |  82 +++++++
 .../bridge/client/YarnJobSubmissionClient.java  | 106 +++++---
 .../runtime/common/files/REEFFileNames.java     |  29 ++-
 19 files changed, 793 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/common/token/avro/SecurityToken.avsc
----------------------------------------------------------------------
diff --git a/lang/common/token/avro/SecurityToken.avsc b/lang/common/token/avro/SecurityToken.avsc
new file mode 100644
index 0000000..5072ff1
--- /dev/null
+++ b/lang/common/token/avro/SecurityToken.avsc
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+[
+  /**
+   * Avro schema for serialized security token
+   */
+  {
+    "namespace": "org.apache.reef.bridge.client",
+
+    "name": "SecurityToken",
+    "doc":  "Security token",
+    "type": "record",
+    "fields": [
+      {
+        "name": "kind",
+        "doc":  "Kind of the security token",
+        "type": "string"
+      },
+      {
+        "name": "service",
+        "doc":  "Token service name",
+        "type": "string"
+      },
+      {
+        "name": "key",
+        "type": "bytes",
+        "doc":  "Token key"
+      },
+      {
+        "name": "password",
+        "type": "bytes",
+        "doc":  "Token password"
+      }
+    ]
+  }
+]

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Client/Avro/YARN/SecurityToken.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Avro/YARN/SecurityToken.cs b/lang/cs/Org.Apache.REEF.Client/Avro/YARN/SecurityToken.cs
new file mode 100644
index 0000000..199a361
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Avro/YARN/SecurityToken.cs
@@ -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.
+
+namespace Org.Apache.REEF.Client.Avro.YARN
+{
+    using System.Runtime.Serialization;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.bridge.client.SecurityToken.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.bridge.client")]
+    public partial class SecurityToken
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.bridge.client.SecurityToken"",""doc"":""Security token"",""fields"":[{""name"":""kind"",""doc"":""Kind of the security token"",""type"":""string""},{""name"":""service"",""doc"":""Token service name"",""type"":""string""},{""name"":""key"",""doc"":""Token key"",""type"":""bytes""},{""name"":""password"",""doc"":""Token password"",""type"":""bytes""}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+      
+        /// <summary>
+        /// Gets or sets the kind field.
+        /// </summary>
+        [DataMember]
+        public string kind { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the service field.
+        /// </summary>
+        [DataMember]
+        public string service { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the key field.
+        /// </summary>
+        [DataMember]
+        public byte[] key { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the password field.
+        /// </summary>
+        [DataMember]
+        public byte[] password { get; set; }
+                
+        /// <summary>
+        /// Initializes a new instance of the <see cref="SecurityToken"/> class.
+        /// </summary>
+        public SecurityToken()
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="SecurityToken"/> class.
+        /// </summary>
+        /// <param name="kind">The kind.</param>
+        /// <param name="service">The service.</param>
+        /// <param name="key">The key.</param>
+        /// <param name="password">The password.</param>
+        public SecurityToken(string kind, string service, byte[] key, byte[] password)
+        {
+            this.kind = kind;
+            this.service = service;
+            this.key = key;
+            this.password = password;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj b/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
index 791af4f..7748895 100644
--- a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
+++ b/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
@@ -119,6 +119,7 @@ under the License.
     <Compile Include="YARN\Parameters\DriverStderrFilePath.cs" />
     <Compile Include="YARN\Parameters\DriverStdoutFilePath.cs" />
     <Compile Include="YARN\Parameters\FileSystemUrl.cs" />
+    <Compile Include="YARN\Parameters\SecurityTokenStrings.cs" />
     <Compile Include="YARN\RestClient\DataModel\KillApplication.cs" />
     <Compile Include="YARN\RestClient\HttpClient.cs" />
     <Compile Include="YARN\RestClient\IDeserializer.cs" />
@@ -136,6 +137,8 @@ under the License.
     <Compile Include="YARN\RestClient\RestRequest.cs" />
     <Compile Include="YARN\RestClient\RestResponse.cs" />
     <Compile Include="YARN\RestClient\HttpClientRetryHandler.cs" />
+    <Compile Include="Avro\YARN\SecurityToken.cs" />
+    <Compile Include="YARN\SecurityTokenWriter.cs" />
     <Compile Include="YARN\WindowsYarnJobCommandProvider.cs" />
     <Compile Include="YARN\JobResource.cs" />
     <Compile Include="YARN\JobSubmissionDirectoryProvider.cs" />

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Client/YARN/Parameters/SecurityTokenParameters.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/Parameters/SecurityTokenParameters.cs b/lang/cs/Org.Apache.REEF.Client/YARN/Parameters/SecurityTokenParameters.cs
index a6a8092..88845a8 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/Parameters/SecurityTokenParameters.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/Parameters/SecurityTokenParameters.cs
@@ -19,13 +19,17 @@ using Org.Apache.REEF.Tang.Annotations;
 
 namespace Org.Apache.REEF.Client.YARN.Parameters
 {
-    [NamedParameter("Security token kind.", defaultValue: "NULL")]
+    [System.Obsolete("TODO[JIRA REEF-1887] Deprecated. Remove in REEF 0.18.")]
+    [NamedParameter("Security token kind.", defaultValue: DefaultTokenKind)]
     public sealed class SecurityTokenKindParameter : Name<string>
     {
+        public const string DefaultTokenKind = "NULL";
     }
 
-    [NamedParameter("Security token service name.", defaultValue: "NULL")]
+    [System.Obsolete("TODO[JIRA REEF-1887] Deprecated. Remove in REEF 0.18.")]
+    [NamedParameter("Security token service name.", defaultValue: DefaultService)]
     public sealed class SecurityTokenServiceParameter : Name<string>
     {
+        public const string DefaultService = "NULL";
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Client/YARN/Parameters/SecurityTokenStrings.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/Parameters/SecurityTokenStrings.cs b/lang/cs/Org.Apache.REEF.Client/YARN/Parameters/SecurityTokenStrings.cs
new file mode 100644
index 0000000..88ad649
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/Parameters/SecurityTokenStrings.cs
@@ -0,0 +1,30 @@
+// 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.
+
+using System.Collections.Generic;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.YARN.Parameters
+{
+    /// <summary>
+    /// Named parameter that contains a set of serialized tokens.
+    /// </summary>
+    [NamedParameter("Serialized SurityToken Info", "SecurityTokenStrings")]
+    public class SecurityTokenStrings : Name<ISet<string>>
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Client/YARN/SecurityTokenWriter.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/SecurityTokenWriter.cs b/lang/cs/Org.Apache.REEF.Client/YARN/SecurityTokenWriter.cs
new file mode 100644
index 0000000..060064c
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/SecurityTokenWriter.cs
@@ -0,0 +1,81 @@
+// 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.
+
+using System.Collections.Generic;
+using System.IO;
+using System.Linq;
+using Newtonsoft.Json;
+using Org.Apache.REEF.Client.YARN.Parameters;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using Microsoft.Hadoop.Avro;
+using Org.Apache.REEF.Client.Avro.YARN;
+using Org.Apache.REEF.Common.Files;
+
+namespace Org.Apache.REEF.Client.YARN
+{
+    /// <summary>
+    /// Serialize a set of security tokens into a file.
+    /// </summary>
+    internal class SecurityTokenWriter
+    {
+        private static readonly Logger Logger = Logger.GetLogger(typeof(SecurityTokenWriter));
+
+        private readonly IAvroSerializer<SecurityToken> _avroSerializer = AvroSerializer.Create<SecurityToken>();
+        private readonly List<SecurityToken> _tokens;
+
+        private readonly string _securityTokensFile;
+
+        /// <summary>
+        /// Injectable constructor that accepts a set of serialized tokens.
+        /// Each serialized token string in the set is a serialized SecurityToken by JsonConvert 
+        /// </summary>
+        /// <param name="serializedTokenStrings">Serialized token strings</param>
+        /// <param name="reefFileNames">REEF file name constants</param>
+        [Inject]
+        private SecurityTokenWriter(REEFFileNames reefFileNames,
+            [Parameter(typeof(SecurityTokenStrings))] ISet<string> serializedTokenStrings)
+        {
+            _securityTokensFile = reefFileNames.GetSecurityTokenFileName();
+            _tokens = serializedTokenStrings.Select(serializedToken =>
+            {
+                var token = JsonConvert.DeserializeObject<SecurityToken>(serializedToken);
+                return new SecurityToken(token.kind, token.service, token.key, token.password);
+            }).ToList();
+        }
+
+        /// <summary>
+        /// Write SecurityToken objects to SecurityTokenFile using IAvroSerializer.
+        /// </summary>
+        public void WriteTokensToFile()
+        {
+            Logger.Log(Level.Verbose, "Write {0} tokens to file: {1}.", _tokens.Count, _securityTokensFile);
+
+            if (_tokens.Count > 0)
+            {
+                using (var stream = File.OpenWrite(_securityTokensFile))
+                {
+                    foreach (var token in _tokens)
+                    {
+                        Logger.Log(Level.Verbose, "Write token {0} to file.", token.kind);
+                        _avroSerializer.Serialize(stream, token);
+                    }
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs
index 69d715e..34e48da 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs
@@ -34,6 +34,7 @@ namespace Org.Apache.REEF.Client.Yarn
         public static readonly OptionalParameter<string> SecurityTokenKind = new OptionalParameter<string>();
         public static readonly OptionalParameter<string> SecurityTokenService = new OptionalParameter<string>();
         public static readonly OptionalImpl<IYarnRestClientCredential> YarnRestClientCredential = new OptionalImpl<IYarnRestClientCredential>();
+        public static readonly OptionalParameter<string> SecurityTokenStr = new OptionalParameter<string>();
 
         /// <summary>
         /// URL for store. For Hadoop file system, it is set in fs.defaultFS as default by YARN environment. Client doesn't need to 
@@ -49,6 +50,7 @@ namespace Org.Apache.REEF.Client.Yarn
             .BindNamedParameter(GenericType<JobSubmissionDirectoryPrefixParameter>.Class, JobSubmissionFolderPrefix)
             .BindNamedParameter(GenericType<SecurityTokenKindParameter>.Class, SecurityTokenKind)
             .BindNamedParameter(GenericType<SecurityTokenServiceParameter>.Class, SecurityTokenService)
+            .BindSetEntry(GenericType<SecurityTokenStrings>.Class, SecurityTokenStr)
             .BindNamedParameter(GenericType<FileSystemUrl>.Class, FileSystemUrl)
             .Build();
 
@@ -61,6 +63,7 @@ namespace Org.Apache.REEF.Client.Yarn
             .BindNamedParameter(GenericType<SecurityTokenKindParameter>.Class, SecurityTokenKind)
             .BindNamedParameter(GenericType<SecurityTokenServiceParameter>.Class, SecurityTokenService)
             .BindNamedParameter(GenericType<FileSystemUrl>.Class, FileSystemUrl)
+            .BindSetEntry(GenericType<SecurityTokenStrings>.Class, SecurityTokenStr)
             .Build();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
index 4559c73..b4e5ad7 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
@@ -158,6 +158,8 @@ namespace Org.Apache.REEF.Client.Yarn
         {
             _driverFolderPreparationHelper.PrepareDriverFolder(jobRequest.AppParameters, driverFolderPath);
 
+            _paramSerializer.WriteSecurityTokens();
+
             // TODO: Remove this when we have a generalized way to pass config to java
             var paramInjector = TangFactory.GetTang().NewInjector(jobRequest.DriverConfigurations.ToArray());
             var submissionJobArgsFilePath = _paramSerializer.SerializeJobFile(jobRequest.JobParameters, paramInjector, driverFolderPath);

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFParamSerializer.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFParamSerializer.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFParamSerializer.cs
index bcaf6fd..c09eaad 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFParamSerializer.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFParamSerializer.cs
@@ -19,13 +19,13 @@ using System.IO;
 using Org.Apache.REEF.Client.API;
 using Org.Apache.REEF.Client.Avro;
 using Org.Apache.REEF.Client.Avro.YARN;
-using Org.Apache.REEF.Client.Yarn;
 using Org.Apache.REEF.Client.YARN.Parameters;
 using Org.Apache.REEF.Common.Avro;
 using Org.Apache.REEF.Common.Files;
 using Org.Apache.REEF.Driver.Bridge;
 using Org.Apache.REEF.Tang.Annotations;
 using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Utilities.Logging;
 using Org.Apache.REEF.Wake.Remote.Parameters;
 
 namespace Org.Apache.REEF.Client.YARN
@@ -35,24 +35,61 @@ namespace Org.Apache.REEF.Client.YARN
     /// </summary>
     internal sealed class YarnREEFParamSerializer
     {
+        private static readonly Logger Logger = Logger.GetLogger(typeof(YarnREEFParamSerializer));
+
         private readonly REEFFileNames _fileNames;
+
+        /// <summary>
+        /// Security token kind name. Used for single token case.
+        /// </summary>
+        [System.Obsolete("TODO[JIRA REEF-1887] Deprecated. Remove in REEF 0.18.")]
         private readonly string _securityTokenKind;
+
+        /// <summary>
+        /// Security token service name. Used for single token case.
+        /// </summary>
+        [System.Obsolete("TODO[JIRA REEF-1887] Deprecated. Remove in REEF 0.18.")]
         private readonly string _securityTokenService;
+
+        /// <summary>
+        /// File system URL
+        /// </summary>
         private readonly string _fileSystemUrl;
+
+        /// <summary>
+        /// Job submission folder relative path
+        /// </summary>
         private readonly string _jobSubmissionPrefix;
 
+        /// <summary>
+        /// Security token writer that parses and writes token information.
+        /// It can process multiple tokens.
+        /// </summary>
+        private readonly SecurityTokenWriter _securityTokenWriter;
+
+        /// <summary>
+        /// Serialize parameters and tokens for Java client.
+        /// </summary>
+        /// <param name="fileNames">REEF file name class which contains file names.</param>
+        /// <param name="securityTokenWriter">SecurityTokenWriter which writes security token info.</param>
+        /// <param name="securityTokenKind">Security token kind name.</param>
+        /// <param name="securityTokenService">Security token service name.</param>
+        /// <param name="fileSystemUrl">File system URL.</param>
+        /// <param name="jobSubmissionPrefix">Job submission folder. e.g: fileSystemUrl/jobSubmissionPrefix/</param>
         [Inject]
         private YarnREEFParamSerializer(
             REEFFileNames fileNames,
+            SecurityTokenWriter securityTokenWriter,
             [Parameter(typeof(SecurityTokenKindParameter))] string securityTokenKind,
             [Parameter(typeof(SecurityTokenServiceParameter))] string securityTokenService,
             [Parameter(typeof(FileSystemUrl))] string fileSystemUrl,
             [Parameter(typeof(JobSubmissionDirectoryPrefixParameter))] string jobSubmissionPrefix)
         {
             _fileNames = fileNames;
+            _securityTokenWriter = securityTokenWriter;
             _jobSubmissionPrefix = jobSubmissionPrefix;
-            _securityTokenKind = securityTokenKind;
             _fileSystemUrl = fileSystemUrl;
+            _securityTokenKind = securityTokenKind;
             _securityTokenService = securityTokenService;
         }
 
@@ -123,9 +160,11 @@ namespace Org.Apache.REEF.Client.YARN
 
             var avroYarnClusterJobSubmissionParameters = new AvroYarnClusterJobSubmissionParameters
             {
-                yarnJobSubmissionParameters = avroYarnJobSubmissionParameters,
+                // TODO[JIRA REEF-1887] Deprecated. Remove in REEF 0.18.
                 securityTokenKind = _securityTokenKind,
                 securityTokenService = _securityTokenService,
+
+                yarnJobSubmissionParameters = avroYarnJobSubmissionParameters,
                 driverMemory = jobParameters.DriverMemoryInMB,
                 maxApplicationSubmissions = jobParameters.MaxApplicationSubmissions,
                 driverStdoutFilePath = string.IsNullOrWhiteSpace(jobParameters.StdoutFilePath.Value) ?
@@ -136,5 +175,13 @@ namespace Org.Apache.REEF.Client.YARN
 
             return AvroJsonSerializer<AvroYarnClusterJobSubmissionParameters>.ToBytes(avroYarnClusterJobSubmissionParameters);
         }
+
+        /// <summary>
+        /// Write Token info.
+        /// </summary>
+        internal void WriteSecurityTokens()
+        {
+            _securityTokenWriter.WriteTokensToFile();
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Common/Files/REEFFileNames.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Files/REEFFileNames.cs b/lang/cs/Org.Apache.REEF.Common/Files/REEFFileNames.cs
index dd5be54..5cb272e 100644
--- a/lang/cs/Org.Apache.REEF.Common/Files/REEFFileNames.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Files/REEFFileNames.cs
@@ -15,6 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
+using System;
 using System.Diagnostics.CodeAnalysis;
 using System.IO;
 using Org.Apache.REEF.Tang.Annotations;
@@ -53,6 +54,7 @@ namespace Org.Apache.REEF.Common.Files
         private const string BRIDGE_EXE_CONFIG_NAME = "Org.Apache.REEF.Bridge.exe.config";
         private const string SECURITY_TOKEN_IDENTIFIER_FILE = "SecurityTokenId";
         private const string SECURITY_TOKEN_PASSWORD_FILE = "SecurityTokenPwd";
+        private const string SECURITY_TOKEN_FILE = "SecurityTokens.json";
         private const string APP_SUBMISSION_PARAMETERS_FILE = "app-submission-params.json";
         private const string JOB_SUBMISSION_PARAMETERS_FILE = "job-submission-params.json";
         private const string YARN_DEFAULT_DRIVER_OUT_VAR = "<LOG_DIR>";
@@ -297,6 +299,7 @@ namespace Org.Apache.REEF.Common.Files
         /// The filename for security token identifier
         /// </summary>
         /// <returns>filename which contains raw bytes of security token identifier</returns>
+        [Obsolete("TODO[JIRA REEF-1887] Use GetSecurityTokenFileName() for consolidated token id and password information. Remove in REEF 0.18.")]
         [Unstable("0.13", "Security token should be handled by .NET only REEF client in the future")]
         public string GetSecurityTokenIdentifierFileName()
         {
@@ -307,6 +310,7 @@ namespace Org.Apache.REEF.Common.Files
         /// The filename for security token password
         /// </summary>
         /// <returns>filename which contains raw bytes of security token password</returns>
+        [Obsolete("TODO[JIRA REEF-1887] Use GetSecurityTokenFileName() for consolidated token id and password information. Remove in REEF 0.18.")]
         [Unstable("0.13", "Security token should be handled by .NET only REEF client in the future")]
         public string GetSecurityTokenPasswordFileName()
         {
@@ -314,6 +318,17 @@ namespace Org.Apache.REEF.Common.Files
         }
 
         /// <summary>
+        /// File name for security token information.
+        /// TODO[JIRA REEF-1887] It supersedes GetSecurityTokenPasswordFileName() and GetSecurityTokenIdentifierFileName().
+        /// Remove this comment line when REEF-1887 is done.
+        /// </summary>
+        /// <returns>Returns security token file name.</returns>
+        public string GetSecurityTokenFileName()
+        {
+            return SECURITY_TOKEN_FILE;
+        }
+
+        /// <summary>
         /// The file name of the PID file created in the current working directory of the process.
         /// This is similar to the file name in the PIDStoreHandler.java
         /// </summary>

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
index b7564f6..917e3eb 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
@@ -16,10 +16,12 @@
 // under the License.
 
 using System.Collections.Generic;
-using System.IO;
 using System.Linq;
+using System.Text;
 using System.Threading;
+using Newtonsoft.Json;
 using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Client.Avro.YARN;
 using Org.Apache.REEF.Client.Common;
 using Org.Apache.REEF.Client.Yarn;
 using Org.Apache.REEF.Client.YARN;
@@ -30,6 +32,7 @@ using Org.Apache.REEF.Tang.Implementations.Configuration;
 using Org.Apache.REEF.Tang.Implementations.Tang;
 using Org.Apache.REEF.Tang.Interface;
 using Org.Apache.REEF.Tang.Util;
+using Org.Apache.REEF.Utilities;
 using Org.Apache.REEF.Utilities.Logging;
 
 namespace Org.Apache.REEF.Examples.HelloREEF
@@ -155,23 +158,24 @@ namespace Org.Apache.REEF.Examples.HelloREEF
         /// <summary>
         /// Get runtime configuration
         /// </summary>
-        /// <returns></returns>
         private static IConfiguration GetRuntimeConfiguration(string[] args)
         {
-            var c = YARNClientConfiguration.ConfigurationModule
-                .Set(YARNClientConfiguration.SecurityTokenKind, TrustedApplicationTokenIdentifier)
-                .Set(YARNClientConfiguration.SecurityTokenService, TrustedApplicationTokenIdentifier)
+            var token = new SecurityToken(
+                TrustedApplicationTokenIdentifier,
+                TrustedApplicationTokenIdentifier,
+                ByteUtilities.StringToByteArrays(args[0]),
+                Encoding.ASCII.GetBytes(args[1]));
+
+            var clientConfig = YARNClientConfiguration.ConfigurationModule
+                .Set(YARNClientConfiguration.SecurityTokenStr, JsonConvert.SerializeObject(token))
                 .Build();
 
-            File.WriteAllText(SecurityTokenId, args[0]);
-            File.WriteAllText(SecurityTokenPwd, args[1]);
-
-            IConfiguration tcpPortConfig = TcpPortConfigurationModule.ConfigurationModule
+            var tcpPortConfig = TcpPortConfigurationModule.ConfigurationModule
                 .Set(TcpPortConfigurationModule.PortRangeStart, args.Length > 2 ? args[2] : DefaultPortRangeStart)
                 .Set(TcpPortConfigurationModule.PortRangeCount, args.Length > 3 ? args[3] : DefaultPortRangeCount)
                 .Build();
 
-            return Configurations.Merge(c, tcpPortConfig);
+            return Configurations.Merge(clientConfig, tcpPortConfig);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/Org.Apache.REEF.Examples.HelloREEF.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/Org.Apache.REEF.Examples.HelloREEF.csproj b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/Org.Apache.REEF.Examples.HelloREEF.csproj
index be7fed1..2cce0ee 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/Org.Apache.REEF.Examples.HelloREEF.csproj
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/Org.Apache.REEF.Examples.HelloREEF.csproj
@@ -18,6 +18,10 @@
     <UseVSHostingProcess>false</UseVSHostingProcess>
   </PropertyGroup>
   <ItemGroup>
+    <Reference Include="Newtonsoft.Json, Version=10.0.0.0, Culture=neutral, PublicKeyToken=30ad4fe6b2a6aeed, processorArchitecture=MSIL">
+      <HintPath>..\packages\Newtonsoft.Json.10.0.3\lib\net45\Newtonsoft.Json.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="System" />
     <Reference Include="System.Core" />
     <Reference Include="System.Xml.Linq" />

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/packages.config b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/packages.config
index d952982..13ca20f 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/packages.config
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/packages.config
@@ -19,4 +19,5 @@ under the License.
 -->
 <packages>
   <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Tests/Functional/Bridge/TestSecurityToken.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Functional/Bridge/TestSecurityToken.cs b/lang/cs/Org.Apache.REEF.Tests/Functional/Bridge/TestSecurityToken.cs
new file mode 100644
index 0000000..8e833fe
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tests/Functional/Bridge/TestSecurityToken.cs
@@ -0,0 +1,240 @@
+// 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.
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using System.Threading;
+using Newtonsoft.Json;
+using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Client.Avro.YARN;
+using Org.Apache.REEF.Client.Common;
+using Org.Apache.REEF.Client.Yarn;
+using Org.Apache.REEF.Client.YARN.RestClient.DataModel;
+using Org.Apache.REEF.Common.Files;
+using Org.Apache.REEF.Driver;
+using Org.Apache.REEF.Examples.HelloREEF;
+using Org.Apache.REEF.Tang.Implementations.Configuration;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+using Org.Apache.REEF.Utilities;
+using Org.Apache.REEF.Utilities.Logging;
+using Xunit;
+
+namespace Org.Apache.REEF.Tests.Functional.Bridge
+{
+    /// <summary>
+    /// Test set security tokens
+    /// </summary>
+    [Collection("FunctionalTests")]
+    public class TestSecurityToken
+    {
+        private static readonly Logger Logger = Logger.GetLogger(typeof(TestSecurityToken));
+
+        private const string DefaultPortRangeStart = "2000";
+        private const string DefaultPortRangeCount = "20";
+
+        private const string Identifier1 = "TrustedApplicationTokenIdentifier";
+        private const string Identifier2 = "TrustedApplicationTokenIdentifier2";
+        private const string TokenKey1 = "TrustedApplication001";
+        private const string TokenKey2 = "TrustedApplication002";
+        private const string Password1 = "none";
+        private const string Password2 = "none";
+
+        /// <summary>
+        /// This is to test pass multiple tokens from client for Yarn application
+        /// </summary>
+        [Fact]
+        [Trait("Environment", "Yarn")]
+        [Trait("Priority", "1")]
+        [Trait("Description", "Run CLR Test on Yarn")]
+        public void TestSetMultipleSecurityToken()
+        {
+            TestRun(GetRuntimeConfigurationForMultipleTokens());
+        }
+
+        /// <summary>
+        /// This is to test write token in old approach for backward compatibility checking
+        /// </summary>
+        [Fact]
+        [Trait("Environment", "Yarn")]
+        [Trait("Priority", "1")]
+        [Trait("Description", "Run CLR Test on Yarn")]
+        [Obsolete("TODO[JIRA REEF-1887] Remove in REEF 0.18.")]
+        public void TestSecurityTokenBackwardCompatibility()
+        {
+            TestRun(GetRuntimeConfigurationBackwardComp());
+        }
+
+        /// <summary>
+        /// This is to test passing one token from client for Yarn application
+        /// </summary>
+        [Fact]
+        [Trait("Environment", "Yarn")]
+        [Trait("Priority", "1")]
+        [Trait("Description", "Run CLR Test on Yarn")]
+        public void TestSetOneSecurityToken()
+        {
+            TestRun(GetRuntimeConfigurationForSingleToken());
+        }
+
+        /// <summary>
+        /// Test run for the runtime in the given injector.
+        /// </summary>
+        /// <param name="config">runtime configuration.</param>
+        private void TestRun(IConfiguration config)
+        {
+            IInjector injector = TangFactory.GetTang().NewInjector(config);
+
+            var reefClient = injector.GetInstance<IREEFClient>();
+            var jobRequestBuilder = injector.GetInstance<JobRequestBuilder>();
+
+            var jobSubmission = jobRequestBuilder
+                .AddDriverConfiguration(GetDriverConfig())
+                .AddGlobalAssemblyForType(typeof(HelloDriver))
+                .SetJobIdentifier("MyTestJob")
+                .SetJavaLogLevel(JavaLoggingSetting.Verbose)
+                .Build();
+
+            var result = reefClient.SubmitAndGetJobStatus(jobSubmission);
+            var state = PullFinalJobStatus(result);
+            Logger.Log(Level.Info, "Application final state : {0}.", state);
+            Assert.Equal(FinalState.SUCCEEDED, state);
+        }
+
+        /// <summary>
+        /// Use HelloDriver in the test
+        /// </summary>
+        /// <returns>Return driver configuration.</returns>
+        private IConfiguration GetDriverConfig()
+        {
+            return DriverConfiguration.ConfigurationModule
+                .Set(DriverConfiguration.OnEvaluatorAllocated, GenericType<HelloDriver>.Class)
+                .Set(DriverConfiguration.OnDriverStarted, GenericType<HelloDriver>.Class)
+                .Build();
+        }
+
+        /// <summary>
+        /// Pull job final status until the Job is done
+        /// </summary>
+        /// <param name="jobSubmitionResult"></param>
+        /// <returns>Return final state of the job.</returns>
+        private FinalState PullFinalJobStatus(IJobSubmissionResult jobSubmitionResult)
+        {
+            int n = 0;
+            var state = jobSubmitionResult.FinalState;
+            while (state.Equals(FinalState.UNDEFINED) && n++ < 100)
+            {
+                Thread.Sleep(3000);
+                state = jobSubmitionResult.FinalState;
+            }
+            return state;
+        }
+
+        /// <summary>
+        /// Get runtime configuration.
+        /// Bind tokens to YarnClientCnfiguration.
+        /// </summary>
+        /// <returns>Return runtime configuration for multiple tokens.</returns>
+        private static IConfiguration GetRuntimeConfigurationForMultipleTokens()
+        {
+            var yarnClientConfigModule = YARNClientConfiguration.ConfigurationModule;
+            foreach (var t in CreateTestTokens())
+            {
+                yarnClientConfigModule = yarnClientConfigModule.Set(YARNClientConfiguration.SecurityTokenStr, t);
+            }
+
+            return Configurations.Merge(yarnClientConfigModule.Build(), TcpPortConfig());
+        }
+
+        /// <summary>
+        /// Get runtime configuration.
+        /// Bind token to YarnClientCnfiguration.
+        /// </summary>
+        /// <returns>Return runtime configuration for single token.</returns>
+        private static IConfiguration GetRuntimeConfigurationForSingleToken()
+        {
+            var yarnClientConfigModule = YARNClientConfiguration.ConfigurationModule
+                .Set(YARNClientConfiguration.SecurityTokenStr, CreateTestToken());
+
+            return Configurations.Merge(yarnClientConfigModule.Build(), TcpPortConfig());
+        }
+
+        /// <summary>
+        /// Get runtime configuration and token with old approach
+        /// </summary>
+        /// <returns>Return runtime configuration for old approach.</returns>
+        [Obsolete("TODO[JIRA REEF-1887] Remove in REEF 0.18.")]
+        private static IConfiguration GetRuntimeConfigurationBackwardComp()
+        {
+            var reefFileNames = TangFactory.GetTang().NewInjector().GetInstance<REEFFileNames>();
+
+            var c = YARNClientConfiguration.ConfigurationModule
+                .Set(YARNClientConfiguration.SecurityTokenKind, Identifier1)
+                .Set(YARNClientConfiguration.SecurityTokenService, Identifier1)
+                .Build();
+
+            File.WriteAllText(reefFileNames.GetSecurityTokenIdentifierFileName(), TokenKey1);
+            File.WriteAllText(reefFileNames.GetSecurityTokenPasswordFileName(), Password1);
+
+            return Configurations.Merge(c, TcpPortConfig());
+        }
+
+        private static IConfiguration TcpPortConfig()
+        {
+            var tcpPortConfig = TcpPortConfigurationModule.ConfigurationModule
+                .Set(TcpPortConfigurationModule.PortRangeStart, DefaultPortRangeStart)
+                .Set(TcpPortConfigurationModule.PortRangeCount, DefaultPortRangeCount)
+                .Build();
+            return tcpPortConfig;
+        }
+
+        internal static IList<string> CreateTestTokens()
+        {
+            var t1 = new SecurityToken(
+                Identifier1,
+                Identifier1,
+                ByteUtilities.StringToByteArrays(TokenKey1),
+                Encoding.ASCII.GetBytes(Password1));
+
+            var t2 = new SecurityToken(
+                Identifier2,
+                Identifier2,
+                ByteUtilities.StringToByteArrays(TokenKey2),
+                Encoding.ASCII.GetBytes(Password2));
+
+            return new List<string>()
+            {
+                JsonConvert.SerializeObject(t1),
+                JsonConvert.SerializeObject(t2)
+            };
+        }
+
+        internal static string CreateTestToken()
+        {
+            var t1 = new SecurityToken(
+                Identifier1,
+                Identifier1,
+                ByteUtilities.StringToByteArrays(TokenKey1),
+                Encoding.ASCII.GetBytes(Password1));
+
+            return JsonConvert.SerializeObject(t1);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj b/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
index a4301b7..252bf75 100644
--- a/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?xml version="1.0" encoding="utf-8"?>
 <!--
 Licensed to the Apache Software Foundation (ASF) under one
 or more contributor license agreements.  See the NOTICE file
@@ -83,6 +83,7 @@ under the License.
     <Compile Include="Functional\Bridge\TestContextStack.cs" />
     <Compile Include="Functional\Bridge\TestDriverConcurrency.cs" />
     <Compile Include="Functional\Bridge\TestFailedEvaluatorEventHandler.cs" />
+    <Compile Include="Functional\Bridge\TestSecurityToken.cs" />
     <Compile Include="Functional\Common\Task\ExceptionTask.cs" />
     <Compile Include="Functional\Common\Task\NullTask.cs" />
     <Compile Include="Functional\Failure\User\ContextStopExceptionTest.cs" />
@@ -236,6 +237,10 @@ under the License.
       <Project>{dec0f0a8-dbef-4ebf-b69c-e2369c15abf1}</Project>
       <Name>Org.Apache.REEF.IO</Name>
     </ProjectReference>
+    <ProjectReference Include="$(SolutionDir)\Org.Apache.REEF.Examples.HelloREEF\Org.Apache.REEF.Examples.HelloREEF.csproj">
+      <Project>{0ff8cee9-b0b6-4a14-9a52-44441be048fe}</Project>
+      <Name>Org.Apache.REEF.Examples.HelloREEF</Name>
+    </ProjectReference>
   </ItemGroup>
   <ItemGroup>
     <Service Include="{82A7F48D-3B50-4B1E-B82E-3ADA8210C358}" />
@@ -244,4 +249,4 @@ under the License.
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />
   <Import Project="$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets" Condition="Exists('$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets')" />
-</Project>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/java/reef-bridge-client/pom.xml
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/pom.xml b/lang/java/reef-bridge-client/pom.xml
index 7621cc1..7bc6a28 100644
--- a/lang/java/reef-bridge-client/pom.xml
+++ b/lang/java/reef-bridge-client/pom.xml
@@ -132,6 +132,7 @@ under the License.
                 <artifactId>avro-maven-plugin</artifactId>
                 <executions>
                     <execution>
+                        <id>generate-local</id>
                         <phase>generate-sources</phase>
                         <goals>
                             <goal>schema</goal>
@@ -141,6 +142,17 @@ under the License.
                             <outputDirectory>${project.basedir}/target/generated-sources/avro/</outputDirectory>
                         </configuration>
                     </execution>
+                    <execution>
+                        <id>generate-token-info</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>schema</goal>
+                        </goals>
+                        <configuration>
+                            <sourceDirectory>${rootPath}/lang/common/token/avro/</sourceDirectory>
+                            <outputDirectory>${project.basedir}/target/generated-sources/avro/</outputDirectory>
+                        </configuration>
+                    </execution>
                 </executions>
             </plugin>
             <plugin>

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/SecurityTokensReader.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/SecurityTokensReader.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/SecurityTokensReader.java
new file mode 100644
index 0000000..a1d908e
--- /dev/null
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/SecurityTokensReader.java
@@ -0,0 +1,82 @@
+/*
+ * 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.reef.bridge.client;
+
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.reef.runtime.common.files.REEFFileNames;
+
+import javax.inject.Inject;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Helper class to read security tokens from file and add them to the user's credentials.
+ * .Net SecurityTokenWriter stores the token info in a file and passes it to the java client.
+ * The method in this class will read this data and add tokens to UserGroupInformation.
+ */
+final class SecurityTokensReader {
+
+  private static final Logger LOG = Logger.getLogger(SecurityTokensReader.class.getName());
+
+  private final DatumReader<SecurityToken> tokenDatumReader = new SpecificDatumReader<>(SecurityToken.class);
+  private final DecoderFactory decoderFactory = new DecoderFactory();
+  private final File securityTokensFile;
+
+  @Inject
+  private SecurityTokensReader(final REEFFileNames reefFileNames) {
+    this.securityTokensFile = new File(reefFileNames.getSecurityTokensFile());
+  }
+
+  /**
+   * Read tokens from a file and add them to the user's credentials.
+   * @param ugi user's credentials to add tokens to.
+   * @throws IOException if there are errors in reading the tokens' file.
+   */
+  void addTokensFromFile(final UserGroupInformation ugi) throws IOException {
+    LOG.log(Level.FINE, "Reading security tokens from file: {0}", this.securityTokensFile);
+
+    try (final FileInputStream stream = new FileInputStream(securityTokensFile)) {
+      final BinaryDecoder decoder = decoderFactory.binaryDecoder(stream, null);
+
+      while (!decoder.isEnd()) {
+        final SecurityToken token = tokenDatumReader.read(null, decoder);
+
+        final Token<TokenIdentifier> yarnToken = new Token<>(
+            token.getKey().array(),
+            token.getPassword().array(),
+            new Text(token.getKind().toString()),
+            new Text(token.getService().toString()));
+
+        LOG.log(Level.FINE, "addToken for {0}", yarnToken.getKind());
+
+        ugi.addToken(yarnToken);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/YarnJobSubmissionClient.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/YarnJobSubmissionClient.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/YarnJobSubmissionClient.java
index fcf03ba..0d71517 100644
--- a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/YarnJobSubmissionClient.java
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/YarnJobSubmissionClient.java
@@ -18,6 +18,7 @@
  */
 package org.apache.reef.bridge.client;
 
+import net.jcip.annotations.Immutable;
 import org.apache.commons.lang.Validate;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -25,6 +26,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -62,7 +64,7 @@ import java.util.logging.Logger;
 /**
  * The Java-side of the C# YARN Job Submission API.
  */
-@SuppressWarnings("checkstyle:hideutilityclassconstructor")
+@Immutable
 public final class YarnJobSubmissionClient {
 
   private static final Logger LOG = Logger.getLogger(YarnJobSubmissionClient.class.getName());
@@ -76,17 +78,21 @@ public final class YarnJobSubmissionClient {
   private final YarnProxyUser yarnProxyUser;
   private final SecurityTokenProvider tokenProvider;
   private final YarnSubmissionParametersFileGenerator jobSubmissionParametersGenerator;
+  private final SecurityTokensReader securityTokensReader;
+  private static final String DEFAULT_TOKEN_KIND = "NULL";
 
   @Inject
-  YarnJobSubmissionClient(@Parameter(DriverIsUnmanaged.class) final boolean isUnmanaged,
-                          @Parameter(DriverLaunchCommandPrefix.class) final List<String> commandPrefixList,
-                          final JobUploader uploader,
-                          final YarnConfiguration yarnConfiguration,
-                          final REEFFileNames fileNames,
-                          final ClasspathProvider classpath,
-                          final YarnProxyUser yarnProxyUser,
-                          final SecurityTokenProvider tokenProvider,
-                          final YarnSubmissionParametersFileGenerator jobSubmissionParametersGenerator) {
+  private YarnJobSubmissionClient(
+      @Parameter(DriverIsUnmanaged.class) final boolean isUnmanaged,
+      @Parameter(DriverLaunchCommandPrefix.class) final List<String> commandPrefixList,
+      final JobUploader uploader,
+      final YarnConfiguration yarnConfiguration,
+      final REEFFileNames fileNames,
+      final ClasspathProvider classpath,
+      final YarnProxyUser yarnProxyUser,
+      final SecurityTokenProvider tokenProvider,
+      final YarnSubmissionParametersFileGenerator jobSubmissionParametersGenerator,
+      final SecurityTokensReader securityTokensReader) {
 
     this.isUnmanaged = isUnmanaged;
     this.commandPrefixList = commandPrefixList;
@@ -97,6 +103,7 @@ public final class YarnJobSubmissionClient {
     this.yarnProxyUser = yarnProxyUser;
     this.tokenProvider = tokenProvider;
     this.jobSubmissionParametersGenerator = jobSubmissionParametersGenerator;
+    this.securityTokensReader = securityTokensReader;
   }
 
   /**
@@ -117,6 +124,9 @@ public final class YarnJobSubmissionClient {
   }
 
   private void launch(final YarnClusterSubmissionFromCS yarnSubmission) throws IOException, YarnException {
+
+    this.setupCredentials(yarnSubmission);
+
     // ------------------------------------------------------------------------
     // Get an application ID
     try (final YarnSubmissionHelper submissionHelper = new YarnSubmissionHelper(
@@ -167,20 +177,64 @@ public final class YarnJobSubmissionClient {
     }
   }
 
-  private static void writeSecurityTokenToUserCredential(
+  private void setupCredentials(
       final YarnClusterSubmissionFromCS yarnSubmission) throws IOException {
-    final UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
-    final REEFFileNames fileNames = new REEFFileNames();
+
+    final File securityTokensFile = new File(fileNames.getSecurityTokensFile());
+
+    if (securityTokensFile.exists()) {
+      LOG.log(Level.INFO, "Writing security tokens to user credential");
+      this.addTokensToCurrentUser();
+    } else if (!yarnSubmission.getTokenKind().equalsIgnoreCase(DEFAULT_TOKEN_KIND)) {
+      // To support backward compatibility
+      LOG.log(Level.INFO, "Writing security token to user credential");
+      this.writeSecurityTokenToUserCredential(yarnSubmission);
+    } else {
+      LOG.log(Level.FINE, "Did not find security token");
+    }
+  }
+
+  private void writeSecurityTokenToUserCredential(
+      final YarnClusterSubmissionFromCS yarnSubmission) throws IOException {
+
     final String securityTokenIdentifierFile = fileNames.getSecurityTokenIdentifierFile();
     final String securityTokenPasswordFile = fileNames.getSecurityTokenPasswordFile();
     final Text tokenKind = new Text(yarnSubmission.getTokenKind());
     final Text tokenService = new Text(yarnSubmission.getTokenService());
-    byte[] identifier = Files.readAllBytes(Paths.get(securityTokenIdentifierFile));
-    byte[] password = Files.readAllBytes(Paths.get(securityTokenPasswordFile));
-    Token token = new Token(identifier, password, tokenKind, tokenService);
+    final byte[] identifier = Files.readAllBytes(Paths.get(securityTokenIdentifierFile));
+    final byte[] password = Files.readAllBytes(Paths.get(securityTokenPasswordFile));
+
+    final Token<TokenIdentifier> token = new Token<>(identifier, password, tokenKind, tokenService);
+
+    final UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
     currentUser.addToken(token);
   }
 
+  private void addTokensToCurrentUser() throws IOException {
+    final UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+    logToken(Level.INFO, "Before adding client tokens,", currentUser);
+    this.securityTokensReader.addTokensFromFile(currentUser);
+    // Log info for debug purpose for now until it is stable then we can change the log level to FINE.
+    logToken(Level.INFO, "After adding client tokens,", currentUser);
+  }
+
+  /**
+   * Log all the tokens in the container for thr user.
+   *
+   * @param logLevel - the log level.
+   * @param msgPrefix - msg prefix for log.
+   * @param user - the UserGroupInformation object.
+   */
+  private static void logToken(final Level logLevel, final String msgPrefix, final UserGroupInformation user) {
+    if (LOG.isLoggable(logLevel)) {
+      LOG.log(logLevel, "{0} number of tokens: [{1}].",
+          new Object[] {msgPrefix, user.getCredentials().numberOfTokens()});
+      for (final org.apache.hadoop.security.token.Token t : user.getCredentials().getAllTokens()) {
+        LOG.log(logLevel, "Token service: {0}, token kind: {1}.", new Object[] {t.getService(), t.getKind()});
+      }
+    }
+  }
+
   /**
    * We leave a file behind in job submission directory so that clr client can figure out
    * the applicationId and yarn rest endpoint.
@@ -243,6 +297,8 @@ public final class YarnJobSubmissionClient {
     out.close();
   }
 
+  private static final Tang TANG = Tang.Factory.getTang();
+
   /**
    * .NET client calls into this main method for job submission.
    * For arguments detail:
@@ -265,14 +321,6 @@ public final class YarnJobSubmissionClient {
             appSubmissionParametersFile, jobSubmissionParametersFile);
 
     LOG.log(Level.INFO, "YARN job submission received from C#: {0}", yarnSubmission);
-    if (!yarnSubmission.getTokenKind().equalsIgnoreCase("NULL")) {
-      // We have to write security token to user credential before YarnJobSubmissionClient is created
-      // as that will need initialization of FileSystem which could need the token.
-      LOG.log(Level.INFO, "Writing security token to user credential");
-      writeSecurityTokenToUserCredential(yarnSubmission);
-    } else{
-      LOG.log(Level.FINE, "Did not find security token");
-    }
 
     if (!yarnSubmission.getFileSystemUrl().equalsIgnoreCase(FileSystemUrl.DEFAULT_VALUE)) {
       LOG.log(Level.INFO, "getFileSystemUrl: {0}", yarnSubmission.getFileSystemUrl());
@@ -281,18 +329,20 @@ public final class YarnJobSubmissionClient {
     }
 
     final List<String> launchCommandPrefix = new ArrayList<String>() {{
-          add(new REEFFileNames().getDriverLauncherExeFile().toString());
+        final REEFFileNames reefFileNames = TANG.newInjector().getInstance(REEFFileNames.class);
+        add(reefFileNames.getDriverLauncherExeFile().getPath());
       }};
 
-    final Configuration yarnJobSubmissionClientConfig = Tang.Factory.getTang().newConfigurationBuilder()
+    final Configuration yarnJobSubmissionClientConfig = TANG.newConfigurationBuilder()
         .bindImplementation(RuntimeClasspathProvider.class, YarnClasspathProvider.class)
         .bindConstructor(org.apache.hadoop.yarn.conf.YarnConfiguration.class, YarnConfigurationConstructor.class)
         .bindNamedParameter(JobSubmissionDirectoryPrefix.class, yarnSubmission.getJobSubmissionDirectoryPrefix())
         .bindNamedParameter(FileSystemUrl.class, yarnSubmission.getFileSystemUrl())
         .bindList(DriverLaunchCommandPrefix.class, launchCommandPrefix)
         .build();
-    final YarnJobSubmissionClient client = Tang.Factory.getTang().newInjector(yarnJobSubmissionClientConfig)
-        .getInstance(YarnJobSubmissionClient.class);
+
+    final YarnJobSubmissionClient client =
+        TANG.newInjector(yarnJobSubmissionClientConfig).getInstance(YarnJobSubmissionClient.class);
 
     client.launch(yarnSubmission);
 

http://git-wip-us.apache.org/repos/asf/reef/blob/85bae60a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/REEFFileNames.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/REEFFileNames.java b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/REEFFileNames.java
index a51ad1c..811bbd4 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/REEFFileNames.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/REEFFileNames.java
@@ -49,16 +49,26 @@ public final class REEFFileNames {
   private static final String EVALUATOR_STDOUT = "evaluator.stdout";
   private static final String DRIVER_HTTP_ENDPOINT_FILE_NAME = "DriverHttpEndpoint.txt";
   private static final String BRIDGE_EXE_NAME = "Org.Apache.REEF.Bridge.exe";
-  private static final String SECURITY_TOKEN_IDENTIFIER_FILE = "SecurityTokenId";
-  private static final String SECURITY_TOKEN_PASSWORD_FILE = "SecurityTokenPwd";
+  private static final String SECURITY_TOKENS_FILE = "SecurityTokens.json";
   private static final String YARN_BOOTSTRAP_APP_PARAM_FILE = "yarn-app-parameters.json";
   private static final String YARN_BOOTSTRAP_JOB_PARAM_FILE = "yarn-job-parameters.json";
 
+  /**
+   * @deprecated TODO[JIRA REEF-1887] Use getSecurityTokensFile() instead. Remove in REEF 0.18.
+   */
+  @Deprecated
+  private static final String SECURITY_TOKEN_IDENTIFIER_FILE = "SecurityTokenId";
+
+  /**
+   * @deprecated TODO[JIRA REEF-1887] Use getSecurityTokensFile() instead. Remove in REEF 0.18.
+   */
+  @Deprecated
+  private static final String SECURITY_TOKEN_PASSWORD_FILE = "SecurityTokenPwd";
+
   @Inject
   public REEFFileNames() {
   }
 
-
   /**
    * @return the filename of the CPP DLL for the bridge.
    */
@@ -222,19 +232,32 @@ public final class REEFFileNames {
 
   /**
    * @return File name that contains the security token identifier
+   * @deprecated TODO[JIRA REEF-1887] Use getSecurityTokensFile() instead. Remove in REEF 0.18.
    */
+  @Deprecated
   public String getSecurityTokenIdentifierFile() {
     return SECURITY_TOKEN_IDENTIFIER_FILE;
   }
 
   /**
    * @return File name that contains the security token password
+   * @deprecated TODO[JIRA REEF-1887] Use getSecurityTokensFile() instead. Remove in REEF 0.18.
    */
+  @Deprecated
   public String getSecurityTokenPasswordFile() {
     return SECURITY_TOKEN_PASSWORD_FILE;
   }
 
   /**
+   * @return Name of the file that contains security tokens and passwords.
+   * TODO[JIRA REEF-1887] This call supersedes getSecurityTokenIdentifierFile()
+   * and getSecurityTokenPasswordFile(). Remove this comment line in REEF 0.18.
+   */
+  public String getSecurityTokensFile() {
+    return SECURITY_TOKENS_FILE;
+  }
+
+  /**
    * @return File name the contains the bootstrap application parameters for YARN job submission
    * without Java dependency.
    */