- distributed_scheduler_protocol.proto
+ distributed_scheduling_am_protocol.proto
yarn_server_common_protos.proto
yarn_server_common_service_protos.proto
yarn_server_common_service_protos.proto
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulerProtocol.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulerProtocol.java
deleted file mode 100644
index 26faa8f..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulerProtocol.java
+++ /dev/null
@@ -1,78 +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.yarn.server.api;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.io.retry.Idempotent;
-import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedRegisterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-
-import java.io.IOException;
-
-/**
- * This protocol extends the ApplicationMasterProtocol. It is
- * used by the LocalScheduler running on the NodeManager to wrap
- * the request / response objects of the registerApplicationMaster
- * and allocate methods of the protocol with addition information
- * required to perform Distributed Scheduling.
- *
- */
-public interface DistributedSchedulerProtocol
- extends ApplicationMasterProtocol {
-
- /**
- * Extends the registerApplicationMaster to wrap the response
- * with additional metadata.
- *
- * @param request ApplicationMaster registration request
- * @return A DistSchedRegisterResponse that contains a standard
- * AM registration response along with additional information required
- * for Distributed Scheduling
- * @throws YarnException
- * @throws IOException
- */
- @Public
- @Unstable
- @Idempotent
- DistSchedRegisterResponse registerApplicationMasterForDistributedScheduling(
- RegisterApplicationMasterRequest request)
- throws YarnException, IOException;
-
- /**
- * Extends the allocate to wrap the response with additional
- * metadata.
- *
- * @param request ApplicationMaster allocate request
- * @return A DistSchedAllocateResponse that contains a standard
- * AM allocate response along with additional information required
- * for Distributed Scheduling
- * @throws YarnException
- * @throws IOException
- */
- @Public
- @Unstable
- @Idempotent
- DistSchedAllocateResponse allocateForDistributedScheduling(
- DistSchedAllocateRequest request) throws YarnException, IOException;
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulerProtocolPB.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulerProtocolPB.java
deleted file mode 100644
index ce7911c..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulerProtocolPB.java
+++ /dev/null
@@ -1,36 +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.yarn.server.api;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB;
-import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService;
-import org.apache.hadoop.yarn.proto.DistributedSchedulerProtocol.DistributedSchedulerProtocolService;
-
-@Private
-@Unstable
-@ProtocolInfo(protocolName = "org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocolPB",
- protocolVersion = 1)
-public interface DistributedSchedulerProtocolPB extends
- DistributedSchedulerProtocolService.BlockingInterface,
- ApplicationMasterProtocolService.BlockingInterface,
- ApplicationMasterProtocolPB {
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulingAMProtocol.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulingAMProtocol.java
new file mode 100644
index 0000000..d1ed1fc
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulingAMProtocol.java
@@ -0,0 +1,87 @@
+/**
+ * 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.yarn.server.api;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.io.retry.Idempotent;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+import java.io.IOException;
+
+/**
+ *
+ * This protocol extends the ApplicationMasterProtocol. It is used
+ * by the DistributedScheduler running on the NodeManager to wrap
+ * the request / response objects of the registerApplicationMaster
+ * and allocate methods of the protocol with additional information
+ * required to perform distributed scheduling.
+ *
+ */
+public interface DistributedSchedulingAMProtocol
+ extends ApplicationMasterProtocol {
+
+ /**
+ *
+ * Extends the registerApplicationMaster to wrap the response
+ * with additional metadata.
+ *
+ *
+ * @param request
+ * ApplicationMaster registration request
+ * @return A RegisterDistributedSchedulingAMResponse that
+ * contains a standard AM registration response along with additional
+ * information required for distributed scheduling
+ * @throws YarnException YarnException
+ * @throws IOException IOException
+ */
+ @Public
+ @Unstable
+ @Idempotent
+ RegisterDistributedSchedulingAMResponse
+ registerApplicationMasterForDistributedScheduling(
+ RegisterApplicationMasterRequest request)
+ throws YarnException, IOException;
+
+ /**
+ *
+ * Extends the allocate to wrap the response with additional
+ * metadata.
+ *
+ *
+ * @param request
+ * ApplicationMaster allocate request
+ * @return A DistributedSchedulingAllocateResponse that contains
+ * a standard AM allocate response along with additional information
+ * required for distributed scheduling
+ * @throws YarnException YarnException
+ * @throws IOException IOException
+ */
+ @Public
+ @Unstable
+ @Idempotent
+ DistributedSchedulingAllocateResponse allocateForDistributedScheduling(
+ DistributedSchedulingAllocateRequest request)
+ throws YarnException, IOException;
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulingAMProtocolPB.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulingAMProtocolPB.java
new file mode 100644
index 0000000..674d4e4
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulingAMProtocolPB.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.yarn.server.api;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB;
+import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService;
+import org.apache.hadoop.yarn.proto.DistributedSchedulingAMProtocol.DistributedSchedulingAMProtocolService;
+
+@Private
+@Unstable
+@ProtocolInfo(protocolName =
+ "org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocolPB",
+ protocolVersion = 1)
+public interface DistributedSchedulingAMProtocolPB extends
+ DistributedSchedulingAMProtocolService.BlockingInterface,
+ ApplicationMasterProtocolService.BlockingInterface,
+ ApplicationMasterProtocolPB {
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java
index c23e27c..8555fc3 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java
@@ -78,7 +78,7 @@ protected InetSocketAddress getRMAddress(YarnConfiguration conf,
YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_ADDRESS,
YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_PORT);
- } else if (protocol == DistributedSchedulerProtocol.class) {
+ } else if (protocol == DistributedSchedulingAMProtocol.class) {
return conf.getSocketAddr(YarnConfiguration.RM_SCHEDULER_ADDRESS,
YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/DistributedSchedulerProtocolPBClientImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/DistributedSchedulerProtocolPBClientImpl.java
deleted file mode 100644
index 0ca61df..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/DistributedSchedulerProtocolPBClientImpl.java
+++ /dev/null
@@ -1,154 +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.yarn.server.api.impl.pb.client;
-
-import com.google.protobuf.ServiceException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos;
-import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedRegisterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
-
-
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistSchedAllocateRequestPBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistSchedAllocateResponsePBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistSchedRegisterResponsePBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .FinishApplicationMasterRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .FinishApplicationMasterResponsePBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .RegisterApplicationMasterRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .RegisterApplicationMasterResponsePBImpl;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.ipc.RPCUtil;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos;
-import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocolPB;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-public class DistributedSchedulerProtocolPBClientImpl implements
- DistributedSchedulerProtocol, Closeable {
-
- private DistributedSchedulerProtocolPB proxy;
-
- public DistributedSchedulerProtocolPBClientImpl(long clientVersion,
- InetSocketAddress addr,
- Configuration conf) throws IOException {
- RPC.setProtocolEngine(conf, DistributedSchedulerProtocolPB.class,
- ProtobufRpcEngine.class);
- proxy = RPC.getProxy(DistributedSchedulerProtocolPB.class, clientVersion,
- addr, conf);
- }
-
- @Override
- public void close() {
- if (this.proxy != null) {
- RPC.stopProxy(this.proxy);
- }
- }
-
- @Override
- public DistSchedRegisterResponse
- registerApplicationMasterForDistributedScheduling(
- RegisterApplicationMasterRequest request)
- throws YarnException, IOException {
- YarnServiceProtos.RegisterApplicationMasterRequestProto requestProto =
- ((RegisterApplicationMasterRequestPBImpl) request).getProto();
- try {
- return new DistSchedRegisterResponsePBImpl(
- proxy.registerApplicationMasterForDistributedScheduling(
- null, requestProto));
- } catch (ServiceException e) {
- RPCUtil.unwrapAndThrowException(e);
- return null;
- }
- }
-
- @Override
- public DistSchedAllocateResponse allocateForDistributedScheduling(
- DistSchedAllocateRequest request) throws YarnException, IOException {
- YarnServerCommonServiceProtos.DistSchedAllocateRequestProto requestProto =
- ((DistSchedAllocateRequestPBImpl) request).getProto();
- try {
- return new DistSchedAllocateResponsePBImpl(
- proxy.allocateForDistributedScheduling(null, requestProto));
- } catch (ServiceException e) {
- RPCUtil.unwrapAndThrowException(e);
- return null;
- }
- }
-
- @Override
- public RegisterApplicationMasterResponse registerApplicationMaster
- (RegisterApplicationMasterRequest request) throws YarnException,
- IOException {
- YarnServiceProtos.RegisterApplicationMasterRequestProto requestProto =
- ((RegisterApplicationMasterRequestPBImpl) request).getProto();
- try {
- return new RegisterApplicationMasterResponsePBImpl(
- proxy.registerApplicationMaster(null, requestProto));
- } catch (ServiceException e) {
- RPCUtil.unwrapAndThrowException(e);
- return null;
- }
- }
-
- @Override
- public FinishApplicationMasterResponse finishApplicationMaster
- (FinishApplicationMasterRequest request) throws YarnException,
- IOException {
- YarnServiceProtos.FinishApplicationMasterRequestProto requestProto =
- ((FinishApplicationMasterRequestPBImpl) request).getProto();
- try {
- return new FinishApplicationMasterResponsePBImpl(
- proxy.finishApplicationMaster(null, requestProto));
- } catch (ServiceException e) {
- RPCUtil.unwrapAndThrowException(e);
- return null;
- }
- }
-
- @Override
- public AllocateResponse allocate(AllocateRequest request) throws
- YarnException, IOException {
- YarnServiceProtos.AllocateRequestProto requestProto =
- ((AllocateRequestPBImpl) request).getProto();
- try {
- return new AllocateResponsePBImpl(proxy.allocate(null, requestProto));
- } catch (ServiceException e) {
- RPCUtil.unwrapAndThrowException(e);
- return null;
- }
- }
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/DistributedSchedulingAMProtocolPBClientImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/DistributedSchedulingAMProtocolPBClientImpl.java
new file mode 100644
index 0000000..66893b3
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/DistributedSchedulingAMProtocolPBClientImpl.java
@@ -0,0 +1,153 @@
+/**
+ * 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.yarn.server.api.impl.pb.client;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos;
+import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistributedSchedulingAllocateRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistributedSchedulingAllocateResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterDistributedSchedulingAMResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocolPB;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+/**
+ * Implementation of {@link DistributedSchedulingAMProtocol}, used when
+ * distributed scheduling is enabled.
+ */
+public class DistributedSchedulingAMProtocolPBClientImpl implements
+ DistributedSchedulingAMProtocol, Closeable {
+
+ private DistributedSchedulingAMProtocolPB proxy;
+
+ public DistributedSchedulingAMProtocolPBClientImpl(long clientVersion,
+ InetSocketAddress addr, Configuration conf) throws IOException {
+ RPC.setProtocolEngine(conf, DistributedSchedulingAMProtocolPB.class,
+ ProtobufRpcEngine.class);
+ proxy = RPC.getProxy(DistributedSchedulingAMProtocolPB.class, clientVersion,
+ addr, conf);
+ }
+
+ @Override
+ public void close() {
+ if (this.proxy != null) {
+ RPC.stopProxy(this.proxy);
+ }
+ }
+
+ @Override
+ public RegisterDistributedSchedulingAMResponse
+ registerApplicationMasterForDistributedScheduling(
+ RegisterApplicationMasterRequest request)
+ throws YarnException, IOException {
+ YarnServiceProtos.RegisterApplicationMasterRequestProto requestProto =
+ ((RegisterApplicationMasterRequestPBImpl) request).getProto();
+ try {
+ return new RegisterDistributedSchedulingAMResponsePBImpl(
+ proxy.registerApplicationMasterForDistributedScheduling(
+ null, requestProto));
+ } catch (ServiceException e) {
+ RPCUtil.unwrapAndThrowException(e);
+ return null;
+ }
+ }
+
+ @Override
+ public DistributedSchedulingAllocateResponse allocateForDistributedScheduling(
+ DistributedSchedulingAllocateRequest request)
+ throws YarnException, IOException {
+ YarnServerCommonServiceProtos.DistributedSchedulingAllocateRequestProto
+ requestProto =
+ ((DistributedSchedulingAllocateRequestPBImpl) request).getProto();
+ try {
+ return new DistributedSchedulingAllocateResponsePBImpl(
+ proxy.allocateForDistributedScheduling(null, requestProto));
+ } catch (ServiceException e) {
+ RPCUtil.unwrapAndThrowException(e);
+ return null;
+ }
+ }
+
+ @Override
+ public RegisterApplicationMasterResponse registerApplicationMaster(
+ RegisterApplicationMasterRequest request)
+ throws YarnException, IOException {
+ YarnServiceProtos.RegisterApplicationMasterRequestProto requestProto =
+ ((RegisterApplicationMasterRequestPBImpl) request).getProto();
+ try {
+ return new RegisterApplicationMasterResponsePBImpl(
+ proxy.registerApplicationMaster(null, requestProto));
+ } catch (ServiceException e) {
+ RPCUtil.unwrapAndThrowException(e);
+ return null;
+ }
+ }
+
+ @Override
+ public FinishApplicationMasterResponse finishApplicationMaster(
+ FinishApplicationMasterRequest request)
+ throws YarnException, IOException {
+ YarnServiceProtos.FinishApplicationMasterRequestProto requestProto =
+ ((FinishApplicationMasterRequestPBImpl) request).getProto();
+ try {
+ return new FinishApplicationMasterResponsePBImpl(
+ proxy.finishApplicationMaster(null, requestProto));
+ } catch (ServiceException e) {
+ RPCUtil.unwrapAndThrowException(e);
+ return null;
+ }
+ }
+
+ @Override
+ public AllocateResponse allocate(AllocateRequest request)
+ throws YarnException, IOException {
+ YarnServiceProtos.AllocateRequestProto requestProto =
+ ((AllocateRequestPBImpl) request).getProto();
+ try {
+ return new AllocateResponsePBImpl(proxy.allocate(null, requestProto));
+ } catch (ServiceException e) {
+ RPCUtil.unwrapAndThrowException(e);
+ return null;
+ }
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/DistributedSchedulerProtocolPBServiceImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/DistributedSchedulerProtocolPBServiceImpl.java
deleted file mode 100644
index 2763259..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/DistributedSchedulerProtocolPBServiceImpl.java
+++ /dev/null
@@ -1,146 +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.yarn.server.api.impl.pb.service;
-
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocol;
-import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos;
-import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocolPB;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedRegisterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords
- .FinishApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistSchedAllocateRequestPBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistSchedAllocateResponsePBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistSchedRegisterResponsePBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .FinishApplicationMasterRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .FinishApplicationMasterResponsePBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProto;
-
-import java.io.IOException;
-
-public class DistributedSchedulerProtocolPBServiceImpl implements
- DistributedSchedulerProtocolPB {
-
- private DistributedSchedulerProtocol real;
-
- public DistributedSchedulerProtocolPBServiceImpl(
- DistributedSchedulerProtocol impl) {
- this.real = impl;
- }
-
- @Override
- public YarnServerCommonServiceProtos.DistSchedRegisterResponseProto
- registerApplicationMasterForDistributedScheduling(RpcController controller,
- RegisterApplicationMasterRequestProto proto) throws
- ServiceException {
- RegisterApplicationMasterRequestPBImpl request = new
- RegisterApplicationMasterRequestPBImpl(proto);
- try {
- DistSchedRegisterResponse response =
- real.registerApplicationMasterForDistributedScheduling(request);
- return ((DistSchedRegisterResponsePBImpl) response).getProto();
- } catch (YarnException e) {
- throw new ServiceException(e);
- } catch (IOException e) {
- throw new ServiceException(e);
- }
- }
-
- @Override
- public YarnServerCommonServiceProtos.DistSchedAllocateResponseProto
- allocateForDistributedScheduling(RpcController controller,
- YarnServerCommonServiceProtos.DistSchedAllocateRequestProto proto)
- throws ServiceException {
- DistSchedAllocateRequestPBImpl request =
- new DistSchedAllocateRequestPBImpl(proto);
- try {
- DistSchedAllocateResponse response = real
- .allocateForDistributedScheduling(request);
- return ((DistSchedAllocateResponsePBImpl) response).getProto();
- } catch (YarnException e) {
- throw new ServiceException(e);
- } catch (IOException e) {
- throw new ServiceException(e);
- }
- }
-
- @Override
- public YarnServiceProtos.AllocateResponseProto allocate(RpcController arg0,
- AllocateRequestProto proto) throws ServiceException {
- AllocateRequestPBImpl request = new AllocateRequestPBImpl(proto);
- try {
- AllocateResponse response = real.allocate(request);
- return ((AllocateResponsePBImpl) response).getProto();
- } catch (YarnException e) {
- throw new ServiceException(e);
- } catch (IOException e) {
- throw new ServiceException(e);
- }
- }
-
- @Override
- public YarnServiceProtos.FinishApplicationMasterResponseProto
- finishApplicationMaster(
- RpcController arg0, YarnServiceProtos
- .FinishApplicationMasterRequestProto proto)
- throws ServiceException {
- FinishApplicationMasterRequestPBImpl request = new
- FinishApplicationMasterRequestPBImpl(proto);
- try {
- FinishApplicationMasterResponse response = real.finishApplicationMaster
- (request);
- return ((FinishApplicationMasterResponsePBImpl) response).getProto();
- } catch (YarnException e) {
- throw new ServiceException(e);
- } catch (IOException e) {
- throw new ServiceException(e);
- }
- }
-
- @Override
- public YarnServiceProtos.RegisterApplicationMasterResponseProto
- registerApplicationMaster(
- RpcController arg0, RegisterApplicationMasterRequestProto proto)
- throws ServiceException {
- RegisterApplicationMasterRequestPBImpl request = new
- RegisterApplicationMasterRequestPBImpl(proto);
- try {
- RegisterApplicationMasterResponse response = real
- .registerApplicationMaster(request);
- return ((RegisterApplicationMasterResponsePBImpl) response).getProto();
- } catch (YarnException e) {
- throw new ServiceException(e);
- } catch (IOException e) {
- throw new ServiceException(e);
- }
- }
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/DistributedSchedulingAMProtocolPBServiceImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/DistributedSchedulingAMProtocolPBServiceImpl.java
new file mode 100644
index 0000000..68cc077
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/DistributedSchedulingAMProtocolPBServiceImpl.java
@@ -0,0 +1,151 @@
+/**
+ * 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.yarn.server.api.impl.pb.service;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos;
+import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistributedSchedulingAllocateRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistributedSchedulingAllocateResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterDistributedSchedulingAMResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProto;
+
+import java.io.IOException;
+
+/**
+ * Implementation of {@link DistributedSchedulingAMProtocolPB}.
+ */
+public class DistributedSchedulingAMProtocolPBServiceImpl implements
+ DistributedSchedulingAMProtocolPB {
+
+ private DistributedSchedulingAMProtocol real;
+
+ public DistributedSchedulingAMProtocolPBServiceImpl(
+ DistributedSchedulingAMProtocol impl) {
+ this.real = impl;
+ }
+
+ @Override
+ public YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProto
+ registerApplicationMasterForDistributedScheduling(
+ RpcController controller, RegisterApplicationMasterRequestProto proto)
+ throws ServiceException {
+ RegisterApplicationMasterRequestPBImpl request = new
+ RegisterApplicationMasterRequestPBImpl(proto);
+ try {
+ RegisterDistributedSchedulingAMResponse response =
+ real.registerApplicationMasterForDistributedScheduling(request);
+ return ((RegisterDistributedSchedulingAMResponsePBImpl) response)
+ .getProto();
+ } catch (YarnException e) {
+ throw new ServiceException(e);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public YarnServerCommonServiceProtos.
+ DistributedSchedulingAllocateResponseProto
+ allocateForDistributedScheduling(RpcController controller,
+ YarnServerCommonServiceProtos.
+ DistributedSchedulingAllocateRequestProto proto)
+ throws ServiceException {
+ DistributedSchedulingAllocateRequestPBImpl request =
+ new DistributedSchedulingAllocateRequestPBImpl(proto);
+ try {
+ DistributedSchedulingAllocateResponse response = real
+ .allocateForDistributedScheduling(request);
+ return ((DistributedSchedulingAllocateResponsePBImpl) response)
+ .getProto();
+ } catch (YarnException e) {
+ throw new ServiceException(e);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public YarnServiceProtos.AllocateResponseProto allocate(RpcController arg0,
+ AllocateRequestProto proto) throws ServiceException {
+ AllocateRequestPBImpl request = new AllocateRequestPBImpl(proto);
+ try {
+ AllocateResponse response = real.allocate(request);
+ return ((AllocateResponsePBImpl) response).getProto();
+ } catch (YarnException e) {
+ throw new ServiceException(e);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public YarnServiceProtos.FinishApplicationMasterResponseProto
+ finishApplicationMaster(
+ RpcController arg0, YarnServiceProtos
+ .FinishApplicationMasterRequestProto proto)
+ throws ServiceException {
+ FinishApplicationMasterRequestPBImpl request = new
+ FinishApplicationMasterRequestPBImpl(proto);
+ try {
+ FinishApplicationMasterResponse response = real.finishApplicationMaster
+ (request);
+ return ((FinishApplicationMasterResponsePBImpl) response).getProto();
+ } catch (YarnException e) {
+ throw new ServiceException(e);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public YarnServiceProtos.RegisterApplicationMasterResponseProto
+ registerApplicationMaster(
+ RpcController arg0, RegisterApplicationMasterRequestProto proto)
+ throws ServiceException {
+ RegisterApplicationMasterRequestPBImpl request = new
+ RegisterApplicationMasterRequestPBImpl(proto);
+ try {
+ RegisterApplicationMasterResponse response = real
+ .registerApplicationMaster(request);
+ return ((RegisterApplicationMasterResponsePBImpl) response).getProto();
+ } catch (YarnException e) {
+ throw new ServiceException(e);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistSchedAllocateRequest.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistSchedAllocateRequest.java
deleted file mode 100644
index 10ff95b..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistSchedAllocateRequest.java
+++ /dev/null
@@ -1,69 +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.yarn.server.api.protocolrecords;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
-import org.apache.hadoop.yarn.api.records.Container;
-
-import java.util.List;
-
-/**
- * Request for a distributed scheduler to notify allocation of containers to
- * the Resource Manager.
- */
-@Public
-@Evolving
-public abstract class DistSchedAllocateRequest {
-
- /**
- * Get the underlying AllocateRequest object.
- * @return Allocate request
- */
- @Public
- @Evolving
- public abstract AllocateRequest getAllocateRequest();
-
- /**
- * Set the underlying AllocateRequest object.
- * @param allocateRequest Allocate request
- */
- @Public
- @Evolving
- public abstract void setAllocateRequest(AllocateRequest allocateRequest);
-
- /**
- * Get the list of newly allocated Container by the
- * Distributed Scheduling component on the NodeManager.
- * @return list of newly allocated Container
- */
- @Public
- @Evolving
- public abstract List getAllocatedContainers();
-
- /**
- * Set the list of newly allocated Container by the
- * Distributed Scheduling component on the NodeManager.
- * @param containers list of newly allocated Container
- */
- @Public
- @Evolving
- public abstract void setAllocatedContainers(List containers);
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistSchedAllocateResponse.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistSchedAllocateResponse.java
deleted file mode 100644
index 5f6e069..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistSchedAllocateResponse.java
+++ /dev/null
@@ -1,58 +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.yarn.server.api.protocolrecords;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.util.Records;
-
-import java.util.List;
-
-@Public
-@Unstable
-public abstract class DistSchedAllocateResponse {
-
- @Public
- @Unstable
- public static DistSchedAllocateResponse newInstance(AllocateResponse
- allResp) {
- DistSchedAllocateResponse response =
- Records.newRecord(DistSchedAllocateResponse.class);
- response.setAllocateResponse(allResp);
- return response;
- }
-
- @Public
- @Unstable
- public abstract void setAllocateResponse(AllocateResponse response);
-
- @Public
- @Unstable
- public abstract AllocateResponse getAllocateResponse();
-
- @Public
- @Unstable
- public abstract void setNodesForScheduling(List nodesForScheduling);
-
- @Public
- @Unstable
- public abstract List getNodesForScheduling();
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistSchedRegisterResponse.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistSchedRegisterResponse.java
deleted file mode 100644
index e4e5138..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistSchedRegisterResponse.java
+++ /dev/null
@@ -1,102 +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.yarn.server.api.protocolrecords;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.protocolrecords
- .RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.util.Records;
-
-import java.util.List;
-
-@Public
-@Unstable
-public abstract class DistSchedRegisterResponse {
-
- @Public
- @Unstable
- public static DistSchedRegisterResponse newInstance
- (RegisterApplicationMasterResponse regAMResp) {
- DistSchedRegisterResponse response =
- Records.newRecord(DistSchedRegisterResponse.class);
- response.setRegisterResponse(regAMResp);
- return response;
- }
-
- @Public
- @Unstable
- public abstract void setRegisterResponse(
- RegisterApplicationMasterResponse resp);
-
- @Public
- @Unstable
- public abstract RegisterApplicationMasterResponse getRegisterResponse();
-
- @Public
- @Unstable
- public abstract void setMinAllocatableCapabilty(Resource minResource);
-
- @Public
- @Unstable
- public abstract Resource getMinAllocatableCapabilty();
-
- @Public
- @Unstable
- public abstract void setMaxAllocatableCapabilty(Resource maxResource);
-
- @Public
- @Unstable
- public abstract Resource getMaxAllocatableCapabilty();
-
- @Public
- @Unstable
- public abstract void setIncrAllocatableCapabilty(Resource maxResource);
-
- @Public
- @Unstable
- public abstract Resource getIncrAllocatableCapabilty();
-
- @Public
- @Unstable
- public abstract void setContainerTokenExpiryInterval(int interval);
-
- @Public
- @Unstable
- public abstract int getContainerTokenExpiryInterval();
-
- @Public
- @Unstable
- public abstract void setContainerIdStart(long containerIdStart);
-
- @Public
- @Unstable
- public abstract long getContainerIdStart();
-
- @Public
- @Unstable
- public abstract void setNodesForScheduling(List nodesForScheduling);
-
- @Public
- @Unstable
- public abstract List getNodesForScheduling();
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistributedSchedulingAllocateRequest.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistributedSchedulingAllocateRequest.java
new file mode 100644
index 0000000..ba94ba9
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistributedSchedulingAllocateRequest.java
@@ -0,0 +1,69 @@
+/**
+ * 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.yarn.server.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.records.Container;
+
+import java.util.List;
+
+/**
+ * Request for a distributed scheduler to notify allocation of containers to
+ * the Resource Manager.
+ */
+@Public
+@Evolving
+public abstract class DistributedSchedulingAllocateRequest {
+
+ /**
+ * Get the underlying AllocateRequest object.
+ * @return Allocate request
+ */
+ @Public
+ @Evolving
+ public abstract AllocateRequest getAllocateRequest();
+
+ /**
+ * Set the underlying AllocateRequest object.
+ * @param allocateRequest Allocate request
+ */
+ @Public
+ @Evolving
+ public abstract void setAllocateRequest(AllocateRequest allocateRequest);
+
+ /**
+ * Get the list of newly allocated Container by the
+ * Distributed Scheduling component on the NodeManager.
+ * @return list of newly allocated Container
+ */
+ @Public
+ @Evolving
+ public abstract List getAllocatedContainers();
+
+ /**
+ * Set the list of newly allocated Container by the
+ * Distributed Scheduling component on the NodeManager.
+ * @param containers list of newly allocated Container
+ */
+ @Public
+ @Evolving
+ public abstract void setAllocatedContainers(List containers);
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistributedSchedulingAllocateResponse.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistributedSchedulingAllocateResponse.java
new file mode 100644
index 0000000..1ce94b0
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DistributedSchedulingAllocateResponse.java
@@ -0,0 +1,58 @@
+/**
+ * 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.yarn.server.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.List;
+
+@Public
+@Unstable
+public abstract class DistributedSchedulingAllocateResponse {
+
+ @Public
+ @Unstable
+ public static DistributedSchedulingAllocateResponse newInstance(
+ AllocateResponse allResp) {
+ DistributedSchedulingAllocateResponse response =
+ Records.newRecord(DistributedSchedulingAllocateResponse.class);
+ response.setAllocateResponse(allResp);
+ return response;
+ }
+
+ @Public
+ @Unstable
+ public abstract void setAllocateResponse(AllocateResponse response);
+
+ @Public
+ @Unstable
+ public abstract AllocateResponse getAllocateResponse();
+
+ @Public
+ @Unstable
+ public abstract void setNodesForScheduling(List nodesForScheduling);
+
+ @Public
+ @Unstable
+ public abstract List getNodesForScheduling();
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterDistributedSchedulingAMResponse.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterDistributedSchedulingAMResponse.java
new file mode 100644
index 0000000..8e4b3e1
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterDistributedSchedulingAMResponse.java
@@ -0,0 +1,101 @@
+/**
+ * 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.yarn.server.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.List;
+
+@Public
+@Unstable
+public abstract class RegisterDistributedSchedulingAMResponse {
+
+ @Public
+ @Unstable
+ public static RegisterDistributedSchedulingAMResponse newInstance
+ (RegisterApplicationMasterResponse regAMResp) {
+ RegisterDistributedSchedulingAMResponse response =
+ Records.newRecord(RegisterDistributedSchedulingAMResponse.class);
+ response.setRegisterResponse(regAMResp);
+ return response;
+ }
+
+ @Public
+ @Unstable
+ public abstract void setRegisterResponse(
+ RegisterApplicationMasterResponse resp);
+
+ @Public
+ @Unstable
+ public abstract RegisterApplicationMasterResponse getRegisterResponse();
+
+ @Public
+ @Unstable
+ public abstract void setMinContainerResource(Resource minResource);
+
+ @Public
+ @Unstable
+ public abstract Resource getMinContainerResource();
+
+ @Public
+ @Unstable
+ public abstract void setMaxContainerResource(Resource maxResource);
+
+ @Public
+ @Unstable
+ public abstract Resource getMaxContainerResource();
+
+ @Public
+ @Unstable
+ public abstract void setIncrContainerResource(Resource maxResource);
+
+ @Public
+ @Unstable
+ public abstract Resource getIncrContainerResource();
+
+ @Public
+ @Unstable
+ public abstract void setContainerTokenExpiryInterval(int interval);
+
+ @Public
+ @Unstable
+ public abstract int getContainerTokenExpiryInterval();
+
+ @Public
+ @Unstable
+ public abstract void setContainerIdStart(long containerIdStart);
+
+ @Public
+ @Unstable
+ public abstract long getContainerIdStart();
+
+ @Public
+ @Unstable
+ public abstract void setNodesForScheduling(List nodesForScheduling);
+
+ @Public
+ @Unstable
+ public abstract List getNodesForScheduling();
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistSchedAllocateRequestPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistSchedAllocateRequestPBImpl.java
deleted file mode 100644
index be386b6..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistSchedAllocateRequestPBImpl.java
+++ /dev/null
@@ -1,185 +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.yarn.server.api.protocolrecords.impl.pb;
-
-import java.util.ArrayList;
-
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
-import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.DistSchedAllocateRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.DistSchedAllocateRequestProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateRequest;
-
-import java.util.Iterator;
-import java.util.List;
-
-/**
- * Implementation of {@link DistSchedAllocateRequest} for a distributed
- * scheduler to notify about the allocation of containers to the Resource
- * Manager.
- */
-public class DistSchedAllocateRequestPBImpl extends DistSchedAllocateRequest {
- private DistSchedAllocateRequestProto.Builder builder = null;
- private boolean viaProto = false;
-
- private DistSchedAllocateRequestProto proto;
- private AllocateRequest allocateRequest;
- private List containers;
-
- public DistSchedAllocateRequestPBImpl() {
- builder = DistSchedAllocateRequestProto.newBuilder();
- }
-
- public DistSchedAllocateRequestPBImpl(DistSchedAllocateRequestProto proto) {
- this.proto = proto;
- this.viaProto = true;
- }
-
- @Override
- public AllocateRequest getAllocateRequest() {
- DistSchedAllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
- if (this.allocateRequest != null) {
- return this.allocateRequest;
- }
- if (!p.hasAllocateRequest()) {
- return null;
- }
- this.allocateRequest = convertFromProtoFormat(p.getAllocateRequest());
- return this.allocateRequest;
- }
-
- @Override
- public void setAllocateRequest(AllocateRequest pAllocateRequest) {
- maybeInitBuilder();
- if (allocateRequest == null) {
- builder.clearAllocateRequest();
- }
- this.allocateRequest = pAllocateRequest;
- }
-
- @Override
- public List getAllocatedContainers() {
- if (this.containers != null) {
- return this.containers;
- }
- initAllocatedContainers();
- return containers;
- }
-
- private void initAllocatedContainers() {
- DistSchedAllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
- List list = p.getAllocatedContainersList();
- this.containers = new ArrayList();
- for (ContainerProto c : list) {
- this.containers.add(convertFromProtoFormat(c));
- }
- }
-
- @Override
- public void setAllocatedContainers(List pContainers) {
- maybeInitBuilder();
- if (pContainers == null || pContainers.isEmpty()) {
- if (this.containers != null) {
- this.containers.clear();
- }
- builder.clearAllocatedContainers();
- return;
- }
- this.containers = new ArrayList<>();
- this.containers.addAll(pContainers);
- }
-
- public DistSchedAllocateRequestProto getProto() {
- mergeLocalToProto();
- proto = viaProto ? proto : builder.build();
- viaProto = true;
- return proto;
- }
-
- private void maybeInitBuilder() {
- if (viaProto || builder == null) {
- builder = DistSchedAllocateRequestProto.newBuilder(proto);
- }
- viaProto = false;
- }
-
- private void mergeLocalToProto() {
- if (viaProto) {
- maybeInitBuilder();
- }
- mergeLocalToBuilder();
- proto = builder.build();
- viaProto = true;
- }
-
- private void mergeLocalToBuilder() {
- if (this.containers != null) {
- builder.clearAllocatedContainers();
- Iterable iterable =
- getContainerProtoIterable(this.containers);
- builder.addAllAllocatedContainers(iterable);
- }
- if (this.allocateRequest != null) {
- builder.setAllocateRequest(
- ((AllocateRequestPBImpl)this.allocateRequest).getProto());
- }
- }
-
- private Iterable getContainerProtoIterable(
- final List newContainersList) {
- maybeInitBuilder();
- return new Iterable() {
- @Override
- public synchronized Iterator iterator() {
- return new Iterator() {
- Iterator iter = newContainersList.iterator();
-
- @Override
- public synchronized boolean hasNext() {
- return iter.hasNext();
- }
-
- @Override
- public synchronized ContainerProto next() {
- return ProtoUtils.convertToProtoFormat(iter.next());
- }
-
- @Override
- public synchronized void remove() {
- throw new UnsupportedOperationException();
-
- }
- };
- }
- };
- }
-
- private ContainerPBImpl convertFromProtoFormat(ContainerProto p) {
- return new ContainerPBImpl(p);
- }
-
- private AllocateRequestPBImpl convertFromProtoFormat(AllocateRequestProto p) {
- return new AllocateRequestPBImpl(p);
- }
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistSchedAllocateResponsePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistSchedAllocateResponsePBImpl.java
deleted file mode 100644
index 3ea4965..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistSchedAllocateResponsePBImpl.java
+++ /dev/null
@@ -1,180 +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.yarn.server.api.protocolrecords.impl.pb;
-
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .AllocateResponsePBImpl;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
-import org.apache.hadoop.yarn.proto.YarnProtos;
-import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos;
-import org.apache.hadoop.yarn.server.api.protocolrecords
- .DistSchedAllocateResponse;
-
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-public class DistSchedAllocateResponsePBImpl extends DistSchedAllocateResponse {
-
- YarnServerCommonServiceProtos.DistSchedAllocateResponseProto proto =
- YarnServerCommonServiceProtos.DistSchedAllocateResponseProto.getDefaultInstance();
- YarnServerCommonServiceProtos.DistSchedAllocateResponseProto.Builder builder = null;
- boolean viaProto = false;
-
- private AllocateResponse allocateResponse;
- private List nodesForScheduling;
-
- public DistSchedAllocateResponsePBImpl() {
- builder = YarnServerCommonServiceProtos.DistSchedAllocateResponseProto.newBuilder();
- }
-
- public DistSchedAllocateResponsePBImpl(YarnServerCommonServiceProtos.DistSchedAllocateResponseProto proto) {
- this.proto = proto;
- viaProto = true;
- }
-
- public YarnServerCommonServiceProtos.DistSchedAllocateResponseProto getProto() {
- mergeLocalToProto();
- proto = viaProto ? proto : builder.build();
- viaProto = true;
- return proto;
- }
-
- private void maybeInitBuilder() {
- if (viaProto || builder == null) {
- builder = YarnServerCommonServiceProtos.DistSchedAllocateResponseProto.newBuilder(proto);
- }
- viaProto = false;
- }
-
- private synchronized void mergeLocalToProto() {
- if (viaProto)
- maybeInitBuilder();
- mergeLocalToBuilder();
- proto = builder.build();
- viaProto = true;
- }
-
- private synchronized void mergeLocalToBuilder() {
- if (this.nodesForScheduling != null) {
- builder.clearNodesForScheduling();
- Iterable iterable =
- getNodeIdProtoIterable(this.nodesForScheduling);
- builder.addAllNodesForScheduling(iterable);
- }
- if (this.allocateResponse != null) {
- builder.setAllocateResponse(
- ((AllocateResponsePBImpl)this.allocateResponse).getProto());
- }
- }
- @Override
- public void setAllocateResponse(AllocateResponse response) {
- maybeInitBuilder();
- if(allocateResponse == null) {
- builder.clearAllocateResponse();
- }
- this.allocateResponse = response;
- }
-
- @Override
- public AllocateResponse getAllocateResponse() {
- if (this.allocateResponse != null) {
- return this.allocateResponse;
- }
-
- YarnServerCommonServiceProtos.DistSchedAllocateResponseProtoOrBuilder p =
- viaProto ? proto : builder;
- if (!p.hasAllocateResponse()) {
- return null;
- }
-
- this.allocateResponse =
- new AllocateResponsePBImpl(p.getAllocateResponse());
- return this.allocateResponse;
- }
-
- @Override
- public void setNodesForScheduling(List nodesForScheduling) {
- maybeInitBuilder();
- if (nodesForScheduling == null || nodesForScheduling.isEmpty()) {
- if (this.nodesForScheduling != null) {
- this.nodesForScheduling.clear();
- }
- builder.clearNodesForScheduling();
- return;
- }
- this.nodesForScheduling = new ArrayList<>();
- this.nodesForScheduling.addAll(nodesForScheduling);
- }
-
- @Override
- public List getNodesForScheduling() {
- if (nodesForScheduling != null) {
- return nodesForScheduling;
- }
- initLocalNodesForSchedulingList();
- return nodesForScheduling;
- }
-
- private synchronized void initLocalNodesForSchedulingList() {
- YarnServerCommonServiceProtos.DistSchedAllocateResponseProtoOrBuilder p =
- viaProto ? proto : builder;
- List list = p.getNodesForSchedulingList();
- nodesForScheduling = new ArrayList<>();
- if (list != null) {
- for (YarnProtos.NodeIdProto t : list) {
- nodesForScheduling.add(ProtoUtils.convertFromProtoFormat(t));
- }
- }
- }
- private synchronized Iterable getNodeIdProtoIterable(
- final List nodeList) {
- maybeInitBuilder();
- return new Iterable() {
- @Override
- public synchronized Iterator iterator() {
- return new Iterator() {
-
- Iterator iter = nodeList.iterator();
-
- @Override
- public boolean hasNext() {
- return iter.hasNext();
- }
-
- @Override
- public YarnProtos.NodeIdProto next() {
- return ProtoUtils.convertToProtoFormat(iter.next());
- }
-
- @Override
- public void remove() {
- throw new UnsupportedOperationException();
- }
- };
- }
- };
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistSchedRegisterResponsePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistSchedRegisterResponsePBImpl.java
deleted file mode 100644
index eec62da..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistSchedRegisterResponsePBImpl.java
+++ /dev/null
@@ -1,308 +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.yarn.server.api.protocolrecords.impl.pb;
-
-import com.google.protobuf.TextFormat;
-
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-
-
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .RegisterApplicationMasterResponsePBImpl;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
-import org.apache.hadoop.yarn.proto.YarnProtos;
-import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos;
-import org.apache.hadoop.yarn.server.api.protocolrecords
- .DistSchedRegisterResponse;
-
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-public class DistSchedRegisterResponsePBImpl extends DistSchedRegisterResponse {
-
- YarnServerCommonServiceProtos.DistSchedRegisterResponseProto proto =
- YarnServerCommonServiceProtos.DistSchedRegisterResponseProto.getDefaultInstance();
- YarnServerCommonServiceProtos.DistSchedRegisterResponseProto.Builder builder = null;
- boolean viaProto = false;
-
- private Resource maxAllocatableCapability;
- private Resource minAllocatableCapability;
- private Resource incrAllocatableCapability;
- private List nodesForScheduling;
- private RegisterApplicationMasterResponse registerApplicationMasterResponse;
-
- public DistSchedRegisterResponsePBImpl() {
- builder = YarnServerCommonServiceProtos.DistSchedRegisterResponseProto.newBuilder();
- }
-
- public DistSchedRegisterResponsePBImpl(YarnServerCommonServiceProtos.DistSchedRegisterResponseProto proto) {
- this.proto = proto;
- viaProto = true;
- }
-
- public YarnServerCommonServiceProtos.DistSchedRegisterResponseProto getProto() {
- mergeLocalToProto();
- proto = viaProto ? proto : builder.build();
- viaProto = true;
- return proto;
- }
-
- private void maybeInitBuilder() {
- if (viaProto || builder == null) {
- builder = YarnServerCommonServiceProtos.DistSchedRegisterResponseProto.newBuilder(proto);
- }
- viaProto = false;
- }
-
- private synchronized void mergeLocalToProto() {
- if (viaProto)
- maybeInitBuilder();
- mergeLocalToBuilder();
- proto = builder.build();
- viaProto = true;
- }
-
- private synchronized void mergeLocalToBuilder() {
- if (this.nodesForScheduling != null) {
- builder.clearNodesForScheduling();
- Iterable iterable =
- getNodeIdProtoIterable(this.nodesForScheduling);
- builder.addAllNodesForScheduling(iterable);
- }
- if (this.maxAllocatableCapability != null) {
- builder.setMaxAllocCapability(
- ProtoUtils.convertToProtoFormat(this.maxAllocatableCapability));
- }
- if (this.minAllocatableCapability != null) {
- builder.setMinAllocCapability(
- ProtoUtils.convertToProtoFormat(this.minAllocatableCapability));
- }
- if (this.incrAllocatableCapability != null) {
- builder.setIncrAllocCapability(
- ProtoUtils.convertToProtoFormat(this.incrAllocatableCapability));
- }
- if (this.registerApplicationMasterResponse != null) {
- builder.setRegisterResponse(
- ((RegisterApplicationMasterResponsePBImpl)
- this.registerApplicationMasterResponse).getProto());
- }
- }
-
- @Override
- public void setRegisterResponse(RegisterApplicationMasterResponse resp) {
- maybeInitBuilder();
- if(registerApplicationMasterResponse == null) {
- builder.clearRegisterResponse();
- }
- this.registerApplicationMasterResponse = resp;
- }
-
- @Override
- public RegisterApplicationMasterResponse getRegisterResponse() {
- if (this.registerApplicationMasterResponse != null) {
- return this.registerApplicationMasterResponse;
- }
-
- YarnServerCommonServiceProtos.DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
- if (!p.hasRegisterResponse()) {
- return null;
- }
-
- this.registerApplicationMasterResponse =
- new RegisterApplicationMasterResponsePBImpl(p.getRegisterResponse());
- return this.registerApplicationMasterResponse;
- }
-
- @Override
- public void setMaxAllocatableCapabilty(Resource maxResource) {
- maybeInitBuilder();
- if(maxAllocatableCapability == null) {
- builder.clearMaxAllocCapability();
- }
- this.maxAllocatableCapability = maxResource;
- }
-
- @Override
- public Resource getMaxAllocatableCapabilty() {
- if (this.maxAllocatableCapability != null) {
- return this.maxAllocatableCapability;
- }
-
- YarnServerCommonServiceProtos.DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
- if (!p.hasMaxAllocCapability()) {
- return null;
- }
-
- this.maxAllocatableCapability =
- ProtoUtils.convertFromProtoFormat(p.getMaxAllocCapability());
- return this.maxAllocatableCapability;
- }
-
- @Override
- public void setMinAllocatableCapabilty(Resource minResource) {
- maybeInitBuilder();
- if(minAllocatableCapability == null) {
- builder.clearMinAllocCapability();
- }
- this.minAllocatableCapability = minResource;
- }
-
- @Override
- public Resource getMinAllocatableCapabilty() {
- if (this.minAllocatableCapability != null) {
- return this.minAllocatableCapability;
- }
-
- YarnServerCommonServiceProtos.DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
- if (!p.hasMinAllocCapability()) {
- return null;
- }
-
- this.minAllocatableCapability =
- ProtoUtils.convertFromProtoFormat(p.getMinAllocCapability());
- return this.minAllocatableCapability;
- }
-
- @Override
- public void setIncrAllocatableCapabilty(Resource incrResource) {
- maybeInitBuilder();
- if(incrAllocatableCapability == null) {
- builder.clearIncrAllocCapability();
- }
- this.incrAllocatableCapability = incrResource;
- }
-
- @Override
- public Resource getIncrAllocatableCapabilty() {
- if (this.incrAllocatableCapability != null) {
- return this.incrAllocatableCapability;
- }
-
- YarnServerCommonServiceProtos.DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
- if (!p.hasIncrAllocCapability()) {
- return null;
- }
-
- this.incrAllocatableCapability =
- ProtoUtils.convertFromProtoFormat(p.getIncrAllocCapability());
- return this.incrAllocatableCapability;
- }
-
- @Override
- public void setContainerTokenExpiryInterval(int interval) {
- maybeInitBuilder();
- builder.setContainerTokenExpiryInterval(interval);
- }
-
- @Override
- public int getContainerTokenExpiryInterval() {
- YarnServerCommonServiceProtos.DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
- if (!p.hasContainerTokenExpiryInterval()) {
- return 0;
- }
- return p.getContainerTokenExpiryInterval();
- }
-
- @Override
- public void setContainerIdStart(long containerIdStart) {
- maybeInitBuilder();
- builder.setContainerIdStart(containerIdStart);
- }
-
- @Override
- public long getContainerIdStart() {
- YarnServerCommonServiceProtos.DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
- if (!p.hasContainerIdStart()) {
- return 0;
- }
- return p.getContainerIdStart();
- }
-
-
- @Override
- public void setNodesForScheduling(List nodesForScheduling) {
- maybeInitBuilder();
- if (nodesForScheduling == null || nodesForScheduling.isEmpty()) {
- if (this.nodesForScheduling != null) {
- this.nodesForScheduling.clear();
- }
- builder.clearNodesForScheduling();
- return;
- }
- this.nodesForScheduling = new ArrayList<>();
- this.nodesForScheduling.addAll(nodesForScheduling);
- }
-
- @Override
- public List getNodesForScheduling() {
- if (nodesForScheduling != null) {
- return nodesForScheduling;
- }
- initLocalNodesForSchedulingList();
- return nodesForScheduling;
- }
-
- private synchronized void initLocalNodesForSchedulingList() {
- YarnServerCommonServiceProtos.DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
- List list = p.getNodesForSchedulingList();
- nodesForScheduling = new ArrayList<>();
- if (list != null) {
- for (YarnProtos.NodeIdProto t : list) {
- nodesForScheduling.add(ProtoUtils.convertFromProtoFormat(t));
- }
- }
- }
- private synchronized Iterable getNodeIdProtoIterable(
- final List nodeList) {
- maybeInitBuilder();
- return new Iterable() {
- @Override
- public synchronized Iterator iterator() {
- return new Iterator() {
-
- Iterator iter = nodeList.iterator();
-
- @Override
- public boolean hasNext() {
- return iter.hasNext();
- }
-
- @Override
- public YarnProtos.NodeIdProto next() {
- return ProtoUtils.convertToProtoFormat(iter.next());
- }
-
- @Override
- public void remove() {
- throw new UnsupportedOperationException();
- }
- };
- }
- };
- }
-
- @Override
- public String toString() {
- return TextFormat.shortDebugString(getProto());
- }
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistributedSchedulingAllocateRequestPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistributedSchedulingAllocateRequestPBImpl.java
new file mode 100644
index 0000000..30fbed3
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistributedSchedulingAllocateRequestPBImpl.java
@@ -0,0 +1,189 @@
+/**
+ * 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.yarn.server.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.DistributedSchedulingAllocateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.DistributedSchedulingAllocateRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
+
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Implementation of {@link DistributedSchedulingAllocateRequest} for a
+ * distributed scheduler to notify about the allocation of containers to the
+ * ResourceManager.
+ */
+public class DistributedSchedulingAllocateRequestPBImpl
+ extends DistributedSchedulingAllocateRequest {
+ private DistributedSchedulingAllocateRequestProto.Builder builder = null;
+ private boolean viaProto = false;
+
+ private DistributedSchedulingAllocateRequestProto proto;
+ private AllocateRequest allocateRequest;
+ private List containers;
+
+ public DistributedSchedulingAllocateRequestPBImpl() {
+ builder = DistributedSchedulingAllocateRequestProto.newBuilder();
+ }
+
+ public DistributedSchedulingAllocateRequestPBImpl(
+ DistributedSchedulingAllocateRequestProto proto) {
+ this.proto = proto;
+ this.viaProto = true;
+ }
+
+ @Override
+ public AllocateRequest getAllocateRequest() {
+ DistributedSchedulingAllocateRequestProtoOrBuilder p =
+ viaProto ? proto : builder;
+ if (this.allocateRequest != null) {
+ return this.allocateRequest;
+ }
+ if (!p.hasAllocateRequest()) {
+ return null;
+ }
+ this.allocateRequest = convertFromProtoFormat(p.getAllocateRequest());
+ return this.allocateRequest;
+ }
+
+ @Override
+ public void setAllocateRequest(AllocateRequest pAllocateRequest) {
+ maybeInitBuilder();
+ if (allocateRequest == null) {
+ builder.clearAllocateRequest();
+ }
+ this.allocateRequest = pAllocateRequest;
+ }
+
+ @Override
+ public List getAllocatedContainers() {
+ if (this.containers != null) {
+ return this.containers;
+ }
+ initAllocatedContainers();
+ return containers;
+ }
+
+ private void initAllocatedContainers() {
+ DistributedSchedulingAllocateRequestProtoOrBuilder p =
+ viaProto ? proto : builder;
+ List list = p.getAllocatedContainersList();
+ this.containers = new ArrayList();
+ for (ContainerProto c : list) {
+ this.containers.add(convertFromProtoFormat(c));
+ }
+ }
+
+ @Override
+ public void setAllocatedContainers(List pContainers) {
+ maybeInitBuilder();
+ if (pContainers == null || pContainers.isEmpty()) {
+ if (this.containers != null) {
+ this.containers.clear();
+ }
+ builder.clearAllocatedContainers();
+ return;
+ }
+ this.containers = new ArrayList<>();
+ this.containers.addAll(pContainers);
+ }
+
+ public DistributedSchedulingAllocateRequestProto getProto() {
+ mergeLocalToProto();
+ proto = viaProto ? proto : builder.build();
+ viaProto = true;
+ return proto;
+ }
+
+ private void maybeInitBuilder() {
+ if (viaProto || builder == null) {
+ builder = DistributedSchedulingAllocateRequestProto.newBuilder(proto);
+ }
+ viaProto = false;
+ }
+
+ private void mergeLocalToProto() {
+ if (viaProto) {
+ maybeInitBuilder();
+ }
+ mergeLocalToBuilder();
+ proto = builder.build();
+ viaProto = true;
+ }
+
+ private void mergeLocalToBuilder() {
+ if (this.containers != null) {
+ builder.clearAllocatedContainers();
+ Iterable iterable =
+ getContainerProtoIterable(this.containers);
+ builder.addAllAllocatedContainers(iterable);
+ }
+ if (this.allocateRequest != null) {
+ builder.setAllocateRequest(
+ ((AllocateRequestPBImpl)this.allocateRequest).getProto());
+ }
+ }
+
+ private Iterable getContainerProtoIterable(
+ final List newContainersList) {
+ maybeInitBuilder();
+ return new Iterable() {
+ @Override
+ public synchronized Iterator iterator() {
+ return new Iterator() {
+ Iterator iter = newContainersList.iterator();
+
+ @Override
+ public synchronized boolean hasNext() {
+ return iter.hasNext();
+ }
+
+ @Override
+ public synchronized ContainerProto next() {
+ return ProtoUtils.convertToProtoFormat(iter.next());
+ }
+
+ @Override
+ public synchronized void remove() {
+ throw new UnsupportedOperationException();
+
+ }
+ };
+ }
+ };
+ }
+
+ private ContainerPBImpl convertFromProtoFormat(ContainerProto p) {
+ return new ContainerPBImpl(p);
+ }
+
+ private AllocateRequestPBImpl convertFromProtoFormat(AllocateRequestProto p) {
+ return new AllocateRequestPBImpl(p);
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistributedSchedulingAllocateResponsePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistributedSchedulingAllocateResponsePBImpl.java
new file mode 100644
index 0000000..18d5073
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistributedSchedulingAllocateResponsePBImpl.java
@@ -0,0 +1,190 @@
+/**
+ * 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.yarn.server.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Implementation of {@link DistributedSchedulingAllocateResponse}.
+ */
+public class DistributedSchedulingAllocateResponsePBImpl extends
+ DistributedSchedulingAllocateResponse {
+
+ YarnServerCommonServiceProtos.DistributedSchedulingAllocateResponseProto
+ proto = YarnServerCommonServiceProtos.
+ DistributedSchedulingAllocateResponseProto.getDefaultInstance();
+ YarnServerCommonServiceProtos.DistributedSchedulingAllocateResponseProto.
+ Builder builder = null;
+ boolean viaProto = false;
+
+ private AllocateResponse allocateResponse;
+ private List nodesForScheduling;
+
+ public DistributedSchedulingAllocateResponsePBImpl() {
+ builder = YarnServerCommonServiceProtos.
+ DistributedSchedulingAllocateResponseProto.newBuilder();
+ }
+
+ public DistributedSchedulingAllocateResponsePBImpl(
+ YarnServerCommonServiceProtos.
+ DistributedSchedulingAllocateResponseProto proto) {
+ this.proto = proto;
+ viaProto = true;
+ }
+
+ public YarnServerCommonServiceProtos.
+ DistributedSchedulingAllocateResponseProto getProto() {
+ mergeLocalToProto();
+ proto = viaProto ? proto : builder.build();
+ viaProto = true;
+ return proto;
+ }
+
+ private void maybeInitBuilder() {
+ if (viaProto || builder == null) {
+ builder = YarnServerCommonServiceProtos.
+ DistributedSchedulingAllocateResponseProto.newBuilder(proto);
+ }
+ viaProto = false;
+ }
+
+ private synchronized void mergeLocalToProto() {
+ if (viaProto)
+ maybeInitBuilder();
+ mergeLocalToBuilder();
+ proto = builder.build();
+ viaProto = true;
+ }
+
+ private synchronized void mergeLocalToBuilder() {
+ if (this.nodesForScheduling != null) {
+ builder.clearNodesForScheduling();
+ Iterable iterable = getNodeIdProtoIterable(
+ this.nodesForScheduling);
+ builder.addAllNodesForScheduling(iterable);
+ }
+ if (this.allocateResponse != null) {
+ builder.setAllocateResponse(
+ ((AllocateResponsePBImpl) this.allocateResponse).getProto());
+ }
+ }
+
+ @Override
+ public void setAllocateResponse(AllocateResponse response) {
+ maybeInitBuilder();
+ if (allocateResponse == null) {
+ builder.clearAllocateResponse();
+ }
+ this.allocateResponse = response;
+ }
+
+ @Override
+ public AllocateResponse getAllocateResponse() {
+ if (this.allocateResponse != null) {
+ return this.allocateResponse;
+ }
+
+ YarnServerCommonServiceProtos.
+ DistributedSchedulingAllocateResponseProtoOrBuilder p =
+ viaProto ? proto : builder;
+ if (!p.hasAllocateResponse()) {
+ return null;
+ }
+
+ this.allocateResponse = new AllocateResponsePBImpl(p.getAllocateResponse());
+ return this.allocateResponse;
+ }
+
+ @Override
+ public void setNodesForScheduling(List nodesForScheduling) {
+ maybeInitBuilder();
+ if (nodesForScheduling == null || nodesForScheduling.isEmpty()) {
+ if (this.nodesForScheduling != null) {
+ this.nodesForScheduling.clear();
+ }
+ builder.clearNodesForScheduling();
+ return;
+ }
+ this.nodesForScheduling = new ArrayList<>();
+ this.nodesForScheduling.addAll(nodesForScheduling);
+ }
+
+ @Override
+ public List getNodesForScheduling() {
+ if (nodesForScheduling != null) {
+ return nodesForScheduling;
+ }
+ initLocalNodesForSchedulingList();
+ return nodesForScheduling;
+ }
+
+ private synchronized void initLocalNodesForSchedulingList() {
+ YarnServerCommonServiceProtos.
+ DistributedSchedulingAllocateResponseProtoOrBuilder p =
+ viaProto ? proto : builder;
+ List list = p.getNodesForSchedulingList();
+ nodesForScheduling = new ArrayList<>();
+ if (list != null) {
+ for (YarnProtos.NodeIdProto t : list) {
+ nodesForScheduling.add(ProtoUtils.convertFromProtoFormat(t));
+ }
+ }
+ }
+
+ private synchronized Iterable getNodeIdProtoIterable(
+ final List nodeList) {
+ maybeInitBuilder();
+ return new Iterable() {
+ @Override
+ public synchronized Iterator iterator() {
+ return new Iterator() {
+
+ Iterator iter = nodeList.iterator();
+
+ @Override
+ public boolean hasNext() {
+ return iter.hasNext();
+ }
+
+ @Override
+ public YarnProtos.NodeIdProto next() {
+ return ProtoUtils.convertToProtoFormat(iter.next());
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+ };
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RegisterDistributedSchedulingAMResponsePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RegisterDistributedSchedulingAMResponsePBImpl.java
new file mode 100644
index 0000000..4aaf99c
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RegisterDistributedSchedulingAMResponsePBImpl.java
@@ -0,0 +1,332 @@
+/**
+ * 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.yarn.server.api.protocolrecords.impl.pb;
+
+import com.google.protobuf.TextFormat;
+
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Implementation of {@link RegisterDistributedSchedulingAMResponse}.
+ */
+public class RegisterDistributedSchedulingAMResponsePBImpl extends
+ RegisterDistributedSchedulingAMResponse {
+
+ YarnServerCommonServiceProtos.RegisterDistributedSchedulingAMResponseProto
+ proto =
+ YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProto
+ .getDefaultInstance();
+ YarnServerCommonServiceProtos.RegisterDistributedSchedulingAMResponseProto.
+ Builder builder = null;
+ boolean viaProto = false;
+
+ private Resource maxContainerResource;
+ private Resource minContainerResource;
+ private Resource incrContainerResource;
+ private List nodesForScheduling;
+ private RegisterApplicationMasterResponse registerApplicationMasterResponse;
+
+ public RegisterDistributedSchedulingAMResponsePBImpl() {
+ builder = YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProto.newBuilder();
+ }
+
+ public RegisterDistributedSchedulingAMResponsePBImpl(
+ YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProto proto) {
+ this.proto = proto;
+ viaProto = true;
+ }
+
+ public YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProto
+ getProto() {
+ mergeLocalToProto();
+ proto = viaProto ? proto : builder.build();
+ viaProto = true;
+ return proto;
+ }
+
+ private void maybeInitBuilder() {
+ if (viaProto || builder == null) {
+ builder = YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProto.newBuilder(proto);
+ }
+ viaProto = false;
+ }
+
+ private synchronized void mergeLocalToProto() {
+ if (viaProto)
+ maybeInitBuilder();
+ mergeLocalToBuilder();
+ proto = builder.build();
+ viaProto = true;
+ }
+
+ private synchronized void mergeLocalToBuilder() {
+ if (this.nodesForScheduling != null) {
+ builder.clearNodesForScheduling();
+ Iterable iterable = getNodeIdProtoIterable(
+ this.nodesForScheduling);
+ builder.addAllNodesForScheduling(iterable);
+ }
+ if (this.maxContainerResource != null) {
+ builder.setMaxContainerResource(ProtoUtils.convertToProtoFormat(
+ this.maxContainerResource));
+ }
+ if (this.minContainerResource != null) {
+ builder.setMinContainerResource(ProtoUtils.convertToProtoFormat(
+ this.minContainerResource));
+ }
+ if (this.incrContainerResource != null) {
+ builder.setIncrContainerResource(
+ ProtoUtils.convertToProtoFormat(this.incrContainerResource));
+ }
+ if (this.registerApplicationMasterResponse != null) {
+ builder.setRegisterResponse(
+ ((RegisterApplicationMasterResponsePBImpl)
+ this.registerApplicationMasterResponse).getProto());
+ }
+ }
+
+ @Override
+ public void setRegisterResponse(RegisterApplicationMasterResponse resp) {
+ maybeInitBuilder();
+ if (registerApplicationMasterResponse == null) {
+ builder.clearRegisterResponse();
+ }
+ this.registerApplicationMasterResponse = resp;
+ }
+
+ @Override
+ public RegisterApplicationMasterResponse getRegisterResponse() {
+ if (this.registerApplicationMasterResponse != null) {
+ return this.registerApplicationMasterResponse;
+ }
+
+ YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProtoOrBuilder p =
+ viaProto ? proto : builder;
+ if (!p.hasRegisterResponse()) {
+ return null;
+ }
+
+ this.registerApplicationMasterResponse =
+ new RegisterApplicationMasterResponsePBImpl(p.getRegisterResponse());
+ return this.registerApplicationMasterResponse;
+ }
+
+ @Override
+ public void setMaxContainerResource(Resource maxResource) {
+ maybeInitBuilder();
+ if (maxContainerResource == null) {
+ builder.clearMaxContainerResource();
+ }
+ this.maxContainerResource = maxResource;
+ }
+
+ @Override
+ public Resource getMaxContainerResource() {
+ if (this.maxContainerResource != null) {
+ return this.maxContainerResource;
+ }
+
+ YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProtoOrBuilder p =
+ viaProto ? proto : builder;
+ if (!p.hasMaxContainerResource()) {
+ return null;
+ }
+
+ this.maxContainerResource = ProtoUtils.convertFromProtoFormat(p
+ .getMaxContainerResource());
+ return this.maxContainerResource;
+ }
+
+ @Override
+ public void setMinContainerResource(Resource minResource) {
+ maybeInitBuilder();
+ if (minContainerResource == null) {
+ builder.clearMinContainerResource();
+ }
+ this.minContainerResource = minResource;
+ }
+
+ @Override
+ public Resource getMinContainerResource() {
+ if (this.minContainerResource != null) {
+ return this.minContainerResource;
+ }
+
+ YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProtoOrBuilder p =
+ viaProto ? proto : builder;
+ if (!p.hasMinContainerResource()) {
+ return null;
+ }
+
+ this.minContainerResource = ProtoUtils.convertFromProtoFormat(p
+ .getMinContainerResource());
+ return this.minContainerResource;
+ }
+
+ @Override
+ public void setIncrContainerResource(Resource incrResource) {
+ maybeInitBuilder();
+ if (incrContainerResource == null) {
+ builder.clearIncrContainerResource();
+ }
+ this.incrContainerResource = incrResource;
+ }
+
+ @Override
+ public Resource getIncrContainerResource() {
+ if (this.incrContainerResource != null) {
+ return this.incrContainerResource;
+ }
+
+ YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProtoOrBuilder p =
+ viaProto ? proto : builder;
+ if (!p.hasIncrContainerResource()) {
+ return null;
+ }
+
+ this.incrContainerResource = ProtoUtils.convertFromProtoFormat(p
+ .getIncrContainerResource());
+ return this.incrContainerResource;
+ }
+
+ @Override
+ public void setContainerTokenExpiryInterval(int interval) {
+ maybeInitBuilder();
+ builder.setContainerTokenExpiryInterval(interval);
+ }
+
+ @Override
+ public int getContainerTokenExpiryInterval() {
+ YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProtoOrBuilder p =
+ viaProto ? proto : builder;
+ if (!p.hasContainerTokenExpiryInterval()) {
+ return 0;
+ }
+ return p.getContainerTokenExpiryInterval();
+ }
+
+ @Override
+ public void setContainerIdStart(long containerIdStart) {
+ maybeInitBuilder();
+ builder.setContainerIdStart(containerIdStart);
+ }
+
+ @Override
+ public long getContainerIdStart() {
+ YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProtoOrBuilder p =
+ viaProto ? proto : builder;
+ if (!p.hasContainerIdStart()) {
+ return 0;
+ }
+ return p.getContainerIdStart();
+ }
+
+ @Override
+ public void setNodesForScheduling(List nodesForScheduling) {
+ maybeInitBuilder();
+ if (nodesForScheduling == null || nodesForScheduling.isEmpty()) {
+ if (this.nodesForScheduling != null) {
+ this.nodesForScheduling.clear();
+ }
+ builder.clearNodesForScheduling();
+ return;
+ }
+ this.nodesForScheduling = new ArrayList<>();
+ this.nodesForScheduling.addAll(nodesForScheduling);
+ }
+
+ @Override
+ public List getNodesForScheduling() {
+ if (nodesForScheduling != null) {
+ return nodesForScheduling;
+ }
+ initLocalNodesForSchedulingList();
+ return nodesForScheduling;
+ }
+
+ private synchronized void initLocalNodesForSchedulingList() {
+ YarnServerCommonServiceProtos.
+ RegisterDistributedSchedulingAMResponseProtoOrBuilder p =
+ viaProto ? proto : builder;
+ List list = p.getNodesForSchedulingList();
+ nodesForScheduling = new ArrayList<>();
+ if (list != null) {
+ for (YarnProtos.NodeIdProto t : list) {
+ nodesForScheduling.add(ProtoUtils.convertFromProtoFormat(t));
+ }
+ }
+ }
+
+ private synchronized Iterable getNodeIdProtoIterable(
+ final List nodeList) {
+ maybeInitBuilder();
+ return new Iterable() {
+ @Override
+ public synchronized Iterator iterator() {
+ return new Iterator() {
+
+ Iterator iter = nodeList.iterator();
+
+ @Override
+ public boolean hasNext() {
+ return iter.hasNext();
+ }
+
+ @Override
+ public YarnProtos.NodeIdProto next() {
+ return ProtoUtils.convertToProtoFormat(iter.next());
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public String toString() {
+ return TextFormat.shortDebugString(getProto());
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/distributed_scheduler_protocol.proto hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/distributed_scheduler_protocol.proto
deleted file mode 100644
index 818eb4a..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/distributed_scheduler_protocol.proto
+++ /dev/null
@@ -1,39 +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.
- */
-
-/**
- * These .proto interfaces are public and stable.
- * Please see http://wiki.apache.org/hadoop/Compatibility
- * for what changes are allowed for a *stable* .proto interface.
- */
-
-option java_package = "org.apache.hadoop.yarn.proto";
-option java_outer_classname = "DistributedSchedulerProtocol";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-package hadoop.yarn;
-
-import "yarn_service_protos.proto";
-import "yarn_server_common_service_protos.proto";
-
-
-service DistributedSchedulerProtocolService {
- rpc registerApplicationMasterForDistributedScheduling (RegisterApplicationMasterRequestProto) returns (DistSchedRegisterResponseProto);
- rpc finishApplicationMaster (FinishApplicationMasterRequestProto) returns (FinishApplicationMasterResponseProto);
- rpc allocateForDistributedScheduling (DistSchedAllocateRequestProto) returns (DistSchedAllocateResponseProto);
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/distributed_scheduling_am_protocol.proto hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/distributed_scheduling_am_protocol.proto
new file mode 100644
index 0000000..97d7529
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/distributed_scheduling_am_protocol.proto
@@ -0,0 +1,38 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are public and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+option java_package = "org.apache.hadoop.yarn.proto";
+option java_outer_classname = "DistributedSchedulingAMProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.yarn;
+
+import "yarn_service_protos.proto";
+import "yarn_server_common_service_protos.proto";
+
+service DistributedSchedulingAMProtocolService {
+ rpc registerApplicationMasterForDistributedScheduling (RegisterApplicationMasterRequestProto) returns (RegisterDistributedSchedulingAMResponseProto);
+ rpc finishApplicationMaster (FinishApplicationMasterRequestProto) returns (FinishApplicationMasterResponseProto);
+ rpc allocateForDistributedScheduling (DistributedSchedulingAllocateRequestProto) returns (DistributedSchedulingAllocateResponseProto);
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
index 3660252..55ac875 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
@@ -26,22 +26,22 @@ import "yarn_protos.proto";
import "yarn_server_common_protos.proto";
import "yarn_service_protos.proto";
-message DistSchedRegisterResponseProto {
+message RegisterDistributedSchedulingAMResponseProto {
optional RegisterApplicationMasterResponseProto register_response = 1;
- optional ResourceProto max_alloc_capability = 2;
- optional ResourceProto min_alloc_capability = 3;
- optional ResourceProto incr_alloc_capability = 4;
+ optional ResourceProto max_container_resource = 2;
+ optional ResourceProto min_container_resource = 3;
+ optional ResourceProto incr_container_resource = 4;
optional int32 container_token_expiry_interval = 5;
optional int64 container_id_start = 6;
repeated NodeIdProto nodes_for_scheduling = 7;
}
-message DistSchedAllocateResponseProto {
+message DistributedSchedulingAllocateResponseProto {
optional AllocateResponseProto allocate_response = 1;
repeated NodeIdProto nodes_for_scheduling = 2;
}
-message DistSchedAllocateRequestProto {
+message DistributedSchedulingAllocateRequestProto {
optional AllocateRequestProto allocate_request = 1;
repeated ContainerProto allocated_containers = 2;
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
index ac360f4..511db16 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
@@ -65,7 +65,7 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
-import org.apache.hadoop.yarn.server.nodemanager.scheduler.LocalScheduler;
+import org.apache.hadoop.yarn.server.nodemanager.scheduler.DistributedScheduler;
import org.apache.hadoop.yarn.server.security.MasterKeyData;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
@@ -467,10 +467,9 @@ protected RequestInterceptor createRequestInterceptorChain() {
interceptorClassNames.add(item.trim());
}
- // Make sure LocalScheduler is present at the beginning
- // of the chain..
+ // Make sure DistributedScheduler is present at the beginning of the chain.
if (this.nmContext.isDistributedSchedulingEnabled()) {
- interceptorClassNames.add(0, LocalScheduler.class.getName());
+ interceptorClassNames.add(0, DistributedScheduler.class.getName());
}
return interceptorClassNames;
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AbstractRequestInterceptor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AbstractRequestInterceptor.java
index 55c65f4..e6c9bbd 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AbstractRequestInterceptor.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AbstractRequestInterceptor.java
@@ -21,12 +21,11 @@
import org.apache.hadoop.conf.Configuration;
import com.google.common.base.Preconditions;
-import org.apache.hadoop.yarn.api.protocolrecords
- .RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedRegisterResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
import java.io.IOException;
@@ -118,8 +117,9 @@ public AMRMProxyApplicationContext getApplicationContext() {
* @throws IOException
*/
@Override
- public DistSchedAllocateResponse allocateForDistributedScheduling(
- DistSchedAllocateRequest request) throws YarnException, IOException {
+ public DistributedSchedulingAllocateResponse allocateForDistributedScheduling(
+ DistributedSchedulingAllocateRequest request)
+ throws YarnException, IOException {
return (this.nextInterceptor != null) ?
this.nextInterceptor.allocateForDistributedScheduling(request) : null;
}
@@ -134,10 +134,10 @@ public DistSchedAllocateResponse allocateForDistributedScheduling(
* @throws IOException
*/
@Override
- public DistSchedRegisterResponse
+ public RegisterDistributedSchedulingAMResponse
registerApplicationMasterForDistributedScheduling(
- RegisterApplicationMasterRequest request)
- throws YarnException, IOException {
+ RegisterApplicationMasterRequest request)
+ throws YarnException, IOException {
return (this.nextInterceptor != null) ? this.nextInterceptor
.registerApplicationMasterForDistributedScheduling(request) : null;
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/DefaultRequestInterceptor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/DefaultRequestInterceptor.java
index debff76..75fe022 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/DefaultRequestInterceptor.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/DefaultRequestInterceptor.java
@@ -43,12 +43,11 @@
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
-import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocol;
+import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
import org.apache.hadoop.yarn.server.api.ServerRMProxy;
-import org.apache.hadoop.yarn.server.api.protocolrecords
- .DistSchedAllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedRegisterResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -63,7 +62,7 @@
AbstractRequestInterceptor {
private static final Logger LOG = LoggerFactory
.getLogger(DefaultRequestInterceptor.class);
- private DistributedSchedulerProtocol rmClient;
+ private DistributedSchedulingAMProtocol rmClient;
private UserGroupInformation user = null;
@Override
@@ -77,13 +76,13 @@ public void init(AMRMProxyApplicationContext appContext) {
user.addToken(appContext.getAMRMToken());
final Configuration conf = this.getConf();
- rmClient =
- user.doAs(new PrivilegedExceptionAction() {
+ rmClient = user.doAs(
+ new PrivilegedExceptionAction() {
@Override
- public DistributedSchedulerProtocol run() throws Exception {
+ public DistributedSchedulingAMProtocol run() throws Exception {
setAMRMTokenService(conf);
return ServerRMProxy.createRMProxy(conf,
- DistributedSchedulerProtocol.class);
+ DistributedSchedulingAMProtocol.class);
}
});
} catch (IOException e) {
@@ -124,7 +123,7 @@ public AllocateResponse allocate(final AllocateRequest request)
}
@Override
- public DistSchedRegisterResponse
+ public RegisterDistributedSchedulingAMResponse
registerApplicationMasterForDistributedScheduling
(RegisterApplicationMasterRequest request) throws YarnException,
IOException {
@@ -134,13 +133,14 @@ public AllocateResponse allocate(final AllocateRequest request)
}
@Override
- public DistSchedAllocateResponse allocateForDistributedScheduling(
- DistSchedAllocateRequest request) throws YarnException, IOException {
+ public DistributedSchedulingAllocateResponse allocateForDistributedScheduling(
+ DistributedSchedulingAllocateRequest request)
+ throws YarnException, IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Forwarding allocateForDistributedScheduling request" +
"to the real YARN RM");
}
- DistSchedAllocateResponse allocateResponse =
+ DistributedSchedulingAllocateResponse allocateResponse =
rmClient.allocateForDistributedScheduling(request);
if (allocateResponse.getAllocateResponse().getAMRMToken() != null) {
updateAMRMToken(allocateResponse.getAllocateResponse().getAMRMToken());
@@ -180,10 +180,10 @@ private void updateAMRMToken(Token token) throws IOException {
@VisibleForTesting
public void setRMClient(final ApplicationMasterProtocol rmClient) {
- if (rmClient instanceof DistributedSchedulerProtocol) {
- this.rmClient = (DistributedSchedulerProtocol)rmClient;
+ if (rmClient instanceof DistributedSchedulingAMProtocol) {
+ this.rmClient = (DistributedSchedulingAMProtocol)rmClient;
} else {
- this.rmClient = new DistributedSchedulerProtocol() {
+ this.rmClient = new DistributedSchedulingAMProtocol() {
@Override
public RegisterApplicationMasterResponse registerApplicationMaster
(RegisterApplicationMasterRequest request) throws YarnException,
@@ -205,7 +205,7 @@ public AllocateResponse allocate(AllocateRequest request) throws
}
@Override
- public DistSchedRegisterResponse
+ public RegisterDistributedSchedulingAMResponse
registerApplicationMasterForDistributedScheduling
(RegisterApplicationMasterRequest request) throws YarnException,
IOException {
@@ -213,8 +213,9 @@ public AllocateResponse allocate(AllocateRequest request) throws
}
@Override
- public DistSchedAllocateResponse allocateForDistributedScheduling(
- DistSchedAllocateRequest request)
+ public DistributedSchedulingAllocateResponse
+ allocateForDistributedScheduling(
+ DistributedSchedulingAllocateRequest request)
throws YarnException, IOException {
throw new IOException("Not Supported !!");
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/RequestInterceptor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/RequestInterceptor.java
index 7a73563..5995af1 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/RequestInterceptor.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/RequestInterceptor.java
@@ -19,14 +19,14 @@
package org.apache.hadoop.yarn.server.nodemanager.amrmproxy;
import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocol;
+import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
/**
* Defines the contract to be implemented by the request intercepter classes,
* that can be used to intercept and inspect messages sent from the application
* master to the resource manager.
*/
-public interface RequestInterceptor extends DistributedSchedulerProtocol,
+public interface RequestInterceptor extends DistributedSchedulingAMProtocol,
Configurable {
/**
* This method is called for initializing the intercepter. This is guaranteed
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/DistributedScheduler.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/DistributedScheduler.java
new file mode 100644
index 0000000..bfb12ee
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/DistributedScheduler.java
@@ -0,0 +1,426 @@
+/**
+ * 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.yarn.server.nodemanager.scheduler;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AMRMProxyApplicationContext;
+import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AbstractRequestInterceptor;
+
+
+
+import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+/**
+ * The DistributedScheduler runs on the NodeManager and is modeled as an
+ * AMRMProxy request interceptor. It is responsible for the
+ * following:
+ *
+ * - Intercept
ApplicationMasterProtocol calls and unwrap the
+ * response objects to extract instructions from the
+ * ClusterMonitor running on the ResourceManager to aid in making
+ * distributed scheduling decisions.
+ * - Call the
OpportunisticContainerAllocator to allocate
+ * containers for the outstanding OPPORTUNISTIC container requests.
+ *
+ */
+public final class DistributedScheduler extends AbstractRequestInterceptor {
+
+ static class PartitionedResourceRequests {
+ private List guaranteed = new ArrayList<>();
+ private List opportunistic = new ArrayList<>();
+ public List getGuaranteed() {
+ return guaranteed;
+ }
+ public List getOpportunistic() {
+ return opportunistic;
+ }
+ }
+
+ static class DistributedSchedulerParams {
+ Resource maxResource;
+ Resource minResource;
+ Resource incrementResource;
+ int containerTokenExpiryInterval;
+ }
+
+ private static final Logger LOG = LoggerFactory
+ .getLogger(DistributedScheduler.class);
+
+ private final static RecordFactory RECORD_FACTORY =
+ RecordFactoryProvider.getRecordFactory(null);
+
+ // Currently just used to keep track of allocated containers.
+ // Can be used for reporting stats later.
+ private Set containersAllocated = new HashSet<>();
+
+ private DistributedSchedulerParams appParams =
+ new DistributedSchedulerParams();
+ private final OpportunisticContainerAllocator.ContainerIdCounter
+ containerIdCounter =
+ new OpportunisticContainerAllocator.ContainerIdCounter();
+ private Map nodeList = new LinkedHashMap<>();
+
+ // Mapping of NodeId to NodeTokens. Populated either from RM response or
+ // generated locally if required.
+ private Map nodeTokens = new HashMap<>();
+ final Set blacklist = new HashSet<>();
+
+ // This maintains a map of outstanding OPPORTUNISTIC Reqs. Key-ed by Priority,
+ // Resource Name (Host/rack/any) and capability. This mapping is required
+ // to match a received Container to an outstanding OPPORTUNISTIC
+ // ResourceRequest (ask).
+ final TreeMap>
+ outstandingOpReqs = new TreeMap<>();
+
+ private ApplicationAttemptId applicationAttemptId;
+ private OpportunisticContainerAllocator containerAllocator;
+ private NMTokenSecretManagerInNM nmSecretManager;
+ private String appSubmitter;
+
+ public void init(AMRMProxyApplicationContext appContext) {
+ super.init(appContext);
+ initLocal(appContext.getApplicationAttemptId(),
+ appContext.getNMCotext().getContainerAllocator(),
+ appContext.getNMCotext().getNMTokenSecretManager(),
+ appContext.getUser());
+ }
+
+ @VisibleForTesting
+ void initLocal(ApplicationAttemptId applicationAttemptId,
+ OpportunisticContainerAllocator containerAllocator,
+ NMTokenSecretManagerInNM nmSecretManager, String appSubmitter) {
+ this.applicationAttemptId = applicationAttemptId;
+ this.containerAllocator = containerAllocator;
+ this.nmSecretManager = nmSecretManager;
+ this.appSubmitter = appSubmitter;
+ }
+
+ /**
+ * Route register call to the corresponding distributed scheduling method viz.
+ * registerApplicationMasterForDistributedScheduling, and return response to
+ * the caller after stripping away Distributed Scheduling information.
+ *
+ * @param request
+ * registration request
+ * @return Allocate Response
+ * @throws YarnException YarnException
+ * @throws IOException IOException
+ */
+ @Override
+ public RegisterApplicationMasterResponse registerApplicationMaster
+ (RegisterApplicationMasterRequest request) throws YarnException,
+ IOException {
+ return registerApplicationMasterForDistributedScheduling(request)
+ .getRegisterResponse();
+ }
+
+ /**
+ * Route allocate call to the allocateForDistributedScheduling method and
+ * return response to the caller after stripping away Distributed Scheduling
+ * information.
+ *
+ * @param request
+ * allocation request
+ * @return Allocate Response
+ * @throws YarnException YarnException
+ * @throws IOException IOException
+ */
+ @Override
+ public AllocateResponse allocate(AllocateRequest request) throws
+ YarnException, IOException {
+ DistributedSchedulingAllocateRequest distRequest = RECORD_FACTORY
+ .newRecordInstance(DistributedSchedulingAllocateRequest.class);
+ distRequest.setAllocateRequest(request);
+ return allocateForDistributedScheduling(distRequest).getAllocateResponse();
+ }
+
+ @Override
+ public FinishApplicationMasterResponse finishApplicationMaster
+ (FinishApplicationMasterRequest request) throws YarnException,
+ IOException {
+ return getNextInterceptor().finishApplicationMaster(request);
+ }
+
+ /**
+ * Check if we already have a NMToken. if Not, generate the Token and
+ * add it to the response
+ */
+ private void updateResponseWithNMTokens(AllocateResponse response,
+ List nmTokens, List allocatedContainers) {
+ List newTokens = new ArrayList<>();
+ if (allocatedContainers.size() > 0) {
+ response.getAllocatedContainers().addAll(allocatedContainers);
+ for (Container alloc : allocatedContainers) {
+ if (!nodeTokens.containsKey(alloc.getNodeId())) {
+ newTokens.add(nmSecretManager.generateNMToken(appSubmitter, alloc));
+ }
+ }
+ List retTokens = new ArrayList<>(nmTokens);
+ retTokens.addAll(newTokens);
+ response.setNMTokens(retTokens);
+ }
+ }
+
+ private PartitionedResourceRequests partitionAskList(List
+ askList) {
+ PartitionedResourceRequests partitionedRequests =
+ new PartitionedResourceRequests();
+ for (ResourceRequest rr : askList) {
+ if (rr.getExecutionTypeRequest().getExecutionType() ==
+ ExecutionType.OPPORTUNISTIC) {
+ partitionedRequests.getOpportunistic().add(rr);
+ } else {
+ partitionedRequests.getGuaranteed().add(rr);
+ }
+ }
+ return partitionedRequests;
+ }
+
+ private void updateParameters(
+ RegisterDistributedSchedulingAMResponse registerResponse) {
+ appParams.minResource = registerResponse.getMinContainerResource();
+ appParams.maxResource = registerResponse.getMaxContainerResource();
+ appParams.incrementResource =
+ registerResponse.getIncrContainerResource();
+ if (appParams.incrementResource == null) {
+ appParams.incrementResource = appParams.minResource;
+ }
+ appParams.containerTokenExpiryInterval = registerResponse
+ .getContainerTokenExpiryInterval();
+
+ containerIdCounter
+ .resetContainerIdCounter(registerResponse.getContainerIdStart());
+ setNodeList(registerResponse.getNodesForScheduling());
+ }
+
+ /**
+ * Takes a list of ResourceRequests (asks), extracts the key information viz.
+ * (Priority, ResourceName, Capability) and adds to the outstanding
+ * OPPORTUNISTIC outstandingOpReqs map. The nested map is required to enforce
+ * the current YARN constraint that only a single ResourceRequest can exist at
+ * a give Priority and Capability.
+ *
+ * @param resourceAsks the list with the {@link ResourceRequest}s
+ */
+ public void addToOutstandingReqs(List resourceAsks) {
+ for (ResourceRequest request : resourceAsks) {
+ Priority priority = request.getPriority();
+
+ // TODO: Extend for Node/Rack locality. We only handle ANY requests now
+ if (!ResourceRequest.isAnyLocation(request.getResourceName())) {
+ continue;
+ }
+
+ if (request.getNumContainers() == 0) {
+ continue;
+ }
+
+ Map reqMap =
+ this.outstandingOpReqs.get(priority);
+ if (reqMap == null) {
+ reqMap = new HashMap<>();
+ this.outstandingOpReqs.put(priority, reqMap);
+ }
+
+ ResourceRequest resourceRequest = reqMap.get(request.getCapability());
+ if (resourceRequest == null) {
+ resourceRequest = request;
+ reqMap.put(request.getCapability(), request);
+ } else {
+ resourceRequest.setNumContainers(
+ resourceRequest.getNumContainers() + request.getNumContainers());
+ }
+ if (ResourceRequest.isAnyLocation(request.getResourceName())) {
+ LOG.info("# of outstandingOpReqs in ANY (at priority = " + priority
+ + ", with capability = " + request.getCapability() + " ) : "
+ + resourceRequest.getNumContainers());
+ }
+ }
+ }
+
+ /**
+ * This method matches a returned list of Container Allocations to any
+ * outstanding OPPORTUNISTIC ResourceRequest.
+ */
+ private void matchAllocationToOutstandingRequest(Resource capability,
+ List allocatedContainers) {
+ for (Container c : allocatedContainers) {
+ containersAllocated.add(c.getId());
+ Map asks =
+ outstandingOpReqs.get(c.getPriority());
+
+ if (asks == null)
+ continue;
+
+ ResourceRequest rr = asks.get(capability);
+ if (rr != null) {
+ rr.setNumContainers(rr.getNumContainers() - 1);
+ if (rr.getNumContainers() == 0) {
+ asks.remove(capability);
+ }
+ }
+ }
+ }
+
+ private void setNodeList(List nodeList) {
+ this.nodeList.clear();
+ addToNodeList(nodeList);
+ }
+
+ private void addToNodeList(List nodes) {
+ for (NodeId n : nodes) {
+ this.nodeList.put(n.getHost(), n);
+ }
+ }
+
+ @Override
+ public RegisterDistributedSchedulingAMResponse
+ registerApplicationMasterForDistributedScheduling(
+ RegisterApplicationMasterRequest request)
+ throws YarnException, IOException {
+ LOG.info("Forwarding registration request to the" +
+ "Distributed Scheduler Service on YARN RM");
+ RegisterDistributedSchedulingAMResponse dsResp = getNextInterceptor()
+ .registerApplicationMasterForDistributedScheduling(request);
+ updateParameters(dsResp);
+ return dsResp;
+ }
+
+ @Override
+ public DistributedSchedulingAllocateResponse allocateForDistributedScheduling(
+ DistributedSchedulingAllocateRequest request)
+ throws YarnException, IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Forwarding allocate request to the" +
+ "Distributed Scheduler Service on YARN RM");
+ }
+ // Partition requests into GUARANTEED and OPPORTUNISTIC reqs
+ PartitionedResourceRequests partitionedAsks =
+ partitionAskList(request.getAllocateRequest().getAskList());
+
+ List releasedContainers =
+ request.getAllocateRequest().getReleaseList();
+ int numReleasedContainers = releasedContainers.size();
+ if (numReleasedContainers > 0) {
+ LOG.info("AttemptID: " + applicationAttemptId + " released: "
+ + numReleasedContainers);
+ containersAllocated.removeAll(releasedContainers);
+ }
+
+ // Also, update black list
+ ResourceBlacklistRequest rbr =
+ request.getAllocateRequest().getResourceBlacklistRequest();
+ if (rbr != null) {
+ blacklist.removeAll(rbr.getBlacklistRemovals());
+ blacklist.addAll(rbr.getBlacklistAdditions());
+ }
+
+ // Add OPPORTUNISTIC reqs to the outstanding reqs
+ addToOutstandingReqs(partitionedAsks.getOpportunistic());
+
+ List allocatedContainers = new ArrayList<>();
+ for (Priority priority : outstandingOpReqs.descendingKeySet()) {
+ // Allocated containers :
+ // Key = Requested Capability,
+ // Value = List of Containers of given Cap (The actual container size
+ // might be different than what is requested.. which is why
+ // we need the requested capability (key) to match against
+ // the outstanding reqs)
+ Map> allocated =
+ containerAllocator.allocate(this.appParams, containerIdCounter,
+ outstandingOpReqs.get(priority).values(), blacklist,
+ applicationAttemptId, nodeList, appSubmitter);
+ for (Map.Entry> e : allocated.entrySet()) {
+ matchAllocationToOutstandingRequest(e.getKey(), e.getValue());
+ allocatedContainers.addAll(e.getValue());
+ }
+ }
+
+ request.setAllocatedContainers(allocatedContainers);
+
+ // Send all the GUARANTEED Reqs to RM
+ request.getAllocateRequest().setAskList(partitionedAsks.getGuaranteed());
+ DistributedSchedulingAllocateResponse dsResp =
+ getNextInterceptor().allocateForDistributedScheduling(request);
+
+ // Update host to nodeId mapping
+ setNodeList(dsResp.getNodesForScheduling());
+ List nmTokens = dsResp.getAllocateResponse().getNMTokens();
+ for (NMToken nmToken : nmTokens) {
+ nodeTokens.put(nmToken.getNodeId(), nmToken);
+ }
+
+ List completedContainers =
+ dsResp.getAllocateResponse().getCompletedContainersStatuses();
+
+ // Only account for opportunistic containers
+ for (ContainerStatus cs : completedContainers) {
+ if (cs.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
+ containersAllocated.remove(cs.getContainerId());
+ }
+ }
+
+ // Check if we have NM tokens for all the allocated containers. If not
+ // generate one and update the response.
+ updateResponseWithNMTokens(
+ dsResp.getAllocateResponse(), nmTokens, allocatedContainers);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(
+ "Number of opportunistic containers currently allocated by" +
+ "application: " + containersAllocated.size());
+ }
+ return dsResp;
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/LocalScheduler.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/LocalScheduler.java
deleted file mode 100644
index ec0e8a4..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/LocalScheduler.java
+++ /dev/null
@@ -1,432 +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.yarn.server.nodemanager.scheduler;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedRegisterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords
- .FinishApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords
- .FinishApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords
- .RegisterApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords
- .RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
-import org.apache.hadoop.yarn.api.records.NMToken;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.nodemanager.amrmproxy
- .AMRMProxyApplicationContext;
-import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AbstractRequestInterceptor;
-
-
-
-import org.apache.hadoop.yarn.server.nodemanager.security
- .NMTokenSecretManagerInNM;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-
-/**
- * The LocalScheduler runs on the NodeManager and is modelled as an
- * AMRMProxy request interceptor. It is responsible for the
- * following :
- *
- * - Intercept
ApplicationMasterProtocol calls and unwrap the
- * response objects to extract instructions from the
- * ClusterManager running on the ResourceManager to aid in making
- * Scheduling scheduling decisions
- * - Call the
OpportunisticContainerAllocator to allocate
- * containers for the opportunistic resource outstandingOpReqs
- *
- */
-public final class LocalScheduler extends AbstractRequestInterceptor {
-
- static class PartitionedResourceRequests {
- private List guaranteed = new ArrayList<>();
- private List opportunistic = new ArrayList<>();
- public List getGuaranteed() {
- return guaranteed;
- }
- public List getOpportunistic() {
- return opportunistic;
- }
- }
-
- static class DistSchedulerParams {
- Resource maxResource;
- Resource minResource;
- Resource incrementResource;
- int containerTokenExpiryInterval;
- }
-
- private static final Logger LOG = LoggerFactory
- .getLogger(LocalScheduler.class);
-
- private final static RecordFactory RECORD_FACTORY =
- RecordFactoryProvider.getRecordFactory(null);
-
- // Currently just used to keep track of allocated Containers
- // Can be used for reporting stats later
- private Set containersAllocated = new HashSet<>();
-
- private DistSchedulerParams appParams = new DistSchedulerParams();
- private final OpportunisticContainerAllocator.ContainerIdCounter containerIdCounter =
- new OpportunisticContainerAllocator.ContainerIdCounter();
- private Map nodeList = new LinkedHashMap<>();
-
- // Mapping of NodeId to NodeTokens. Populated either from RM response or
- // generated locally if required.
- private Map nodeTokens = new HashMap<>();
- final Set blacklist = new HashSet<>();
-
- // This maintains a map of outstanding OPPORTUNISTIC Reqs. Key-ed by Priority,
- // Resource Name (Host/rack/any) and capability. This mapping is required
- // to match a received Container to an outstanding OPPORTUNISTIC
- // ResourceRequests (ask)
- final TreeMap>
- outstandingOpReqs = new TreeMap<>();
-
- private ApplicationAttemptId applicationAttemptId;
- private OpportunisticContainerAllocator containerAllocator;
- private NMTokenSecretManagerInNM nmSecretManager;
- private String appSubmitter;
-
- public void init(AMRMProxyApplicationContext appContext) {
- super.init(appContext);
- initLocal(appContext.getApplicationAttemptId(),
- appContext.getNMCotext().getContainerAllocator(),
- appContext.getNMCotext().getNMTokenSecretManager(),
- appContext.getUser());
- }
-
- @VisibleForTesting
- void initLocal(ApplicationAttemptId applicationAttemptId,
- OpportunisticContainerAllocator containerAllocator,
- NMTokenSecretManagerInNM nmSecretManager, String appSubmitter) {
- this.applicationAttemptId = applicationAttemptId;
- this.containerAllocator = containerAllocator;
- this.nmSecretManager = nmSecretManager;
- this.appSubmitter = appSubmitter;
- }
-
- /**
- * Route register call to the corresponding distributed scheduling method viz.
- * registerApplicationMasterForDistributedScheduling, and return response to
- * the caller after stripping away Distributed Scheduling information.
- *
- * @param request
- * registration request
- * @return Allocate Response
- * @throws YarnException
- * @throws IOException
- */
- @Override
- public RegisterApplicationMasterResponse registerApplicationMaster
- (RegisterApplicationMasterRequest request) throws YarnException,
- IOException {
- return registerApplicationMasterForDistributedScheduling(request)
- .getRegisterResponse();
- }
-
- /**
- * Route allocate call to the allocateForDistributedScheduling method and
- * return response to the caller after stripping away Distributed Scheduling
- * information.
- *
- * @param request
- * allocation request
- * @return Allocate Response
- * @throws YarnException
- * @throws IOException
- */
- @Override
- public AllocateResponse allocate(AllocateRequest request) throws
- YarnException, IOException {
- DistSchedAllocateRequest distRequest =
- RECORD_FACTORY.newRecordInstance(DistSchedAllocateRequest.class);
- distRequest.setAllocateRequest(request);
- return allocateForDistributedScheduling(distRequest).getAllocateResponse();
- }
-
- @Override
- public FinishApplicationMasterResponse finishApplicationMaster
- (FinishApplicationMasterRequest request) throws YarnException,
- IOException {
- return getNextInterceptor().finishApplicationMaster(request);
- }
-
- /**
- * Check if we already have a NMToken. if Not, generate the Token and
- * add it to the response
- * @param response
- * @param nmTokens
- * @param allocatedContainers
- */
- private void updateResponseWithNMTokens(AllocateResponse response,
- List nmTokens, List allocatedContainers) {
- List newTokens = new ArrayList<>();
- if (allocatedContainers.size() > 0) {
- response.getAllocatedContainers().addAll(allocatedContainers);
- for (Container alloc : allocatedContainers) {
- if (!nodeTokens.containsKey(alloc.getNodeId())) {
- newTokens.add(nmSecretManager.generateNMToken(appSubmitter, alloc));
- }
- }
- List retTokens = new ArrayList<>(nmTokens);
- retTokens.addAll(newTokens);
- response.setNMTokens(retTokens);
- }
- }
-
- private PartitionedResourceRequests partitionAskList(List
- askList) {
- PartitionedResourceRequests partitionedRequests =
- new PartitionedResourceRequests();
- for (ResourceRequest rr : askList) {
- if (rr.getExecutionTypeRequest().getExecutionType() ==
- ExecutionType.OPPORTUNISTIC) {
- partitionedRequests.getOpportunistic().add(rr);
- } else {
- partitionedRequests.getGuaranteed().add(rr);
- }
- }
- return partitionedRequests;
- }
-
- private void updateParameters(
- DistSchedRegisterResponse registerResponse) {
- appParams.minResource = registerResponse.getMinAllocatableCapabilty();
- appParams.maxResource = registerResponse.getMaxAllocatableCapabilty();
- appParams.incrementResource =
- registerResponse.getIncrAllocatableCapabilty();
- if (appParams.incrementResource == null) {
- appParams.incrementResource = appParams.minResource;
- }
- appParams.containerTokenExpiryInterval = registerResponse
- .getContainerTokenExpiryInterval();
-
- containerIdCounter
- .resetContainerIdCounter(registerResponse.getContainerIdStart());
- setNodeList(registerResponse.getNodesForScheduling());
- }
-
- /**
- * Takes a list of ResourceRequests (asks), extracts the key information viz.
- * (Priority, ResourceName, Capability) and adds it the outstanding
- * OPPORTUNISTIC outstandingOpReqs map. The nested map is required to enforce
- * the current YARN constraint that only a single ResourceRequest can exist at
- * a give Priority and Capability
- * @param resourceAsks
- */
- public void addToOutstandingReqs(List resourceAsks) {
- for (ResourceRequest request : resourceAsks) {
- Priority priority = request.getPriority();
-
- // TODO: Extend for Node/Rack locality. We only handle ANY requests now
- if (!ResourceRequest.isAnyLocation(request.getResourceName())) {
- continue;
- }
-
- if (request.getNumContainers() == 0) {
- continue;
- }
-
- Map reqMap =
- this.outstandingOpReqs.get(priority);
- if (reqMap == null) {
- reqMap = new HashMap<>();
- this.outstandingOpReqs.put(priority, reqMap);
- }
-
- ResourceRequest resourceRequest = reqMap.get(request.getCapability());
- if (resourceRequest == null) {
- resourceRequest = request;
- reqMap.put(request.getCapability(), request);
- } else {
- resourceRequest.setNumContainers(
- resourceRequest.getNumContainers() + request.getNumContainers());
- }
- if (ResourceRequest.isAnyLocation(request.getResourceName())) {
- LOG.info("# of outstandingOpReqs in ANY (at priority = " + priority
- + ", with capability = " + request.getCapability() + " ) : "
- + resourceRequest.getNumContainers());
- }
- }
- }
-
- /**
- * This method matches a returned list of Container Allocations to any
- * outstanding OPPORTUNISTIC ResourceRequest
- * @param capability
- * @param allocatedContainers
- */
- public void matchAllocationToOutstandingRequest(Resource capability,
- List allocatedContainers) {
- for (Container c : allocatedContainers) {
- containersAllocated.add(c.getId());
- Map asks =
- outstandingOpReqs.get(c.getPriority());
-
- if (asks == null)
- continue;
-
- ResourceRequest rr = asks.get(capability);
- if (rr != null) {
- rr.setNumContainers(rr.getNumContainers() - 1);
- if (rr.getNumContainers() == 0) {
- asks.remove(capability);
- }
- }
- }
- }
-
- private void setNodeList(List nodeList) {
- this.nodeList.clear();
- addToNodeList(nodeList);
- }
-
- private void addToNodeList(List nodes) {
- for (NodeId n : nodes) {
- this.nodeList.put(n.getHost(), n);
- }
- }
-
- @Override
- public DistSchedRegisterResponse
- registerApplicationMasterForDistributedScheduling(
- RegisterApplicationMasterRequest request)
- throws YarnException, IOException {
- LOG.info("Forwarding registration request to the" +
- "Distributed Scheduler Service on YARN RM");
- DistSchedRegisterResponse dsResp = getNextInterceptor()
- .registerApplicationMasterForDistributedScheduling(request);
- updateParameters(dsResp);
- return dsResp;
- }
-
- @Override
- public DistSchedAllocateResponse allocateForDistributedScheduling(
- DistSchedAllocateRequest request) throws YarnException, IOException {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Forwarding allocate request to the" +
- "Distributed Scheduler Service on YARN RM");
- }
- // Partition requests into GUARANTEED and OPPORTUNISTIC reqs
- PartitionedResourceRequests partitionedAsks = partitionAskList(
- request.getAllocateRequest().getAskList());
-
- List releasedContainers =
- request.getAllocateRequest().getReleaseList();
- int numReleasedContainers = releasedContainers.size();
- if (numReleasedContainers > 0) {
- LOG.info("AttemptID: " + applicationAttemptId + " released: "
- + numReleasedContainers);
- containersAllocated.removeAll(releasedContainers);
- }
-
- // Also, update black list
- ResourceBlacklistRequest rbr =
- request.getAllocateRequest().getResourceBlacklistRequest();
- if (rbr != null) {
- blacklist.removeAll(rbr.getBlacklistRemovals());
- blacklist.addAll(rbr.getBlacklistAdditions());
- }
-
- // Add OPPORTUNISTIC reqs to the outstanding reqs
- addToOutstandingReqs(partitionedAsks.getOpportunistic());
-
- List allocatedContainers = new ArrayList<>();
- for (Priority priority : outstandingOpReqs.descendingKeySet()) {
- // Allocated containers :
- // Key = Requested Capability,
- // Value = List of Containers of given Cap (The actual container size
- // might be different than what is requested.. which is why
- // we need the requested capability (key) to match against
- // the outstanding reqs)
- Map> allocated =
- containerAllocator.allocate(this.appParams, containerIdCounter,
- outstandingOpReqs.get(priority).values(), blacklist,
- applicationAttemptId, nodeList, appSubmitter);
- for (Map.Entry> e : allocated.entrySet()) {
- matchAllocationToOutstandingRequest(e.getKey(), e.getValue());
- allocatedContainers.addAll(e.getValue());
- }
- }
- request.setAllocatedContainers(allocatedContainers);
-
- // Send all the GUARANTEED Reqs to RM
- request.getAllocateRequest().setAskList(partitionedAsks.getGuaranteed());
- DistSchedAllocateResponse dsResp =
- getNextInterceptor().allocateForDistributedScheduling(request);
-
- // Update host to nodeId mapping
- setNodeList(dsResp.getNodesForScheduling());
- List nmTokens = dsResp.getAllocateResponse().getNMTokens();
- for (NMToken nmToken : nmTokens) {
- nodeTokens.put(nmToken.getNodeId(), nmToken);
- }
-
- List completedContainers =
- dsResp.getAllocateResponse().getCompletedContainersStatuses();
-
- // Only account for opportunistic containers
- for (ContainerStatus cs : completedContainers) {
- if (cs.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
- containersAllocated.remove(cs.getContainerId());
- }
- }
-
- // Check if we have NM tokens for all the allocated containers. If not
- // generate one and update the response.
- updateResponseWithNMTokens(
- dsResp.getAllocateResponse(), nmTokens, allocatedContainers);
-
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "Number of opportunistic containers currently allocated by" +
- "application: " + containersAllocated.size());
- }
- return dsResp;
- }
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/OpportunisticContainerAllocator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/OpportunisticContainerAllocator.java
index 22a6a24..ce5bda0 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/OpportunisticContainerAllocator.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/OpportunisticContainerAllocator.java
@@ -29,7 +29,7 @@
import org.apache.hadoop.yarn.server.api.ContainerType;
import org.apache.hadoop.yarn.server.nodemanager.Context;
import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
-import org.apache.hadoop.yarn.server.nodemanager.scheduler.LocalScheduler.DistSchedulerParams;
+import org.apache.hadoop.yarn.server.nodemanager.scheduler.DistributedScheduler.DistributedSchedulerParams;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -37,15 +37,17 @@
import java.net.InetSocketAddress;
import java.util.*;
+import java.util.Map.Entry;
import java.util.concurrent.atomic.AtomicLong;
/**
- * The OpportunisticContainerAllocator allocates containers on a given list
- * of Nodes after it modifies the container sizes to within allowable limits
- * specified by the ClusterManager running on the RM. It tries to
- * distribute the containers as evenly as possible. It also uses the
- * NMTokenSecretManagerInNM to generate the required NM tokens for
- * the allocated containers
+ *
+ * The OpportunisticContainerAllocator allocates containers on a given list of
+ * nodes, after modifying the container sizes to respect the limits set by the
+ * ResourceManager. It tries to distribute the containers as evenly as possible.
+ * It also uses the NMTokenSecretManagerInNM to generate the
+ * required NM tokens for the allocated containers.
+ *
*/
public class OpportunisticContainerAllocator {
@@ -78,15 +80,15 @@ public OpportunisticContainerAllocator(NodeStatusUpdater nodeStatusUpdater,
this.webpagePort = webpagePort;
}
- public Map> allocate(DistSchedulerParams appParams,
- ContainerIdCounter idCounter, Collection resourceAsks,
- Set blacklist, ApplicationAttemptId appAttId,
- Map allNodes, String userName) throws YarnException {
+ public Map> allocate(
+ DistributedSchedulerParams appParams, ContainerIdCounter idCounter,
+ Collection resourceAsks, Set blacklist,
+ ApplicationAttemptId appAttId, Map allNodes,
+ String userName) throws YarnException {
Map> containers = new HashMap<>();
- Set nodesAllocated = new HashSet<>();
for (ResourceRequest anyAsk : resourceAsks) {
allocateOpportunisticContainers(appParams, idCounter, blacklist, appAttId,
- allNodes, userName, containers, nodesAllocated, anyAsk);
+ allNodes, userName, containers, anyAsk);
LOG.info("Opportunistic allocation requested for ["
+ "priority=" + anyAsk.getPriority()
+ ", num_containers=" + anyAsk.getNumContainers()
@@ -96,30 +98,30 @@ public OpportunisticContainerAllocator(NodeStatusUpdater nodeStatusUpdater,
return containers;
}
- private void allocateOpportunisticContainers(DistSchedulerParams appParams,
- ContainerIdCounter idCounter, Set blacklist,
- ApplicationAttemptId id, Map allNodes, String userName,
- Map> containers, Set nodesAllocated,
- ResourceRequest anyAsk) throws YarnException {
+ private void allocateOpportunisticContainers(
+ DistributedSchedulerParams appParams, ContainerIdCounter idCounter,
+ Set blacklist, ApplicationAttemptId id,
+ Map allNodes, String userName,
+ Map> containers, ResourceRequest anyAsk)
+ throws YarnException {
int toAllocate = anyAsk.getNumContainers()
- - (containers.isEmpty() ?
- 0 : containers.get(anyAsk.getCapability()).size());
+ - (containers.isEmpty() ? 0 :
+ containers.get(anyAsk.getCapability()).size());
- List topKNodesLeft = new ArrayList<>();
- for (String s : allNodes.keySet()) {
- // Bias away from whatever we have already allocated and respect blacklist
- if (nodesAllocated.contains(s) || blacklist.contains(s)) {
+ List nodesForScheduling = new ArrayList<>();
+ for (Entry nodeEntry : allNodes.entrySet()) {
+ // Do not use blacklisted nodes for scheduling.
+ if (blacklist.contains(nodeEntry.getKey())) {
continue;
}
- topKNodesLeft.add(s);
+ nodesForScheduling.add(nodeEntry.getValue());
}
int numAllocated = 0;
- int nextNodeToAllocate = 0;
+ int nextNodeToSchedule = 0;
for (int numCont = 0; numCont < toAllocate; numCont++) {
- String topNode = topKNodesLeft.get(nextNodeToAllocate);
- nextNodeToAllocate++;
- nextNodeToAllocate %= topKNodesLeft.size();
- NodeId nodeId = allNodes.get(topNode);
+ nextNodeToSchedule++;
+ nextNodeToSchedule %= nodesForScheduling.size();
+ NodeId nodeId = nodesForScheduling.get(nextNodeToSchedule);
Container container = buildContainer(appParams, idCounter, anyAsk, id,
userName, nodeId);
List cList = containers.get(anyAsk.getCapability());
@@ -134,7 +136,7 @@ private void allocateOpportunisticContainers(DistSchedulerParams appParams,
LOG.info("Allocated " + numAllocated + " opportunistic containers.");
}
- private Container buildContainer(DistSchedulerParams appParams,
+ private Container buildContainer(DistributedSchedulerParams appParams,
ContainerIdCounter idCounter, ResourceRequest rr, ApplicationAttemptId id,
String userName, NodeId nodeId) throws YarnException {
ContainerId cId =
@@ -165,7 +167,7 @@ private Container buildContainer(DistSchedulerParams appParams,
return container;
}
- private Resource normalizeCapability(DistSchedulerParams appParams,
+ private Resource normalizeCapability(DistributedSchedulerParams appParams,
ResourceRequest ask) {
return Resources.normalize(RESOURCE_CALCULATOR,
ask.getCapability(), appParams.minResource, appParams.maxResource,
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestDistributedScheduler.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestDistributedScheduler.java
new file mode 100644
index 0000000..b093b3b
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestDistributedScheduler.java
@@ -0,0 +1,270 @@
+/**
+ * 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.yarn.server.nodemanager.scheduler;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
+import org.apache.hadoop.yarn.server.api.records.MasterKey;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
+import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.RequestInterceptor;
+import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Test cases for {@link DistributedScheduler}.
+ */
+public class TestDistributedScheduler {
+
+ @Test
+ public void testDistributedScheduler() throws Exception {
+
+ Configuration conf = new Configuration();
+ DistributedScheduler distributedScheduler = new DistributedScheduler();
+
+ RequestInterceptor finalReqIntcptr = setup(conf, distributedScheduler);
+
+ registerAM(distributedScheduler, finalReqIntcptr, Arrays.asList(
+ NodeId.newInstance("a", 1), NodeId.newInstance("b", 2)));
+
+ final AtomicBoolean flipFlag = new AtomicBoolean(true);
+ Mockito.when(
+ finalReqIntcptr.allocateForDistributedScheduling(
+ Mockito.any(DistributedSchedulingAllocateRequest.class)))
+ .thenAnswer(new Answer() {
+ @Override
+ public DistributedSchedulingAllocateResponse answer(
+ InvocationOnMock invocationOnMock) throws Throwable {
+ flipFlag.set(!flipFlag.get());
+ if (flipFlag.get()) {
+ return createAllocateResponse(Arrays.asList(
+ NodeId.newInstance("c", 3), NodeId.newInstance("d", 4)));
+ } else {
+ return createAllocateResponse(Arrays.asList(
+ NodeId.newInstance("d", 4), NodeId.newInstance("c", 3)));
+ }
+ }
+ });
+
+ AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
+ ResourceRequest guaranteedReq =
+ createResourceRequest(ExecutionType.GUARANTEED, 5, "*");
+
+ ResourceRequest opportunisticReq =
+ createResourceRequest(ExecutionType.OPPORTUNISTIC, 4, "*");
+
+ allocateRequest.setAskList(Arrays.asList(guaranteedReq, opportunisticReq));
+
+ // Verify 4 containers were allocated
+ AllocateResponse allocateResponse =
+ distributedScheduler.allocate(allocateRequest);
+ Assert.assertEquals(4, allocateResponse.getAllocatedContainers().size());
+
+ // Verify equal distribution on hosts a and b, and none on c or d
+ Map> allocs = mapAllocs(allocateResponse, 4);
+ Assert.assertEquals(2, allocs.get(NodeId.newInstance("a", 1)).size());
+ Assert.assertEquals(2, allocs.get(NodeId.newInstance("b", 2)).size());
+ Assert.assertNull(allocs.get(NodeId.newInstance("c", 3)));
+ Assert.assertNull(allocs.get(NodeId.newInstance("d", 4)));
+
+ // New Allocate request
+ allocateRequest = Records.newRecord(AllocateRequest.class);
+ opportunisticReq =
+ createResourceRequest(ExecutionType.OPPORTUNISTIC, 6, "*");
+ allocateRequest.setAskList(Arrays.asList(guaranteedReq, opportunisticReq));
+
+ // Verify 6 containers were allocated
+ allocateResponse = distributedScheduler.allocate(allocateRequest);
+ Assert.assertEquals(6, allocateResponse.getAllocatedContainers().size());
+
+ // Verify new containers are equally distribution on hosts c and d,
+ // and none on a or b
+ allocs = mapAllocs(allocateResponse, 6);
+ Assert.assertEquals(3, allocs.get(NodeId.newInstance("c", 3)).size());
+ Assert.assertEquals(3, allocs.get(NodeId.newInstance("d", 4)).size());
+ Assert.assertNull(allocs.get(NodeId.newInstance("a", 1)));
+ Assert.assertNull(allocs.get(NodeId.newInstance("b", 2)));
+
+ // Ensure the DistributedScheduler respects the list order..
+ // The first request should be allocated to "d" since it is ranked higher
+ // The second request should be allocated to "c" since the ranking is
+ // flipped on every allocate response.
+ allocateRequest = Records.newRecord(AllocateRequest.class);
+ opportunisticReq =
+ createResourceRequest(ExecutionType.OPPORTUNISTIC, 1, "*");
+ allocateRequest.setAskList(Arrays.asList(guaranteedReq, opportunisticReq));
+ allocateResponse = distributedScheduler.allocate(allocateRequest);
+ allocs = mapAllocs(allocateResponse, 1);
+ Assert.assertEquals(1, allocs.get(NodeId.newInstance("d", 4)).size());
+
+ allocateRequest = Records.newRecord(AllocateRequest.class);
+ opportunisticReq =
+ createResourceRequest(ExecutionType.OPPORTUNISTIC, 1, "*");
+ allocateRequest.setAskList(Arrays.asList(guaranteedReq, opportunisticReq));
+ allocateResponse = distributedScheduler.allocate(allocateRequest);
+ allocs = mapAllocs(allocateResponse, 1);
+ Assert.assertEquals(1, allocs.get(NodeId.newInstance("c", 3)).size());
+
+ allocateRequest = Records.newRecord(AllocateRequest.class);
+ opportunisticReq =
+ createResourceRequest(ExecutionType.OPPORTUNISTIC, 1, "*");
+ allocateRequest.setAskList(Arrays.asList(guaranteedReq, opportunisticReq));
+ allocateResponse = distributedScheduler.allocate(allocateRequest);
+ allocs = mapAllocs(allocateResponse, 1);
+ Assert.assertEquals(1, allocs.get(NodeId.newInstance("d", 4)).size());
+ }
+
+ private void registerAM(DistributedScheduler distributedScheduler,
+ RequestInterceptor finalReqIntcptr, List nodeList)
+ throws Exception {
+ RegisterDistributedSchedulingAMResponse distSchedRegisterResponse =
+ Records.newRecord(RegisterDistributedSchedulingAMResponse.class);
+ distSchedRegisterResponse.setRegisterResponse(
+ Records.newRecord(RegisterApplicationMasterResponse.class));
+ distSchedRegisterResponse.setContainerTokenExpiryInterval(12345);
+ distSchedRegisterResponse.setContainerIdStart(0);
+ distSchedRegisterResponse.setMaxContainerResource(
+ Resource.newInstance(1024, 4));
+ distSchedRegisterResponse.setMinContainerResource(
+ Resource.newInstance(512, 2));
+ distSchedRegisterResponse.setNodesForScheduling(nodeList);
+ Mockito.when(
+ finalReqIntcptr.registerApplicationMasterForDistributedScheduling(
+ Mockito.any(RegisterApplicationMasterRequest.class)))
+ .thenReturn(distSchedRegisterResponse);
+
+ distributedScheduler.registerApplicationMaster(
+ Records.newRecord(RegisterApplicationMasterRequest.class));
+ }
+
+ private RequestInterceptor setup(Configuration conf,
+ DistributedScheduler distributedScheduler) {
+ NodeStatusUpdater nodeStatusUpdater = Mockito.mock(NodeStatusUpdater.class);
+ Mockito.when(nodeStatusUpdater.getRMIdentifier()).thenReturn(12345l);
+ Context context = Mockito.mock(Context.class);
+ NMContainerTokenSecretManager nmContainerTokenSecretManager = new
+ NMContainerTokenSecretManager(conf);
+ MasterKey mKey = new MasterKey() {
+ @Override
+ public int getKeyId() {
+ return 1;
+ }
+ @Override
+ public void setKeyId(int keyId) {}
+ @Override
+ public ByteBuffer getBytes() {
+ return ByteBuffer.allocate(8);
+ }
+ @Override
+ public void setBytes(ByteBuffer bytes) {}
+ };
+ nmContainerTokenSecretManager.setMasterKey(mKey);
+ Mockito.when(context.getContainerTokenSecretManager()).thenReturn
+ (nmContainerTokenSecretManager);
+ OpportunisticContainerAllocator containerAllocator =
+ new OpportunisticContainerAllocator(nodeStatusUpdater, context, 7777);
+
+ NMTokenSecretManagerInNM nmTokenSecretManagerInNM =
+ new NMTokenSecretManagerInNM();
+ nmTokenSecretManagerInNM.setMasterKey(mKey);
+ distributedScheduler.initLocal(
+ ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1),
+ containerAllocator, nmTokenSecretManagerInNM, "test");
+
+ RequestInterceptor finalReqIntcptr = Mockito.mock(RequestInterceptor.class);
+ distributedScheduler.setNextInterceptor(finalReqIntcptr);
+ return finalReqIntcptr;
+ }
+
+ private ResourceRequest createResourceRequest(ExecutionType execType,
+ int numContainers, String resourceName) {
+ ResourceRequest opportunisticReq = Records.newRecord(ResourceRequest.class);
+ opportunisticReq.setExecutionTypeRequest(
+ ExecutionTypeRequest.newInstance(execType, true));
+ opportunisticReq.setNumContainers(numContainers);
+ opportunisticReq.setCapability(Resource.newInstance(1024, 4));
+ opportunisticReq.setPriority(Priority.newInstance(100));
+ opportunisticReq.setRelaxLocality(true);
+ opportunisticReq.setResourceName(resourceName);
+ return opportunisticReq;
+ }
+
+ private DistributedSchedulingAllocateResponse createAllocateResponse(
+ List nodes) {
+ DistributedSchedulingAllocateResponse distSchedAllocateResponse =
+ Records.newRecord(DistributedSchedulingAllocateResponse.class);
+ distSchedAllocateResponse
+ .setAllocateResponse(Records.newRecord(AllocateResponse.class));
+ distSchedAllocateResponse.setNodesForScheduling(nodes);
+ return distSchedAllocateResponse;
+ }
+
+ private Map> mapAllocs(
+ AllocateResponse allocateResponse, int expectedSize) throws Exception {
+ Assert.assertEquals(expectedSize,
+ allocateResponse.getAllocatedContainers().size());
+ Map> allocs = new HashMap<>();
+ for (Container c : allocateResponse.getAllocatedContainers()) {
+ ContainerTokenIdentifier cTokId = BuilderUtils
+ .newContainerTokenIdentifier(c.getContainerToken());
+ Assert.assertEquals(
+ c.getNodeId().getHost() + ":" + c.getNodeId().getPort(),
+ cTokId.getNmHostAddress());
+ List cIds = allocs.get(c.getNodeId());
+ if (cIds == null) {
+ cIds = new ArrayList<>();
+ allocs.put(c.getNodeId(), cIds);
+ }
+ cIds.add(c.getId());
+ }
+ return allocs;
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestLocalScheduler.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestLocalScheduler.java
deleted file mode 100644
index 8de849b..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestLocalScheduler.java
+++ /dev/null
@@ -1,270 +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.yarn.server.nodemanager.scheduler;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
-import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedRegisterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords
- .RegisterApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords
- .RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.server.api.records.MasterKey;
-import org.apache.hadoop.yarn.server.nodemanager.Context;
-import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
-import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.RequestInterceptor;
-import org.apache.hadoop.yarn.server.nodemanager.security
- .NMContainerTokenSecretManager;
-import org.apache.hadoop.yarn.server.nodemanager.security
- .NMTokenSecretManagerInNM;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.apache.hadoop.yarn.util.Records;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-public class TestLocalScheduler {
-
- @Test
- public void testLocalScheduler() throws Exception {
-
- Configuration conf = new Configuration();
- LocalScheduler localScheduler = new LocalScheduler();
-
- RequestInterceptor finalReqIntcptr = setup(conf, localScheduler);
-
- registerAM(localScheduler, finalReqIntcptr, Arrays.asList(
- NodeId.newInstance("a", 1), NodeId.newInstance("b", 2)));
-
- final AtomicBoolean flipFlag = new AtomicBoolean(false);
- Mockito.when(
- finalReqIntcptr.allocateForDistributedScheduling(
- Mockito.any(DistSchedAllocateRequest.class)))
- .thenAnswer(new Answer() {
- @Override
- public DistSchedAllocateResponse answer(InvocationOnMock
- invocationOnMock) throws Throwable {
- flipFlag.set(!flipFlag.get());
- if (flipFlag.get()) {
- return createAllocateResponse(Arrays.asList(
- NodeId.newInstance("c", 3), NodeId.newInstance("d", 4)));
- } else {
- return createAllocateResponse(Arrays.asList(
- NodeId.newInstance("d", 4), NodeId.newInstance("c", 3)));
- }
- }
- });
-
- AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
- ResourceRequest guaranteedReq =
- createResourceRequest(ExecutionType.GUARANTEED, 5, "*");
-
- ResourceRequest opportunisticReq =
- createResourceRequest(ExecutionType.OPPORTUNISTIC, 4, "*");
- allocateRequest.setAskList(Arrays.asList(guaranteedReq, opportunisticReq));
-
- // Verify 4 containers were allocated
- AllocateResponse allocateResponse =
- localScheduler.allocate(allocateRequest);
- Assert.assertEquals(4, allocateResponse.getAllocatedContainers().size());
-
- // Verify equal distribution on hosts a and b
- // And None on c and d
- Map> allocs = mapAllocs(allocateResponse, 4);
- Assert.assertEquals(2, allocs.get(NodeId.newInstance("a", 1)).size());
- Assert.assertEquals(2, allocs.get(NodeId.newInstance("b", 2)).size());
- Assert.assertNull(allocs.get(NodeId.newInstance("c", 3)));
- Assert.assertNull(allocs.get(NodeId.newInstance("d", 4)));
-
- // New Allocate request
- allocateRequest = Records.newRecord(AllocateRequest.class);
- opportunisticReq =
- createResourceRequest(ExecutionType.OPPORTUNISTIC, 6, "*");
- allocateRequest.setAskList(Arrays.asList(guaranteedReq, opportunisticReq));
-
- // Verify 6 containers were allocated
- allocateResponse = localScheduler.allocate(allocateRequest);
- Assert.assertEquals(6, allocateResponse.getAllocatedContainers().size());
-
- // Verify New containers are equally distribution on hosts c and d
- // And None on a and b
- allocs = mapAllocs(allocateResponse, 6);
- Assert.assertEquals(3, allocs.get(NodeId.newInstance("c", 3)).size());
- Assert.assertEquals(3, allocs.get(NodeId.newInstance("d", 4)).size());
- Assert.assertNull(allocs.get(NodeId.newInstance("a", 1)));
- Assert.assertNull(allocs.get(NodeId.newInstance("b", 2)));
-
- // Ensure the LocalScheduler respects the list order..
- // The first request should be allocated to "d" since it is ranked higher
- // The second request should be allocated to "c" since the ranking is
- // flipped on every allocate response.
- allocateRequest = Records.newRecord(AllocateRequest.class);
- opportunisticReq =
- createResourceRequest(ExecutionType.OPPORTUNISTIC, 1, "*");
- allocateRequest.setAskList(Arrays.asList(guaranteedReq, opportunisticReq));
- allocateResponse = localScheduler.allocate(allocateRequest);
- allocs = mapAllocs(allocateResponse, 1);
- Assert.assertEquals(1, allocs.get(NodeId.newInstance("d", 4)).size());
-
- allocateRequest = Records.newRecord(AllocateRequest.class);
- opportunisticReq =
- createResourceRequest(ExecutionType.OPPORTUNISTIC, 1, "*");
- allocateRequest.setAskList(Arrays.asList(guaranteedReq, opportunisticReq));
- allocateResponse = localScheduler.allocate(allocateRequest);
- allocs = mapAllocs(allocateResponse, 1);
- Assert.assertEquals(1, allocs.get(NodeId.newInstance("c", 3)).size());
-
- allocateRequest = Records.newRecord(AllocateRequest.class);
- opportunisticReq =
- createResourceRequest(ExecutionType.OPPORTUNISTIC, 1, "*");
- allocateRequest.setAskList(Arrays.asList(guaranteedReq, opportunisticReq));
- allocateResponse = localScheduler.allocate(allocateRequest);
- allocs = mapAllocs(allocateResponse, 1);
- Assert.assertEquals(1, allocs.get(NodeId.newInstance("d", 4)).size());
- }
-
- private void registerAM(LocalScheduler localScheduler, RequestInterceptor
- finalReqIntcptr, List nodeList) throws Exception {
- DistSchedRegisterResponse distSchedRegisterResponse =
- Records.newRecord(DistSchedRegisterResponse.class);
- distSchedRegisterResponse.setRegisterResponse(
- Records.newRecord(RegisterApplicationMasterResponse.class));
- distSchedRegisterResponse.setContainerTokenExpiryInterval(12345);
- distSchedRegisterResponse.setContainerIdStart(0);
- distSchedRegisterResponse.setMaxAllocatableCapabilty(
- Resource.newInstance(1024, 4));
- distSchedRegisterResponse.setMinAllocatableCapabilty(
- Resource.newInstance(512, 2));
- distSchedRegisterResponse.setNodesForScheduling(nodeList);
- Mockito.when(
- finalReqIntcptr.registerApplicationMasterForDistributedScheduling(
- Mockito.any(RegisterApplicationMasterRequest.class)))
- .thenReturn(distSchedRegisterResponse);
-
- localScheduler.registerApplicationMaster(
- Records.newRecord(RegisterApplicationMasterRequest.class));
- }
-
- private RequestInterceptor setup(Configuration conf, LocalScheduler
- localScheduler) {
- NodeStatusUpdater nodeStatusUpdater = Mockito.mock(NodeStatusUpdater.class);
- Mockito.when(nodeStatusUpdater.getRMIdentifier()).thenReturn(12345l);
- Context context = Mockito.mock(Context.class);
- NMContainerTokenSecretManager nmContainerTokenSecretManager = new
- NMContainerTokenSecretManager(conf);
- MasterKey mKey = new MasterKey() {
- @Override
- public int getKeyId() {
- return 1;
- }
- @Override
- public void setKeyId(int keyId) {}
- @Override
- public ByteBuffer getBytes() {
- return ByteBuffer.allocate(8);
- }
- @Override
- public void setBytes(ByteBuffer bytes) {}
- };
- nmContainerTokenSecretManager.setMasterKey(mKey);
- Mockito.when(context.getContainerTokenSecretManager()).thenReturn
- (nmContainerTokenSecretManager);
- OpportunisticContainerAllocator containerAllocator =
- new OpportunisticContainerAllocator(nodeStatusUpdater, context, 7777);
-
- NMTokenSecretManagerInNM nmTokenSecretManagerInNM =
- new NMTokenSecretManagerInNM();
- nmTokenSecretManagerInNM.setMasterKey(mKey);
- localScheduler.initLocal(
- ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1),
- containerAllocator, nmTokenSecretManagerInNM, "test");
-
- RequestInterceptor finalReqIntcptr = Mockito.mock(RequestInterceptor.class);
- localScheduler.setNextInterceptor(finalReqIntcptr);
- return finalReqIntcptr;
- }
-
- private ResourceRequest createResourceRequest(ExecutionType execType,
- int numContainers, String resourceName) {
- ResourceRequest opportunisticReq = Records.newRecord(ResourceRequest.class);
- opportunisticReq.setExecutionTypeRequest(
- ExecutionTypeRequest.newInstance(execType, true));
- opportunisticReq.setNumContainers(numContainers);
- opportunisticReq.setCapability(Resource.newInstance(1024, 4));
- opportunisticReq.setPriority(Priority.newInstance(100));
- opportunisticReq.setRelaxLocality(true);
- opportunisticReq.setResourceName(resourceName);
- return opportunisticReq;
- }
-
- private DistSchedAllocateResponse createAllocateResponse(List nodes) {
- DistSchedAllocateResponse distSchedAllocateResponse = Records.newRecord
- (DistSchedAllocateResponse.class);
- distSchedAllocateResponse.setAllocateResponse(
- Records.newRecord(AllocateResponse.class));
- distSchedAllocateResponse.setNodesForScheduling(nodes);
- return distSchedAllocateResponse;
- }
-
- private Map> mapAllocs(AllocateResponse
- allocateResponse, int expectedSize) throws Exception {
- Assert.assertEquals(expectedSize,
- allocateResponse.getAllocatedContainers().size());
- Map> allocs = new HashMap<>();
- for (Container c : allocateResponse.getAllocatedContainers()) {
- ContainerTokenIdentifier cTokId = BuilderUtils
- .newContainerTokenIdentifier(c.getContainerToken());
- Assert.assertEquals(
- c.getNodeId().getHost() + ":" + c.getNodeId().getPort(),
- cTokId.getNmHostAddress());
- List cIds = allocs.get(c.getNodeId());
- if (cIds == null) {
- cIds = new ArrayList<>();
- allocs.put(c.getNodeId(), cIds);
- }
- cIds.add(c.getId());
- }
- return allocs;
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DistributedSchedulingAMService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DistributedSchedulingAMService.java
new file mode 100644
index 0000000..5c6485f
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DistributedSchedulingAMService.java
@@ -0,0 +1,361 @@
+/**
+ * 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.yarn.server.resourcemanager;
+
+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.ipc.Server;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
+import org.apache.hadoop.yarn.api.impl.pb.service.ApplicationMasterProtocolPBServiceImpl;
+
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService;
+
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.NodeQueueLoadMonitor;
+
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.QueueLimitCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The DistributedSchedulingAMService is started instead of the
+ * ApplicationMasterService if distributed scheduling is enabled for the YARN
+ * cluster.
+ * It extends the functionality of the ApplicationMasterService by servicing
+ * clients (AMs and AMRMProxy request interceptors) that understand the
+ * DistributedSchedulingProtocol.
+ */
+public class DistributedSchedulingAMService extends ApplicationMasterService
+ implements DistributedSchedulingAMProtocol, EventHandler {
+
+ private static final Log LOG =
+ LogFactory.getLog(DistributedSchedulingAMService.class);
+
+ private final NodeQueueLoadMonitor nodeMonitor;
+
+ private final ConcurrentHashMap> rackToNode =
+ new ConcurrentHashMap<>();
+ private final ConcurrentHashMap> hostToNode =
+ new ConcurrentHashMap<>();
+ private final int k;
+
+ public DistributedSchedulingAMService(RMContext rmContext,
+ YarnScheduler scheduler) {
+ super(DistributedSchedulingAMService.class.getName(), rmContext, scheduler);
+ this.k = rmContext.getYarnConfiguration().getInt(
+ YarnConfiguration.DIST_SCHEDULING_NODES_NUMBER_USED,
+ YarnConfiguration.DIST_SCHEDULING_NODES_NUMBER_USED_DEFAULT);
+ long nodeSortInterval = rmContext.getYarnConfiguration().getLong(
+ YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS,
+ YarnConfiguration.
+ NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS_DEFAULT);
+ NodeQueueLoadMonitor.LoadComparator comparator =
+ NodeQueueLoadMonitor.LoadComparator.valueOf(
+ rmContext.getYarnConfiguration().get(
+ YarnConfiguration.NM_CONTAINER_QUEUING_LOAD_COMPARATOR,
+ YarnConfiguration.
+ NM_CONTAINER_QUEUING_LOAD_COMPARATOR_DEFAULT));
+
+ NodeQueueLoadMonitor topKSelector =
+ new NodeQueueLoadMonitor(nodeSortInterval, comparator);
+
+ float sigma = rmContext.getYarnConfiguration()
+ .getFloat(YarnConfiguration.NM_CONTAINER_QUEUING_LIMIT_STDEV,
+ YarnConfiguration.NM_CONTAINER_QUEUING_LIMIT_STDEV_DEFAULT);
+
+ int limitMin, limitMax;
+
+ if (comparator == NodeQueueLoadMonitor.LoadComparator.QUEUE_LENGTH) {
+ limitMin = rmContext.getYarnConfiguration()
+ .getInt(YarnConfiguration.NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH,
+ YarnConfiguration.
+ NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH_DEFAULT);
+ limitMax = rmContext.getYarnConfiguration()
+ .getInt(YarnConfiguration.NM_CONTAINER_QUEUING_MAX_QUEUE_LENGTH,
+ YarnConfiguration.
+ NM_CONTAINER_QUEUING_MAX_QUEUE_LENGTH_DEFAULT);
+ } else {
+ limitMin = rmContext.getYarnConfiguration()
+ .getInt(
+ YarnConfiguration.NM_CONTAINER_QUEUING_MIN_QUEUE_WAIT_TIME_MS,
+ YarnConfiguration.
+ NM_CONTAINER_QUEUING_MIN_QUEUE_WAIT_TIME_MS_DEFAULT);
+ limitMax = rmContext.getYarnConfiguration()
+ .getInt(
+ YarnConfiguration.NM_CONTAINER_QUEUING_MAX_QUEUE_WAIT_TIME_MS,
+ YarnConfiguration.
+ NM_CONTAINER_QUEUING_MAX_QUEUE_WAIT_TIME_MS_DEFAULT);
+ }
+
+ topKSelector.initThresholdCalculator(sigma, limitMin, limitMax);
+ this.nodeMonitor = topKSelector;
+ }
+
+ @Override
+ public Server getServer(YarnRPC rpc, Configuration serverConf,
+ InetSocketAddress addr, AMRMTokenSecretManager secretManager) {
+ Server server = rpc.getServer(DistributedSchedulingAMProtocol.class, this,
+ addr, serverConf, secretManager,
+ serverConf.getInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT,
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT));
+ // To support application running on NMs that DO NOT support
+ // Dist Scheduling... The server multiplexes both the
+ // ApplicationMasterProtocol as well as the DistributedSchedulingProtocol
+ ((RPC.Server) server).addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+ ApplicationMasterProtocolPB.class,
+ ApplicationMasterProtocolService.newReflectiveBlockingService(
+ new ApplicationMasterProtocolPBServiceImpl(this)));
+ return server;
+ }
+
+ @Override
+ public RegisterApplicationMasterResponse registerApplicationMaster
+ (RegisterApplicationMasterRequest request) throws YarnException,
+ IOException {
+ return super.registerApplicationMaster(request);
+ }
+
+ @Override
+ public FinishApplicationMasterResponse finishApplicationMaster
+ (FinishApplicationMasterRequest request) throws YarnException,
+ IOException {
+ return super.finishApplicationMaster(request);
+ }
+
+ @Override
+ public AllocateResponse allocate(AllocateRequest request) throws
+ YarnException, IOException {
+ return super.allocate(request);
+ }
+
+ @Override
+ public RegisterDistributedSchedulingAMResponse
+ registerApplicationMasterForDistributedScheduling(
+ RegisterApplicationMasterRequest request) throws YarnException,
+ IOException {
+ RegisterApplicationMasterResponse response =
+ registerApplicationMaster(request);
+ RegisterDistributedSchedulingAMResponse dsResp = recordFactory
+ .newRecordInstance(RegisterDistributedSchedulingAMResponse.class);
+ dsResp.setRegisterResponse(response);
+ dsResp.setMinContainerResource(
+ Resource.newInstance(
+ getConfig().getInt(
+ YarnConfiguration.DIST_SCHEDULING_MIN_CONTAINER_MEMORY_MB,
+ YarnConfiguration.
+ DIST_SCHEDULING_MIN_CONTAINER_MEMORY_MB_DEFAULT),
+ getConfig().getInt(
+ YarnConfiguration.DIST_SCHEDULING_MIN_CONTAINER_VCORES,
+ YarnConfiguration.DIST_SCHEDULING_MIN_CONTAINER_VCORES_DEFAULT)
+ )
+ );
+ dsResp.setMaxContainerResource(
+ Resource.newInstance(
+ getConfig().getInt(
+ YarnConfiguration.DIST_SCHEDULING_MAX_MEMORY_MB,
+ YarnConfiguration.DIST_SCHEDULING_MAX_MEMORY_MB_DEFAULT),
+ getConfig().getInt(
+ YarnConfiguration.DIST_SCHEDULING_MAX_CONTAINER_VCORES,
+ YarnConfiguration.DIST_SCHEDULING_MAX_CONTAINER_VCORES_DEFAULT)
+ )
+ );
+ dsResp.setIncrContainerResource(
+ Resource.newInstance(
+ getConfig().getInt(
+ YarnConfiguration.DIST_SCHEDULING_INCR_CONTAINER_MEMORY_MB,
+ YarnConfiguration.
+ DIST_SCHEDULING_INCR_CONTAINER_MEMORY_MB_DEFAULT),
+ getConfig().getInt(
+ YarnConfiguration.DIST_SCHEDULING_INCR_CONTAINER_VCORES,
+ YarnConfiguration.DIST_SCHEDULING_INCR_CONTAINER_VCORES_DEFAULT)
+ )
+ );
+ dsResp.setContainerTokenExpiryInterval(
+ getConfig().getInt(
+ YarnConfiguration.DIST_SCHEDULING_CONTAINER_TOKEN_EXPIRY_MS,
+ YarnConfiguration.
+ DIST_SCHEDULING_CONTAINER_TOKEN_EXPIRY_MS_DEFAULT));
+ dsResp.setContainerIdStart(
+ this.rmContext.getEpoch() << ResourceManager.EPOCH_BIT_SHIFT);
+
+ // Set nodes to be used for scheduling
+ dsResp.setNodesForScheduling(
+ this.nodeMonitor.selectLeastLoadedNodes(this.k));
+ return dsResp;
+ }
+
+ @Override
+ public DistributedSchedulingAllocateResponse allocateForDistributedScheduling(
+ DistributedSchedulingAllocateRequest request)
+ throws YarnException, IOException {
+ List distAllocContainers = request.getAllocatedContainers();
+ for (Container container : distAllocContainers) {
+ // Create RMContainer
+ SchedulerApplicationAttempt appAttempt =
+ ((AbstractYarnScheduler) rmContext.getScheduler())
+ .getCurrentAttemptForContainer(container.getId());
+ RMContainer rmContainer = new RMContainerImpl(container,
+ appAttempt.getApplicationAttemptId(), container.getNodeId(),
+ appAttempt.getUser(), rmContext, true);
+ appAttempt.addRMContainer(container.getId(), rmContainer);
+ rmContainer.handle(
+ new RMContainerEvent(container.getId(),
+ RMContainerEventType.LAUNCHED));
+ }
+ AllocateResponse response = allocate(request.getAllocateRequest());
+ DistributedSchedulingAllocateResponse dsResp = recordFactory
+ .newRecordInstance(DistributedSchedulingAllocateResponse.class);
+ dsResp.setAllocateResponse(response);
+ dsResp.setNodesForScheduling(
+ this.nodeMonitor.selectLeastLoadedNodes(this.k));
+ return dsResp;
+ }
+
+ private void addToMapping(ConcurrentHashMap> mapping,
+ String rackName, NodeId nodeId) {
+ if (rackName != null) {
+ mapping.putIfAbsent(rackName, new HashSet());
+ Set nodeIds = mapping.get(rackName);
+ synchronized (nodeIds) {
+ nodeIds.add(nodeId);
+ }
+ }
+ }
+
+ private void removeFromMapping(ConcurrentHashMap> mapping,
+ String rackName, NodeId nodeId) {
+ if (rackName != null) {
+ Set nodeIds = mapping.get(rackName);
+ synchronized (nodeIds) {
+ nodeIds.remove(nodeId);
+ }
+ }
+ }
+
+ @Override
+ public void handle(SchedulerEvent event) {
+ switch (event.getType()) {
+ case NODE_ADDED:
+ if (!(event instanceof NodeAddedSchedulerEvent)) {
+ throw new RuntimeException("Unexpected event type: " + event);
+ }
+ NodeAddedSchedulerEvent nodeAddedEvent = (NodeAddedSchedulerEvent) event;
+ nodeMonitor.addNode(nodeAddedEvent.getContainerReports(),
+ nodeAddedEvent.getAddedRMNode());
+ addToMapping(rackToNode, nodeAddedEvent.getAddedRMNode().getRackName(),
+ nodeAddedEvent.getAddedRMNode().getNodeID());
+ addToMapping(hostToNode, nodeAddedEvent.getAddedRMNode().getHostName(),
+ nodeAddedEvent.getAddedRMNode().getNodeID());
+ break;
+ case NODE_REMOVED:
+ if (!(event instanceof NodeRemovedSchedulerEvent)) {
+ throw new RuntimeException("Unexpected event type: " + event);
+ }
+ NodeRemovedSchedulerEvent nodeRemovedEvent =
+ (NodeRemovedSchedulerEvent) event;
+ nodeMonitor.removeNode(nodeRemovedEvent.getRemovedRMNode());
+ removeFromMapping(rackToNode,
+ nodeRemovedEvent.getRemovedRMNode().getRackName(),
+ nodeRemovedEvent.getRemovedRMNode().getNodeID());
+ removeFromMapping(hostToNode,
+ nodeRemovedEvent.getRemovedRMNode().getHostName(),
+ nodeRemovedEvent.getRemovedRMNode().getNodeID());
+ break;
+ case NODE_UPDATE:
+ if (!(event instanceof NodeUpdateSchedulerEvent)) {
+ throw new RuntimeException("Unexpected event type: " + event);
+ }
+ NodeUpdateSchedulerEvent nodeUpdatedEvent = (NodeUpdateSchedulerEvent)
+ event;
+ nodeMonitor.updateNode(nodeUpdatedEvent.getRMNode());
+ break;
+ case NODE_RESOURCE_UPDATE:
+ if (!(event instanceof NodeResourceUpdateSchedulerEvent)) {
+ throw new RuntimeException("Unexpected event type: " + event);
+ }
+ NodeResourceUpdateSchedulerEvent nodeResourceUpdatedEvent =
+ (NodeResourceUpdateSchedulerEvent) event;
+ nodeMonitor.updateNodeResource(nodeResourceUpdatedEvent.getRMNode(),
+ nodeResourceUpdatedEvent.getResourceOption());
+ break;
+
+ // <-- IGNORED EVENTS : START -->
+ case APP_ADDED:
+ break;
+ case APP_REMOVED:
+ break;
+ case APP_ATTEMPT_ADDED:
+ break;
+ case APP_ATTEMPT_REMOVED:
+ break;
+ case CONTAINER_EXPIRED:
+ break;
+ case NODE_LABELS_UPDATE:
+ break;
+ // <-- IGNORED EVENTS : END -->
+ default:
+ LOG.error("Unknown event arrived at DistributedSchedulingAMService: "
+ + event.toString());
+ }
+
+ }
+
+ public QueueLimitCalculator getNodeManagerQueueLimitCalculator() {
+ return nodeMonitor.getThresholdCalculator();
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DistributedSchedulingService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DistributedSchedulingService.java
deleted file mode 100644
index 5aabddc..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DistributedSchedulingService.java
+++ /dev/null
@@ -1,358 +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.yarn.server.resourcemanager;
-
-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.ipc.Server;
-import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocol;
-import org.apache.hadoop.yarn.api.impl.pb.service.ApplicationMasterProtocolPBServiceImpl;
-
-
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedRegisterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService;
-
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.NodeQueueLoadMonitor;
-
-
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.QueueLimitCalculator;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- * The DistributedSchedulingService is started instead of the
- * ApplicationMasterService if DistributedScheduling is enabled for the YARN
- * cluster.
- * It extends the functionality of the ApplicationMasterService by servicing
- * clients (AMs and AMRMProxy request interceptors) that understand the
- * DistributedSchedulingProtocol.
- */
-public class DistributedSchedulingService extends ApplicationMasterService
- implements DistributedSchedulerProtocol, EventHandler {
-
- private static final Log LOG =
- LogFactory.getLog(DistributedSchedulingService.class);
-
- private final NodeQueueLoadMonitor nodeMonitor;
-
- private final ConcurrentHashMap> rackToNode =
- new ConcurrentHashMap<>();
- private final ConcurrentHashMap> hostToNode =
- new ConcurrentHashMap<>();
- private final int k;
-
- public DistributedSchedulingService(RMContext rmContext,
- YarnScheduler scheduler) {
- super(DistributedSchedulingService.class.getName(), rmContext, scheduler);
- this.k = rmContext.getYarnConfiguration().getInt(
- YarnConfiguration.DIST_SCHEDULING_TOP_K,
- YarnConfiguration.DIST_SCHEDULING_TOP_K_DEFAULT);
- long nodeSortInterval = rmContext.getYarnConfiguration().getLong(
- YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS,
- YarnConfiguration.
- NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS_DEFAULT);
- NodeQueueLoadMonitor.LoadComparator comparator =
- NodeQueueLoadMonitor.LoadComparator.valueOf(
- rmContext.getYarnConfiguration().get(
- YarnConfiguration.NM_CONTAINER_QUEUING_LOAD_COMPARATOR,
- YarnConfiguration.
- NM_CONTAINER_QUEUING_LOAD_COMPARATOR_DEFAULT));
-
- NodeQueueLoadMonitor topKSelector =
- new NodeQueueLoadMonitor(nodeSortInterval, comparator);
-
- float sigma = rmContext.getYarnConfiguration()
- .getFloat(YarnConfiguration.NM_CONTAINER_QUEUING_LIMIT_STDEV,
- YarnConfiguration.NM_CONTAINER_QUEUING_LIMIT_STDEV_DEFAULT);
-
- int limitMin, limitMax;
-
- if (comparator == NodeQueueLoadMonitor.LoadComparator.QUEUE_LENGTH) {
- limitMin = rmContext.getYarnConfiguration()
- .getInt(YarnConfiguration.NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH,
- YarnConfiguration.
- NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH_DEFAULT);
- limitMax = rmContext.getYarnConfiguration()
- .getInt(YarnConfiguration.NM_CONTAINER_QUEUING_MAX_QUEUE_LENGTH,
- YarnConfiguration.
- NM_CONTAINER_QUEUING_MAX_QUEUE_LENGTH_DEFAULT);
- } else {
- limitMin = rmContext.getYarnConfiguration()
- .getInt(
- YarnConfiguration.NM_CONTAINER_QUEUING_MIN_QUEUE_WAIT_TIME_MS,
- YarnConfiguration.
- NM_CONTAINER_QUEUING_MIN_QUEUE_WAIT_TIME_MS_DEFAULT);
- limitMax = rmContext.getYarnConfiguration()
- .getInt(
- YarnConfiguration.NM_CONTAINER_QUEUING_MAX_QUEUE_WAIT_TIME_MS,
- YarnConfiguration.
- NM_CONTAINER_QUEUING_MAX_QUEUE_WAIT_TIME_MS_DEFAULT);
- }
-
- topKSelector.initThresholdCalculator(sigma, limitMin, limitMax);
- this.nodeMonitor = topKSelector;
- }
-
- @Override
- public Server getServer(YarnRPC rpc, Configuration serverConf,
- InetSocketAddress addr, AMRMTokenSecretManager secretManager) {
- Server server = rpc.getServer(DistributedSchedulerProtocol.class, this,
- addr, serverConf, secretManager,
- serverConf.getInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT,
- YarnConfiguration.DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT));
- // To support application running on NMs that DO NOT support
- // Dist Scheduling... The server multiplexes both the
- // ApplicationMasterProtocol as well as the DistributedSchedulingProtocol
- ((RPC.Server) server).addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
- ApplicationMasterProtocolPB.class,
- ApplicationMasterProtocolService.newReflectiveBlockingService(
- new ApplicationMasterProtocolPBServiceImpl(this)));
- return server;
- }
-
- @Override
- public RegisterApplicationMasterResponse registerApplicationMaster
- (RegisterApplicationMasterRequest request) throws YarnException,
- IOException {
- return super.registerApplicationMaster(request);
- }
-
- @Override
- public FinishApplicationMasterResponse finishApplicationMaster
- (FinishApplicationMasterRequest request) throws YarnException,
- IOException {
- return super.finishApplicationMaster(request);
- }
-
- @Override
- public AllocateResponse allocate(AllocateRequest request) throws
- YarnException, IOException {
- return super.allocate(request);
- }
-
- @Override
- public DistSchedRegisterResponse
- registerApplicationMasterForDistributedScheduling(
- RegisterApplicationMasterRequest request) throws YarnException,
- IOException {
- RegisterApplicationMasterResponse response =
- registerApplicationMaster(request);
- DistSchedRegisterResponse dsResp = recordFactory
- .newRecordInstance(DistSchedRegisterResponse.class);
- dsResp.setRegisterResponse(response);
- dsResp.setMinAllocatableCapabilty(
- Resource.newInstance(
- getConfig().getInt(
- YarnConfiguration.DIST_SCHEDULING_MIN_MEMORY,
- YarnConfiguration.DIST_SCHEDULING_MIN_MEMORY_DEFAULT),
- getConfig().getInt(
- YarnConfiguration.DIST_SCHEDULING_MIN_VCORES,
- YarnConfiguration.DIST_SCHEDULING_MIN_VCORES_DEFAULT)
- )
- );
- dsResp.setMaxAllocatableCapabilty(
- Resource.newInstance(
- getConfig().getInt(
- YarnConfiguration.DIST_SCHEDULING_MAX_MEMORY,
- YarnConfiguration.DIST_SCHEDULING_MAX_MEMORY_DEFAULT),
- getConfig().getInt(
- YarnConfiguration.DIST_SCHEDULING_MAX_VCORES,
- YarnConfiguration.DIST_SCHEDULING_MAX_VCORES_DEFAULT)
- )
- );
- dsResp.setIncrAllocatableCapabilty(
- Resource.newInstance(
- getConfig().getInt(
- YarnConfiguration.DIST_SCHEDULING_INCR_MEMORY,
- YarnConfiguration.DIST_SCHEDULING_INCR_MEMORY_DEFAULT),
- getConfig().getInt(
- YarnConfiguration.DIST_SCHEDULING_INCR_VCORES,
- YarnConfiguration.DIST_SCHEDULING_INCR_VCORES_DEFAULT)
- )
- );
- dsResp.setContainerTokenExpiryInterval(
- getConfig().getInt(
- YarnConfiguration.DIST_SCHEDULING_CONTAINER_TOKEN_EXPIRY_MS,
- YarnConfiguration.
- DIST_SCHEDULING_CONTAINER_TOKEN_EXPIRY_MS_DEFAULT));
- dsResp.setContainerIdStart(
- this.rmContext.getEpoch() << ResourceManager.EPOCH_BIT_SHIFT);
-
- // Set nodes to be used for scheduling
- dsResp.setNodesForScheduling(
- this.nodeMonitor.selectLeastLoadedNodes(this.k));
- return dsResp;
- }
-
- @Override
- public DistSchedAllocateResponse allocateForDistributedScheduling(
- DistSchedAllocateRequest request) throws YarnException, IOException {
- List distAllocContainers = request.getAllocatedContainers();
- for (Container container : distAllocContainers) {
- // Create RMContainer
- SchedulerApplicationAttempt appAttempt =
- ((AbstractYarnScheduler) rmContext.getScheduler())
- .getCurrentAttemptForContainer(container.getId());
- RMContainer rmContainer = new RMContainerImpl(container,
- appAttempt.getApplicationAttemptId(), container.getNodeId(),
- appAttempt.getUser(), rmContext, true);
- appAttempt.addRMContainer(container.getId(), rmContainer);
- rmContainer.handle(
- new RMContainerEvent(container.getId(),
- RMContainerEventType.LAUNCHED));
- }
- AllocateResponse response = allocate(request.getAllocateRequest());
- DistSchedAllocateResponse dsResp = recordFactory.newRecordInstance
- (DistSchedAllocateResponse.class);
- dsResp.setAllocateResponse(response);
- dsResp.setNodesForScheduling(
- this.nodeMonitor.selectLeastLoadedNodes(this.k));
- return dsResp;
- }
-
- private void addToMapping(ConcurrentHashMap> mapping,
- String rackName, NodeId nodeId) {
- if (rackName != null) {
- mapping.putIfAbsent(rackName, new HashSet());
- Set nodeIds = mapping.get(rackName);
- synchronized (nodeIds) {
- nodeIds.add(nodeId);
- }
- }
- }
-
- private void removeFromMapping(ConcurrentHashMap> mapping,
- String rackName, NodeId nodeId) {
- if (rackName != null) {
- Set nodeIds = mapping.get(rackName);
- synchronized (nodeIds) {
- nodeIds.remove(nodeId);
- }
- }
- }
-
- @Override
- public void handle(SchedulerEvent event) {
- switch (event.getType()) {
- case NODE_ADDED:
- if (!(event instanceof NodeAddedSchedulerEvent)) {
- throw new RuntimeException("Unexpected event type: " + event);
- }
- NodeAddedSchedulerEvent nodeAddedEvent = (NodeAddedSchedulerEvent) event;
- nodeMonitor.addNode(nodeAddedEvent.getContainerReports(),
- nodeAddedEvent.getAddedRMNode());
- addToMapping(rackToNode, nodeAddedEvent.getAddedRMNode().getRackName(),
- nodeAddedEvent.getAddedRMNode().getNodeID());
- addToMapping(hostToNode, nodeAddedEvent.getAddedRMNode().getHostName(),
- nodeAddedEvent.getAddedRMNode().getNodeID());
- break;
- case NODE_REMOVED:
- if (!(event instanceof NodeRemovedSchedulerEvent)) {
- throw new RuntimeException("Unexpected event type: " + event);
- }
- NodeRemovedSchedulerEvent nodeRemovedEvent =
- (NodeRemovedSchedulerEvent) event;
- nodeMonitor.removeNode(nodeRemovedEvent.getRemovedRMNode());
- removeFromMapping(rackToNode,
- nodeRemovedEvent.getRemovedRMNode().getRackName(),
- nodeRemovedEvent.getRemovedRMNode().getNodeID());
- removeFromMapping(hostToNode,
- nodeRemovedEvent.getRemovedRMNode().getHostName(),
- nodeRemovedEvent.getRemovedRMNode().getNodeID());
- break;
- case NODE_UPDATE:
- if (!(event instanceof NodeUpdateSchedulerEvent)) {
- throw new RuntimeException("Unexpected event type: " + event);
- }
- NodeUpdateSchedulerEvent nodeUpdatedEvent = (NodeUpdateSchedulerEvent)
- event;
- nodeMonitor.updateNode(nodeUpdatedEvent.getRMNode());
- break;
- case NODE_RESOURCE_UPDATE:
- if (!(event instanceof NodeResourceUpdateSchedulerEvent)) {
- throw new RuntimeException("Unexpected event type: " + event);
- }
- NodeResourceUpdateSchedulerEvent nodeResourceUpdatedEvent =
- (NodeResourceUpdateSchedulerEvent) event;
- nodeMonitor.updateNodeResource(nodeResourceUpdatedEvent.getRMNode(),
- nodeResourceUpdatedEvent.getResourceOption());
- break;
-
- // <-- IGNORED EVENTS : START -->
- case APP_ADDED:
- break;
- case APP_REMOVED:
- break;
- case APP_ATTEMPT_ADDED:
- break;
- case APP_ATTEMPT_REMOVED:
- break;
- case CONTAINER_EXPIRED:
- break;
- case NODE_LABELS_UPDATE:
- break;
- // <-- IGNORED EVENTS : END -->
- default:
- LOG.error("Unknown event arrived at DistributedSchedulingService: "
- + event.toString());
- }
-
- }
-
- public QueueLimitCalculator getNodeManagerQueueLimitCalculator() {
- return nodeMonitor.getThresholdCalculator();
- }
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 0c1df33..4509045 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -1180,12 +1180,12 @@ protected ApplicationMasterService createApplicationMasterService() {
if (this.rmContext.getYarnConfiguration().getBoolean(
YarnConfiguration.DIST_SCHEDULING_ENABLED,
YarnConfiguration.DIST_SCHEDULING_ENABLED_DEFAULT)) {
- DistributedSchedulingService distributedSchedulingService = new
- DistributedSchedulingService(this.rmContext, scheduler);
+ DistributedSchedulingAMService distributedSchedulingService = new
+ DistributedSchedulingAMService(this.rmContext, scheduler);
EventDispatcher distSchedulerEventDispatcher =
new EventDispatcher(distributedSchedulingService,
- DistributedSchedulingService.class.getName());
- // Add an event dispoatcher for the DistributedSchedulingService
+ DistributedSchedulingAMService.class.getName());
+ // Add an event dispatcher for the DistributedSchedulingAMService
// to handle node updates/additions and removals.
// Since the SchedulerEvent is currently a super set of theses,
// we register interest for it..
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index 3764664..c677345 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -44,6 +44,7 @@
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
@@ -93,8 +94,8 @@ public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
this.queue = queue;
this.user = user;
this.activeUsersManager = activeUsersManager;
- this.containerIdCounter =
- new AtomicLong(epoch << EPOCH_BIT_SHIFT);
+ this.containerIdCounter = new AtomicLong(
+ epoch << ResourceManager.EPOCH_BIT_SHIFT);
this.appResourceUsage = appResourceUsage;
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index a3e0f9a..6e8c7ff 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -822,7 +822,7 @@ protected ApplicationMasterService createApplicationMasterService() {
if (this.rmContext.getYarnConfiguration().getBoolean(
YarnConfiguration.DIST_SCHEDULING_ENABLED,
YarnConfiguration.DIST_SCHEDULING_ENABLED_DEFAULT)) {
- return new DistributedSchedulingService(getRMContext(), scheduler) {
+ return new DistributedSchedulingAMService(getRMContext(), scheduler) {
@Override
protected void serviceStart() {
// override to not start rpc handler
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingAMService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingAMService.java
new file mode 100644
index 0000000..0213a94
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingAMService.java
@@ -0,0 +1,269 @@
+/**
+ * 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.yarn.server.resourcemanager;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistributedSchedulingAllocateRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistributedSchedulingAllocateResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterDistributedSchedulingAMResponsePBImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Test cases for {@link DistributedSchedulingAMService}.
+ */
+public class TestDistributedSchedulingAMService {
+
+ // Test if the DistributedSchedulingAMService can handle both DSProtocol as
+ // well as AMProtocol clients
+ @Test
+ public void testRPCWrapping() throws Exception {
+ Configuration conf = new Configuration();
+ conf.set(YarnConfiguration.IPC_RPC_IMPL, HadoopYarnProtoRPC.class
+ .getName());
+ YarnRPC rpc = YarnRPC.create(conf);
+ String bindAddr = "localhost:0";
+ InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
+ conf.setSocketAddr(YarnConfiguration.RM_SCHEDULER_ADDRESS, addr);
+ final RecordFactory factory = RecordFactoryProvider.getRecordFactory(null);
+ final RMContext rmContext = new RMContextImpl() {
+ @Override
+ public AMLivelinessMonitor getAMLivelinessMonitor() {
+ return null;
+ }
+
+ @Override
+ public Configuration getYarnConfiguration() {
+ return new YarnConfiguration();
+ }
+ };
+ Container c = factory.newRecordInstance(Container.class);
+ c.setExecutionType(ExecutionType.OPPORTUNISTIC);
+ c.setId(
+ ContainerId.newContainerId(
+ ApplicationAttemptId.newInstance(
+ ApplicationId.newInstance(12345, 1), 2), 3));
+ AllocateRequest allReq =
+ (AllocateRequestPBImpl)factory.newRecordInstance(AllocateRequest.class);
+ allReq.setAskList(Arrays.asList(
+ ResourceRequest.newInstance(Priority.UNDEFINED, "a",
+ Resource.newInstance(1, 2), 1, true, "exp",
+ ExecutionTypeRequest.newInstance(
+ ExecutionType.OPPORTUNISTIC, true))));
+ DistributedSchedulingAMService service =
+ createService(factory, rmContext, c);
+ Server server = service.getServer(rpc, conf, addr, null);
+ server.start();
+
+ // Verify that the DistrubutedSchedulingService can handle vanilla
+ // ApplicationMasterProtocol clients
+ RPC.setProtocolEngine(conf, ApplicationMasterProtocolPB.class,
+ ProtobufRpcEngine.class);
+ ApplicationMasterProtocolPB ampProxy =
+ RPC.getProxy(ApplicationMasterProtocolPB
+ .class, 1, NetUtils.getConnectAddress(server), conf);
+ RegisterApplicationMasterResponse regResp =
+ new RegisterApplicationMasterResponsePBImpl(
+ ampProxy.registerApplicationMaster(null,
+ ((RegisterApplicationMasterRequestPBImpl)factory
+ .newRecordInstance(
+ RegisterApplicationMasterRequest.class)).getProto()));
+ Assert.assertEquals("dummyQueue", regResp.getQueue());
+ FinishApplicationMasterResponse finishResp =
+ new FinishApplicationMasterResponsePBImpl(
+ ampProxy.finishApplicationMaster(null,
+ ((FinishApplicationMasterRequestPBImpl)factory
+ .newRecordInstance(
+ FinishApplicationMasterRequest.class)).getProto()
+ ));
+ Assert.assertEquals(false, finishResp.getIsUnregistered());
+ AllocateResponse allocResp =
+ new AllocateResponsePBImpl(
+ ampProxy.allocate(null,
+ ((AllocateRequestPBImpl)factory
+ .newRecordInstance(AllocateRequest.class)).getProto())
+ );
+ List allocatedContainers = allocResp.getAllocatedContainers();
+ Assert.assertEquals(1, allocatedContainers.size());
+ Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
+ allocatedContainers.get(0).getExecutionType());
+ Assert.assertEquals(12345, allocResp.getNumClusterNodes());
+
+
+ // Verify that the DistrubutedSchedulingService can handle the
+ // DistributedSchedulingAMProtocol clients as well
+ RPC.setProtocolEngine(conf, DistributedSchedulingAMProtocolPB.class,
+ ProtobufRpcEngine.class);
+ DistributedSchedulingAMProtocolPB dsProxy =
+ RPC.getProxy(DistributedSchedulingAMProtocolPB
+ .class, 1, NetUtils.getConnectAddress(server), conf);
+
+ RegisterDistributedSchedulingAMResponse dsRegResp =
+ new RegisterDistributedSchedulingAMResponsePBImpl(
+ dsProxy.registerApplicationMasterForDistributedScheduling(null,
+ ((RegisterApplicationMasterRequestPBImpl)factory
+ .newRecordInstance(RegisterApplicationMasterRequest.class))
+ .getProto()));
+ Assert.assertEquals(54321l, dsRegResp.getContainerIdStart());
+ Assert.assertEquals(4,
+ dsRegResp.getMaxContainerResource().getVirtualCores());
+ Assert.assertEquals(1024,
+ dsRegResp.getMinContainerResource().getMemorySize());
+ Assert.assertEquals(2,
+ dsRegResp.getIncrContainerResource().getVirtualCores());
+
+ DistributedSchedulingAllocateRequestPBImpl distAllReq =
+ (DistributedSchedulingAllocateRequestPBImpl)factory.newRecordInstance(
+ DistributedSchedulingAllocateRequest.class);
+ distAllReq.setAllocateRequest(allReq);
+ distAllReq.setAllocatedContainers(Arrays.asList(c));
+ DistributedSchedulingAllocateResponse dsAllocResp =
+ new DistributedSchedulingAllocateResponsePBImpl(
+ dsProxy.allocateForDistributedScheduling(null,
+ distAllReq.getProto()));
+ Assert.assertEquals(
+ "h1", dsAllocResp.getNodesForScheduling().get(0).getHost());
+
+ FinishApplicationMasterResponse dsfinishResp =
+ new FinishApplicationMasterResponsePBImpl(
+ dsProxy.finishApplicationMaster(null,
+ ((FinishApplicationMasterRequestPBImpl) factory
+ .newRecordInstance(FinishApplicationMasterRequest.class))
+ .getProto()));
+ Assert.assertEquals(
+ false, dsfinishResp.getIsUnregistered());
+ }
+
+ private DistributedSchedulingAMService createService(final RecordFactory
+ factory, final RMContext rmContext, final Container c) {
+ return new DistributedSchedulingAMService(rmContext, null) {
+ @Override
+ public RegisterApplicationMasterResponse registerApplicationMaster(
+ RegisterApplicationMasterRequest request) throws
+ YarnException, IOException {
+ RegisterApplicationMasterResponse resp = factory.newRecordInstance(
+ RegisterApplicationMasterResponse.class);
+ // Dummy Entry to Assert that we get this object back
+ resp.setQueue("dummyQueue");
+ return resp;
+ }
+
+ @Override
+ public FinishApplicationMasterResponse finishApplicationMaster(
+ FinishApplicationMasterRequest request) throws YarnException,
+ IOException {
+ FinishApplicationMasterResponse resp = factory.newRecordInstance(
+ FinishApplicationMasterResponse.class);
+ // Dummy Entry to Assert that we get this object back
+ resp.setIsUnregistered(false);
+ return resp;
+ }
+
+ @Override
+ public AllocateResponse allocate(AllocateRequest request) throws
+ YarnException, IOException {
+ AllocateResponse response = factory.newRecordInstance(
+ AllocateResponse.class);
+ response.setNumClusterNodes(12345);
+ response.setAllocatedContainers(Arrays.asList(c));
+ return response;
+ }
+
+ @Override
+ public RegisterDistributedSchedulingAMResponse
+ registerApplicationMasterForDistributedScheduling(
+ RegisterApplicationMasterRequest request)
+ throws YarnException, IOException {
+ RegisterDistributedSchedulingAMResponse resp = factory
+ .newRecordInstance(RegisterDistributedSchedulingAMResponse.class);
+ resp.setContainerIdStart(54321L);
+ resp.setMaxContainerResource(Resource.newInstance(4096, 4));
+ resp.setMinContainerResource(Resource.newInstance(1024, 1));
+ resp.setIncrContainerResource(Resource.newInstance(2048, 2));
+ return resp;
+ }
+
+ @Override
+ public DistributedSchedulingAllocateResponse
+ allocateForDistributedScheduling(
+ DistributedSchedulingAllocateRequest request)
+ throws YarnException, IOException {
+ List askList =
+ request.getAllocateRequest().getAskList();
+ List allocatedContainers = request.getAllocatedContainers();
+ Assert.assertEquals(1, allocatedContainers.size());
+ Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
+ allocatedContainers.get(0).getExecutionType());
+ Assert.assertEquals(1, askList.size());
+ Assert.assertTrue(askList.get(0)
+ .getExecutionTypeRequest().getEnforceExecutionType());
+ DistributedSchedulingAllocateResponse resp = factory
+ .newRecordInstance(DistributedSchedulingAllocateResponse.class);
+ resp.setNodesForScheduling(
+ Arrays.asList(NodeId.newInstance("h1", 1234)));
+ return resp;
+ }
+ };
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingService.java
deleted file mode 100644
index 4716bab..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingService.java
+++ /dev/null
@@ -1,271 +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.yarn.server.resourcemanager;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .AllocateResponsePBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .FinishApplicationMasterRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .FinishApplicationMasterResponsePBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .RegisterApplicationMasterRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
- .RegisterApplicationMasterResponsePBImpl;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocolPB;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedRegisterResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistSchedAllocateRequestPBImpl;
-
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb
- .DistSchedAllocateResponsePBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb
- .DistSchedRegisterResponsePBImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt
- .AMLivelinessMonitor;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Arrays;
-import java.util.List;
-
-public class TestDistributedSchedulingService {
-
- // Test if the DistributedSchedulingService can handle both DSProtocol as
- // well as AMProtocol clients
- @Test
- public void testRPCWrapping() throws Exception {
- Configuration conf = new Configuration();
- conf.set(YarnConfiguration.IPC_RPC_IMPL, HadoopYarnProtoRPC.class
- .getName());
- YarnRPC rpc = YarnRPC.create(conf);
- String bindAddr = "localhost:0";
- InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
- conf.setSocketAddr(YarnConfiguration.RM_SCHEDULER_ADDRESS, addr);
- final RecordFactory factory = RecordFactoryProvider.getRecordFactory(null);
- final RMContext rmContext = new RMContextImpl() {
- @Override
- public AMLivelinessMonitor getAMLivelinessMonitor() {
- return null;
- }
-
- @Override
- public Configuration getYarnConfiguration() {
- return new YarnConfiguration();
- }
- };
- Container c = factory.newRecordInstance(Container.class);
- c.setExecutionType(ExecutionType.OPPORTUNISTIC);
- c.setId(
- ContainerId.newContainerId(
- ApplicationAttemptId.newInstance(
- ApplicationId.newInstance(12345, 1), 2), 3));
- AllocateRequest allReq =
- (AllocateRequestPBImpl)factory.newRecordInstance(AllocateRequest.class);
- allReq.setAskList(Arrays.asList(
- ResourceRequest.newInstance(Priority.UNDEFINED, "a",
- Resource.newInstance(1, 2), 1, true, "exp",
- ExecutionTypeRequest.newInstance(
- ExecutionType.OPPORTUNISTIC, true))));
- DistributedSchedulingService service = createService(factory, rmContext, c);
- Server server = service.getServer(rpc, conf, addr, null);
- server.start();
-
- // Verify that the DistrubutedSchedulingService can handle vanilla
- // ApplicationMasterProtocol clients
- RPC.setProtocolEngine(conf, ApplicationMasterProtocolPB.class,
- ProtobufRpcEngine.class);
- ApplicationMasterProtocolPB ampProxy =
- RPC.getProxy(ApplicationMasterProtocolPB
- .class, 1, NetUtils.getConnectAddress(server), conf);
- RegisterApplicationMasterResponse regResp =
- new RegisterApplicationMasterResponsePBImpl(
- ampProxy.registerApplicationMaster(null,
- ((RegisterApplicationMasterRequestPBImpl)factory
- .newRecordInstance(
- RegisterApplicationMasterRequest.class)).getProto()));
- Assert.assertEquals("dummyQueue", regResp.getQueue());
- FinishApplicationMasterResponse finishResp =
- new FinishApplicationMasterResponsePBImpl(
- ampProxy.finishApplicationMaster(null,
- ((FinishApplicationMasterRequestPBImpl)factory
- .newRecordInstance(
- FinishApplicationMasterRequest.class)).getProto()
- ));
- Assert.assertEquals(false, finishResp.getIsUnregistered());
- AllocateResponse allocResp =
- new AllocateResponsePBImpl(
- ampProxy.allocate(null,
- ((AllocateRequestPBImpl)factory
- .newRecordInstance(AllocateRequest.class)).getProto())
- );
- List allocatedContainers = allocResp.getAllocatedContainers();
- Assert.assertEquals(1, allocatedContainers.size());
- Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
- allocatedContainers.get(0).getExecutionType());
- Assert.assertEquals(12345, allocResp.getNumClusterNodes());
-
-
- // Verify that the DistrubutedSchedulingService can handle the
- // DistributedSchedulerProtocol clients as well
- RPC.setProtocolEngine(conf, DistributedSchedulerProtocolPB.class,
- ProtobufRpcEngine.class);
- DistributedSchedulerProtocolPB dsProxy =
- RPC.getProxy(DistributedSchedulerProtocolPB
- .class, 1, NetUtils.getConnectAddress(server), conf);
-
- DistSchedRegisterResponse dsRegResp =
- new DistSchedRegisterResponsePBImpl(
- dsProxy.registerApplicationMasterForDistributedScheduling(null,
- ((RegisterApplicationMasterRequestPBImpl)factory
- .newRecordInstance(RegisterApplicationMasterRequest.class))
- .getProto()));
- Assert.assertEquals(54321l, dsRegResp.getContainerIdStart());
- Assert.assertEquals(4,
- dsRegResp.getMaxAllocatableCapabilty().getVirtualCores());
- Assert.assertEquals(1024,
- dsRegResp.getMinAllocatableCapabilty().getMemorySize());
- Assert.assertEquals(2,
- dsRegResp.getIncrAllocatableCapabilty().getVirtualCores());
-
- DistSchedAllocateRequestPBImpl distAllReq =
- (DistSchedAllocateRequestPBImpl)factory.newRecordInstance(
- DistSchedAllocateRequest.class);
- distAllReq.setAllocateRequest(allReq);
- distAllReq.setAllocatedContainers(Arrays.asList(c));
- DistSchedAllocateResponse dsAllocResp =
- new DistSchedAllocateResponsePBImpl(
- dsProxy.allocateForDistributedScheduling(null,
- distAllReq.getProto()));
- Assert.assertEquals(
- "h1", dsAllocResp.getNodesForScheduling().get(0).getHost());
-
- FinishApplicationMasterResponse dsfinishResp =
- new FinishApplicationMasterResponsePBImpl(
- dsProxy.finishApplicationMaster(null,
- ((FinishApplicationMasterRequestPBImpl) factory
- .newRecordInstance(FinishApplicationMasterRequest.class))
- .getProto()));
- Assert.assertEquals(
- false, dsfinishResp.getIsUnregistered());
- }
-
- private DistributedSchedulingService createService(final RecordFactory
- factory, final RMContext rmContext, final Container c) {
- return new DistributedSchedulingService(rmContext, null) {
- @Override
- public RegisterApplicationMasterResponse registerApplicationMaster(
- RegisterApplicationMasterRequest request) throws
- YarnException, IOException {
- RegisterApplicationMasterResponse resp = factory.newRecordInstance(
- RegisterApplicationMasterResponse.class);
- // Dummy Entry to Assert that we get this object back
- resp.setQueue("dummyQueue");
- return resp;
- }
-
- @Override
- public FinishApplicationMasterResponse finishApplicationMaster(
- FinishApplicationMasterRequest request) throws YarnException,
- IOException {
- FinishApplicationMasterResponse resp = factory.newRecordInstance(
- FinishApplicationMasterResponse.class);
- // Dummy Entry to Assert that we get this object back
- resp.setIsUnregistered(false);
- return resp;
- }
-
- @Override
- public AllocateResponse allocate(AllocateRequest request) throws
- YarnException, IOException {
- AllocateResponse response = factory.newRecordInstance(
- AllocateResponse.class);
- response.setNumClusterNodes(12345);
- response.setAllocatedContainers(Arrays.asList(c));
- return response;
- }
-
- @Override
- public DistSchedRegisterResponse
- registerApplicationMasterForDistributedScheduling(
- RegisterApplicationMasterRequest request) throws
- YarnException, IOException {
- DistSchedRegisterResponse resp = factory.newRecordInstance(
- DistSchedRegisterResponse.class);
- resp.setContainerIdStart(54321L);
- resp.setMaxAllocatableCapabilty(Resource.newInstance(4096, 4));
- resp.setMinAllocatableCapabilty(Resource.newInstance(1024, 1));
- resp.setIncrAllocatableCapabilty(Resource.newInstance(2048, 2));
- return resp;
- }
-
- @Override
- public DistSchedAllocateResponse allocateForDistributedScheduling(
- DistSchedAllocateRequest request) throws YarnException, IOException {
- List askList =
- request.getAllocateRequest().getAskList();
- List allocatedContainers = request.getAllocatedContainers();
- Assert.assertEquals(1, allocatedContainers.size());
- Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
- allocatedContainers.get(0).getExecutionType());
- Assert.assertEquals(1, askList.size());
- Assert.assertTrue(askList.get(0)
- .getExecutionTypeRequest().getEnforceExecutionType());
- DistSchedAllocateResponse resp =
- factory.newRecordInstance(DistSchedAllocateResponse.class);
- resp.setNodesForScheduling(
- Arrays.asList(NodeId.newInstance("h1", 1234)));
- return resp;
- }
- };
- }
-}