Преглед изворни кода

MAPREDUCE-5237. Add methods that were removed from ClusterStatus back into 2.x so as to be compatible with 1.x. Contributed by Zhijie Shen.
Incompatible as ClusterStatus.UNINITIALIZED_MEMORY_VALUE is a long now and so breaks 0.23.x but it shouldn't be a big deal in reality.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1487175 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli пре 12 година
родитељ
комит
0edae7a96a

+ 5 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -156,6 +156,11 @@ Release 2.0.5-beta - UNRELEASED
     when compared to 1.x. This breaks 0.23.x users of one API in Job. (Mayank
     Bansal via vinodkv)
 
+    MAPREDUCE-5237. Add methods that were removed from ClusterStatus back into
+    2.x so as to be compatible with 1.x. Incompatible as
+    ClusterStatus.UNINITIALIZED_MEMORY_VALUE is a long now and so breaks 0.23.x
+    but it shouldn't be a big deal in reality. (Zhijie Shen via vinodkv)
+
   NEW FEATURES
 
     HADOOP-8562. Enhancements to support Hadoop on Windows Server and Windows

+ 43 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ClusterStatus.java

@@ -23,14 +23,13 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.mapreduce.ClusterMetrics;
-import org.apache.hadoop.mapreduce.TaskTrackerInfo;
 import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
 import org.apache.hadoop.util.StringInterner;
 
@@ -175,7 +174,7 @@ public class ClusterStatus implements Writable {
     
   }
   
-  public static final int UNINITIALIZED_MEMORY_VALUE = -1;
+  public static final long UNINITIALIZED_MEMORY_VALUE = -1;
   
   private int numActiveTrackers;
   private Collection<String> activeTrackers = new ArrayList<String>();
@@ -316,7 +315,33 @@ public class ClusterStatus implements Writable {
     }
     return blacklistedTrackers;
   }
-  
+
+  /**
+   * Get the names of graylisted task trackers in the cluster.
+   *
+   * The gray list of trackers is no longer available on M/R 2.x. The function
+   * is kept to be compatible with M/R 1.x applications.
+   *
+   * @return an empty graylisted task trackers in the cluster.
+   */
+  @Deprecated
+  public Collection<String> getGraylistedTrackerNames() {
+    return Collections.emptySet();
+  }
+
+  /**
+   * Get the number of graylisted task trackers in the cluster.
+   *
+   * The gray list of trackers is no longer available on M/R 2.x. The function
+   * is kept to be compatible with M/R 1.x applications.
+   *
+   * @return 0 graylisted task trackers in the cluster.
+   */
+  @Deprecated
+  public int getGraylistedTrackers() {
+    return 0;
+  }
+
   /**
    * Get the number of blacklisted task trackers in the cluster.
    * 
@@ -413,6 +438,20 @@ public class ClusterStatus implements Writable {
     return blacklistedTrackersInfo;
   }
 
+  /**
+   * Get the current state of the <code>JobTracker</code>,
+   * as {@link JobTracker.State}
+   *
+   * {@link JobTracker.State} should no longer be used on M/R 2.x. The function
+   * is kept to be compatible with M/R 1.x applications.
+   *
+   * @return the invalid state of the <code>JobTracker</code>.
+   */
+  @Deprecated
+  public JobTracker.State getJobTrackerState() {
+    return JobTracker.State.RUNNING;
+  }
+
   public void write(DataOutput out) throws IOException {
     if (activeTrackers.size() == 0) {
       out.writeInt(numActiveTrackers);

+ 35 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobTracker.java

@@ -0,0 +1,35 @@
+/**
+ * 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;
+
+/**
+ * <code>JobTracker</code> is no longer used since M/R 2.x. This is a dummy
+ * JobTracker class, which is used to be compatible with M/R 1.x applications.
+ */
+public class JobTracker {
+
+  /**
+   * <code>State</code> is no longer used since M/R 2.x. It is kept in case
+   * that M/R 1.x applications may still use it.
+   */
+  public static enum State {
+    INITIALIZING, RUNNING
+  }
+
+}

+ 41 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestClusterStatus.java

@@ -0,0 +1,41 @@
+/**
+ * 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 junit.framework.Assert;
+
+import org.junit.Test;
+
+public class TestClusterStatus {
+
+  private ClusterStatus clusterStatus = new ClusterStatus();
+
+  @SuppressWarnings("deprecation")
+  @Test (timeout = 1000)
+  public void testGraylistedTrackers() {
+    Assert.assertEquals(0, clusterStatus.getGraylistedTrackers());
+    Assert.assertTrue(clusterStatus.getGraylistedTrackerNames().isEmpty());
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test (timeout = 1000)
+  public void testJobTrackerState() {
+    Assert.assertEquals(JobTracker.State.RUNNING,
+        clusterStatus.getJobTrackerState());
+  }
+}