You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by ju...@apache.org on 2014/11/19 23:30:15 UTC

incubator-reef git commit: [REEF-47]: Log the REEF Version in all processes.

Repository: incubator-reef
Updated Branches:
  refs/heads/master 57d3e3e12 -> 622de531f


[REEF-47]: Log the REEF Version in all processes.

The following changes are made to that end:

  * The new method `REEFVersion.logVersion()` is added which performs the
    actual logging.
  * On the client, this method is called in the constructor of
    `REEFImplementation`.
  * On the Driver and Evaluator, the method is called in the constructor of
    `LaunchClass`

Also, this removes the method `REEF.getVersion()` and its implementation in
`REEFImplementation` as that method wasn't used.

Related JIRAs:
  [REEF-47](https://issues.apache.org/jira/browse/REEF-47)

This is to closes #22

Author:    Markus Weimer <we...@apache.org>


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

Branch: refs/heads/master
Commit: 622de531f0c085629c917cfac0fae87b28dc753d
Parents: 57d3e3e
Author: Markus Weimer <we...@apache.org>
Authored: Wed Nov 19 12:59:58 2014 -0800
Committer: Julia Wang <jw...@yahoo.com>
Committed: Wed Nov 19 14:24:41 2014 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/reef/client/REEF.java  |  5 ----
 .../common/client/JobSubmissionHelper.java      | 12 +-------
 .../common/client/REEFImplementation.java       | 13 ++++-----
 .../reef/runtime/common/launch/LaunchClass.java |  5 +++-
 .../java/org/apache/reef/util/REEFVersion.java  | 30 +++++++++++++++-----
 5 files changed, 34 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/622de531/reef-common/src/main/java/org/apache/reef/client/REEF.java
----------------------------------------------------------------------
diff --git a/reef-common/src/main/java/org/apache/reef/client/REEF.java b/reef-common/src/main/java/org/apache/reef/client/REEF.java
index 9020035..5a783f0 100644
--- a/reef-common/src/main/java/org/apache/reef/client/REEF.java
+++ b/reef-common/src/main/java/org/apache/reef/client/REEF.java
@@ -52,9 +52,4 @@ public interface REEF extends AutoCloseable {
    * @param driverConf The driver configuration: including everything it needs to execute.  @see DriverConfiguration
    */
   public void submit(final Configuration driverConf);
-
-  /**
-   * @return the version of REEF running.
-   */
-  public String getVersion();
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/622de531/reef-common/src/main/java/org/apache/reef/runtime/common/client/JobSubmissionHelper.java
----------------------------------------------------------------------
diff --git a/reef-common/src/main/java/org/apache/reef/runtime/common/client/JobSubmissionHelper.java b/reef-common/src/main/java/org/apache/reef/runtime/common/client/JobSubmissionHelper.java
index fa06be5..ee9a105 100644
--- a/reef-common/src/main/java/org/apache/reef/runtime/common/client/JobSubmissionHelper.java
+++ b/reef-common/src/main/java/org/apache/reef/runtime/common/client/JobSubmissionHelper.java
@@ -58,13 +58,10 @@ final class JobSubmissionHelper {
   private static final Logger LOG = Logger.getLogger(JobSubmissionHelper.class.getName());
 
   private final ConfigurationSerializer configurationSerializer;
-  private final REEFVersion version;
 
   @Inject
-  JobSubmissionHelper(final ConfigurationSerializer configurationSerializer,
-                      final REEFVersion version) {
+  JobSubmissionHelper(final ConfigurationSerializer configurationSerializer) {
     this.configurationSerializer = configurationSerializer;
-    this.version = version;
   }
 
   /**
@@ -163,11 +160,4 @@ final class JobSubmissionHelper {
     }
     return jarFile;
   }
-
-  /**
-   * @return the version string for REEF.
-   */
-  String getVersion() {
-    return version.getVersion();
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/622de531/reef-common/src/main/java/org/apache/reef/runtime/common/client/REEFImplementation.java
----------------------------------------------------------------------
diff --git a/reef-common/src/main/java/org/apache/reef/runtime/common/client/REEFImplementation.java b/reef-common/src/main/java/org/apache/reef/runtime/common/client/REEFImplementation.java
index a44441d..f0074f9 100644
--- a/reef-common/src/main/java/org/apache/reef/runtime/common/client/REEFImplementation.java
+++ b/reef-common/src/main/java/org/apache/reef/runtime/common/client/REEFImplementation.java
@@ -28,10 +28,12 @@ import org.apache.reef.runtime.common.launch.parameters.ErrorHandlerRID;
 import org.apache.reef.tang.Configuration;
 import org.apache.reef.tang.annotations.Name;
 import org.apache.reef.tang.annotations.NamedParameter;
+import org.apache.reef.util.REEFVersion;
 import org.apache.reef.util.logging.LoggingScope;
 import org.apache.reef.util.logging.LoggingScopeFactory;
 
 import javax.inject.Inject;
+import java.util.logging.Level;
 import java.util.logging.Logger;
 
 @ClientSide
@@ -53,6 +55,7 @@ public final class REEFImplementation implements REEF {
    * @param jobSubmissionHelper
    * @param jobStatusMessageHandler is passed only to make sure it is instantiated
    * @param clientWireUp
+   * @param reefVersion             provides the current version of REEF.
    */
   @Inject
   REEFImplementation(final JobSubmissionHandler jobSubmissionHandler,
@@ -60,13 +63,15 @@ public final class REEFImplementation implements REEF {
                      final JobSubmissionHelper jobSubmissionHelper,
                      final JobStatusMessageHandler jobStatusMessageHandler,
                      final ClientWireUp clientWireUp,
-                     final LoggingScopeFactory loggingScopeFactory) {
+                     final LoggingScopeFactory loggingScopeFactory,
+                     final REEFVersion reefVersion) {
     this.jobSubmissionHandler = jobSubmissionHandler;
     this.runningJobs = runningJobs;
     this.jobSubmissionHelper = jobSubmissionHelper;
     this.clientWireUp = clientWireUp;
     clientWireUp.performWireUp();
     this.loggingScopeFactory = loggingScopeFactory;
+    reefVersion.logVersion();
   }
 
   @Override
@@ -97,12 +102,6 @@ public final class REEFImplementation implements REEF {
     }
   }
 
-  @Override
-  public String getVersion() {
-    return this.jobSubmissionHelper.getVersion();
-  }
-
-
   @NamedParameter(doc = "The driver remote identifier.")
   public final static class DriverRemoteIdentifier implements Name<String> {
   }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/622de531/reef-common/src/main/java/org/apache/reef/runtime/common/launch/LaunchClass.java
----------------------------------------------------------------------
diff --git a/reef-common/src/main/java/org/apache/reef/runtime/common/launch/LaunchClass.java b/reef-common/src/main/java/org/apache/reef/runtime/common/launch/LaunchClass.java
index de2a799..8c40439 100644
--- a/reef-common/src/main/java/org/apache/reef/runtime/common/launch/LaunchClass.java
+++ b/reef-common/src/main/java/org/apache/reef/runtime/common/launch/LaunchClass.java
@@ -28,6 +28,7 @@ import org.apache.reef.tang.annotations.Name;
 import org.apache.reef.tang.annotations.NamedParameter;
 import org.apache.reef.tang.annotations.Parameter;
 import org.apache.reef.tang.formats.ConfigurationSerializer;
+import org.apache.reef.util.REEFVersion;
 import org.apache.reef.wake.profiler.WakeProfiler;
 import org.apache.reef.wake.remote.RemoteConfiguration;
 import org.apache.reef.wake.time.Clock;
@@ -62,7 +63,9 @@ public final class LaunchClass implements AutoCloseable, Runnable {
               final @Parameter(ErrorHandlerRID.class) String errorHandlerID,
               final @Parameter(ClockConfigurationPath.class) String evaluatorConfigurationPath,
               final @Parameter(ProfilingEnabled.class) boolean enableProfiling,
-              final ConfigurationSerializer configurationSerializer) {
+              final ConfigurationSerializer configurationSerializer,
+              final REEFVersion reefVersion) {
+    reefVersion.logVersion();
     this.remoteManager = remoteManager;
     this.launchID = launchID;
     this.errorHandlerID = errorHandlerID;

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/622de531/reef-common/src/main/java/org/apache/reef/util/REEFVersion.java
----------------------------------------------------------------------
diff --git a/reef-common/src/main/java/org/apache/reef/util/REEFVersion.java b/reef-common/src/main/java/org/apache/reef/util/REEFVersion.java
index dd77bbc..1b74e7f 100644
--- a/reef-common/src/main/java/org/apache/reef/util/REEFVersion.java
+++ b/reef-common/src/main/java/org/apache/reef/util/REEFVersion.java
@@ -43,6 +43,29 @@ public final class REEFVersion {
     this.version = loadVersion();
   }
 
+  /**
+   * Logs the version of REEF into the log Level INFO.
+   */
+  public void logVersion() {
+    this.logVersion(Level.INFO);
+  }
+
+  /**
+   * Logs the version of REEF into the given logLevel.
+   *
+   * @param logLevel The level to use in the log.
+   */
+  public void logVersion(final Level logLevel) {
+    LOG.log(logLevel, "REEF Version: {0}", this.version);
+  }
+
+  /**
+   * @return the version string for REEF.
+   */
+  public String getVersion() {
+    return version;
+  }
+
   private static String loadVersion() {
     String version;
     try (final InputStream is = Thread.currentThread().getContextClassLoader().getResourceAsStream(FILENAME)) {
@@ -58,11 +81,4 @@ public final class REEFVersion {
     }
     return version;
   }
-
-  /**
-   * @return the version string for REEF.
-   */
-  public String getVersion() {
-    return version;
-  }
 }