int launch_docker_container_as_user()

in hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c [2063:2343]


int launch_docker_container_as_user(const char * user, const char *app_id,
                              const char *container_id, const char *work_dir,
                              const char *script_name, const char *cred_file,
                              const int https,
                              const char *keystore_file, const char *truststore_file,
                              const char *pid_file, char* const* local_dirs,
                              char* const* log_dirs, const char *command_file) {
  int exit_code = -1;
  char *script_file_dest = NULL;
  char *cred_file_dest = NULL;
  char *keystore_file_dest = NULL;
  char *truststore_file_dest = NULL;
  char *exit_code_file = NULL;
  char *docker_command_with_binary = NULL;
  char *docker_inspect_command = NULL;
  char *docker_inspect_exitcode_command = NULL;
  int container_file_source =-1;
  int cred_file_source = -1;
  int keystore_file_source = -1;
  int truststore_file_source = -1;
  int use_entry_point = 0;

  gid_t user_gid = getegid();
  uid_t prev_uid = geteuid();

  char **docker_command = NULL;
  char *docker_binary = NULL;

  fprintf(LOGFILE, "Creating script paths...\n");
  exit_code = create_script_paths(
    work_dir, script_name, cred_file, https, keystore_file, truststore_file, &script_file_dest, &cred_file_dest,
    &keystore_file_dest, &truststore_file_dest, &container_file_source, &cred_file_source, &keystore_file_source, &truststore_file_source);
  if (exit_code != 0) {
    fprintf(ERRORFILE, "Could not create script path\n");
    goto cleanup;
  }

  fprintf(LOGFILE, "Creating local dirs...\n");
  exit_code = create_local_dirs(user, app_id, container_id,
    work_dir, script_name, cred_file, https, keystore_file, truststore_file, local_dirs, log_dirs,
    1, script_file_dest, cred_file_dest, keystore_file_dest, truststore_file_dest,
    container_file_source, cred_file_source, keystore_file_source, truststore_file_source);
  if (exit_code != 0) {
    fprintf(ERRORFILE, "Could not create local files and directories %d %d\n", container_file_source, cred_file_source);
    goto cleanup;
  }

  exit_code = create_user_filecache_dirs(user, local_dirs);
  if (exit_code != 0) {
    fprintf(ERRORFILE, "Could not create user filecache directory\n");
    goto cleanup;
  }

  exit_code = create_yarn_sysfs(user, app_id, container_id, work_dir, local_dirs);
  if (exit_code != 0) {
    fprintf(ERRORFILE, "Could not create user yarn sysfs directory\n");
    exit(-1);
  }

  docker_command = construct_docker_command(command_file);
  docker_binary = get_docker_binary(&CFG);

  fprintf(LOGFILE, "Getting exit code file...\n");
  exit_code_file = get_exit_code_file(pid_file);
  if (NULL == exit_code_file) {
    exit_code = OUT_OF_MEMORY;
    fprintf(ERRORFILE, "Container out of memory\n");
    goto cleanup;
  }

  fprintf(LOGFILE, "Changing effective user to root...\n");
  if (change_effective_user(0, user_gid) != 0) {
    fprintf(ERRORFILE, "Could not change to effective users %d, %d\n", 0, user_gid);
    goto cleanup;
  }

  use_entry_point = get_use_entry_point_flag();
  char *so = init_log_path(chosen_container_log_dir, "stdout.txt");
  if (so == NULL) {
    exit_code = UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
    goto cleanup;
  }
  char *se = init_log_path(chosen_container_log_dir, "stderr.txt");
  if (se == NULL) {
    exit_code = UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
    goto cleanup;
  }

  docker_command_with_binary = flatten(docker_command);

  if (docker_command_with_binary == NULL) {
    fprintf (ERRORFILE, "Could not flatten docker command.\n");
    exit_code = UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
    goto cleanup;
  }

  // Launch container
  pid_t child_pid = fork();
  if (child_pid == -1) {
    fprintf (ERRORFILE,
      "Could not invoke docker %s.\n", docker_command_with_binary);
    exit_code = UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
    goto cleanup;
  }

  if (child_pid == 0) {
    FILE* so_fd = fopen(so, "a+");
    if (so_fd == NULL) {
      fprintf(ERRORFILE, "Could not append to %s\n", so);
      exit_code = UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
      _exit(exit_code);
    }
    FILE* se_fd = fopen(se, "a+");
    if (se_fd == NULL) {
      fprintf(ERRORFILE, "Could not append to %s\n", se);
      exit_code = UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
      fclose(so_fd);
      _exit(exit_code);
    }
    // if entry point is enabled, clone docker command output
    // to stdout.txt and stderr.txt for yarn.
    if (use_entry_point) {
      fprintf(so_fd, "Launching docker container...\n");
      fprintf(so_fd, "Docker run command: %s\n", docker_command_with_binary);
      if (dup2(fileno(so_fd), fileno(stdout)) == -1) {
        fprintf(ERRORFILE, "Could not append to stdout.txt\n");
        fclose(so_fd);
        _exit(UNABLE_TO_EXECUTE_CONTAINER_SCRIPT);
      }
      if (dup2(fileno(se_fd), fileno(stderr)) == -1) {
        fprintf(ERRORFILE, "Could not append to stderr.txt\n");
        fclose(se_fd);
        _exit(UNABLE_TO_EXECUTE_CONTAINER_SCRIPT);
      }
    }
    fclose(so_fd);
    fclose(se_fd);
    execvp(docker_binary, docker_command);
    fprintf(ERRORFILE, "failed to execute docker command! error: %s\n", strerror(errno));
    _exit(UNABLE_TO_EXECUTE_CONTAINER_SCRIPT);
  } else {
    if (use_entry_point) {
      int pid = 0;
      int res = 0;
      int count = 0;
      int max_retries = get_max_retries(&CFG);
      docker_inspect_command = make_string(
          "%s inspect --format {{.State.Pid}} %s",
          docker_binary, container_id);
      // check for docker container pid
      while (count < max_retries) {
        fprintf(LOGFILE, "Inspecting docker container...\n");
        fprintf(LOGFILE, "Docker inspect command: %s\n", docker_inspect_command);
        FILE* inspect_docker = popen(docker_inspect_command, "r");
        res = fscanf (inspect_docker, "%d", &pid);
        fprintf(LOGFILE, "pid from docker inspect: %d\n", pid);
        if (pclose (inspect_docker) != 0 || res <= 0) {
          fprintf (ERRORFILE,
              "Could not inspect docker to get pid %s.\n", docker_inspect_command);
          exit_code = UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
        } else {
          if (pid != 0) {
            break;
          }
        }
        sleep(3);
        count++;
      }
    } else {
      exit_code = wait_and_get_exit_code(child_pid);
      if (exit_code != 0) {
        exit_code = UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
        goto cleanup;
      }
    }
  }

  docker_inspect_command = make_string(
    "%s inspect --format {{.State.Pid}} %s",
    docker_binary, container_id);

  fprintf(LOGFILE, "Inspecting docker container...\n");
  fprintf(LOGFILE, "Docker inspect command: %s\n", docker_inspect_command);
  FILE* inspect_docker = popen(docker_inspect_command, "r");
  int pid = 0;
  int res = fscanf (inspect_docker, "%d", &pid);
  fprintf(LOGFILE, "pid from docker inspect: %d\n", pid);
  if (pclose (inspect_docker) != 0 || res <= 0)
  {
    fprintf (ERRORFILE,
     "Could not inspect docker to get pid %s.\n", docker_inspect_command);
    exit_code = UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
    goto cleanup;
  }

  if (pid != 0) {
    // write pid to pidfile
    fprintf(LOGFILE, "Writing pid file...\n");
    if (pid_file == NULL
        || write_pid_to_file_as_nm(pid_file, (pid_t)pid) != 0) {
      exit_code = WRITE_PIDFILE_FAILED;
      fprintf(ERRORFILE, "Could not write pid to %s\n", pid_file);
      goto cleanup;
    }

    fprintf(LOGFILE, "Waiting for docker container to finish.\n");

    // wait for pid to finish
#ifdef __linux
    char* proc_pid_path = make_string("%s/%d", PROC_PATH, pid);
    while (dir_exists(proc_pid_path) == 0) {
      sleep(1);
    }
    if (dir_exists(proc_pid_path) == -1) {
      fprintf(ERRORFILE, "Error occurred checking %s\n", proc_pid_path);
    }
#else
    while (kill(pid,0) == 0) {
      sleep(1);
    }
#endif
  }

  // discover container exit code
  docker_inspect_exitcode_command = make_string(
    "%s inspect --format {{.State.ExitCode}} %s",
  docker_binary, container_id);
  fprintf(LOGFILE, "Obtaining the exit code...\n");
  fprintf(LOGFILE, "Docker inspect command: %s\n", docker_inspect_exitcode_command);
  int count = 0;
  int max_retries = get_max_retries(&CFG);
  while (count < max_retries) {
    FILE* inspect_exitcode_docker = popen(docker_inspect_exitcode_command, "r");
    res = fscanf (inspect_exitcode_docker, "%d", &exit_code);
    if (pclose (inspect_exitcode_docker) != 0 || res <= 0) {
      fprintf (ERRORFILE, "Could not inspect docker to get Exit code %s.\n", docker_inspect_exitcode_command);
      exit_code = -1;
    } else {
      if (exit_code != 0) {
        break;
      }
    }
    sleep(3);
    count++;
  }
  fprintf(LOGFILE, "Exit code from docker inspect: %d\n", exit_code);

cleanup:

  if (exit_code_file != NULL && write_exit_code_file_as_nm(exit_code_file, exit_code) < 0) {
    fprintf (ERRORFILE,
      "Could not write exit code to file %s.\n", exit_code_file);
  }
  fprintf(LOGFILE, "Wrote the exit code %d to %s\n", exit_code, exit_code_file);

  // Drop root privileges
  if (change_effective_user(prev_uid, user_gid) != 0) {
    fprintf(ERRORFILE,
      "Could not change to effective users %d, %d\n", prev_uid, user_gid);
  }

#if HAVE_FCLOSEALL
  fcloseall();
#else
  // only those fds are opened assuming no bug
  fclose(LOGFILE);
  fclose(ERRORFILE);
  fclose(stdin);
  fclose(stdout);
  fclose(stderr);
#endif
  free(exit_code_file);
  free(script_file_dest);
  free(cred_file_dest);
  free(keystore_file_dest);
  free(truststore_file_dest);
  free(docker_command_with_binary);
  free(docker_inspect_command);
  free_values(docker_command);
  return exit_code;
}