This repository has been archived by the owner on Jul 15, 2019. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 7
/
ExecDriver.java
715 lines (623 loc) · 24.5 KB
/
ExecDriver.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
/**
* 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.
*/
package org.apache.hadoop.hive.ql.exec;
import java.io.*;
import java.util.*;
import java.net.URI;
import java.net.URLEncoder;
import java.net.URLDecoder;
import java.net.URL;
import java.net.URLClassLoader;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.io.*;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.ql.plan.mapredWork;
import org.apache.hadoop.hive.ql.plan.exprNodeDesc;
import org.apache.hadoop.hive.ql.plan.partitionDesc;
import org.apache.hadoop.hive.ql.plan.tableDesc;
import org.apache.hadoop.hive.ql.metadata.HiveException;
import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
import org.apache.hadoop.hive.ql.history.HiveHistory.Keys;
import org.apache.hadoop.hive.ql.io.*;
import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
import org.apache.hadoop.hive.ql.session.SessionState;
import org.apache.log4j.BasicConfigurator;
import org.apache.log4j.varia.NullAppender;
import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
public class ExecDriver extends Task<mapredWork> implements Serializable {
private static final long serialVersionUID = 1L;
transient protected JobConf job;
/**
* Constructor when invoked from QL
*/
public ExecDriver() {
super();
}
public static String getRealFiles(Configuration conf) {
// fill in local files to be added to the task environment
SessionState ss = SessionState.get();
Set<String> files = (ss == null) ? null : ss.list_resource(
SessionState.ResourceType.FILE, null);
if (files != null) {
ArrayList<String> realFiles = new ArrayList<String>(files.size());
for (String one : files) {
try {
realFiles.add(Utilities.realFile(one, conf));
} catch (IOException e) {
throw new RuntimeException("Cannot validate file " + one
+ "due to exception: " + e.getMessage(), e);
}
}
return StringUtils.join(realFiles, ",");
} else {
return "";
}
}
/**
* Initialization when invoked from QL
*/
public void initialize(HiveConf conf) {
super.initialize(conf);
job = new JobConf(conf, ExecDriver.class);
String realFiles = getRealFiles(job);
if (realFiles != null && realFiles.length() > 0) {
job.set("tmpfiles", realFiles);
// workaround for hadoop-17 - jobclient only looks at commandlineconfig
Configuration commandConf = JobClient.getCommandLineConfig();
if (commandConf != null) {
commandConf.set("tmpfiles", realFiles);
}
}
}
/**
* Constructor/Initialization for invocation as independent utility
*/
public ExecDriver(mapredWork plan, JobConf job, boolean isSilent)
throws HiveException {
setWork(plan);
this.job = job;
LOG = LogFactory.getLog(this.getClass().getName());
console = new LogHelper(LOG, isSilent);
}
/**
* A list of the currently running jobs spawned in this Hive instance that is
* used to kill all running jobs in the event of an unexpected shutdown -
* i.e., the JVM shuts down while there are still jobs running.
*/
public static HashMap<String, String> runningJobKillURIs = new HashMap<String, String>();
/**
* In Hive, when the user control-c's the command line, any running jobs
* spawned from that command line are best-effort killed.
*
* This static constructor registers a shutdown thread to iterate over all the
* running job kill URLs and do a get on them.
*
*/
static {
if (new org.apache.hadoop.conf.Configuration().getBoolean(
"webinterface.private.actions", false)) {
Runtime.getRuntime().addShutdownHook(new Thread() {
public void run() {
for (Iterator<String> elems = runningJobKillURIs.values().iterator(); elems
.hasNext();) {
String uri = elems.next();
try {
System.err.println("killing job with: " + uri);
int retCode = ((java.net.HttpURLConnection) new java.net.URL(uri)
.openConnection()).getResponseCode();
if (retCode != 200) {
System.err.println("Got an error trying to kill job with URI: "
+ uri + " = " + retCode);
}
} catch (Exception e) {
System.err.println("trying to kill job, caught: " + e);
// do nothing
}
}
}
});
}
}
/**
* from StreamJob.java
*/
public void jobInfo(RunningJob rj) {
if (job.get("mapred.job.tracker", "local").equals("local")) {
console.printInfo("Job running in-process (local Hadoop)");
} else {
String hp = job.get("mapred.job.tracker");
if (SessionState.get() != null) {
SessionState.get().getHiveHistory().setTaskProperty(
SessionState.get().getQueryId(), getId(),
Keys.TASK_HADOOP_ID, rj.getJobID());
}
console.printInfo("Starting Job = " + rj.getJobID() + ", Tracking URL = "
+ rj.getTrackingURL());
console.printInfo("Kill Command = "
+ HiveConf.getVar(job, HiveConf.ConfVars.HADOOPBIN)
+ " job -Dmapred.job.tracker=" + hp + " -kill " + rj.getJobID());
}
}
/**
* from StreamJob.java
*/
public RunningJob jobProgress(JobClient jc, RunningJob rj) throws IOException {
String lastReport = "";
while (!rj.isComplete()) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
}
rj = jc.getJob(rj.getJobID());
String report = null;
report = " map = " + Math.round(rj.mapProgress() * 100) + "%, reduce ="
+ Math.round(rj.reduceProgress() * 100) + "%";
if (!report.equals(lastReport)) {
SessionState ss = SessionState.get();
if (ss != null) {
ss.getHiveHistory().setTaskCounters(
SessionState.get().getQueryId(), getId(), rj);
ss.getHiveHistory().setTaskProperty(
SessionState.get().getQueryId(), getId(),
Keys.TASK_HADOOP_PROGRESS, report);
ss.getHiveHistory().progressTask(
SessionState.get().getQueryId(), this);
}
console.printInfo(report);
lastReport = report;
}
}
return rj;
}
/**
* Estimate the number of reducers needed for this job, based on job input,
* and configuration parameters.
* @return the number of reducers.
*/
public int estimateNumberOfReducers(HiveConf hive, JobConf job, mapredWork work) throws IOException {
if (hive == null) {
hive = new HiveConf();
}
long bytesPerReducer = hive.getLongVar(HiveConf.ConfVars.BYTESPERREDUCER);
int maxReducers = hive.getIntVar(HiveConf.ConfVars.MAXREDUCERS);
long totalInputFileSize = getTotalInputFileSize(job, work);
LOG.info("BytesPerReducer=" + bytesPerReducer + " maxReducers=" + maxReducers
+ " totalInputFileSize=" + totalInputFileSize);
int reducers = (int)((totalInputFileSize + bytesPerReducer - 1) / bytesPerReducer);
reducers = Math.max(1, reducers);
reducers = Math.min(maxReducers, reducers);
return reducers;
}
/**
* Set the number of reducers for the mapred work.
*/
protected void setNumberOfReducers() throws IOException {
// this is a temporary hack to fix things that are not fixed in the compiler
Integer numReducersFromWork = work.getNumReduceTasks();
if(work.getReducer() == null) {
console.printInfo("Number of reduce tasks is set to 0 since there's no reduce operator");
work.setNumReduceTasks(Integer.valueOf(0));
} else {
if (numReducersFromWork >= 0) {
console.printInfo("Number of reduce tasks determined at compile time: " + work.getNumReduceTasks());
} else if (job.getNumReduceTasks() > 0) {
int reducers = job.getNumReduceTasks();
work.setNumReduceTasks(reducers);
console.printInfo("Number of reduce tasks not specified. Defaulting to jobconf value of: " + reducers);
} else {
int reducers = estimateNumberOfReducers(conf, job, work);
work.setNumReduceTasks(reducers);
console.printInfo("Number of reduce tasks not specified. Estimated from input data size: " + reducers);
}
console.printInfo("In order to change the average load for a reducer (in bytes):");
console.printInfo(" set " + HiveConf.ConfVars.BYTESPERREDUCER.varname + "=<number>");
console.printInfo("In order to limit the maximum number of reducers:");
console.printInfo(" set " + HiveConf.ConfVars.MAXREDUCERS.varname + "=<number>");
console.printInfo("In order to set a constant number of reducers:");
console.printInfo(" set " + HiveConf.ConfVars.HADOOPNUMREDUCERS + "=<number>");
}
}
/**
* Add new elements to the classpath
*
* @param newPaths
* Array of classpath elements
*/
private static void addToClassPath(String[] newPaths, boolean local) throws Exception {
Thread curThread = Thread.currentThread();
URLClassLoader loader = (URLClassLoader) curThread.getContextClassLoader();
List<URL> curPath = Arrays.asList(loader.getURLs());
ArrayList<URL> newPath = new ArrayList<URL>();
for (String onestr : newPaths) {
// special processing for hadoop-17. file:// needs to be removed
if (local) {
if (StringUtils.indexOf(onestr, "file://") == 0)
onestr = StringUtils.substring(onestr, 7);
}
URL oneurl = (new File(onestr)).toURL();
if (!curPath.contains(oneurl)) {
newPath.add(oneurl);
}
}
loader = new URLClassLoader(newPath.toArray(new URL[0]), loader);
curThread.setContextClassLoader(loader);
}
/**
* Calculate the total size of input files.
* @param job the hadoop job conf.
* @return the total size in bytes.
* @throws IOException
*/
public long getTotalInputFileSize(JobConf job, mapredWork work) throws IOException {
long r = 0;
// For each input path, calculate the total size.
for (String path: work.getPathToAliases().keySet()) {
try {
Path p = new Path(path);
FileSystem fs = p.getFileSystem(job);
ContentSummary cs = fs.getContentSummary(p);
r += cs.getLength();
} catch (IOException e) {
LOG.info("Cannot get size of " + path + ". Safely ignored.");
}
}
return r;
}
/**
* Execute a query plan using Hadoop
*/
public int execute() {
try {
setNumberOfReducers();
} catch(IOException e) {
String statusMesg = "IOException while accessing HDFS to estimate the number of reducers: "
+ e.getMessage();
console.printError(statusMesg, "\n"
+ org.apache.hadoop.util.StringUtils.stringifyException(e));
return 1;
}
String invalidReason = work.isInvalid();
if (invalidReason != null) {
throw new RuntimeException("Plan invalid, Reason: " + invalidReason);
}
String hiveScratchDir = HiveConf.getVar(job, HiveConf.ConfVars.SCRATCHDIR);
Path jobScratchDir = new Path(hiveScratchDir + Utilities.randGen.nextInt());
FileOutputFormat.setOutputPath(job, jobScratchDir);
job.setMapperClass(ExecMapper.class);
job.setMapOutputKeyClass(HiveKey.class);
// LazySimpleSerDe writes to Text
// Revert to DynamicSerDe: job.setMapOutputValueClass(BytesWritable.class);
job.setMapOutputValueClass(Text.class);
job.setNumReduceTasks(work.getNumReduceTasks().intValue());
job.setReducerClass(ExecReducer.class);
job.setInputFormat(org.apache.hadoop.hive.ql.io.HiveInputFormat.class);
// No-Op - we don't really write anything here ..
job.setOutputKeyClass(Text.class);
job.setOutputValueClass(Text.class);
String auxJars = HiveConf.getVar(job, HiveConf.ConfVars.HIVEAUXJARS);
if (StringUtils.isNotBlank(auxJars)) {
LOG.info("adding libjars: " + auxJars);
job.set("tmpjars", auxJars);
}
int returnVal = 0;
RunningJob rj = null, orig_rj = null;
boolean success = false;
try {
addInputPaths(job, work, hiveScratchDir);
Utilities.setMapRedWork(job, work);
// remove the pwd from conf file so that job tracker doesn't show this logs
String pwd = job.get(HiveConf.ConfVars.METASTOREPWD.varname);
if (pwd != null)
job.set(HiveConf.ConfVars.METASTOREPWD.varname, "HIVE");
JobClient jc = new JobClient(job);
// make this client wait if job trcker is not behaving well.
Throttle.checkJobTracker(job, LOG);
orig_rj = rj = jc.submitJob(job);
// replace it back
if (pwd != null)
job.set(HiveConf.ConfVars.METASTOREPWD.varname, pwd);
// add to list of running jobs so in case of abnormal shutdown can kill
// it.
runningJobKillURIs.put(rj.getJobID(), rj.getTrackingURL()
+ "&action=kill");
jobInfo(rj);
rj = jobProgress(jc, rj);
if(rj == null) {
// in the corner case where the running job has disappeared from JT memory
// remember that we did actually submit the job.
rj = orig_rj;
success = false;
} else {
success = rj.isSuccessful();
}
String statusMesg = "Ended Job = " + rj.getJobID();
if (!success) {
statusMesg += " with errors";
returnVal = 2;
console.printError(statusMesg);
} else {
console.printInfo(statusMesg);
}
} catch (Exception e) {
String mesg = " with exception '" + Utilities.getNameMessage(e) + "'";
if (rj != null) {
mesg = "Ended Job = " + rj.getJobID() + mesg;
} else {
mesg = "Job Submission failed" + mesg;
}
// Has to use full name to make sure it does not conflict with
// org.apache.commons.lang.StringUtils
console.printError(mesg, "\n"
+ org.apache.hadoop.util.StringUtils.stringifyException(e));
success = false;
returnVal = 1;
} finally {
Utilities.clearMapRedWork(job);
try {
FileSystem fs = jobScratchDir.getFileSystem(job);
fs.delete(jobScratchDir, true);
if (returnVal != 0 && rj != null) {
rj.killJob();
}
runningJobKillURIs.remove(rj.getJobID());
} catch (Exception e) {
}
}
try {
if (rj != null) {
if(work.getAliasToWork() != null) {
for(Operator<? extends Serializable> op:
work.getAliasToWork().values()) {
op.jobClose(job, success);
}
}
if(work.getReducer() != null) {
work.getReducer().jobClose(job, success);
}
}
} catch (Exception e) {
// jobClose needs to execute successfully otherwise fail task
if(success) {
success = false;
returnVal = 3;
String mesg = "Job Commit failed with exception '" + Utilities.getNameMessage(e) + "'";
console.printError(mesg, "\n"
+ org.apache.hadoop.util.StringUtils.stringifyException(e));
}
}
return (returnVal);
}
private static void printUsage() {
System.out
.println("ExecDriver -plan <plan-file> [-jobconf k1=v1 [-jobconf k2=v2] ...] "
+ "[-files <file1>[,<file2>] ...]");
System.exit(1);
}
public static void main(String[] args) throws IOException, HiveException {
String planFileName = null;
ArrayList<String> jobConfArgs = new ArrayList<String>();
boolean isSilent = false;
String files = null;
try {
for (int i = 0; i < args.length; i++) {
if (args[i].equals("-plan")) {
planFileName = args[++i];
System.out.println("plan = " + planFileName);
} else if (args[i].equals("-jobconf")) {
jobConfArgs.add(args[++i]);
} else if (args[i].equals("-silent")) {
isSilent = true;
} else if (args[i].equals("-files")) {
files = args[++i];
}
}
} catch (IndexOutOfBoundsException e) {
System.err.println("Missing argument to option");
printUsage();
}
// If started from main(), and isSilent is on, we should not output
// any logs.
if (isSilent) {
BasicConfigurator.resetConfiguration();
BasicConfigurator.configure(new NullAppender());
}
if (planFileName == null) {
System.err.println("Must specify Plan File Name");
printUsage();
}
JobConf conf = new JobConf(ExecDriver.class);
for (String one : jobConfArgs) {
int eqIndex = one.indexOf('=');
if (eqIndex != -1) {
try {
conf.set(one.substring(0, eqIndex), URLDecoder.decode(one
.substring(eqIndex + 1), "UTF-8"));
} catch (UnsupportedEncodingException e) {
System.err.println("Unexpected error " + e.getMessage()
+ " while encoding " + one.substring(eqIndex + 1));
System.exit(3);
}
}
}
if (files != null) {
conf.set("tmpfiles", files);
}
URI pathURI = (new Path(planFileName)).toUri();
InputStream pathData;
if (StringUtils.isEmpty(pathURI.getScheme())) {
// default to local file system
pathData = new FileInputStream(planFileName);
} else {
// otherwise may be in hadoop ..
FileSystem fs = FileSystem.get(conf);
pathData = fs.open(new Path(planFileName));
}
// workaround for hadoop-17 - libjars are not added to classpath. this
// affects local
// mode execution
boolean localMode = HiveConf.getVar(conf, HiveConf.ConfVars.HADOOPJT)
.equals("local");
if (localMode) {
String auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS);
if (StringUtils.isNotBlank(auxJars)) {
try {
addToClassPath(StringUtils.split(auxJars, ","), true);
} catch (Exception e) {
throw new HiveException(e.getMessage(), e);
}
}
}
mapredWork plan = Utilities.deserializeMapRedWork(pathData);
ExecDriver ed = new ExecDriver(plan, conf, isSilent);
int ret = ed.execute();
if (ret != 0) {
System.out.println("Job Failed");
System.exit(2);
}
}
/**
* Given a Hive Configuration object - generate a command line fragment for
* passing such configuration information to ExecDriver
*/
public static String generateCmdLine(HiveConf hconf) {
try {
StringBuilder sb = new StringBuilder();
Properties deltaP = hconf.getChangedProperties();
boolean localMode = hconf.getVar(HiveConf.ConfVars.HADOOPJT).equals(
"local");
String hadoopSysDir = "mapred.system.dir";
String hadoopWorkDir = "mapred.local.dir";
for (Object one : deltaP.keySet()) {
String oneProp = (String) one;
if (localMode
&& (oneProp.equals(hadoopSysDir) || oneProp.equals(hadoopWorkDir)))
continue;
String oneValue = deltaP.getProperty(oneProp);
sb.append("-jobconf ");
sb.append(oneProp);
sb.append("=");
sb.append(URLEncoder.encode(oneValue, "UTF-8"));
sb.append(" ");
}
// Multiple concurrent local mode job submissions can cause collisions in
// working dirs
// Workaround is to rename map red working dir to a temp dir in such a
// case
if (localMode) {
sb.append("-jobconf ");
sb.append(hadoopSysDir);
sb.append("=");
sb.append(URLEncoder.encode(hconf.get(hadoopSysDir) + "/"
+ Utilities.randGen.nextInt(), "UTF-8"));
sb.append(" ");
sb.append("-jobconf ");
sb.append(hadoopWorkDir);
sb.append("=");
sb.append(URLEncoder.encode(hconf.get(hadoopWorkDir) + "/"
+ Utilities.randGen.nextInt(), "UTF-8"));
}
return sb.toString();
} catch (UnsupportedEncodingException e) {
throw new RuntimeException(e);
}
}
@Override
public boolean isMapRedTask() {
return true;
}
@Override
public boolean hasReduce() {
mapredWork w = getWork();
return w.getReducer() != null;
}
private void addInputPaths(JobConf job, mapredWork work, String hiveScratchDir) throws Exception {
FileSystem inpFs = FileSystem.get(job);
int numEmptyPaths = 0;
// If the query references non-existent partitions
if (work.getPathToAliases().isEmpty() &&
!work.getAliasToWork().isEmpty()) {
String oneAlias = (String)work.getAliasToWork().keySet().toArray()[0];
Class<? extends HiveOutputFormat> outFileFormat = (Class<? extends HiveOutputFormat>)Class.forName("org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat");
String newFile = hiveScratchDir + File.separator + (++numEmptyPaths);
Path newPath = new Path(newFile);
LOG.info("Changed input file to " + newPath.toString());
// add a dummy work
Map<String, ArrayList<String>> pathToAliases = work.getPathToAliases();
ArrayList<String> newList = new ArrayList<String>();
newList.add(oneAlias);
pathToAliases.put(newPath.toString(), newList);
Map<String,partitionDesc> pathToPartitionInfo = work.getPathToPartitionInfo();
partitionDesc pDesc = new partitionDesc();
tableDesc tDesc = new tableDesc(LazySimpleSerDe.class,
SequenceFileInputFormat.class,
SequenceFileOutputFormat.class,
new Properties());
pDesc.setTableDesc(tDesc);
pathToPartitionInfo.put(newPath.toString(), pDesc);
RecordWriter recWriter = outFileFormat.newInstance().getHiveRecordWriter(job, newPath, Text.class, false, new Properties(), null);
recWriter.close(false);
FileInputFormat.addInputPaths(job, newPath.toString());
}
else {
List<String> emptyPaths = new ArrayList<String>();
for (String onefile : work.getPathToAliases().keySet()) {
LOG.info("Adding input file " + onefile);
// If the input file does not exist, replace it by a empty file
Path dirPath = new Path(onefile);
boolean emptyInput = true;
if (inpFs.exists(dirPath)) {
FileStatus[] fStats = inpFs.listStatus(dirPath);
if (fStats.length > 0)
emptyInput = false;
}
if (emptyInput)
emptyPaths.add(onefile);
else
FileInputFormat.addInputPaths(job, onefile);
}
for (String emptyFile : emptyPaths) {
Class<? extends HiveOutputFormat> outFileFormat = work.getPathToPartitionInfo().get(emptyFile).getTableDesc().getOutputFileFormatClass();
String newFile = hiveScratchDir + File.separator + (++numEmptyPaths);
Path newPath = new Path(newFile);
LOG.info("Changed input file to " + newPath.toString());
// toggle the work
LinkedHashMap<String, ArrayList<String>> pathToAliases = work.getPathToAliases();
pathToAliases.put(newPath.toUri().toString(), pathToAliases.get(emptyFile));
pathToAliases.remove(emptyFile);
work.setPathToAliases(pathToAliases);
LinkedHashMap<String,partitionDesc> pathToPartitionInfo = work.getPathToPartitionInfo();
pathToPartitionInfo.put(newPath.toUri().toString(), pathToPartitionInfo.get(emptyFile));
pathToPartitionInfo.remove(emptyFile);
work.setPathToPartitionInfo(pathToPartitionInfo);
String onefile = newPath.toString();
RecordWriter recWriter = outFileFormat.newInstance().getHiveRecordWriter(job, newPath, Text.class, false, new Properties(), null);
recWriter.close(false);
FileInputFormat.addInputPaths(job, onefile);
}
}
}
}