浏览代码

HDDS-1019. Use apache/hadoop-runner image to test ozone secure cluster. Contributed by Xiaoyu Yao.

Xiaoyu Yao 6 年之前
父节点
当前提交
0e450202a6

+ 3 - 16
hadoop-ozone/dist/src/main/compose/ozonesecure/docker-compose.yaml

@@ -35,11 +35,7 @@ services:
       command: ["hadoop", "kms"]
       command: ["hadoop", "kms"]
 
 
   datanode:
   datanode:
-    build:
-      context: docker-image/runner
-      dockerfile: Dockerfile
-      args:
-        buildno: 1
+    image: apache/hadoop-runner
     volumes:
     volumes:
       - ../..:/opt/hadoop
       - ../..:/opt/hadoop
     hostname: datanode
     hostname: datanode
@@ -49,12 +45,7 @@ services:
     env_file:
     env_file:
       - docker-config
       - docker-config
   ozoneManager:
   ozoneManager:
-    build:
-      context: docker-image/runner
-      dockerfile: Dockerfile
-      args:
-        buildno: 1
-    image: om
+    image: apache/hadoop-runner
     hostname: om
     hostname: om
     volumes:
     volumes:
       - ../..:/opt/hadoop
       - ../..:/opt/hadoop
@@ -67,11 +58,7 @@ services:
       - docker-config
       - docker-config
     command: ["/opt/hadoop/bin/ozone","om"]
     command: ["/opt/hadoop/bin/ozone","om"]
   scm:
   scm:
-    build:
-      context: docker-image/runner
-      dockerfile: Dockerfile
-      args:
-        buildno: 1
+    image: apache/hadoop-runner
     hostname: scm
     hostname: scm
     volumes:
     volumes:
       - ../..:/opt/hadoop
       - ../..:/opt/hadoop

+ 1 - 1
hadoop-ozone/dist/src/main/compose/ozonesecure/docker-config

@@ -103,7 +103,7 @@ LOG4J2.PROPERTIES_rootLogger.appenderRefs=stdout
 LOG4J2.PROPERTIES_rootLogger.appenderRef.stdout.ref=STDOUT
 LOG4J2.PROPERTIES_rootLogger.appenderRef.stdout.ref=STDOUT
 
 
 OZONE_DATANODE_SECURE_USER=root
 OZONE_DATANODE_SECURE_USER=root
-CONF_DIR=/etc/security/keytabs
+KEYTAB_DIR=/etc/security/keytabs
 KERBEROS_KEYTABS=dn om scm HTTP testuser
 KERBEROS_KEYTABS=dn om scm HTTP testuser
 KERBEROS_KEYSTORES=hadoop
 KERBEROS_KEYSTORES=hadoop
 KERBEROS_SERVER=kdc
 KERBEROS_SERVER=kdc

+ 0 - 39
hadoop-ozone/dist/src/main/compose/ozonesecure/docker-image/runner/Dockerfile

@@ -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 lsfor the specific language governing permissions and
-# limitations under the License.
-
-FROM openjdk:8-jdk
-RUN apt-get update && apt-get install -y jq curl python python-pip sudo && apt-get clean
-RUN pip install robotframework
-RUN wget -O /usr/local/bin/dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.0/dumb-init_1.2.0_amd64
-RUN chmod +x /usr/local/bin/dumb-init
-RUN mkdir -p /etc/security/keytabs && chmod -R a+wr /etc/security/keytabs
-ENV JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64/
-ENV PATH $PATH:/opt/hadoop/bin
-
-RUN apt-get install -y jsvc
-ENV JSVC_HOME=/usr/bin
-ADD scripts/krb5.conf /etc/
-RUN apt-get install -y krb5-user
-
-RUN addgroup --gid 1000 hadoop
-RUN adduser --disabled-password --gecos "" --uid 1000 hadoop --gid 1000 --home /opt/hadoop
-RUN echo "hadoop ALL=(ALL) NOPASSWD: ALL" >> /etc/sudoers
-ADD scripts /opt/
-
-WORKDIR /opt/hadoop
-
-VOLUME /data
-ENTRYPOINT ["/usr/local/bin/dumb-init", "--", "/opt/starter.sh"]

+ 0 - 26
hadoop-ozone/dist/src/main/compose/ozonesecure/docker-image/runner/build.sh

@@ -1,26 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-
-DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
-set -e
-mkdir -p build
-if [ ! -d "$DIR/build/apache-rat-0.12" ]; then
-  wget "http://xenia.sote.hu/ftp/mirrors/www.apache.org/creadur/apache-rat-0.12/apache-rat-0.12-bin.tar.gz -O $DIR/build/apache-rat.tar.gz"
-  cd "$DIR"/build || exit
-  tar zvxf apache-rat.tar.gz
-fi
-java -jar "$DIR"/build/apache-rat-0.12/apache-rat-0.12.jar "$DIR" -e public -e apache-rat-0.12 -e .git -e .gitignore
-docker build -t ahadoop/runner .

+ 0 - 115
hadoop-ozone/dist/src/main/compose/ozonesecure/docker-image/runner/scripts/envtoconf.py

@@ -1,115 +0,0 @@
-#!/usr/bin/python
-#
-# 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.
-#
-
-"""convert environment variables to config"""
-
-import os
-import re
-
-import argparse
-
-import sys
-import transformation
-
-class Simple(object):
-  """Simple conversion"""
-  def __init__(self, args):
-    parser = argparse.ArgumentParser()
-    parser.add_argument("--destination", help="Destination directory", required=True)
-    self.args = parser.parse_args(args=args)
-    # copy the default files to file.raw in destination directory
-
-    self.known_formats = ['xml', 'properties', 'yaml', 'yml', 'env', "sh", "cfg", 'conf']
-    self.output_dir = self.args.destination
-
-    self.configurables = {}
-
-  def destination_file_path(self, name, extension):
-    """destination file path"""
-    return os.path.join(self.output_dir, "{}.{}".format(name, extension))
-
-  def write_env_var(self, name, extension, key, value):
-    """Write environment variables"""
-    with open(self.destination_file_path(name, extension) + ".raw", "a") as myfile:
-      myfile.write("{}: {}\n".format(key, value))
-
-  def process_envs(self):
-    """Process environment variables"""
-    for key in os.environ.keys():
-      pattern = re.compile("[_\\.]")
-      parts = pattern.split(key)
-      extension = None
-      name = parts[0].lower()
-      if len(parts) > 1:
-        extension = parts[1].lower()
-        config_key = key[len(name) + len(extension) + 2:].strip()
-      if extension and "!" in extension:
-        splitted = extension.split("!")
-        extension = splitted[0]
-        fmt = splitted[1]
-        config_key = key[len(name) + len(extension) + len(fmt) + 3:].strip()
-      else:
-        fmt = extension
-
-      if extension and extension in self.known_formats:
-        if name not in self.configurables.keys():
-          with open(self.destination_file_path(name, extension) + ".raw", "w") as myfile:
-            myfile.write("")
-        self.configurables[name] = (extension, fmt)
-        self.write_env_var(name, extension, config_key, os.environ[key])
-      else:
-        for configurable_name in self.configurables:
-          if key.lower().startswith(configurable_name.lower()):
-            self.write_env_var(configurable_name,
-                               self.configurables[configurable_name],
-                               key[len(configurable_name) + 1:],
-                               os.environ[key])
-
-  def transform(self):
-    """transform"""
-    for configurable_name in self.configurables:
-      name = configurable_name
-      extension, fmt = self.configurables[name]
-
-      destination_path = self.destination_file_path(name, extension)
-
-      with open(destination_path + ".raw", "r") as myfile:
-        content = myfile.read()
-        transformer_func = getattr(transformation, "to_" + fmt)
-        content = transformer_func(content)
-        with open(destination_path, "w") as myfile:
-          myfile.write(content)
-
-  def main(self):
-    """main"""
-
-    # add the
-    self.process_envs()
-
-    # copy file.ext.raw to file.ext in the destination directory, and
-    # transform to the right format (eg. key: value ===> XML)
-    self.transform()
-
-
-def main():
-  """main"""
-  Simple(sys.argv[1:]).main()
-
-
-if __name__ == '__main__':
-  Simple(sys.argv[1:]).main()

+ 0 - 38
hadoop-ozone/dist/src/main/compose/ozonesecure/docker-image/runner/scripts/krb5.conf

@@ -1,38 +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.
-
-[logging]
- default = FILE:/var/log/krb5libs.log
- kdc = FILE:/var/log/krb5kdc.log
- admin_server = FILE:/var/log/kadmind.log
-
-[libdefaults]
- dns_canonicalize_hostname = false
- dns_lookup_realm = false
- ticket_lifetime = 24h
- renew_lifetime = 7d
- forwardable = true
- rdns = false
- default_realm = EXAMPLE.COM
-
-[realms]
- EXAMPLE.COM = {
-  kdc = SERVER
-  admin_server = SERVER
- }
-
-[domain_realm]
- .example.com = EXAMPLE.COM
- example.com = EXAMPLE.COM

+ 0 - 100
hadoop-ozone/dist/src/main/compose/ozonesecure/docker-image/runner/scripts/starter.sh

@@ -1,100 +0,0 @@
-#!/usr/bin/env bash
-##
-# 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.
-##
-set -e
-
-
-DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
-echo "Setting up enviorment!!"
-
-if [ -n "$KERBEROS_ENABLED" ]; then
-  echo "Setting up kerberos!!"
-  KERBEROS_SERVER=${KERBEROS_SERVER:-krb5}
-  ISSUER_SERVER=${ISSUER_SERVER:-$KERBEROS_SERVER\:8081}
-
-  echo "KDC ISSUER_SERVER => $ISSUER_SERVER"
-
-  if [ -n "$SLEEP_SECONDS" ]; then
-    echo "Sleeping for $(SLEEP_SECONDS) seconds"
-    sleep "$SLEEP_SECONDS"
-  fi
-
-
-  while true
-    do
-      STATUS=$(curl -s -o /dev/null -w '%{http_code}' http://"$ISSUER_SERVER"/keytab/test/test)
-      if [ "$STATUS" -eq 200 ]; then
-        echo "Got 200, KDC service ready!!"
-        break
-      else
-        echo "Got $STATUS :( KDC service not ready yet..."
-      fi
-      sleep 5
-    done
-
-    HOST_NAME=$(hostname -f)
-    export HOST_NAME
-    for NAME in ${KERBEROS_KEYTABS}; do
-      echo "Download $NAME/$HOSTNAME@EXAMPLE.COM keytab file to $CONF_DIR/$NAME.keytab"
-      wget "http://$ISSUER_SERVER/keytab/$HOST_NAME/$NAME" -O "$CONF_DIR/$NAME.keytab"
-      klist -kt "$CONF_DIR/$NAME.keytab"
-      KERBEROS_ENABLED=true
-    done
-
-    sed "s/SERVER/$KERBEROS_SERVER/g" "$DIR"/krb5.conf | sudo tee /etc/krb5.conf
-fi
-
-#To avoid docker volume permission problems
-sudo chmod o+rwx /data
-
-"$DIR"/envtoconf.py --destination /opt/hadoop/etc/hadoop
-
-if [ -n "$ENSURE_NAMENODE_DIR" ]; then
-  CLUSTERID_OPTS=""
-  if [ -n "$ENSURE_NAMENODE_CLUSTERID" ]; then
-    CLUSTERID_OPTS="-clusterid $ENSURE_NAMENODE_CLUSTERID"
-  fi
-  if [ ! -d "$ENSURE_NAMENODE_DIR" ]; then
-    /opt/hadoop/bin/hdfs namenode -format -force "$CLUSTERID_OPTS"
-  fi
-fi
-
-if [ -n "$ENSURE_STANDBY_NAMENODE_DIR" ]; then
-  if [ ! -d "$ENSURE_STANDBY_NAMENODE_DIR" ]; then
-    /opt/hadoop/bin/hdfs namenode -bootstrapStandby
-  fi
-fi
-
-if [ -n "$ENSURE_SCM_INITIALIZED" ]; then
-  if [ ! -f "$ENSURE_SCM_INITIALIZED" ]; then
-    /opt/hadoop/bin/ozone scm --init
-  fi
-fi
-
-if [ -n "$ENSURE_OM_INITIALIZED" ]; then
-  if [ ! -f "$ENSURE_OM_INITIALIZED" ]; then
-    #To make sure SCM is running in dockerized environment we will sleep
-    # Could be removed after HDFS-13203
-    echo "Waiting 15 seconds for SCM startup"
-    sleep 15
-    /opt/hadoop/bin/ozone om --init
-  fi
-fi
-
-echo 'setup finished'
-"$@"

+ 0 - 150
hadoop-ozone/dist/src/main/compose/ozonesecure/docker-image/runner/scripts/transformation.py

@@ -1,150 +0,0 @@
-#!/usr/bin/python
-#
-# 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.
-#
-
-"""This module transform properties into different format"""
-def render_yaml(yaml_root, prefix=""):
-  """render yaml"""
-  result = ""
-  if isinstance(yaml_root, dict):
-    if prefix:
-      result += "\n"
-      for key in yaml_root:
-        result += "{}{}: {}".format(prefix, key, render_yaml(
-            yaml_root[key], prefix + "   "))
-  elif isinstance(yaml_root, list):
-    result += "\n"
-    for item in yaml_root:
-      result += prefix + " - " + render_yaml(item, prefix + " ")
-  else:
-    result += "{}\n".format(yaml_root)
-  return result
-
-
-def to_yaml(content):
-  """transform to yaml"""
-  props = process_properties(content)
-
-  keys = props.keys()
-  yaml_props = {}
-  for key in keys:
-    parts = key.split(".")
-    node = yaml_props
-    prev_part = None
-    parent_node = {}
-    for part in parts[:-1]:
-      if part.isdigit():
-        if isinstance(node, dict):
-          parent_node[prev_part] = []
-          node = parent_node[prev_part]
-        while len(node) <= int(part):
-          node.append({})
-        parent_node = node
-        node = node[int(node)]
-      else:
-        if part not in node:
-          node[part] = {}
-        parent_node = node
-        node = node[part]
-      prev_part = part
-    if parts[-1].isdigit():
-      if isinstance(node, dict):
-        parent_node[prev_part] = []
-        node = parent_node[prev_part]
-      node.append(props[key])
-    else:
-      node[parts[-1]] = props[key]
-
-  return render_yaml(yaml_props)
-
-
-def to_yml(content):
-  """transform to yml"""
-  return to_yaml(content)
-
-
-def to_properties(content):
-  """transform to properties"""
-  result = ""
-  props = process_properties(content)
-  for key, val in props.items():
-    result += "{}: {}\n".format(key, val)
-  return result
-
-
-def to_env(content):
-  """transform to environment variables"""
-  result = ""
-  props = process_properties(content)
-  for key, val in props:
-    result += "{}={}\n".format(key, val)
-  return result
-
-
-def to_sh(content):
-  """transform to shell"""
-  result = ""
-  props = process_properties(content)
-  for key, val in props:
-    result += "export {}=\"{}\"\n".format(key, val)
-  return result
-
-
-def to_cfg(content):
-  """transform to config"""
-  result = ""
-  props = process_properties(content)
-  for key, val in props:
-    result += "{}={}\n".format(key, val)
-  return result
-
-
-def to_conf(content):
-  """transform to configuration"""
-  result = ""
-  props = process_properties(content)
-  for key, val in props:
-    result += "export {}={}\n".format(key, val)
-  return result
-
-
-def to_xml(content):
-  """transform to xml"""
-  result = "<configuration>\n"
-  props = process_properties(content)
-  for key in props:
-    result += "<property><name>{0}</name><value>{1}</value></property>\n". \
-      format(key, props[key])
-  result += "</configuration>"
-  return result
-
-
-def process_properties(content, sep=': ', comment_char='#'):
-  """
-  Read the file passed as parameter as a properties file.
-  """
-  props = {}
-  for line in content.split("\n"):
-    sline = line.strip()
-    if sline and not sline.startswith(comment_char):
-      key_value = sline.split(sep)
-      key = key_value[0].strip()
-      value = sep.join(key_value[1:]).strip().strip('"')
-      props[key] = value
-
-  return props