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 2018/06/14 01:30:42 UTC

reef git commit: [REEF-2031] Azure Batch credentials exposed in job-submission-params.json in REEF.NET

Repository: reef
Updated Branches:
  refs/heads/master 7e1ec477a -> ec738ad33


[REEF-2031] Azure Batch credentials exposed in job-submission-params.json in REEF.NET

When running REEF .NET on Azure Batch, Azure Batch key and Azure Storage
account key are exposed in job-submission-params.json file. Those keys
are unnecessary as environment variables "AZ_BATCH_AUTHENTICATION_TOKEN"
is set for Azure Batch token, and "AZURE_STORAGE_CONTAINER_SAS_TOKEN_ENV" is set for Azure Storage Account token.

JIRA:
  [REEF-2031](httaps://issues.apache.org/jira/browse/REEF-2031)

Pull request:
  This closes #1469


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

Branch: refs/heads/master
Commit: ec738ad33ed0e71872e0699c668ad87116cf3bd8
Parents: 7e1ec47
Author: Chenxi Zhao <33...@users.noreply.github.com>
Authored: Tue Jun 5 13:20:57 2018 -0700
Committer: Sergiy Matusevych <mo...@apache.org>
Committed: Wed Jun 13 18:23:56 2018 -0700

----------------------------------------------------------------------
 .../AvroAzureBatchJobSubmissionParameters.cs    | 20 +------
 .../AzureBatch/Util/JobJarMaker.cs              |  5 --
 .../src/main/avro/JobSubmissionParameters.avsc  |  5 +-
 ...zureBatchBootstrapDriverConfigGenerator.java | 35 ++-----------
 .../client/AzureBatchBootstrapREEFLauncher.java | 55 +++++++++++++-------
 5 files changed, 41 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/ec738ad3/lang/cs/Org.Apache.REEF.Client/Avro/AzureBatch/AvroAzureBatchJobSubmissionParameters.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Avro/AzureBatch/AvroAzureBatchJobSubmissionParameters.cs b/lang/cs/Org.Apache.REEF.Client/Avro/AzureBatch/AvroAzureBatchJobSubmissionParameters.cs
index 78bfe4a..6409dc8 100644
--- a/lang/cs/Org.Apache.REEF.Client/Avro/AzureBatch/AvroAzureBatchJobSubmissionParameters.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Avro/AzureBatch/AvroAzureBatchJobSubmissionParameters.cs
@@ -30,7 +30,7 @@ namespace Org.Apache.REEF.Client.Avro.AzureBatch
     [DataContract(Namespace = "org.apache.reef.reef.bridge.client.avro")]
     public sealed class AvroAzureBatchJobSubmissionParameters
     {
-        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.reef.bridge.client.avro.AvroAzureBatchJobSubmissionParameters"",""doc"":""Job submission parameters used by the Azure Batch runtime"",""fields"":[{""name"":""sharedJobSubmissionParameters"",""type"":{""type"":""record"",""name"":""org.apache.reef.reef.bridge.client.avro.AvroJobSubmissionParameters"",""doc"":""General cross-language job submission parameters shared by all runtimes"",""fields"":[{""name"":""jobId"",""type"":""string""},{""name"":""jobSubmissionFolder"",""type"":""string""}]}},{""name"":""AzureBatchAccountKey"",""type"":""string""},{""name"":""AzureBatchAccountName"",""type"":""string""},{""name"":""AzureBatchAccountUri"",""type"":""string""},{""name"":""AzureBatchPoolId"",""type"":""string""},{""name"":""AzureStorageAccountKey"",""type"":""string""},{""name"":""AzureStorageAccountName"",""type"":""string""},{""name"":""AzureStorageContainerName"",""type"":""string""},{""name"":
 ""AzureBatchIsWindows"",""type"":""boolean""}]}";
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.reef.bridge.client.avro.AvroAzureBatchJobSubmissionParameters"",""doc"":""Job submission parameters used by the Azure Batch runtime"",""fields"":[{""name"":""sharedJobSubmissionParameters"",""type"":{""type"":""record"",""name"":""org.apache.reef.reef.bridge.client.avro.AvroJobSubmissionParameters"",""doc"":""General cross-language job submission parameters shared by all runtimes"",""fields"":[{""name"":""jobId"",""type"":""string""},{""name"":""jobSubmissionFolder"",""type"":""string""}]}},{""name"":""AzureBatchAccountName"",""type"":""string""},{""name"":""AzureBatchAccountUri"",""type"":""string""},{""name"":""AzureBatchPoolId"",""type"":""string""},{""name"":""AzureStorageAccountName"",""type"":""string""},{""name"":""AzureStorageContainerName"",""type"":""string""}]}";
 
         /// <summary>
         /// Gets the schema.
@@ -50,12 +50,6 @@ namespace Org.Apache.REEF.Client.Avro.AzureBatch
         public AvroJobSubmissionParameters sharedJobSubmissionParameters { get; set; }
 
         /// <summary>
-        /// Gets or sets the AzureBatchAccountKey field.
-        /// </summary>
-        [DataMember]
-        public string AzureBatchAccountKey { get; set; }
-
-        /// <summary>
         /// Gets or sets the AzureBatchAccountName field.
         /// </summary>
         [DataMember]
@@ -74,12 +68,6 @@ namespace Org.Apache.REEF.Client.Avro.AzureBatch
         public string AzureBatchPoolId { get; set; }
 
         /// <summary>
-        /// Gets or sets the AzureStorageAccountKey field.
-        /// </summary>
-        [DataMember]
-        public string AzureStorageAccountKey { get; set; }
-
-        /// <summary>
         /// Gets or sets the AzureStorageAccountName field.
         /// </summary>
         [DataMember]
@@ -92,12 +80,6 @@ namespace Org.Apache.REEF.Client.Avro.AzureBatch
         public string AzureStorageContainerName { get; set; }
 
         /// <summary>
-        /// Gets or sets the AzureBatchIsWindows field.
-        /// </summary>
-        [DataMember]
-        public bool AzureBatchIsWindows { get; set; }
-
-        /// <summary>
         /// Initializes a new instance of the <see cref="AvroAzureBatchJobSubmissionParameters"/> class.
         /// </summary>
         public AvroAzureBatchJobSubmissionParameters()

http://git-wip-us.apache.org/repos/asf/reef/blob/ec738ad3/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/JobJarMaker.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/JobJarMaker.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/JobJarMaker.cs
index 48e62e5..4d434d7 100644
--- a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/JobJarMaker.cs
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/JobJarMaker.cs
@@ -40,11 +40,9 @@ namespace Org.Apache.REEF.Client.AzureBatch.Util
             IResourceArchiveFileGenerator resourceArchiveFileGenerator,
             DriverFolderPreparationHelper driverFolderPreparationHelper,
             REEFFileNames fileNames,
-            [Parameter(typeof(AzureBatchAccountKey))] string azureBatchAccountKey,
             [Parameter(typeof(AzureBatchAccountName))] string azureBatchAccountName,
             [Parameter(typeof(AzureBatchAccountUri))] string azureBatchAccountUri,
             [Parameter(typeof(AzureBatchPoolId))] string azureBatchPoolId,
-            [Parameter(typeof(AzureStorageAccountKey))] string azureStorageAccountKey,
             [Parameter(typeof(AzureStorageAccountName))] string azureStorageAccountName,
             [Parameter(typeof(AzureStorageContainerName))] string azureStorageContainerName)
         {
@@ -53,14 +51,11 @@ namespace Org.Apache.REEF.Client.AzureBatch.Util
             _fileNames = fileNames;
             _avroAzureBatchJobSubmissionParameters = new AvroAzureBatchJobSubmissionParameters
             {
-                AzureBatchAccountKey = azureBatchAccountKey,
                 AzureBatchAccountName = azureBatchAccountName,
                 AzureBatchAccountUri = azureBatchAccountUri,
                 AzureBatchPoolId = azureBatchPoolId,
-                AzureStorageAccountKey = azureStorageAccountKey,
                 AzureStorageAccountName = azureStorageAccountName,
                 AzureStorageContainerName = azureStorageContainerName,
-                AzureBatchIsWindows = true
             };
         }
 

http://git-wip-us.apache.org/repos/asf/reef/blob/ec738ad3/lang/java/reef-bridge-client/src/main/avro/JobSubmissionParameters.avsc
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/avro/JobSubmissionParameters.avsc b/lang/java/reef-bridge-client/src/main/avro/JobSubmissionParameters.avsc
index e2a30f3..d3bda9f 100644
--- a/lang/java/reef-bridge-client/src/main/avro/JobSubmissionParameters.avsc
+++ b/lang/java/reef-bridge-client/src/main/avro/JobSubmissionParameters.avsc
@@ -73,14 +73,11 @@
       "doc": "Cross-language submission parameters to the Azure Batch runtime",
       "fields": [
         { "name": "sharedJobSubmissionParameters", "type": "AvroJobSubmissionParameters" },
-        { "name": "AzureBatchAccountKey", "type": "string" },
         { "name": "AzureBatchAccountName", "type": "string" },
         { "name": "AzureBatchAccountUri", "type": "string" },
         { "name": "AzureBatchPoolId", "type": "string" },
-        { "name": "AzureStorageAccountKey", "type": "string" },
         { "name": "AzureStorageAccountName", "type": "string" },
-        { "name": "AzureStorageContainerName", "type": "string" },
-        { "name": "AzureBatchIsWindows", "type": "boolean" }
+        { "name": "AzureStorageContainerName", "type": "string" }
       ]
   }
 ]

http://git-wip-us.apache.org/repos/asf/reef/blob/ec738ad3/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/AzureBatchBootstrapDriverConfigGenerator.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/AzureBatchBootstrapDriverConfigGenerator.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/AzureBatchBootstrapDriverConfigGenerator.java
index c054722..5f09abd 100644
--- a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/AzureBatchBootstrapDriverConfigGenerator.java
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/AzureBatchBootstrapDriverConfigGenerator.java
@@ -18,9 +18,6 @@
  */
 package org.apache.reef.bridge.client;
 
-import org.apache.avro.io.DecoderFactory;
-import org.apache.avro.io.JsonDecoder;
-import org.apache.avro.specific.SpecificDatumReader;
 import org.apache.reef.reef.bridge.client.avro.AvroAzureBatchJobSubmissionParameters;
 import org.apache.reef.runtime.common.client.DriverConfigurationProvider;
 import org.apache.reef.runtime.common.driver.parameters.ClientRemoteIdentifier;
@@ -28,9 +25,6 @@ import org.apache.reef.tang.Configuration;
 
 import javax.inject.Inject;
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -49,15 +43,10 @@ final class AzureBatchBootstrapDriverConfigGenerator {
     this.driverConfigurationProvider = driverConfigurationProvider;
   }
 
-  Configuration getDriverConfigurationFromParams(final String bootstrapJobArgsLocation) throws IOException {
-
-    final File bootstrapJobArgsFile = new File(bootstrapJobArgsLocation).getCanonicalFile();
-
-    final AvroAzureBatchJobSubmissionParameters azureBatchBootstrapJobArgs =
-        readAzureBatchJobSubmissionParametersFromFile(bootstrapJobArgsFile);
-
-    final String jobId = azureBatchBootstrapJobArgs.getSharedJobSubmissionParameters().getJobId().toString();
-    final File jobFolder = new File(azureBatchBootstrapJobArgs
+  Configuration getDriverConfigurationFromParams(
+      final AvroAzureBatchJobSubmissionParameters avroAzureBatchJobSubmissionParameters) {
+    final String jobId = avroAzureBatchJobSubmissionParameters.getSharedJobSubmissionParameters().getJobId().toString();
+    final File jobFolder = new File(avroAzureBatchJobSubmissionParameters
         .getSharedJobSubmissionParameters().getJobSubmissionFolder().toString());
 
     LOG.log(Level.INFO, "jobFolder {0} jobId {1}.", new Object[]{jobFolder.toURI(), jobId});
@@ -66,20 +55,4 @@ final class AzureBatchBootstrapDriverConfigGenerator {
         jobFolder.toURI(), ClientRemoteIdentifier.NONE, jobId,
         Constants.DRIVER_CONFIGURATION_WITH_HTTP_AND_NAMESERVER);
   }
-
-  private AvroAzureBatchJobSubmissionParameters readAzureBatchJobSubmissionParametersFromFile(final File file)
-      throws IOException {
-    try (final FileInputStream fileInputStream = new FileInputStream(file)) {
-      return readAzureBatchSubmissionParametersFromInputStream(fileInputStream);
-    }
-  }
-
-  private static AvroAzureBatchJobSubmissionParameters readAzureBatchSubmissionParametersFromInputStream(
-      final InputStream inputStream) throws IOException {
-    final JsonDecoder decoder = DecoderFactory.get().jsonDecoder(
-        AvroAzureBatchJobSubmissionParameters.getClassSchema(), inputStream);
-    final SpecificDatumReader<AvroAzureBatchJobSubmissionParameters> reader = new SpecificDatumReader<>(
-        AvroAzureBatchJobSubmissionParameters.class);
-    return reader.read(null, decoder);
-  }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/ec738ad3/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/AzureBatchBootstrapREEFLauncher.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/AzureBatchBootstrapREEFLauncher.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/AzureBatchBootstrapREEFLauncher.java
index 35a1e59..4f2a3f6 100644
--- a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/AzureBatchBootstrapREEFLauncher.java
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/AzureBatchBootstrapREEFLauncher.java
@@ -23,10 +23,21 @@ import org.apache.avro.io.JsonDecoder;
 import org.apache.avro.specific.SpecificDatumReader;
 import org.apache.reef.annotations.audience.Interop;
 import org.apache.reef.reef.bridge.client.avro.AvroAzureBatchJobSubmissionParameters;
-import org.apache.reef.runtime.azbatch.client.AzureBatchRuntimeConfiguration;
-import org.apache.reef.runtime.azbatch.client.AzureBatchRuntimeConfigurationCreator;
+import org.apache.reef.runtime.azbatch.AzureBatchClasspathProvider;
+import org.apache.reef.runtime.azbatch.AzureBatchJVMPathProvider;
+import org.apache.reef.runtime.azbatch.client.AzureBatchDriverConfigurationProviderImpl;
+import org.apache.reef.runtime.azbatch.parameters.AzureBatchAccountName;
+import org.apache.reef.runtime.azbatch.parameters.AzureBatchAccountUri;
+import org.apache.reef.runtime.azbatch.parameters.AzureBatchPoolId;
+import org.apache.reef.runtime.azbatch.parameters.AzureStorageAccountName;
+import org.apache.reef.runtime.azbatch.parameters.AzureStorageContainerName;
+import org.apache.reef.runtime.azbatch.util.command.CommandBuilder;
+import org.apache.reef.runtime.azbatch.util.command.WindowsCommandBuilder;
 import org.apache.reef.runtime.common.REEFEnvironment;
+import org.apache.reef.runtime.common.client.DriverConfigurationProvider;
 import org.apache.reef.runtime.common.evaluator.PIDStoreStartHandler;
+import org.apache.reef.runtime.common.files.RuntimeClasspathProvider;
+import org.apache.reef.runtime.common.files.RuntimePathProvider;
 import org.apache.reef.runtime.common.launch.REEFErrorHandler;
 import org.apache.reef.runtime.common.launch.REEFMessageCodec;
 import org.apache.reef.tang.Configuration;
@@ -73,9 +84,10 @@ public final class AzureBatchBootstrapREEFLauncher {
       throw fatal(message, new IllegalArgumentException(message));
     }
 
-    final File partialConfigFile = new File(args[0]);
+    final AvroAzureBatchJobSubmissionParameters avroAzureBatchJobSubmissionParameters =
+        readAvroJobSubmissionParameters(new File(args[0]));
     final AzureBatchBootstrapDriverConfigGenerator azureBatchBootstrapDriverConfigGenerator =
-        TANG.newInjector(generateConfigurationFromJobSubmissionParameters(partialConfigFile))
+        TANG.newInjector(generateConfiguration(avroAzureBatchJobSubmissionParameters))
             .getInstance(AzureBatchBootstrapDriverConfigGenerator.class);
 
     final Configuration launcherConfig =
@@ -87,7 +99,8 @@ public final class AzureBatchBootstrapREEFLauncher {
             .build();
 
     try (final REEFEnvironment reef = REEFEnvironment.fromConfiguration(
-        azureBatchBootstrapDriverConfigGenerator.getDriverConfigurationFromParams(args[0]), launcherConfig)) {
+        azureBatchBootstrapDriverConfigGenerator.getDriverConfigurationFromParams(
+            avroAzureBatchJobSubmissionParameters), launcherConfig)) {
       reef.run();
     } catch (final InjectionException ex) {
       throw fatal("Unable to configure and start REEFEnvironment.", ex);
@@ -98,33 +111,35 @@ public final class AzureBatchBootstrapREEFLauncher {
     System.exit(0); // TODO[REEF-1715]: Should be able to exit cleanly at the end of main()
   }
 
-  private static Configuration generateConfigurationFromJobSubmissionParameters(final File params) throws IOException {
-
+  private static AvroAzureBatchJobSubmissionParameters readAvroJobSubmissionParameters(
+      final File paramsFile) throws IOException {
     final AvroAzureBatchJobSubmissionParameters avroAzureBatchJobSubmissionParameters;
-
-    try (final FileInputStream fileInputStream = new FileInputStream(params)) {
+    try (final FileInputStream fileInputStream = new FileInputStream(paramsFile)) {
       final JsonDecoder decoder = DecoderFactory.get().jsonDecoder(
           AvroAzureBatchJobSubmissionParameters.getClassSchema(), fileInputStream);
       final SpecificDatumReader<AvroAzureBatchJobSubmissionParameters> reader =
           new SpecificDatumReader<>(AvroAzureBatchJobSubmissionParameters.class);
       avroAzureBatchJobSubmissionParameters = reader.read(null, decoder);
     }
+    return avroAzureBatchJobSubmissionParameters;
+  }
 
-    return AzureBatchRuntimeConfigurationCreator
-        .getOrCreateAzureBatchRuntimeConfiguration(avroAzureBatchJobSubmissionParameters.getAzureBatchIsWindows())
-        .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_NAME,
+  private static Configuration generateConfiguration(
+      final AvroAzureBatchJobSubmissionParameters avroAzureBatchJobSubmissionParameters) {
+    return TANG.newConfigurationBuilder()
+        .bindImplementation(DriverConfigurationProvider.class, AzureBatchDriverConfigurationProviderImpl.class)
+        .bindImplementation(RuntimeClasspathProvider.class, AzureBatchClasspathProvider.class)
+        .bindImplementation(RuntimePathProvider.class, AzureBatchJVMPathProvider.class)
+        .bindImplementation(CommandBuilder.class, WindowsCommandBuilder.class)
+        .bindNamedParameter(AzureBatchAccountName.class,
             avroAzureBatchJobSubmissionParameters.getAzureBatchAccountName().toString())
-        .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_KEY,
-            avroAzureBatchJobSubmissionParameters.getAzureBatchAccountKey().toString())
-        .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_URI,
+        .bindNamedParameter(AzureBatchAccountUri.class,
             avroAzureBatchJobSubmissionParameters.getAzureBatchAccountUri().toString())
-        .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_POOL_ID,
+        .bindNamedParameter(AzureBatchPoolId.class,
             avroAzureBatchJobSubmissionParameters.getAzureBatchPoolId().toString())
-        .set(AzureBatchRuntimeConfiguration.AZURE_STORAGE_ACCOUNT_NAME,
+        .bindNamedParameter(AzureStorageAccountName.class,
             avroAzureBatchJobSubmissionParameters.getAzureStorageAccountName().toString())
-        .set(AzureBatchRuntimeConfiguration.AZURE_STORAGE_ACCOUNT_KEY,
-            avroAzureBatchJobSubmissionParameters.getAzureStorageAccountKey().toString())
-        .set(AzureBatchRuntimeConfiguration.AZURE_STORAGE_CONTAINER_NAME,
+        .bindNamedParameter(AzureStorageContainerName.class,
             avroAzureBatchJobSubmissionParameters.getAzureStorageContainerName().toString())
         .build();
   }