소스 검색

YARN-7201. Added an apache httpd example YARN service. Contributed by Billie Rinaldi

Jian He 7 년 전
부모
커밋
c70e5df100
10개의 변경된 파일367개의 추가작업 그리고 1개의 파일을 삭제
  1. 62 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd-no-dns/httpd-no-dns.json
  2. 24 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd-no-dns/httpd-proxy-no-dns.conf
  3. 24 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd/httpd-proxy.conf
  4. 55 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd/httpd.json
  5. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/ServiceApiConstants.java
  6. 31 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
  7. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java
  8. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java
  9. 159 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Examples.md
  10. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Overview.md

+ 62 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd-no-dns/httpd-no-dns.json

@@ -0,0 +1,62 @@
+{
+  "name": "httpd-service-no-dns",
+  "lifetime": "3600",
+  "components": [
+    {
+      "name": "httpd",
+      "number_of_containers": 2,
+      "artifact": {
+        "id": "centos/httpd-24-centos7:latest",
+        "type": "DOCKER"
+      },
+      "launch_command": "/usr/bin/run-httpd",
+      "resource": {
+        "cpus": 1,
+        "memory": "1024"
+      },
+      "readiness_check": {
+        "type": "HTTP",
+        "props": {
+          "url": "http://${THIS_HOST}:8080"
+        }
+      },
+      "configuration": {
+        "files": [
+          {
+            "type": "ENV",
+            "dest_file": "/var/www/html/index.html",
+            "props": {
+              "content": "<html><header><title>Title</title></header><body>Hello from ${COMPONENT_INSTANCE_NAME}!</body></html>"
+            }
+          }
+        ]
+      }
+    },
+    {
+      "name": "httpd-proxy",
+      "number_of_containers": 1,
+      "dependencies": [ "httpd" ],
+      "artifact": {
+        "id": "centos/httpd-24-centos7:latest",
+        "type": "DOCKER"
+      },
+      "launch_command": "/usr/bin/run-httpd",
+      "resource": {
+        "cpus": 1,
+        "memory": "1024"
+      },
+      "configuration": {
+        "files": [
+          {
+            "type": "TEMPLATE",
+            "dest_file": "/etc/httpd/conf.d/httpd-proxy.conf",
+            "src_file": "httpd-proxy-no-dns.conf"
+          }
+        ]
+      }
+    }
+  ],
+  "quicklinks": {
+    "Apache HTTP Server": "http://httpd-proxy-0.${SERVICE_NAME}.${USER}.${DOMAIN}:8080"
+  }
+}

+ 24 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd-no-dns/httpd-proxy-no-dns.conf

@@ -0,0 +1,24 @@
+#
+# 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.
+
+<Proxy balancer://test>
+  BalancerMember http://${HTTPD-0_IP}:8080
+  BalancerMember http://${HTTPD-1_IP}:8080
+  ProxySet lbmethod=bytraffic
+</Proxy>
+
+ProxyPass "/"  "balancer://test/"
+ProxyPassReverse "/"  "balancer://test/"

+ 24 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd/httpd-proxy.conf

@@ -0,0 +1,24 @@
+#
+# 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.
+
+<Proxy balancer://test>
+  BalancerMember http://httpd-0.${SERVICE_NAME}.${USER}.${DOMAIN}:8080
+  BalancerMember http://httpd-1.${SERVICE_NAME}.${USER}.${DOMAIN}:8080
+  ProxySet lbmethod=bytraffic
+</Proxy>
+
+ProxyPass "/"  "balancer://test/"
+ProxyPassReverse "/"  "balancer://test/"

+ 55 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd/httpd.json

@@ -0,0 +1,55 @@
+{
+  "name": "httpd-service",
+  "lifetime": "3600",
+  "components": [
+    {
+      "name": "httpd",
+      "number_of_containers": 2,
+      "artifact": {
+        "id": "centos/httpd-24-centos7:latest",
+        "type": "DOCKER"
+      },
+      "launch_command": "/usr/bin/run-httpd",
+      "resource": {
+        "cpus": 1,
+        "memory": "1024"
+      },
+      "configuration": {
+        "files": [
+          {
+            "type": "ENV",
+            "dest_file": "/var/www/html/index.html",
+            "props": {
+              "content": "<html><header><title>Title</title></header><body>Hello from ${COMPONENT_INSTANCE_NAME}!</body></html>"
+            }
+          }
+        ]
+      }
+    },
+    {
+      "name": "httpd-proxy",
+      "number_of_containers": 1,
+      "artifact": {
+        "id": "centos/httpd-24-centos7:latest",
+        "type": "DOCKER"
+      },
+      "launch_command": "/usr/bin/run-httpd",
+      "resource": {
+        "cpus": 1,
+        "memory": "1024"
+      },
+      "configuration": {
+        "files": [
+          {
+            "type": "TEMPLATE",
+            "dest_file": "/etc/httpd/conf.d/httpd-proxy.conf",
+            "src_file": "httpd-proxy.conf"
+          }
+        ]
+      }
+    }
+  ],
+  "quicklinks": {
+    "Apache HTTP Server": "http://httpd-proxy-0.${SERVICE_NAME}.${USER}.${DOMAIN}:8080"
+  }
+}

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/ServiceApiConstants.java

@@ -52,6 +52,11 @@ public interface ServiceApiConstants {
 
   String CONTAINER_ID = $("CONTAINER_ID");
 
+  // Templates for component instance host/IP
+  String COMPONENT_HOST = $("%s_HOST");
+
+  String COMPONENT_IP = $("%s_IP");
+
   // Constants for default cluster ZK
   String CLUSTER_ZK_QUORUM = $("CLUSTER_ZK_QUORUM");
 

+ 31 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java

@@ -47,8 +47,10 @@ import org.slf4j.LoggerFactory;
 
 import java.text.MessageFormat;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -58,6 +60,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import static org.apache.hadoop.yarn.api.records.ContainerExitStatus.*;
+import static org.apache.hadoop.yarn.service.api.ServiceApiConstants.*;
 import static org.apache.hadoop.yarn.service.component.ComponentEventType.*;
 import static org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType.START;
 import static org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType.STOP;
@@ -386,6 +389,34 @@ public class Component implements EventHandler<ComponentEvent> {
     return true;
   }
 
+  public Map<String, String> getDependencyHostIpTokens() {
+    Map<String, String> tokens = new HashMap<>();
+    List<String> dependencies = componentSpec.getDependencies();
+    if (SliderUtils.isEmpty(dependencies)) {
+      return tokens;
+    }
+    for (String dependency : dependencies) {
+      Collection<ComponentInstance> instances = scheduler.getAllComponents()
+          .get(dependency).getAllComponentInstances().values();
+      for (ComponentInstance instance : instances) {
+        if (instance.getContainerStatus() == null) {
+          continue;
+        }
+        if (SliderUtils.isEmpty(instance.getContainerStatus().getIPs()) ||
+            SliderUtils.isUnset(instance.getContainerStatus().getHost())) {
+          continue;
+        }
+        String ip = instance.getContainerStatus().getIPs().get(0);
+        String host = instance.getContainerStatus().getHost();
+        tokens.put(String.format(COMPONENT_IP,
+            instance.getCompInstanceName().toUpperCase()), ip);
+        tokens.put(String.format(COMPONENT_HOST,
+            instance.getCompInstanceName().toUpperCase()), host);
+      }
+    }
+    return tokens;
+  }
+
   private void incRunningContainers() {
     componentMetrics.containersRunning.incr();
     scheduler.getServiceMetrics().containersRunning.incr();

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java

@@ -397,6 +397,7 @@ public class ProviderUtils implements YarnServiceConstants {
     tokens.put(CONTAINER_ID, instance.getContainer().getId().toString());
     tokens.put(COMPONENT_ID,
         String.valueOf(instance.getCompInstanceId().getId()));
+    tokens.putAll(instance.getComponent().getDependencyHostIpTokens());
     return tokens;
   }
 }

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java

@@ -1550,6 +1550,10 @@ public class RegistryDNS extends AbstractService implements DNSOperations,
   private final RegistryCommand removeRecordCommand = new RegistryCommand() {
     @Override
     public void exec(Zone zone, Record record) throws IOException {
+      if (zone == null) {
+        LOG.error("Unable to remove record because zone is null: {}", record);
+        return;
+      }
       zone.removeRecord(record);
       LOG.info("Removed {}", record);
       if (isDNSSECEnabled()) {

+ 159 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Examples.md

@@ -0,0 +1,159 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# YARN Service Examples
+
+This document describes some example service definitions (`Yarnfile`).
+
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
+
+## Apache web server - httpd (with registry DNS)
+Below is the `Yarnfile` for a service called `httpd-service` with two `httpd` instances.
+There is also an httpd proxy instance (httpd-proxy-0) that proxies between the other two httpd instances (httpd-0 and httpd-1).
+
+Note this example requires registry DNS.
+```
+{
+  "name": "httpd-service",
+  "lifetime": "3600",
+  "components": [
+    {
+      "name": "httpd",
+      "number_of_containers": 2,
+      "artifact": {
+        "id": "centos/httpd-24-centos7:latest",
+        "type": "DOCKER"
+      },
+      "launch_command": "/usr/bin/run-httpd",
+      "resource": {
+        "cpus": 1,
+        "memory": "1024"
+      },
+      "configuration": {
+        "files": [
+          {
+            "type": "ENV",
+            "dest_file": "/var/www/html/index.html",
+            "props": {
+              "content": "<html><header><title>Title</title></header><body>Hello from ${COMPONENT_INSTANCE_NAME}!</body></html>"
+            }
+          }
+        ]
+      }
+    },
+    {
+      "name": "httpd-proxy",
+      "number_of_containers": 1,
+      "artifact": {
+        "id": "centos/httpd-24-centos7:latest",
+        "type": "DOCKER"
+      },
+      "launch_command": "/usr/bin/run-httpd",
+      "resource": {
+        "cpus": 1,
+        "memory": "1024"
+      },
+      "configuration": {
+        "files": [
+          {
+            "type": "TEMPLATE",
+            "dest_file": "/etc/httpd/conf.d/httpd-proxy.conf",
+            "src_file": "httpd-proxy.conf"
+          }
+        ]
+      }
+    }
+  ],
+  "quicklinks": {
+    "Apache HTTP Server": "http://httpd-proxy-0.${SERVICE_NAME}.${USER}.${DOMAIN}:8080"
+  }
+}
+```
+This `Yarnfile` is already included in the Hadoop distribution, along with the required configuration template `httpd-proxy.conf`.
+First upload the configuration template file to HDFS:
+```
+hdfs dfs -copyFromLocal ${HADOOP_YARN_HOME}/share/hadoop/yarn/yarn-service-examples/httpd/httpd-proxy.conf .
+```
+
+The proxy configuration template looks like the following and will configure the httpd-proxy-0 container to balance between the httpd-0 and httpd-1 containers evenly:
+```
+<Proxy balancer://test>
+  BalancerMember http://httpd-0.${SERVICE_NAME}.${USER}.${DOMAIN}:8080
+  BalancerMember http://httpd-1.${SERVICE_NAME}.${USER}.${DOMAIN}:8080
+  ProxySet lbmethod=bytraffic
+</Proxy>
+
+ProxyPass "/"  "balancer://test/"
+ProxyPassReverse "/"  "balancer://test/"
+```
+
+Then run the service with the command:
+```
+yarn service create [service-name] --example httpd
+```
+where `service-name` is optional. If omitted, it uses the name defined in the `Yarnfile`.
+
+Once the service is running, navigate to `http://httpd-proxy-0.${SERVICE_NAME}.${USER}.${DOMAIN}:8080` to see the root page.
+The pages should alternately show "Hello from httpd-0!" or "Hello from httpd-1!"
+
+The individual httpd URLs can also be visited, `http://httpd-0.${SERVICE_NAME}.${USER}.${DOMAIN}:8080` and `http://httpd-1.${SERVICE_NAME}.${USER}.${DOMAIN}:8080`.
+
+If unsure of your hostnames, visit the apiserver REST endpoint `http://<apiserver host>:9191/ws/v1/services/httpd-service`.
+
+## Apache web server - httpd (without registry DNS)
+
+A similar IP-based example is provided for environments that do not have registry DNS set up.
+The service name for this example is `httpd-service-no-dns`.
+There are a couple of additions to the `Yarnfile` for the `httpd-service` described above.
+A readiness check is added for the `httpd` component:
+```
+      "readiness_check": {
+        "type": "HTTP",
+        "props": {
+          "url": "http://${THIS_HOST}:8080"
+        }
+      },
+```
+and `httpd` is added as a dependency for the `httpd-proxy` component:
+```
+      "dependencies": [ "httpd" ],
+```
+
+This means that the httpd-proxy-0 instance will not be started until after an HTTP probe has succeeded for the httpd-0 and httpd-1 containers.
+This is necessary so that the IPs of the containers can be used in the configuration of httpd-proxy-0.
+The proxy configuration is similar to that of the previous example, with the BalancerMember lines changed as follows:
+```
+  BalancerMember http://${HTTPD-0_IP}:8080
+  BalancerMember http://${HTTPD-1_IP}:8080
+```
+
+Note that IP and HOST variables such as `${HTTPD-0_IP}` and `${HTTPD-0_HOST}` should only be used by a component that has a dependency on the named component (`httpd` in this case) AND should only be used when the named component specifies a readiness check.
+Here, `httpd-proxy` has a dependency on `httpd` and `httpd` has an HTTP readiness check.
+Without the dependency and readiness check, the httpd-proxy-0 container would be started in parallel with the httpd-0 and http-1 containers, and the IPs and hosts would not be assigned yet for httpd-0 and httpd-1.
+
+Other variables can be used by any component.
+
+Before creating the service, upload the proxy configuration to HDFS:
+```
+hdfs dfs -copyFromLocal ${HADOOP_YARN_HOME}/share/hadoop/yarn/yarn-service-examples/httpd-no-dns/httpd-proxy-no-dns.conf .
+```
+
+Then run the service with the command:
+```
+yarn service create [service-name] --example httpd-no-dns
+```
+where `service-name` is optional. If omitted, it uses the name defined in the `Yarnfile`.
+
+Look up your IPs at the apiserver REST endpoint `http://<apiserver host>:9191/ws/v1/services/httpd-service`.
+Then visit port 8080 for each IP to view the pages.

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Overview.md

@@ -52,7 +52,8 @@ The benefits of combining these workloads are two-fold:
 
 * [Concepts](Concepts.md): Describes the internals of the framework and some features in YARN core to support running services on YARN.
 * [Service REST API](YarnServiceAPI.md): The API doc for deploying/managing services on YARN.
-* [Service Discovery](ServiceDiscovery.md): Deep dives into the YARN DNS internals
+* [Service Discovery](ServiceDiscovery.md): Deep dives into the YARN DNS internals.
+* [Examples](Examples.md): List some example service definitions (`Yarnfile`).