瀏覽代碼

HDDS-1635. Maintain docker entrypoint and envtoconf inside ozone project (#894)

Elek, Márton 6 年之前
父節點
當前提交
3c9a5e7b16

+ 1 - 0
hadoop-ozone/dist/dev-support/bin/dist-layout-stitching

@@ -96,6 +96,7 @@ run cp "${ROOT}/hadoop-hdds/common/src/main/resources/network-topology-nodegroup
 run cp "${ROOT}/hadoop-common-project/hadoop-common/src/main/bin/hadoop" "bin/"
 run cp "${ROOT}/hadoop-common-project/hadoop-common/src/main/bin/hadoop.cmd" "bin/"
 run cp "${ROOT}/hadoop-ozone/common/src/main/bin/ozone" "bin/"
+run cp -r "${ROOT}/hadoop-ozone/dist/src/main/dockerbin" "bin/"
 
 run cp "${ROOT}/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh" "libexec/"
 run cp "${ROOT}/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd" "libexec/"

+ 149 - 0
hadoop-ozone/dist/src/main/dockerbin/entrypoint.sh

@@ -0,0 +1,149 @@
+#!/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 )"
+
+if [ -n "$SLEEP_SECONDS" ]; then
+   echo "Sleeping for $SLEEP_SECONDS seconds"
+   sleep "$SLEEP_SECONDS"
+fi
+
+#
+# You can wait for an other TCP port with these settings.
+#
+# Example:
+#
+# export WAITFOR=localhost:9878
+#
+# With an optional parameter, you can also set the maximum
+# time of waiting with (in seconds) with WAITFOR_TIMEOUT.
+# (The default is 300 seconds / 5 minutes.)
+if [ -n "$WAITFOR" ]; then
+  echo "Waiting for the service $WAITFOR"
+  WAITFOR_HOST=$(printf "%s\n" "$WAITFOR"| cut -d : -f 1)
+  WAITFOR_PORT=$(printf "%s\n" "$WAITFOR"| cut -d : -f 2)
+  for i in $(seq "${WAITFOR_TIMEOUT:-300}" -1 0) ; do
+    set +e
+    nc -z "$WAITFOR_HOST" "$WAITFOR_PORT" > /dev/null 2>&1
+    result=$?
+    set -e
+    if [ $result -eq 0 ] ; then
+      break
+    fi
+    sleep 1
+  done
+  if [ "$i" -eq 0 ]; then
+     echo "Waiting for service $WAITFOR is timed out." >&2
+     exit 1
+  f
+  fi
+fi
+
+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
+
+  if [ -z "$KEYTAB_DIR" ]; then
+    KEYTAB_DIR='/etc/security/keytabs'
+  fi
+  while true
+    do
+      set +e
+      STATUS=$(curl -s -o /dev/null -w '%{http_code}' http://"$ISSUER_SERVER"/keytab/test/test)
+      set -e
+      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 $KEYTAB_DIR/$NAME.keytab"
+      wget "http://$ISSUER_SERVER/keytab/$HOST_NAME/$NAME" -O "$KEYTAB_DIR/$NAME.keytab"
+      klist -kt "$KEYTAB_DIR/$NAME.keytab"
+      KERBEROS_ENABLED=true
+    done
+
+    #Optional: let's try to adjust the krb5.conf
+    sudo sed -i "s/krb5/$KERBEROS_SERVER/g" "/etc/krb5.conf" || true
+fi
+
+CONF_DESTINATION_DIR="${HADOOP_CONF_DIR:-/opt/hadoop/etc/hadoop}"
+
+#Try to copy the defaults
+set +e
+if [[ -d "/opt/ozone/etc/hadoop" ]]; then
+   cp /opt/hadoop/etc/hadoop/* "$CONF_DESTINATION_DIR/" > /dev/null 2>&1
+elif [[ -d "/opt/hadoop/etc/hadoop" ]]; then
+   cp /opt/hadoop/etc/hadoop/* "$CONF_DESTINATION_DIR/" > /dev/null 2>&1
+fi
+set -e
+
+"$DIR"/envtoconf.py --destination "$CONF_DESTINATION_DIR"
+
+if [ -n "$ENSURE_SCM_INITIALIZED" ]; then
+  if [ ! -f "$ENSURE_SCM_INITIALIZED" ]; then
+    # Improve om and scm start up options
+    /opt/hadoop/bin/ozone scm --init || /opt/hadoop/bin/ozone scm -init
+  fi
+fi
+
+if [ -n "$ENSURE_OM_INITIALIZED" ]; then
+  if [ ! -f "$ENSURE_OM_INITIALIZED" ]; then
+    # Improve om and scm start up options
+    /opt/hadoop/bin/ozone om --init ||  /opt/hadoop/bin/ozone om -createObjectStore
+  fi
+fi
+
+# Supports byteman script to instrument hadoop process with byteman script
+#
+#
+if [ -n "$BYTEMAN_SCRIPT" ] || [ -n "$BYTEMAN_SCRIPT_URL" ]; then
+
+  export PATH=$PATH:$BYTEMAN_DIR/bin
+
+  if [ -n "$BYTEMAN_SCRIPT_URL" ]; then
+    wget "$BYTEMAN_SCRIPT_URL" -O /tmp/byteman.btm
+    export BYTEMAN_SCRIPT=/tmp/byteman.btm
+  fi
+
+  if [ ! -f "$BYTEMAN_SCRIPT" ]; then
+    echo "ERROR: The defined $BYTEMAN_SCRIPT does not exist!!!"
+    exit 255
+  fi
+
+  AGENT_STRING="-javaagent:/opt/byteman.jar=script:$BYTEMAN_SCRIPT"
+  export HADOOP_OPTS="$AGENT_STRING $HADOOP_OPTS"
+  echo "Process is instrumented with adding $AGENT_STRING to HADOOP_OPTS"
+fi
+
+exec "$@"

+ 117 - 0
hadoop-ozone/dist/src/main/dockerbin/envtoconf.py

@@ -0,0 +1,117 @@
+#!/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.excluded_envs = ['HADOOP_CONF_DIR']
+    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():
+      if key in self.excluded_envs:
+          continue
+      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()

+ 150 - 0
hadoop-ozone/dist/src/main/dockerbin/transformation.py

@@ -0,0 +1,150 @@
+#!/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