001 /**
002 * Copyright (c) 2010 Yahoo! Inc. All rights reserved.
003 * Licensed under the Apache License, Version 2.0 (the "License");
004 * you may not use this file except in compliance with the License.
005 * You may obtain a copy of the License at
006 *
007 * http://www.apache.org/licenses/LICENSE-2.0
008 *
009 * Unless required by applicable law or agreed to in writing, software
010 * distributed under the License is distributed on an "AS IS" BASIS,
011 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
012 * See the License for the specific language governing permissions and
013 * limitations under the License. See accompanying LICENSE file.
014 */
015 package org.apache.oozie.action.hadoop;
016
017 import org.apache.pig.Main;
018 import org.apache.hadoop.conf.Configuration;
019 import org.apache.hadoop.fs.Path;
020 import org.apache.hadoop.mapred.JobClient;
021
022 import java.io.FileNotFoundException;
023 import java.io.OutputStream;
024 import java.io.FileOutputStream;
025 import java.io.BufferedReader;
026 import java.io.FileReader;
027 import java.io.File;
028 import java.io.IOException;
029 import java.util.HashSet;
030 import java.util.Map;
031 import java.util.List;
032 import java.util.ArrayList;
033 import java.util.Properties;
034 import java.util.Set;
035 import java.net.URL;
036
037 public class PigMain extends LauncherMain {
038 private static final Set<String> DISALLOWED_PIG_OPTIONS = new HashSet<String>();
039
040 static {
041 DISALLOWED_PIG_OPTIONS.add("-4");
042 DISALLOWED_PIG_OPTIONS.add("-log4jconf");
043 DISALLOWED_PIG_OPTIONS.add("-e");
044 DISALLOWED_PIG_OPTIONS.add("-execute");
045 DISALLOWED_PIG_OPTIONS.add("-f");
046 DISALLOWED_PIG_OPTIONS.add("-file");
047 DISALLOWED_PIG_OPTIONS.add("-l");
048 DISALLOWED_PIG_OPTIONS.add("-logfile");
049 DISALLOWED_PIG_OPTIONS.add("-r");
050 DISALLOWED_PIG_OPTIONS.add("-dryrun");
051 DISALLOWED_PIG_OPTIONS.add("-x");
052 DISALLOWED_PIG_OPTIONS.add("-exectype");
053 DISALLOWED_PIG_OPTIONS.add("-P");
054 DISALLOWED_PIG_OPTIONS.add("-propertyFile");
055 }
056
057 public static void main(String[] args) throws Exception {
058 run(PigMain.class, args);
059 }
060
061 protected void run(String[] args) throws Exception {
062 System.out.println();
063 System.out.println("Oozie Pig action configuration");
064 System.out.println("=================================================================");
065
066 // loading action conf prepared by Oozie
067 Configuration actionConf = new Configuration(false);
068
069 String actionXml = System.getProperty("oozie.action.conf.xml");
070
071 if (actionXml == null) {
072 throw new RuntimeException("Missing Java System Property [oozie.action.conf.xml]");
073 }
074 if (!new File(actionXml).exists()) {
075 throw new RuntimeException("Action Configuration XML file [" + actionXml + "] does not exist");
076 }
077
078 actionConf.addResource(new Path("file:///", actionXml));
079
080 Properties pigProperties = new Properties();
081 for (Map.Entry<String, String> entry : actionConf) {
082 pigProperties.setProperty(entry.getKey(), entry.getValue());
083 }
084
085 //propagate delegation related props from launcher job to Pig job
086 if (System.getenv("HADOOP_TOKEN_FILE_LOCATION") != null) {
087 pigProperties.setProperty("mapreduce.job.credentials.binary", System.getenv("HADOOP_TOKEN_FILE_LOCATION"));
088 System.out.println("------------------------");
089 System.out.println("Setting env property for mapreduce.job.credentials.binary to:"
090 + System.getenv("HADOOP_TOKEN_FILE_LOCATION"));
091 System.out.println("------------------------");
092 System.setProperty("mapreduce.job.credentials.binary", System.getenv("HADOOP_TOKEN_FILE_LOCATION"));
093 }
094 else {
095 System.out.println("Non-kerberoes execution");
096 }
097
098 OutputStream os = new FileOutputStream("pig.properties");
099 pigProperties.store(os, "");
100 os.close();
101
102 System.out.println();
103 System.out.println("pig.properties content:");
104 System.out.println("------------------------");
105 pigProperties.store(System.out, "");
106 System.out.flush();
107 System.out.println("------------------------");
108 System.out.println();
109
110 List<String> arguments = new ArrayList<String>();
111 String script = actionConf.get("oozie.pig.script");
112
113 if (script == null) {
114 throw new RuntimeException("Action Configuration does not have [oozie.pig.script] property");
115 }
116
117 if (!new File(script).exists()) {
118 throw new RuntimeException("Error: Pig script file [" + script + "] does not exist");
119 }
120
121 System.out.println("Pig script [" + script + "] content: ");
122 System.out.println("------------------------");
123 BufferedReader br = new BufferedReader(new FileReader(script));
124 String line = br.readLine();
125 while (line != null) {
126 System.out.println(line);
127 line = br.readLine();
128 }
129 br.close();
130 System.out.println("------------------------");
131 System.out.println();
132
133 arguments.add("-file");
134 arguments.add(script);
135 String[] params = MapReduceMain.getStrings(actionConf, "oozie.pig.params");
136 for (String param : params) {
137 arguments.add("-param");
138 arguments.add(param);
139 }
140
141 String hadoopJobId = System.getProperty("oozie.launcher.job.id");
142 if (hadoopJobId == null) {
143 throw new RuntimeException("Launcher Hadoop Job ID system property not set");
144 }
145
146 String logFile = new File("pig-oozie-" + hadoopJobId + ".log").getAbsolutePath();
147
148 URL log4jFile = Thread.currentThread().getContextClassLoader().getResource("log4j.properties");
149 if (log4jFile != null) {
150
151 String pigLogLevel = actionConf.get("oozie.pig.log.level", "INFO");
152
153 // append required PIG properties to the default hadoop log4j file
154 Properties hadoopProps = new Properties();
155 hadoopProps.load(log4jFile.openStream());
156 hadoopProps.setProperty("log4j.logger.org.apache.pig", pigLogLevel + ", A, B");
157 hadoopProps.setProperty("log4j.appender.A", "org.apache.log4j.ConsoleAppender");
158 hadoopProps.setProperty("log4j.appender.A.layout", "org.apache.log4j.PatternLayout");
159 hadoopProps.setProperty("log4j.appender.A.layout.ConversionPattern", "%-4r [%t] %-5p %c %x - %m%n");
160 hadoopProps.setProperty("log4j.appender.B", "org.apache.log4j.FileAppender");
161 hadoopProps.setProperty("log4j.appender.B.file", logFile);
162 hadoopProps.setProperty("log4j.appender.B.layout", "org.apache.log4j.PatternLayout");
163 hadoopProps.setProperty("log4j.appender.B.layout.ConversionPattern", "%-4r [%t] %-5p %c %x - %m%n");
164
165 String localProps = new File("piglog4j.properties").getAbsolutePath();
166 OutputStream os1 = new FileOutputStream(localProps);
167 hadoopProps.store(os1, "");
168 os1.close();
169
170 arguments.add("-log4jconf");
171 arguments.add(localProps);
172
173 // print out current directory
174 File localDir = new File(localProps).getParentFile();
175 System.out.println("Current (local) dir = " + localDir.getAbsolutePath());
176 }
177 else {
178 System.out.println("log4jfile is null");
179 }
180
181 String pigLog = "pig-" + hadoopJobId + ".log";
182 arguments.add("-logfile");
183 arguments.add(pigLog);
184
185 String[] pigArgs = MapReduceMain.getStrings(actionConf, "oozie.pig.args");
186 for (String pigArg : pigArgs) {
187 if (DISALLOWED_PIG_OPTIONS.contains(pigArg)) {
188 throw new RuntimeException("Error: Pig argument " + pigArg + " is not supported");
189 }
190 arguments.add(pigArg);
191 }
192
193 System.out.println("Pig command arguments :");
194 for (String arg : arguments) {
195 System.out.println(" " + arg);
196 }
197
198 System.out.println("=================================================================");
199 System.out.println();
200 System.out.println(">>> Invoking Pig command line now >>>");
201 System.out.println();
202 System.out.flush();
203
204 try {
205 System.out.println();
206 runPigJob(new String[] { "-version" });
207 }
208 catch (SecurityException ex) {
209 LauncherSecurityManager.reset();
210 }
211 System.out.println();
212 System.out.flush();
213 try {
214 runPigJob(arguments.toArray(new String[arguments.size()]));
215 }
216 catch (SecurityException ex) {
217 if (LauncherSecurityManager.getExitInvoked()) {
218 if (LauncherSecurityManager.getExitCode() != 0) {
219 System.err.println();
220 System.err.println("Pig logfile dump:");
221 System.err.println();
222 try {
223 BufferedReader reader = new BufferedReader(new FileReader(pigLog));
224 line = reader.readLine();
225 while (line != null) {
226 System.err.println(line);
227 line = reader.readLine();
228 }
229 reader.close();
230 }
231 catch (FileNotFoundException e) {
232 System.err.println("pig log file: " + pigLog + " not found.");
233 }
234 throw ex;
235 }
236 }
237 }
238
239 System.out.println();
240 System.out.println("<<< Invocation of Pig command completed <<<");
241 System.out.println();
242
243 // harvesting and recording Hadoop Job IDs
244 Properties jobIds = getHadoopJobIds(logFile);
245 File file = new File(System.getProperty("oozie.action.output.properties"));
246 os = new FileOutputStream(file);
247 jobIds.store(os, "");
248 os.close();
249 System.out.println(" Hadoop Job IDs executed by Pig: " + jobIds.getProperty("hadoopJobs"));
250 System.out.println();
251 }
252
253 protected void runPigJob(String[] args) throws Exception {
254 // running as from the command line
255 Main.main(args);
256 }
257
258 public static void setPigScript(Configuration conf, String script, String[] params, String[] args) {
259 conf.set("oozie.pig.script", script);
260 MapReduceMain.setStrings(conf, "oozie.pig.params", params);
261 MapReduceMain.setStrings(conf, "oozie.pig.args", args);
262 }
263
264 private static final String JOB_ID_LOG_PREFIX = "HadoopJobId: ";
265
266 protected Properties getHadoopJobIds(String logFile) throws IOException {
267 int jobCount = 0;
268 Properties props = new Properties();
269 StringBuffer sb = new StringBuffer(100);
270 if (new File(logFile).exists() == false) {
271 System.err.println("pig log file: " + logFile + " not present. Therefore no Hadoop jobids found");
272 props.setProperty("hadoopJobs", "");
273 }
274 else {
275 BufferedReader br = new BufferedReader(new FileReader(logFile));
276 String line = br.readLine();
277 String separator = "";
278 while (line != null) {
279 if (line.contains(JOB_ID_LOG_PREFIX)) {
280 int jobIdStarts = line.indexOf(JOB_ID_LOG_PREFIX) + JOB_ID_LOG_PREFIX.length();
281 String jobId = line.substring(jobIdStarts);
282 int jobIdEnds = jobId.indexOf(" ");
283 if (jobIdEnds > -1) {
284 jobId = jobId.substring(0, jobId.indexOf(" "));
285 }
286 sb.append(separator).append(jobId);
287 separator = ",";
288 }
289 line = br.readLine();
290 }
291 br.close();
292 props.setProperty("hadoopJobs", sb.toString());
293 }
294 return props;
295 }
296
297 }