001 /** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018 package org.apache.oozie.action.hadoop; 019 020 import java.io.BufferedReader; 021 import java.io.BufferedWriter; 022 import java.io.File; 023 import java.io.FileOutputStream; 024 import java.io.FileReader; 025 import java.io.FileWriter; 026 import java.io.IOException; 027 import java.io.OutputStream; 028 import java.net.URL; 029 import java.util.ArrayList; 030 import java.util.HashMap; 031 import java.util.List; 032 import java.util.Map; 033 import java.util.Map.Entry; 034 import java.util.Properties; 035 036 import org.apache.hadoop.conf.Configuration; 037 import org.apache.hadoop.fs.Path; 038 import org.apache.hadoop.hive.cli.CliDriver; 039 040 public class HiveMain extends LauncherMain { 041 public static final String USER_HIVE_DEFAULT_FILE = "oozie-user-hive-default.xml"; 042 043 public static final String HIVE_L4J_PROPS = "hive-log4j.properties"; 044 public static final String HIVE_EXEC_L4J_PROPS = "hive-exec-log4j.properties"; 045 public static final String HIVE_SITE_CONF = "hive-site.xml"; 046 private static final String HIVE_SCRIPT = "oozie.hive.script"; 047 private static final String HIVE_PARAMS = "oozie.hive.params"; 048 049 public static void main(String[] args) throws Exception { 050 run(HiveMain.class, args); 051 } 052 053 private static Configuration initActionConf() { 054 // Loading action conf prepared by Oozie 055 Configuration hiveConf = new Configuration(false); 056 057 String actionXml = System.getProperty("oozie.action.conf.xml"); 058 059 if (actionXml == null) { 060 throw new RuntimeException("Missing Java System Property [oozie.action.conf.xml]"); 061 } 062 if (!new File(actionXml).exists()) { 063 throw new RuntimeException("Action Configuration XML file [" + actionXml + "] does not exist"); 064 } else { 065 System.out.println("Using action configuration file " + actionXml); 066 } 067 068 hiveConf.addResource(new Path("file:///", actionXml)); 069 070 // Propagate delegation related props from launcher job to Hive job 071 String delegationToken = System.getenv("HADOOP_TOKEN_FILE_LOCATION"); 072 if (delegationToken != null) { 073 hiveConf.set("mapreduce.job.credentials.binary", delegationToken); 074 System.out.println("------------------------"); 075 System.out.println("Setting env property for mapreduce.job.credentials.binary to: " + delegationToken); 076 System.out.println("------------------------"); 077 System.setProperty("mapreduce.job.credentials.binary", delegationToken); 078 } else { 079 System.out.println("Non-Kerberos execution"); 080 } 081 082 // Have to explicitly unset this property or Hive will not set it. 083 hiveConf.set("mapred.job.name", ""); 084 085 // See https://issues.apache.org/jira/browse/HIVE-1411 086 hiveConf.set("datanucleus.plugin.pluginRegistryBundleCheck", "LOG"); 087 088 // to force hive to use the jobclient to submit the job, never using HADOOPBIN (to do localmode) 089 hiveConf.setBoolean("hive.exec.mode.local.auto", false); 090 091 return hiveConf; 092 } 093 094 public static String setUpHiveLog4J(Configuration hiveConf) throws IOException { 095 //Logfile to capture job IDs 096 String hadoopJobId = System.getProperty("oozie.launcher.job.id"); 097 if (hadoopJobId == null) { 098 throw new RuntimeException("Launcher Hadoop Job ID system property not set"); 099 } 100 101 String logFile = new File("hive-oozie-" + hadoopJobId + ".log").getAbsolutePath(); 102 103 Properties hadoopProps = new Properties(); 104 105 // Preparing log4j configuration 106 URL log4jFile = Thread.currentThread().getContextClassLoader().getResource("log4j.properties"); 107 if (log4jFile != null) { 108 // getting hadoop log4j configuration 109 hadoopProps.load(log4jFile.openStream()); 110 } 111 112 String logLevel = hiveConf.get("oozie.hive.log.level", "INFO"); 113 114 hadoopProps.setProperty("log4j.logger.org.apache.hadoop.hive", logLevel + ", A"); 115 hadoopProps.setProperty("log4j.logger.hive", logLevel + ", A"); 116 hadoopProps.setProperty("log4j.logger.DataNucleus", logLevel + ", A"); 117 hadoopProps.setProperty("log4j.logger.DataStore", logLevel + ", A"); 118 hadoopProps.setProperty("log4j.logger.JPOX", logLevel + ", A"); 119 hadoopProps.setProperty("log4j.appender.A", "org.apache.log4j.ConsoleAppender"); 120 hadoopProps.setProperty("log4j.appender.A.layout", "org.apache.log4j.PatternLayout"); 121 hadoopProps.setProperty("log4j.appender.A.layout.ConversionPattern", "%-4r [%t] %-5p %c %x - %m%n"); 122 123 hadoopProps.setProperty("log4j.appender.jobid", "org.apache.log4j.FileAppender"); 124 hadoopProps.setProperty("log4j.appender.jobid.file", logFile); 125 hadoopProps.setProperty("log4j.appender.jobid.layout", "org.apache.log4j.PatternLayout"); 126 hadoopProps.setProperty("log4j.appender.jobid.layout.ConversionPattern", "%-4r [%t] %-5p %c %x - %m%n"); 127 hadoopProps.setProperty("log4j.logger.org.apache.hadoop.hive.ql.exec", "INFO, jobid"); 128 129 String localProps = new File(HIVE_L4J_PROPS).getAbsolutePath(); 130 OutputStream os1 = new FileOutputStream(localProps); 131 hadoopProps.store(os1, ""); 132 os1.close(); 133 134 localProps = new File(HIVE_EXEC_L4J_PROPS).getAbsolutePath(); 135 os1 = new FileOutputStream(localProps); 136 hadoopProps.store(os1, ""); 137 os1.close(); 138 return logFile; 139 } 140 141 public static Configuration setUpHiveSite() throws Exception { 142 Configuration hiveConf = initActionConf(); 143 144 // Write the action configuration out to hive-site.xml 145 OutputStream os = new FileOutputStream(HIVE_SITE_CONF); 146 hiveConf.writeXml(os); 147 os.close(); 148 149 System.out.println(); 150 System.out.println("Hive Configuration Properties:"); 151 System.out.println("------------------------"); 152 for (Entry<String, String> entry : hiveConf) { 153 System.out.println(entry.getKey() + "=" + entry.getValue()); 154 } 155 System.out.flush(); 156 System.out.println("------------------------"); 157 System.out.println(); 158 return hiveConf; 159 } 160 161 protected void run(String[] args) throws Exception { 162 if (System.getenv("HADOOP_HOME") == null) { 163 throw new RuntimeException("'HADOOP_HOME' environment variable undefined, Hive cannot run"); 164 } 165 System.out.println(); 166 System.out.println("Oozie Hive action configuration"); 167 System.out.println("================================================================="); 168 169 Configuration hiveConf = setUpHiveSite(); 170 171 List<String> arguments = new ArrayList<String>(); 172 String scriptPath = hiveConf.get(HIVE_SCRIPT); 173 174 if (scriptPath == null) { 175 throw new RuntimeException("Action Configuration does not have [oozie.hive.script] property"); 176 } 177 178 if (!new File(scriptPath).exists()) { 179 throw new RuntimeException("Hive script file [" + scriptPath + "] does not exist"); 180 } 181 182 // check if hive-default.xml is in the classpath, if not look for oozie-hive-default.xml 183 // in the current directory (it will be there if the Hive action has the 'oozie.hive.defaults' 184 // property) and rename it to hive-default.xml 185 if (Thread.currentThread().getContextClassLoader().getResource("hive-default.xml") == null) { 186 File userProvidedDefault = new File(USER_HIVE_DEFAULT_FILE); 187 if (userProvidedDefault.exists()) { 188 if (!userProvidedDefault.renameTo(new File("hive-default.xml"))) { 189 throw new RuntimeException( 190 "Could not rename user provided Hive defaults file to 'hive-default.xml'"); 191 } 192 System.out.println("Using 'hive-default.xml' defined in the Hive action"); 193 } 194 else { 195 throw new RuntimeException( 196 "Hive JAR does not bundle a 'hive-default.xml' and Hive action does not define one"); 197 } 198 } 199 else { 200 System.out.println("Using 'hive-default.xml' defined in the Hive JAR"); 201 File userProvidedDefault = new File(USER_HIVE_DEFAULT_FILE); 202 if (userProvidedDefault.exists()) { 203 System.out.println("WARNING: Ignoring user provided Hive defaults"); 204 } 205 } 206 System.out.println(); 207 208 String logFile = setUpHiveLog4J(hiveConf); 209 210 // print out current directory & its contents 211 File localDir = new File("dummy").getAbsoluteFile().getParentFile(); 212 System.out.println("Current (local) dir = " + localDir.getAbsolutePath()); 213 System.out.println("------------------------"); 214 for (String file : localDir.list()) { 215 System.out.println(" " + file); 216 } 217 System.out.println("------------------------"); 218 System.out.println(); 219 220 // Prepare the Hive Script 221 String script = readStringFromFile(scriptPath); 222 System.out.println(); 223 System.out.println("Original script [" + scriptPath + "] content: "); 224 System.out.println("------------------------"); 225 System.out.println(script); 226 System.out.println("------------------------"); 227 System.out.println(); 228 229 String[] params = MapReduceMain.getStrings(hiveConf, HIVE_PARAMS); 230 if (params.length > 0) { 231 Map<String, String> varMap = new HashMap<String, String>(); 232 System.out.println("Parameters:"); 233 System.out.println("------------------------"); 234 for (String param : params) { 235 System.out.println(" " + param); 236 237 int idx = param.indexOf('='); 238 if (idx == -1) { 239 throw new RuntimeException("Parameter expression must contain an assignment: " + param); 240 } else if (idx == 0) { 241 throw new RuntimeException("Parameter value not specified: " + param); 242 } 243 String var = param.substring(0, idx); 244 String val = param.substring(idx + 1, param.length()); 245 varMap.put(var, val); 246 } 247 System.out.println("------------------------"); 248 System.out.println(); 249 250 String resolvedScript = substitute(varMap, script); 251 scriptPath = scriptPath + ".sub"; 252 writeStringToFile(scriptPath, resolvedScript); 253 254 System.out.println("Resolved script [" + scriptPath + "] content: "); 255 System.out.println("------------------------"); 256 System.out.println(resolvedScript); 257 System.out.println("------------------------"); 258 System.out.println(); 259 } 260 261 arguments.add("-f"); 262 arguments.add(scriptPath); 263 264 265 System.out.println("Hive command arguments :"); 266 for (String arg : arguments) { 267 System.out.println(" " + arg); 268 } 269 System.out.println(); 270 271 System.out.println("================================================================="); 272 System.out.println(); 273 System.out.println(">>> Invoking Hive command line now >>>"); 274 System.out.println(); 275 System.out.flush(); 276 277 try { 278 runHive(arguments.toArray(new String[arguments.size()])); 279 } 280 catch (SecurityException ex) { 281 if (LauncherSecurityManager.getExitInvoked()) { 282 if (LauncherSecurityManager.getExitCode() != 0) { 283 throw ex; 284 } 285 } 286 } 287 288 System.out.println("\n<<< Invocation of Hive command completed <<<\n"); 289 290 // harvesting and recording Hadoop Job IDs 291 Properties jobIds = getHadoopJobIds(logFile, JOB_ID_LOG_PREFIX); 292 File file = new File(System.getProperty("oozie.action.output.properties")); 293 OutputStream os = new FileOutputStream(file); 294 jobIds.store(os, ""); 295 os.close(); 296 System.out.println(" Hadoop Job IDs executed by Hive: " + jobIds.getProperty("hadoopJobs")); 297 System.out.println(); 298 } 299 300 private void runHive(String[] args) throws Exception { 301 CliDriver.main(args); 302 } 303 304 public static void setHiveScript(Configuration conf, String script, String[] params) { 305 conf.set(HIVE_SCRIPT, script); 306 MapReduceMain.setStrings(conf, HIVE_PARAMS, params); 307 } 308 309 private static String readStringFromFile(String filePath) throws IOException { 310 String line; 311 BufferedReader br = new BufferedReader(new FileReader(filePath)); 312 StringBuilder sb = new StringBuilder(); 313 String sep = System.getProperty("line.separator"); 314 while ((line = br.readLine()) != null) { 315 sb.append(line).append(sep); 316 } 317 return sb.toString(); 318 } 319 320 private static void writeStringToFile(String filePath, String str) throws IOException { 321 BufferedWriter out = new BufferedWriter(new FileWriter(filePath)); 322 out.write(str); 323 out.close(); 324 } 325 326 static String substitute(Map<String, String> vars, String expr) { 327 for (Map.Entry<String, String> entry : vars.entrySet()) { 328 String var = "${" + entry.getKey() + "}"; 329 String value = entry.getValue(); 330 expr = expr.replace(var, value); 331 } 332 return expr; 333 } 334 335 //TODO: Hive should provide a programmatic way of spitting out Hadoop jobs 336 private static final String JOB_ID_LOG_PREFIX = "Ended Job = "; 337 338 public static Properties getHadoopJobIds(String logFile, String prefix) throws IOException { 339 Properties props = new Properties(); 340 StringBuffer sb = new StringBuffer(100); 341 if (!new File(logFile).exists()) { 342 System.err.println("hive log file: " + logFile + " not present. Therefore no Hadoop jobids found"); 343 props.setProperty("hadoopJobs", ""); 344 } 345 else { 346 BufferedReader br = new BufferedReader(new FileReader(logFile)); 347 String line = br.readLine(); 348 String separator = ""; 349 while (line != null) { 350 if (line.contains(prefix)) { 351 int jobIdStarts = line.indexOf(prefix) + prefix.length(); 352 String jobId = line.substring(jobIdStarts).trim(); 353 354 //Doing this because Hive now does things like ConditionalTask which are not Hadoop jobs. 355 if (jobId.startsWith("job_")) { 356 sb.append(separator).append(jobId); 357 separator = ","; 358 } 359 } 360 line = br.readLine(); 361 } 362 br.close(); 363 props.setProperty("hadoopJobs", sb.toString()); 364 } 365 return props; 366 } 367 368 }