瀏覽代碼

MAPREDUCE-461. Enable service-plugins for JobTracker. Contributed by Fredrik Hedberg. Backported by Brandon Li.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1@1389631 13f79535-47bb-0310-9956-ffa450edef68
Vinod Kumar Vavilapalli 13 年之前
父節點
當前提交
7068e92976

+ 3 - 0
CHANGES.txt

@@ -30,6 +30,9 @@ Release 1.2.0 - unreleased
     all failed tasks as KILLED for future retries and doesn't accept new job
     submissions. (acmurthy)
 
+    MAPREDUCE-461. Enable service-plugins for JobTracker. (Fredrik Hedberg and
+    Brandon Li via vinodkv)
+
   IMPROVEMENTS
 
     HDFS-3515. Port HDFS-1457 to branch-1. (eli)

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

@@ -107,6 +107,7 @@ import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.mortbay.util.ajax.JSON;
@@ -217,6 +218,8 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   private final List<JobInProgressListener> jobInProgressListeners =
     new CopyOnWriteArrayList<JobInProgressListener>();
 
+  private List<ServicePlugin> plugins;
+  
   private static final LocalDirAllocator lDirAlloc = 
                               new LocalDirAllocator("mapred.local.dir");
   //system directory is completely owned by the JobTracker
@@ -2042,7 +2045,6 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
       hdfsMonitor = new HDFSMonitorThread(this.conf, this, this.fs);
       hdfsMonitor.start();
     }
-
   }
   
   JobTracker(final JobConf conf, String identifier, Clock clock, QueueManager qm) 
@@ -2211,7 +2213,18 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
             DNSToSwitchMapping.class), conf);
     this.numTaskCacheLevels = conf.getInt("mapred.task.cache.levels", 
         NetworkTopology.DEFAULT_HOST_LEVEL);
-
+    
+    plugins = conf.getInstances("mapreduce.jobtracker.plugins",
+        ServicePlugin.class);
+    for (ServicePlugin p : plugins) {
+      try {
+        p.start(this);
+        LOG.info("Started plug-in " + p + " of type " + p.getClass());
+      } catch (Throwable t) {
+        LOG.warn("ServicePlugin " + p + " of type " + p.getClass()
+            + " could not be started", t);
+      }
+    }
   }
 
   private static SimpleDateFormat getDateFormat() {
@@ -2363,6 +2376,17 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   }
 
   void close() throws IOException {
+    if (plugins != null) {
+      for (ServicePlugin p : plugins) {
+        try {
+          p.stop();
+          LOG.info("Stopped plug-in " + p + " of type " + p.getClass());
+        } catch (Throwable t) {
+          LOG.warn("ServicePlugin " + p + " of type " + p.getClass()
+              + " could not be stopped", t);
+        }
+      }
+    }
     if (this.infoServer != null) {
       LOG.info("Stopping infoServer");
       try {

+ 90 - 0
src/test/org/apache/hadoop/mapred/TestJobTrackerPlugins.java

@@ -0,0 +1,90 @@
+/**
+ * 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 junit.framework.TestCase;
+
+//import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.util.ServicePlugin;
+import org.junit.Test;
+
+public class TestJobTrackerPlugins extends TestCase {
+  
+  static class FakeServicePlugin implements ServicePlugin {
+
+    private static FakeServicePlugin instance;
+    
+    public static FakeServicePlugin getInstance() {
+      return instance;
+    }
+    
+    private Object service;
+    private boolean stopped;
+    
+    public Object getService() {
+      return service;
+    }
+    
+    public boolean isStopped() {
+      return stopped;
+    }
+    
+    public FakeServicePlugin() {
+      // store static reference to instance so we can retrieve it in the test
+      instance = this;
+    }
+    
+    @Override
+    public void start(Object service) {
+      this.service = service;
+    }
+
+    @Override
+    public void stop() {
+      stopped = true;
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+  }
+  
+  @Test
+  public void test() throws Exception {
+    JobConf conf = new JobConf();
+    conf.set("mapred.job.tracker", "localhost:0");
+    conf.set("mapred.job.tracker.http.address", "0.0.0.0:0");
+    conf.setClass("mapreduce.jobtracker.plugins", FakeServicePlugin.class,
+        ServicePlugin.class);
+    
+    assertNull("Plugin not created", FakeServicePlugin.getInstance());
+    
+    JobTracker jobTracker = JobTracker.startTracker(conf);
+    assertNotNull("Plugin created", FakeServicePlugin.getInstance());
+    assertSame("Service is jobTracker",
+        FakeServicePlugin.getInstance().getService(), jobTracker);
+    assertFalse("Plugin not stopped",
+        FakeServicePlugin.getInstance().isStopped());
+    
+    jobTracker.close();
+    assertTrue("Plugin stopped", FakeServicePlugin.getInstance().isStopped());
+  }
+
+}