Browse Source

HADOOP-8209. Add option to relax build-version check for branch-1. Contributed by Eli Collins

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1@1325547 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 13 years ago
parent
commit
c0ff8671c3

+ 2 - 0
CHANGES.txt

@@ -21,6 +21,8 @@ Release 1.1.0 - unreleased
 
     HDFS-3055. Implement recovery mode (Colin Patrick McCabe via todd)
 
+    HADOOP-8209. Add option to relax build-version check for branch-1. (eli)
+
   IMPROVEMENTS
 
     MAPREDUCE-3597. [Rumen] Provide a way to access other info of history file

+ 14 - 0
src/core/core-default.xml

@@ -552,4 +552,18 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.relaxed.worker.version.check</name>
+  <value>false</value>
+  <description>
+    By default datanodes refuse to connect to namenodes if their build
+    revision (svn revision) do not match, and tasktrackers refuse to
+    connect to jobtrackers if their build version (version, revision,
+    user, and source checksum) do not match. This option changes the
+    behavior of hadoop workers to only check for a version match (eg
+    "1.0.2") but ignore the other build fields (revision, user, and
+    source checksum).
+  </description>
+</property>
+
 </configuration>

+ 5 - 0
src/core/org/apache/hadoop/fs/CommonConfigurationKeys.java

@@ -64,5 +64,10 @@ public class CommonConfigurationKeys {
   /** Default value for IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY */
   public static final int IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_DEFAULT =
       256 * 1024;
+
+  /** See src/core/core-default.xml */
+  public static final String HADOOP_RELAXED_VERSION_CHECK_KEY =
+      "hadoop.relaxed.worker.version.check";
+  public static final boolean HADOOP_RELAXED_VERSION_CHECK_DEFAULT = false;
 }
 

+ 2 - 2
src/core/org/apache/hadoop/util/VersionInfo.java

@@ -89,8 +89,8 @@ public class VersionInfo {
   }
   
   /**
-   * Returns the buildVersion which includes version, 
-   * revision, user and date. 
+   * Returns the full version string containing version,
+   * revision, user and source checksum. 
    */
   public static String getBuildVersion(){
     return VersionInfo.getVersion() + 

+ 0 - 7
src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -35,9 +35,6 @@ import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.util.VersionInfo;
-
-
 
 /**
  * Storage information file.
@@ -837,10 +834,6 @@ public abstract class Storage extends StorageInfo {
     return false;
   }
 
-  public static String getBuildVersion() {
-    return VersionInfo.getRevision();
-  }
-
   public static String getRegistrationID(StorageInfo storage) {
     return "NS-" + Integer.toString(storage.getNamespaceID())
       + "-" + Integer.toString(storage.getLayoutVersion())

+ 42 - 7
src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -56,6 +56,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
@@ -233,6 +234,7 @@ public class DataNode extends Configured
   boolean isBlockTokenInitialized = false;
   final String userWithLocalPathAccess;
   private boolean connectToDnViaHostname;
+  private boolean relaxedVersionCheck;
 
   /**
    * Testing hook that allows tests to delay the sending of blockReceived RPCs
@@ -347,6 +349,10 @@ public class DataNode extends Configured
                                              true);
     this.writePacketSize = conf.getInt("dfs.write.packet.size", 64*1024);
 
+    this.relaxedVersionCheck = conf.getBoolean(
+        CommonConfigurationKeys.HADOOP_RELAXED_VERSION_CHECK_KEY,
+        CommonConfigurationKeys.HADOOP_RELAXED_VERSION_CHECK_DEFAULT);
+
     InetSocketAddress socAddr = DataNode.getStreamingAddr(conf);
     int tmpPort = socAddr.getPort();
     storage = new DataStorage();
@@ -548,6 +554,34 @@ public class DataNode extends Configured
            SocketChannel.open().socket() : new Socket();                                   
   }
   
+  /**
+   * @return true if this datanode is permitted to connect to
+   *    the given namenode version
+   */
+  boolean isPermittedVersion(NamespaceInfo nsInfo) {
+    boolean versionMatch =
+      nsInfo.getVersion().equals(VersionInfo.getVersion());
+    boolean revisionMatch =
+      nsInfo.getRevision().equals(VersionInfo.getRevision());
+
+    if (revisionMatch && !versionMatch) {
+      throw new AssertionError("Invalid build. The revisions match" +
+          " but the NN version is " + nsInfo.getVersion() +
+          " and the DN version is " + VersionInfo.getVersion());
+    }
+    if (relaxedVersionCheck) {
+      if (versionMatch && !revisionMatch) {
+        LOG.info("Permitting datanode revision " + VersionInfo.getRevision() +
+            " to connect to namenode revision " + nsInfo.getRevision() +
+            " because " + CommonConfigurationKeys.HADOOP_RELAXED_VERSION_CHECK_KEY +
+            " is enabled");
+      }
+      return versionMatch;
+    } else {
+      return revisionMatch;
+    }
+  }
+
   private NamespaceInfo handshake() throws IOException {
     NamespaceInfo nsInfo = new NamespaceInfo();
     while (shouldRun) {
@@ -561,13 +595,14 @@ public class DataNode extends Configured
         } catch (InterruptedException ie) {}
       }
     }
-    String errorMsg = null;
-    // verify build version
-    if( ! nsInfo.getBuildVersion().equals( Storage.getBuildVersion() )) {
-      errorMsg = "Incompatible build versions: namenode BV = " 
-        + nsInfo.getBuildVersion() + "; datanode BV = "
-        + Storage.getBuildVersion();
-      LOG.fatal( errorMsg );
+    if (!isPermittedVersion(nsInfo)) {
+      String errorMsg = "Incompatible versions: namenode version " +
+        nsInfo.getVersion() + " revision " + nsInfo.getRevision() +
+        " datanode version " + VersionInfo.getVersion() + " revision " +
+        VersionInfo.getRevision() + " and " +
+        CommonConfigurationKeys.HADOOP_RELAXED_VERSION_CHECK_KEY +
+        " is " + (relaxedVersionCheck ? "enabled" : "not enabled");
+      LOG.fatal(errorMsg);
       notifyNamenode(DatanodeProtocol.NOTIFY, errorMsg);  
       throw new IOException( errorMsg );
     }

+ 2 - 5
src/hdfs/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java

@@ -41,12 +41,9 @@ import org.apache.hadoop.security.KerberosInfo;
     clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
 public interface DatanodeProtocol extends VersionedProtocol {
   /**
-   * 25: Serialized format of BlockTokenIdentifier changed to contain
-   *     multiple blocks within a single BlockTokenIdentifier
-   *     
-   *     (bumped to 25 to bring in line with trunk)
+   * 26: Added an additional member to NamespaceInfo
    */
-  public static final long versionID = 25L;
+  public static final long versionID = 26L;
   
   // error code
   final static int NOTIFY = 0;

+ 16 - 9
src/hdfs/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java

@@ -23,12 +23,12 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.hadoop.hdfs.protocol.FSConstants;
-import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;
+import org.apache.hadoop.util.VersionInfo;
 
 /**
  * NamespaceInfo is returned by the name-node in reply 
@@ -36,22 +36,27 @@ import org.apache.hadoop.io.WritableFactory;
  * 
  */
 public class NamespaceInfo extends StorageInfo implements Writable {
-  String  buildVersion;
+  String revision;
+  String version;
   int distributedUpgradeVersion;
 
   public NamespaceInfo() {
     super();
-    buildVersion = null;
   }
   
   public NamespaceInfo(int nsID, long cT, int duVersion) {
-    super(FSConstants.LAYOUT_VERSION, nsID, cT);
-    buildVersion = Storage.getBuildVersion();
+    super(FSConstants.LAYOUT_VERSION, nsID, cT); 
+    version = VersionInfo.getVersion();
+    revision = VersionInfo.getRevision();
     this.distributedUpgradeVersion = duVersion;
   }
   
-  public String getBuildVersion() {
-    return buildVersion;
+  public String getVersion() {
+    return version;
+  }
+
+  public String getRevision() {
+    return revision;
   }
 
   public int getDistributedUpgradeVersion() {
@@ -70,7 +75,8 @@ public class NamespaceInfo extends StorageInfo implements Writable {
   }
 
   public void write(DataOutput out) throws IOException {
-    UTF8.writeString(out, getBuildVersion());
+    UTF8.writeString(out, getVersion());
+    UTF8.writeString(out, getRevision());
     out.writeInt(getLayoutVersion());
     out.writeInt(getNamespaceID());
     out.writeLong(getCTime());
@@ -78,7 +84,8 @@ public class NamespaceInfo extends StorageInfo implements Writable {
   }
 
   public void readFields(DataInput in) throws IOException {
-    buildVersion = UTF8.readString(in);
+    version = UTF8.readString(in);
+    revision = UTF8.readString(in);
     layoutVersion = in.readInt();
     namespaceID = in.readInt();
     cTime = in.readLong();

+ 8 - 3
src/mapred/org/apache/hadoop/mapred/InterTrackerProtocol.java

@@ -72,8 +72,9 @@ interface InterTrackerProtocol extends VersionedProtocol {
    * Version 29: Adding available memory and CPU usage information on TT to
    *             TaskTrackerStatus for MAPREDUCE-1218
    * Version 30: Adding disk failure to TaskTrackerStatus for MAPREDUCE-3015
+   * Version 31: Adding version methods for HADOOP-8209
    */
-  public static final long versionID = 30L;
+  public static final long versionID = 31L;
   
   public final static int TRACKERS_OK = 0;
   public final static int UNKNOWN_TASKTRACKER = 1;
@@ -144,9 +145,13 @@ interface InterTrackerProtocol extends VersionedProtocol {
    */
   public String getSystemDir();
   
-  
   /**
-   * Returns the buildVersion of the JobTracker 
+   * Returns the VersionInfo build version of the JobTracker 
    */
   public String getBuildVersion() throws IOException;
+
+  /**
+   * Returns the VersionInfo version of the JobTracker
+   */
+  public String getVIVersion() throws IOException;
 }

+ 7 - 2
src/mapred/org/apache/hadoop/mapred/JobTracker.java

@@ -3290,8 +3290,13 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   ////////////////////////////////////////////////////
   // InterTrackerProtocol
   ////////////////////////////////////////////////////
-  
-  public String getBuildVersion() throws IOException{
+
+  // Just returns the VersionInfo version (unlike MXBean#getVersion)
+  public String getVIVersion() throws IOException {
+    return VersionInfo.getVersion();
+  }
+
+  public String getBuildVersion() throws IOException {
     return VersionInfo.getBuildVersion();
   }
 

+ 44 - 6
src/mapred/org/apache/hadoop/mapred/TaskTracker.java

@@ -62,6 +62,7 @@ import org.apache.hadoop.filecache.TaskDistributedCacheManager;
 import org.apache.hadoop.filecache.TrackerDistributedCacheManager;
 import org.apache.hadoop.mapreduce.server.tasktracker.*;
 import org.apache.hadoop.mapreduce.server.tasktracker.userlogs.*;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -406,6 +407,11 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol,
    */
   private long diskHealthCheckInterval;
 
+  /**
+   * Whether the TT performs a full or relaxed version check with the JT.
+   */
+  private boolean relaxedVersionCheck;
+
   /*
    * A list of commitTaskActions for whom commit response has been received 
    */
@@ -1417,6 +1423,9 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol,
    */
   public TaskTracker(JobConf conf) throws IOException, InterruptedException {
     originalConf = conf;
+    relaxedVersionCheck = conf.getBoolean(
+        CommonConfigurationKeys.HADOOP_RELAXED_VERSION_CHECK_KEY,
+        CommonConfigurationKeys.HADOOP_RELAXED_VERSION_CHECK_DEFAULT);
     FILE_CACHE_SIZE = conf.getInt("mapred.tasktracker.file.cache.size", 2000);
     maxMapSlots = conf.getInt(
                   "mapred.tasktracker.map.tasks.maximum", 2);
@@ -1582,6 +1591,32 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol,
     return (heartbeatInterval / (numFinishedTasks * oobHeartbeatDamper + 1));
   }
   
+  /**
+   * @return true if this tasktracker is permitted to connect to
+   *    the given jobtracker version
+   */
+  boolean isPermittedVersion(String jtBuildVersion, String jtVersion) {
+    boolean buildVersionMatch =
+      jtBuildVersion.equals(VersionInfo.getBuildVersion());
+    boolean versionMatch = jtVersion.equals(VersionInfo.getVersion());
+    if (buildVersionMatch && !versionMatch) {
+      throw new AssertionError("Invalid build. The build versions match" +
+          " but the JT version is " + jtVersion +
+          " and the TT version is " + VersionInfo.getVersion());
+    }
+    if (relaxedVersionCheck) {
+      if (!buildVersionMatch && versionMatch) {
+        LOG.info("Permitting tasktracker revision " + VersionInfo.getRevision() +
+            " to connect to jobtracker " + jtBuildVersion + " because " +
+            CommonConfigurationKeys.HADOOP_RELAXED_VERSION_CHECK_KEY +
+            " is enabled");
+      }
+      return versionMatch;
+    } else {
+      return buildVersionMatch;
+    }
+  }
+
   /**
    * Main service loop.  Will stay in this loop forever.
    */
@@ -1615,15 +1650,18 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol,
         }
 
         // If the TaskTracker is just starting up:
-        // 1. Verify the buildVersion
+        // 1. Verify the versions matches with the JobTracker
         // 2. Get the system directory & filesystem
         if(justInited) {
-          String jobTrackerBV = jobClient.getBuildVersion();
-          if(!VersionInfo.getBuildVersion().equals(jobTrackerBV)) {
+          String jtBuildVersion = jobClient.getBuildVersion();
+          String jtVersion = jobClient.getVIVersion();
+          if (!isPermittedVersion(jtBuildVersion, jtVersion)) {
             String msg = "Shutting down. Incompatible buildVersion." +
-            "\nJobTracker's: " + jobTrackerBV + 
-            "\nTaskTracker's: "+ VersionInfo.getBuildVersion();
-            LOG.error(msg);
+              "\nJobTracker's: " + jtBuildVersion + 
+              "\nTaskTracker's: "+ VersionInfo.getBuildVersion() +
+              " and " + CommonConfigurationKeys.HADOOP_RELAXED_VERSION_CHECK_KEY +
+              " is " + (relaxedVersionCheck ? "enabled" : "not enabled");
+            LOG.fatal(msg);
             try {
               jobClient.reportTaskTrackerError(taskTrackerName, null, msg);
             } catch(Exception e ) {

+ 119 - 0
src/test/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVersionCheck.java

@@ -0,0 +1,119 @@
+/**
+ * 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.hadoop.hdfs.server.datanode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+/**
+ * Test the version check the DN performs when connecting to the NN
+ */
+public class TestDataNodeVersionCheck {
+
+  /**
+   * Test the default DN version checking
+   */
+  @Test
+  public void testDefaultVersionCheck() throws IOException {
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration conf = new Configuration();
+      cluster = new MiniDFSCluster(conf, 1, true, null);
+
+      DataNode dn = cluster.getDataNodes().get(0);
+    
+      final NamespaceInfo currInfo = new NamespaceInfo(0, 0, 0);
+      assertTrue(dn.isPermittedVersion(currInfo));
+
+      // Different revisions are not permitted
+      NamespaceInfo infoDiffRev = new NamespaceInfo(0, 0, 0) {
+                @Override public String getRevision() { return "bogus"; }
+      };      
+      assertFalse("Different revision is not permitted",
+          dn.isPermittedVersion(infoDiffRev));
+
+      // Different versions are not permitted
+      NamespaceInfo infoDiffVersion = new NamespaceInfo(0, 0, 0) {
+        @Override public String getVersion() { return "bogus"; }
+        @Override public String getRevision() { return "bogus"; }
+      };
+      assertFalse("Different version is not permitted",
+          dn.isPermittedVersion(infoDiffVersion));
+
+      // A bogus version (matching revision but not version)
+      NamespaceInfo bogusVersion = new NamespaceInfo(0, 0, 0) {
+        @Override public String getVersion() { return "bogus"; }
+      };
+      try {
+        dn.isPermittedVersion(bogusVersion);
+        fail("Matched revision with mismatched version");
+      } catch (AssertionError ae) {
+        // Expected
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+  
+  /**
+   * Test the "relaxed" DN version checking
+   */
+  @Test
+  public void testRelaxedVersionCheck() throws IOException {
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration conf = new Configuration();
+      conf.setBoolean(
+          CommonConfigurationKeys.HADOOP_RELAXED_VERSION_CHECK_KEY, true);
+      cluster = new MiniDFSCluster(conf, 1, true, null);
+      
+      DataNode dn = cluster.getDataNodes().get(0);
+    
+      final NamespaceInfo currInfo = new NamespaceInfo(0, 0, 0);
+      assertTrue(dn.isPermittedVersion(currInfo));
+
+      // Different revisions are permitted
+      NamespaceInfo infoDiffRev = new NamespaceInfo(0, 0, 0) {
+        @Override public String getRevision() { return "bogus"; }
+      };      
+      assertTrue("Different revisions should be permitted",
+          dn.isPermittedVersion(infoDiffRev));
+
+      // Different versions are not permitted
+      NamespaceInfo infoDiffVersion = new NamespaceInfo(0, 0, 0) {
+        @Override public String getVersion() { return "bogus"; }
+        @Override public String getRevision() { return "bogus"; }
+      };
+      assertFalse("Different version is not permitted",
+          dn.isPermittedVersion(infoDiffVersion));
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+}

+ 91 - 0
src/test/org/apache/hadoop/mapred/TestTaskTrackerVersionCheck.java

@@ -0,0 +1,91 @@
+/**
+ * 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.hadoop.mapred;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.util.VersionInfo;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+/**
+ * Test the version check the TT performs when connecting to the JT
+ */
+public class TestTaskTrackerVersionCheck {
+
+  /**
+   * Test the default TT version checking
+   */
+  @Test
+  public void testDefaultVersionCheck() throws IOException {
+    MiniMRCluster mr = null;
+    try {
+      JobConf jtConf = new JobConf();
+      mr = new MiniMRCluster(1, "file:///", 1, null, null, jtConf);
+      TaskTracker tt = mr.getTaskTrackerRunner(0).getTaskTracker();
+      String currBuildVersion = VersionInfo.getBuildVersion();
+      String currVersion = VersionInfo.getVersion();
+
+      assertTrue(tt.isPermittedVersion(currBuildVersion, currVersion));
+      assertFalse("We disallow different versions",
+          tt.isPermittedVersion(currBuildVersion+"x", currVersion+"x"));
+      assertFalse("We disallow different full versions with same version",
+          tt.isPermittedVersion(currBuildVersion+"x", currVersion));      
+      try {
+        tt.isPermittedVersion(currBuildVersion, currVersion+"x");
+        fail("Matched full version with mismatched version");
+      } catch (AssertionError ae) {
+        // Expected. The versions should always match if the full
+        // versions match as the full version contains the version.
+      }
+    } finally {
+      if (mr != null) {
+        mr.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test the "relaxed" TT version checking
+   */
+  @Test
+  public void testRelaxedVersionCheck() throws IOException {
+    MiniMRCluster mr = null;
+    try {
+      JobConf jtConf = new JobConf();
+      jtConf.setBoolean(
+          CommonConfigurationKeys.HADOOP_RELAXED_VERSION_CHECK_KEY, true);
+      mr = new MiniMRCluster(1, "file:///", 1, null, null, jtConf);
+      TaskTracker tt = mr.getTaskTrackerRunner(0).getTaskTracker();
+      String currFullVersion = VersionInfo.getBuildVersion();
+      String currVersion = VersionInfo.getVersion();
+
+      assertTrue(tt.isPermittedVersion(currFullVersion, currVersion));
+      assertFalse("We dissallow different versions",
+          tt.isPermittedVersion(currFullVersion+"x", currVersion+"x"));
+      assertTrue("We allow different full versions with same version",
+          tt.isPermittedVersion(currFullVersion+"x", currVersion));
+    } finally {
+      if (mr != null) {
+        mr.shutdown();
+      }
+    }
+  }
+}