Ver código fonte

MAPREDUCE-2400. Remove Cluster's dependency on JobTracker via a ServiceProvider for the actual implementation. Contributed by Tom White.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/MR-279@1145677 13f79535-47bb-0310-9956-ffa450edef68
Arun Murthy 14 anos atrás
pai
commit
3aba3821d0
17 arquivos alterados com 317 adições e 174 exclusões
  1. 3 0
      mapreduce/CHANGES.txt
  2. 1 0
      mapreduce/build.xml
  3. 66 0
      mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobTrackerClientProtocolProvider.java
  4. 0 105
      mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/ClientFactory.java
  5. 20 8
      mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
  6. 2 0
      mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRConfig.java
  7. 37 0
      mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/protocol/ClientProtocolProvider.java
  8. 28 0
      mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/resources/META-INF/services/org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider
  9. 8 0
      mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  10. 0 34
      mapreduce/mr-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YarnClientFactory.java
  11. 50 0
      mapreduce/mr-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YarnClientProtocolProvider.java
  12. 28 0
      mapreduce/mr-client/hadoop-mapreduce-client-jobclient/src/main/resources/META-INF/services/org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider
  13. 2 3
      mapreduce/mr-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
  14. 1 4
      mapreduce/mr-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
  15. 14 0
      mapreduce/src/java/META-INF/services/org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider
  16. 0 20
      mapreduce/src/java/org/apache/hadoop/mapred/LocalClientFactory.java
  17. 57 0
      mapreduce/src/java/org/apache/hadoop/mapred/LocalClientProtocolProvider.java

+ 3 - 0
mapreduce/CHANGES.txt

@@ -5,6 +5,9 @@ Trunk (unreleased changes)
 
     MAPREDUCE-279
 
+    MAPREDUCE-2400. Remove Cluster's dependency on JobTracker via a 
+    ServiceProvider for the actual implementation. (tomwhite via acmurthy) 
+ 
     MAPREDUCE-2628. Add compiled on date to NM and RM info/about page.
  
     MAPREDUCE-2645. Updates to MRv2 INSTALL and README documentation.

+ 1 - 0
mapreduce/build.xml

@@ -384,6 +384,7 @@
     
     <copy todir="${build.classes}">
       <fileset dir="${mapred.src.dir}" includes="**/*.properties"/>
+      <fileset dir="${mapred.src.dir}" includes="**/META-INF/services/*"/>
       <fileset dir="${mapred.src.dir}" includes="mapred-default.xml"/>
       <fileset dir="${mapred.src.dir}" includes="mapred-queues-default.xml"/>
     </copy>

+ 66 - 0
mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobTrackerClientProtocolProvider.java

@@ -0,0 +1,66 @@
+/**
+ * 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 java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+
+@InterfaceAudience.Private
+public class JobTrackerClientProtocolProvider extends ClientProtocolProvider {
+
+  @Override
+  public ClientProtocol create(Configuration conf) throws IOException {
+    String framework = conf.get(MRConfig.FRAMEWORK_NAME);
+    if (framework != null && !framework.equals("classic")) {
+      return null;
+    }
+    String tracker = conf.get(MRConfig.MASTER_ADDRESS, "local");
+    if (!"local".equals(tracker)) {
+      return createRPCProxy(Master.getMasterAddress(conf), conf);
+    }
+    return null;
+  }
+
+  @Override
+  public ClientProtocol create(InetSocketAddress addr, Configuration conf) throws IOException {
+    return createRPCProxy(addr, conf);
+  }
+  
+  private ClientProtocol createRPCProxy(InetSocketAddress addr,
+      Configuration conf) throws IOException {
+    return (ClientProtocol) RPC.getProxy(ClientProtocol.class,
+      ClientProtocol.versionID, addr, UserGroupInformation.getCurrentUser(),
+      conf, NetUtils.getSocketFactory(conf, ClientProtocol.class));
+  }
+
+  @Override
+  public void close(ClientProtocol clientProtocol) throws IOException {
+    RPC.stopProxy(clientProtocol);
+  }
+
+}

+ 0 - 105
mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/ClientFactory.java

@@ -1,105 +0,0 @@
-/*
- * 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.mapreduce;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.mapred.Master;
-import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.UserGroupInformation;
-
-/**
- * Class to instantiate ClientProtocol proxy handle.
- *
- */
-public abstract class ClientFactory {
-  private static final Log LOG = LogFactory.getLog(ClientFactory.class);
-  
-  @SuppressWarnings("unchecked")
-  public static ClientProtocol create(Configuration conf) throws IOException {
-    Class<ClientFactory> factory = (Class<ClientFactory>) conf.getClass(
-        "mapreduce.clientfactory.class.name", 
-        DefaultClientFactory.class); 
-    try {
-      return factory.newInstance().createClient(conf);
-    } catch (Exception e) {
-      throw new IOException("Could not create ClientProtocol using factory: "
-          + factory.getName(), e);
-    }
-  }
-
-  protected abstract ClientProtocol createClient(Configuration conf) 
-  throws IOException;
-
-  //the default factory handles the backward compatibility
-  public static class DefaultClientFactory extends ClientFactory {
-
-    @Override
-    protected ClientProtocol createClient(Configuration conf)
-    throws IOException {
-      String tracker = conf.get(MRConfig.MASTER_ADDRESS);
-      if ("local".equals(tracker)) {
-        return createLocalClient(conf);
-      } else {
-        return createJTClient(conf);
-      }
-    }
-  }
-
-  public static class NullDefaultClientFactory extends ClientFactory {
-
-    @Override
-    protected ClientProtocol createClient(Configuration conf)
-    throws IOException {
-      throw new IOException("Local Client factory not set. No Default");
-    }	  
-  }
-
-  public ClientProtocol createLocalClient(Configuration conf) 
-  throws IOException {
-    conf.setInt("mapreduce.job.maps", 1);
-    /* find the local job runner in config */	
-    @SuppressWarnings("unchecked")
-    Class<ClientFactory> localfactory = (Class<ClientFactory>) conf.getClass(
-        "mapreduce.local.clientfactory.class.name", NullDefaultClientFactory.class);
-    try {
-      return localfactory.newInstance().createClient(conf);
-    } catch(Exception e) {
-      throw new IOException("could not create local client", e);
-    }
-  }
-
-  public ClientProtocol createJTClient(Configuration conf) throws IOException {
-    return createJTClient(Master.getMasterAddress(conf), conf);
-  }
-
-  public ClientProtocol createJTClient(InetSocketAddress addr,
-      Configuration conf) throws IOException {
-    return (ClientProtocol) RPC.getProxy(ClientProtocol.class,
-        ClientProtocol.versionID, addr, 
-        UserGroupInformation.getCurrentUser(), conf,
-        NetUtils.getSocketFactory(conf, ClientProtocol.class));
-  }
-}

+ 20 - 8
mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java

@@ -23,6 +23,7 @@ import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.ServiceLoader;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -36,8 +37,8 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Master;
-import org.apache.hadoop.mapreduce.ClientFactory.DefaultClientFactory;
 import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider;
 import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.mapreduce.util.ConfigUtil;
 import org.apache.hadoop.net.NetUtils;
@@ -53,6 +54,7 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 @InterfaceStability.Evolving
 public class Cluster {
   private static final Log LOG = LogFactory.getLog(Cluster.class);
+  private ClientProtocolProvider clientProtocolProvider;
   private ClientProtocol client;
   private UserGroupInformation ugi;
   private Configuration conf;
@@ -68,15 +70,28 @@ public class Cluster {
   public Cluster(Configuration conf) throws IOException {
     this.conf = conf;
     this.ugi = UserGroupInformation.getCurrentUser();
-    client = ClientFactory.create(conf);
+    for (ClientProtocolProvider provider : ServiceLoader.load(ClientProtocolProvider.class)) {
+      ClientProtocol clientProtocol = provider.create(conf);
+      if (clientProtocol != null) {
+        clientProtocolProvider = provider;
+        client = clientProtocol;
+        break;
+      }
+    }
   }
 
   public Cluster(InetSocketAddress jobTrackAddr, Configuration conf) 
       throws IOException {
     this.conf = conf;
     this.ugi = UserGroupInformation.getCurrentUser();
-    client = (new DefaultClientFactory()).createJTClient(jobTrackAddr, 
-        conf); 
+    for (ClientProtocolProvider provider : ServiceLoader.load(ClientProtocolProvider.class)) {
+      ClientProtocol clientProtocol = provider.create(jobTrackAddr, conf);
+      if (clientProtocol != null) {
+        clientProtocolProvider = provider;
+        client = clientProtocol;
+        break;
+      }
+    }
   }
 
   
@@ -92,10 +107,7 @@ public class Cluster {
    * Close the <code>Cluster</code>.
    */
   public synchronized void close() throws IOException {
-    // FIXME
-    //if (!(client instanceof LocalJobRunner)) {
-      RPC.stopProxy(client);
-    //}
+    clientProtocolProvider.close(client);
   }
 
   private Job[] getJobs(JobStatus[] stats) throws IOException {

+ 2 - 0
mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRConfig.java

@@ -72,4 +72,6 @@ public interface MRConfig {
   public static final String MASTER_USER_NAME = 
     "mapreduce.jobtracker.kerberos.principal";
 
+  public static final String FRAMEWORK_NAME  = "mapreduce.framework.name";
+
 }

+ 37 - 0
mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/protocol/ClientProtocolProvider.java

@@ -0,0 +1,37 @@
+/**
+ * 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.mapreduce.protocol;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+@InterfaceAudience.Private
+public abstract class ClientProtocolProvider {
+  
+  public abstract ClientProtocol create(Configuration conf) throws IOException;
+  
+  public abstract ClientProtocol create(InetSocketAddress addr,
+      Configuration conf) throws IOException;
+
+  public abstract void close(ClientProtocol clientProtocol) throws IOException;
+
+}

+ 28 - 0
mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/resources/META-INF/services/org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider

@@ -0,0 +1,28 @@
+#
+#   Licensed 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.
+#
+org.apache.hadoop.mapred.JobTrackerClientProtocolProvider
+#
+#   Licensed 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.
+#
+org.apache.hadoop.mapred.JobTrackerClientProtocolProvider

+ 8 - 0
mapreduce/mr-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -1164,4 +1164,12 @@
   </description>
 </property>
 
+<property>
+  <name>mapreduce.framework.name</name>
+  <value>yarn</value>
+  <description>The runtime framework for executing MapReduce jobs.
+  Can be one of local, classic or yarn.
+  </description>
+</property>
+
 </configuration>

+ 0 - 34
mapreduce/mr-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YarnClientFactory.java

@@ -1,34 +0,0 @@
-/**
-* 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.conf.Configuration;
-import org.apache.hadoop.mapreduce.ClientFactory;
-import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
-
-public class YarnClientFactory extends ClientFactory {
-
-    @Override
-    protected ClientProtocol createClient(Configuration conf)
-        throws IOException {
-      return new YARNRunner(conf);
-    }
-  }

+ 50 - 0
mapreduce/mr-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YarnClientProtocolProvider.java

@@ -0,0 +1,50 @@
+/**
+ * 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 java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider;
+
+public class YarnClientProtocolProvider extends ClientProtocolProvider {
+
+  @Override
+  public ClientProtocol create(Configuration conf) throws IOException {
+    if ("yarn".equals(conf.get(MRConfig.FRAMEWORK_NAME))) {
+      return new YARNRunner(conf);
+    }
+    return null;
+  }
+
+  @Override
+  public ClientProtocol create(InetSocketAddress addr, Configuration conf)
+      throws IOException {
+    return create(conf);
+  }
+
+  @Override
+  public void close(ClientProtocol clientProtocol) throws IOException {
+    // nothing to do
+  }
+
+}

+ 28 - 0
mapreduce/mr-client/hadoop-mapreduce-client-jobclient/src/main/resources/META-INF/services/org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider

@@ -0,0 +1,28 @@
+#
+#   Licensed 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.
+#
+org.apache.hadoop.mapred.YarnClientProtocolProvider
+#
+#   Licensed 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.
+#
+org.apache.hadoop.mapred.YarnClientProtocolProvider

+ 2 - 3
mapreduce/mr-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java

@@ -29,8 +29,8 @@ import org.apache.avro.ipc.Server;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.ClientFactory;
 import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptResponse;
@@ -116,8 +116,7 @@ public class TestClientRedirect {
   public void testRedirect() throws Exception {
     
     Configuration conf = new YarnConfiguration();
-    conf.setClass("mapreduce.clientfactory.class.name",
-        YarnClientFactory.class, ClientFactory.class);
+    conf.set(MRConfig.FRAMEWORK_NAME, "yarn");
     conf.set(YarnConfiguration.APPSMANAGER_ADDRESS, RMADDRESS);
     conf.set(JHConfig.HS_BIND_ADDRESS, HSHOSTADDRESS);
     RMService rmService = new RMService("test");

+ 1 - 4
mapreduce/mr-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java

@@ -25,8 +25,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.ShuffleHandler;
-import org.apache.hadoop.mapred.YarnClientFactory;
-import org.apache.hadoop.mapreduce.ClientFactory;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.hs.JobHistoryServer;
@@ -62,8 +60,7 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
 
   @Override
   public void init(Configuration conf) {
-    conf.setClass("mapreduce.clientfactory.class.name",
-        YarnClientFactory.class, ClientFactory.class);
+    conf.set(MRConfig.FRAMEWORK_NAME, "yarn");
     conf.setStrings(MRConstants.NM_HOSTS_CONF_KEY,
         new String[] { NMConfig.DEFAULT_NM_BIND_ADDRESS });
     conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name"));

+ 14 - 0
mapreduce/src/java/META-INF/services/org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider

@@ -0,0 +1,14 @@
+#
+#   Licensed 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.
+#
+org.apache.hadoop.mapred.LocalClientProtocolProvider

+ 0 - 20
mapreduce/src/java/org/apache/hadoop/mapred/LocalClientFactory.java

@@ -1,20 +0,0 @@
-package org.apache.hadoop.mapred;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.ClientFactory;
-import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
-
-/**
- * Factory responsible for local job runner clients.
- *
- */
-public class LocalClientFactory extends ClientFactory {
-
-	@Override
-	protected ClientProtocol createClient(Configuration conf)
-			throws IOException {
-		return new LocalJobRunner(conf);
-	}
-}

+ 57 - 0
mapreduce/src/java/org/apache/hadoop/mapred/LocalClientProtocolProvider.java

@@ -0,0 +1,57 @@
+/**
+ * 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 java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider;
+
+@InterfaceAudience.Private
+public class LocalClientProtocolProvider extends ClientProtocolProvider {
+
+  @Override
+  public ClientProtocol create(Configuration conf) throws IOException {
+    String framework = conf.get(MRConfig.FRAMEWORK_NAME);
+    if (framework != null && !framework.equals("local")) {
+      return null;
+    }
+    String tracker = conf.get(MRConfig.MASTER_ADDRESS, "local");
+    if ("local".equals(tracker)) {
+      conf.setInt("mapreduce.job.maps", 1);
+      return new LocalJobRunner(conf);
+    }
+    return null;
+  }
+
+  @Override
+  public ClientProtocol create(InetSocketAddress addr, Configuration conf) {
+    return null; // LocalJobRunner doesn't use a socket
+  }
+
+  @Override
+  public void close(ClientProtocol clientProtocol) {
+    // no clean up required
+  }
+
+}