protected void run()

in sharelib/hive2/src/main/java/org/apache/oozie/action/hadoop/Hive2Main.java [109:259]


    protected void run(String[] args) throws Exception {
        System.out.println();
        System.out.println("Oozie Hive 2 action configuration");
        System.out.println("=================================================================");
        System.out.println();

        Configuration actionConf = initActionConf();

        //Logfile to capture job IDs
        String hadoopJobId = System.getProperty("oozie.launcher.job.id");
        if (hadoopJobId == null) {
            throw new RuntimeException("Launcher Hadoop Job ID system property not set");
        }
        String logFile = new File("hive2-oozie-" + hadoopJobId + ".log").getAbsolutePath();

        List<String> arguments = new ArrayList<>();
        String jdbcUrl = actionConf.get(Hive2ActionExecutor.HIVE2_JDBC_URL);
        if (jdbcUrl == null) {
            throw new RuntimeException("Action Configuration does not have [" +  Hive2ActionExecutor.HIVE2_JDBC_URL
                    + "] property");
        }
        arguments.add("-u");
        arguments.add(jdbcUrl);

        // Use the user who is running the map task
        String username = actionConf.get("user.name");
        arguments.add("-n");
        arguments.add(username);

        String password = actionConf.get(Hive2ActionExecutor.HIVE2_PASSWORD);
        if (password == null) {
            // Have to pass something or Beeline might interactively prompt, which we don't want
            password = "DUMMY";
        }
        arguments.add("-p");
        arguments.add(password);

        // We always use the same driver
        arguments.add("-d");
        arguments.add("org.apache.hive.jdbc.HiveDriver");

        String scriptPath = actionConf.get(Hive2ActionExecutor.HIVE2_SCRIPT);
        String query = actionConf.get(Hive2ActionExecutor.HIVE2_QUERY);
        if (scriptPath != null) {
            if (!new File(scriptPath).exists()) {
                throw new RuntimeException("Hive 2 script file [" + scriptPath + "] does not exist");
            }
            // print out current directory & its contents
            File localDir = new File("dummy").getAbsoluteFile().getParentFile();
            System.out.println("Current (local) dir = " + localDir.getAbsolutePath());
            System.out.println("------------------------");
            String[] files = localDir.list();
            if (files != null) {
                for (String file : files) {
                    System.out.println("  " + file);
                }
            }
            System.out.println("------------------------");
            System.out.println();
            // Prepare the Hive Script
            String script = readStringFromFile(scriptPath);
            System.out.println();
            System.out.println("Script [" + scriptPath + "] content: ");
            System.out.println("------------------------");
            System.out.println(script);
            System.out.println("------------------------");
            System.out.println();
            arguments.add("-f");
            arguments.add(scriptPath);
        } else if (query != null) {
            System.out.println("Query: ");
            System.out.println("------------------------");
            System.out.println(query);
            System.out.println("------------------------");
            System.out.println();
            String filename = createScriptFile(query);
            arguments.add("-f");
            arguments.add(filename);
        } else {
            throw new RuntimeException("Action Configuration does not have ["
                +  Hive2ActionExecutor.HIVE2_SCRIPT + "], or ["
                +  Hive2ActionExecutor.HIVE2_QUERY + "] property");
        }

        // Pass any parameters to Beeline via arguments
        String[] params = ActionUtils.getStrings(actionConf, Hive2ActionExecutor.HIVE2_PARAMS);
        if (params.length > 0) {
            System.out.println("Parameters:");
            System.out.println("------------------------");
            for (String param : params) {
                System.out.println("  " + param);

                int idx = param.indexOf('=');
                if (idx == -1) {
                    throw new RuntimeException("Parameter expression must contain an assignment: " + param);
                } else if (idx == 0) {
                    throw new RuntimeException("Parameter value not specified: " + param);
                }
                arguments.add("--hivevar");
                arguments.add(param);
            }
            System.out.println("------------------------");
            System.out.println();
        }

        // This tells BeeLine to look for a delegation token; otherwise it won't and will fail in secure mode because there are no
        // Kerberos credentials.  In non-secure mode, this argument is ignored so we can simply always pass it.
        arguments.add("-a");
        arguments.add("delegationToken");

        String[] beelineArgs = ActionUtils.getStrings(actionConf, Hive2ActionExecutor.HIVE2_ARGS);
        for (String beelineArg : beelineArgs) {
            if (DISALLOWED_BEELINE_OPTIONS.contains(beelineArg)) {
                throw new RuntimeException("Error: Beeline argument " + beelineArg + " is not supported");
            }
            arguments.add(beelineArg);
        }

        // Propagate MR job tag if defined
        if (actionConf.get(LauncherMain.MAPREDUCE_JOB_TAGS) != null ) {
            arguments.add("--hiveconf");
            arguments.add("mapreduce.job.tags=" + actionConf.get(LauncherMain.MAPREDUCE_JOB_TAGS));
        }
        // Propagate "oozie.*" configs (but not "oozie.launcher.*" nor "oozie.hive2.*")
        for (Map.Entry<String, String> oozieConfig : actionConf.getValByRegex("^oozie\\.(?!launcher|hive2).+").entrySet()) {
            arguments.add("--hiveconf");
            arguments.add(oozieConfig.getKey() + "=" + oozieConfig.getValue());
        }

        System.out.println("Beeline command arguments :");
        for (String arg : arguments) {
            System.out.println("             " + arg);
        }
        System.out.println();

        LauncherMain.killChildYarnJobs(actionConf);

        System.out.println("=================================================================");
        System.out.println();
        System.out.println(">>> Invoking Beeline command line now >>>");
        System.out.println();
        System.out.flush();

        try {
            runBeeline(arguments.toArray(new String[arguments.size()]), logFile);
        }
        finally {
            System.out.println("\n<<< Invocation of Beeline command completed <<<\n");
            writeExternalChildIDs(logFile, HIVE2_JOB_IDS_PATTERNS, "Beeline");
        }
    }