forked from apache/hive
-
Notifications
You must be signed in to change notification settings - Fork 0
/
FileSinkOperator.java
1862 lines (1694 loc) · 76.6 KB
/
FileSinkOperator.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
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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 static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_TEMPORARY_TABLE_STORAGE;
import static org.apache.hadoop.hive.ql.security.authorization.HiveCustomStorageHandlerUtils.setWriteOperation;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.BitSet;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import java.util.Set;
import java.util.function.BiFunction;
import com.google.common.collect.Lists;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.common.FileUtils;
import org.apache.hadoop.hive.common.StatsSetupConst;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConfUtil;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
import org.apache.hadoop.hive.ql.CompilationOpContext;
import org.apache.hadoop.hive.ql.ErrorMsg;
import org.apache.hadoop.hive.ql.exec.Utilities.MissingBucketsContext;
import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
import org.apache.hadoop.hive.ql.io.AcidUtils;
import org.apache.hadoop.hive.ql.io.BucketCodec;
import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
import org.apache.hadoop.hive.ql.io.HiveKey;
import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
import org.apache.hadoop.hive.ql.io.HivePartitioner;
import org.apache.hadoop.hive.ql.io.RecordUpdater;
import org.apache.hadoop.hive.ql.io.StatsProvidingRecordWriter;
import org.apache.hadoop.hive.ql.io.StreamingOutputFormat;
import org.apache.hadoop.hive.ql.io.arrow.ArrowWrapperWritable;
import org.apache.hadoop.hive.ql.io.orc.OrcRecordUpdater;
import org.apache.hadoop.hive.ql.metadata.HiveException;
import org.apache.hadoop.hive.ql.metadata.HiveFatalException;
import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler;
import org.apache.hadoop.hive.ql.metadata.HiveUtils;
import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
import org.apache.hadoop.hive.ql.plan.FileSinkDesc.DPSortState;
import org.apache.hadoop.hive.ql.plan.ListBucketingCtx;
import org.apache.hadoop.hive.ql.plan.PlanUtils;
import org.apache.hadoop.hive.ql.plan.SkewedColumnPositionPair;
import org.apache.hadoop.hive.ql.plan.TableDesc;
import org.apache.hadoop.hive.ql.plan.api.OperatorType;
import org.apache.hadoop.hive.ql.stats.StatsCollectionContext;
import org.apache.hadoop.hive.ql.stats.StatsPublisher;
import org.apache.hadoop.hive.serde2.*;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
import org.apache.hadoop.hive.serde2.objectinspector.StructField;
import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.SubStructObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
import org.apache.hadoop.hive.shims.HadoopShims.StoragePolicyShim;
import org.apache.hadoop.hive.shims.HadoopShims.StoragePolicyValue;
import org.apache.hadoop.hive.shims.ShimLoader;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hive.common.util.HiveStringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* File Sink operator implementation.
**/
@SuppressWarnings("deprecation")
public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
Serializable, IConfigureJobConf {
public static final Logger LOG = LoggerFactory.getLogger(FileSinkOperator.class);
protected transient HashMap<String, FSPaths> valToPaths;
protected transient int numDynParts;
protected transient List<String> dpColNames;
protected transient DynamicPartitionCtx dpCtx;
protected transient boolean isCompressed;
protected transient boolean isTemporary;
protected transient Path parent;
protected transient HiveOutputFormat<?, ?> hiveOutputFormat;
protected transient Path specPath;
protected transient String unionPath;
protected transient boolean isUnionDp;
protected transient int dpStartCol; // start column # for DP columns
protected transient List<String> dpVals; // array of values corresponding to DP columns
protected transient List<Object> dpWritables;
protected transient RecordWriter[] rowOutWriters; // row specific RecordWriters
protected transient int maxPartitions;
protected transient ListBucketingCtx lbCtx;
protected transient boolean isSkewedStoredAsSubDirectories;
protected transient boolean[] statsFromRecordWriter;
protected transient boolean isCollectRWStats;
private transient FSPaths prevFsp;
private transient FSPaths fpaths;
private StructField recIdField; // field to find record identifier in
private StructField bucketField; // field bucket is in in record id
private StructObjectInspector recIdInspector; // OI for inspecting record id
private IntObjectInspector bucketInspector; // OI for inspecting bucket id
protected transient long numRows = 0;
protected transient long cntr = 1;
protected transient long logEveryNRows = 0;
protected transient int rowIndex = 0;
private transient Path destTablePath;
private transient boolean isInsertOverwrite;
private transient String counterGroup;
private transient BiFunction<Object[], ObjectInspector[], Integer> hashFunc;
public static final String TOTAL_TABLE_ROWS_WRITTEN = "TOTAL_TABLE_ROWS_WRITTEN";
private transient Set<String> dynamicPartitionSpecs = new HashSet<>();
/**
* Counters.
*/
public static enum Counter {
RECORDS_OUT
}
/**
* RecordWriter.
*
*/
public static interface RecordWriter {
void write(Writable w) throws IOException;
void close(boolean abort) throws IOException;
}
public class FSPaths implements Cloneable {
private Path tmpPathRoot;
private String subdirBeforeTxn, subdirAfterTxn;
private final String subdirForTxn;
private Path taskOutputTempPathRoot;
Path[] outPaths;
// The bucket files we successfully wrote to in this writer
Path[] outPathsCommitted;
Path[] finalPaths;
RecordWriter[] outWriters;
RecordUpdater[] updaters;
Stat stat;
int acidLastBucket = -1;
int acidFileOffset = -1;
private boolean isMmTable;
private boolean isDirectInsert;
private AcidUtils.Operation acidOperation;
private boolean isInsertOverwrite;
String dpDirForCounters;
public FSPaths(Path specPath, boolean isMmTable, boolean isDirectInsert, boolean isInsertOverwrite,
AcidUtils.Operation acidOperation) {
this.isMmTable = isMmTable;
this.isDirectInsert = isDirectInsert;
this.acidOperation = acidOperation;
this.isInsertOverwrite = isInsertOverwrite;
if (!isMmTable && !isDirectInsert) {
tmpPathRoot = Utilities.toTempPath(specPath);
taskOutputTempPathRoot = Utilities.toTaskTempPath(specPath);
subdirForTxn = null;
} else {
tmpPathRoot = specPath;
taskOutputTempPathRoot = null; // Should not be used.
if (isMmTable) {
subdirForTxn = AcidUtils.baseOrDeltaSubdir(conf.getInsertOverwrite(),
conf.getTableWriteId(), conf.getTableWriteId(), conf.getStatementId());
} else {
/**
* For direct write to final path during ACID insert, we create the delta directories
* later when we create the RecordUpdater using AcidOutputFormat.Options
*/
subdirForTxn = null;
}
}
if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) {
Utilities.FILE_OP_LOGGER.trace("new FSPaths for " + numFiles
+ " files, dynParts = " + bDynParts + " (spec path " + specPath + ")");
}
outPaths = new Path[numFiles];
outPathsCommitted = new Path[numFiles];
finalPaths = new Path[numFiles];
outWriters = new RecordWriter[numFiles];
updaters = new RecordUpdater[numFiles];
LOG.debug("Created slots for {}", numFiles);
stat = new Stat();
}
public void closeWriters(boolean abort, List<Path> deleteDeltas) throws HiveException {
Exception exception = null;
for (int idx = 0; idx < outWriters.length; idx++) {
if (outWriters[idx] != null) {
try {
outWriters[idx].close(abort);
updateProgress();
} catch (IOException e) {
exception = e;
LOG.error("Error closing " + outWriters[idx].toString(), e);
// continue closing others
}
}
}
for (int i = 0; i < updaters.length; i++) {
if (updaters[i] != null) {
SerDeStats stats = updaters[i].getStats();
// Ignore 0 row files except in case of insert overwrite or delete or update
if (isDirectInsert
&& (stats.getRowCount() > 0 || isInsertOverwrite || AcidUtils.Operation.DELETE.equals(acidOperation)
|| AcidUtils.Operation.UPDATE.equals(acidOperation))) {
OrcRecordUpdater recordUpdater = (OrcRecordUpdater) updaters[i];
switch (acidOperation) {
case INSERT:
outPathsCommitted[i] = recordUpdater.getUpdatedFilePath();
break;
case UPDATE:
// In case of update operation, we need both the deleteFilePath and the updatedFilePath.
// The updateFilePath will be added to the outPathsCommitted array and the deleteFilePath
// will be collected in a separate list.
outPathsCommitted[i] = recordUpdater.getUpdatedFilePath();
if (deleteDeltas != null) {
deleteDeltas.add(recordUpdater.getDeleteFilePath());
LOG.debug("The following path has been added to the deleteDeltas list: "
+ recordUpdater.getDeleteFilePath().toString());
}
break;
case DELETE:
// In case of delete operation, the deleteFilePath has to be used, not the updatedFilePath
outPathsCommitted[i] = recordUpdater.getDeleteFilePath();
break;
default:
break;
}
LOG.debug(
"The following path has been added to the outPathsCommitted array: " + outPathsCommitted[i].toString());
}
try {
updaters[i].close(abort);
} catch (IOException e) {
exception = e;
LOG.error("Error closing " + updaters[i].toString(), e);
// continue closing others
}
}
}
// Made an attempt to close all writers.
if (exception != null) {
throw new HiveException(exception);
}
}
private void commit(FileSystem fs, List<Path> commitPaths, List<Path> deleteDeltas) throws HiveException {
for (int idx = 0; idx < outPaths.length; ++idx) {
try {
if (outPaths[idx] != null) {
commitOneOutPath(idx, fs, commitPaths);
}
} catch (IOException e) {
throw new HiveException("Unable to commit output from: " +
outPaths[idx] + " to: " + finalPaths[idx], e);
}
}
if (isDirectInsert && AcidUtils.Operation.UPDATE.equals(conf.getAcidOperation())) {
// In case of an update, the outPathsCommitted array only contains the delta directory,
// but not the delete delta directory. The files from the delete delta directory has to be
// added to the commitPath list, otherwise it will be cleaned up.
commitPaths.addAll(deleteDeltas);
}
}
private void commitOneOutPath(int idx, FileSystem fs, List<Path> commitPaths)
throws IOException, HiveException {
if ((bDynParts || isSkewedStoredAsSubDirectories)
&& !fs.exists(finalPaths[idx].getParent())) {
if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) {
Utilities.FILE_OP_LOGGER.trace("commit making path for dyn/skew: " + finalPaths[idx].getParent());
}
FileUtils.mkdir(fs, finalPaths[idx].getParent(), hconf);
}
if(outPaths[idx] != null && fs.exists(outPaths[idx])) {
if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) {
Utilities.FILE_OP_LOGGER.trace("committing " + outPaths[idx] + " to " + finalPaths[idx] + " (mm table ="
+ isMmTable + ", direct insert = " + isDirectInsert + ")");
}
if (isMmTable) {
assert outPaths[idx].equals(finalPaths[idx]);
commitPaths.add(outPaths[idx]);
} else if (isDirectInsert && outPathsCommitted[idx] != null) {
if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) {
Utilities.FILE_OP_LOGGER
.trace("committing " + outPathsCommitted[idx] + " (direct insert = " + isDirectInsert + ")");
}
commitPaths.add(outPathsCommitted[idx]);
} else if (!isDirectInsert && !fs.rename(outPaths[idx], finalPaths[idx])) {
FileStatus fileStatus = FileUtils.getFileStatusOrNull(fs, finalPaths[idx]);
if (fileStatus != null) {
LOG.warn("Target path " + finalPaths[idx] + " with a size " + fileStatus.getLen() + " exists. Trying to delete it.");
if (!fs.delete(finalPaths[idx], true)) {
throw new HiveException("Unable to delete existing target output: " + finalPaths[idx]);
}
}
if (!fs.rename(outPaths[idx], finalPaths[idx])) {
throw new HiveException("Unable to rename output from: "
+ outPaths[idx] + " to: " + finalPaths[idx]);
}
}
}
updateProgress();
}
public void abortWritersAndUpdaters(FileSystem fs, boolean abort, boolean delete) throws HiveException {
for (int idx = 0; idx < outWriters.length; idx++) {
if (outWriters[idx] != null) {
try {
LOG.debug("Aborted: closing: " + outWriters[idx].toString());
outWriters[idx].close(abort);
if (delete) {
fs.delete(outPaths[idx], true);
}
updateProgress();
} catch (IOException e) {
throw new HiveException(e);
}
}
}
for (int idx = 0; idx < updaters.length; idx++) {
if (updaters[idx] != null) {
try {
LOG.debug("Aborted: closing: " + updaters[idx].toString());
updaters[idx].close(abort);
if (delete) {
fs.delete(outPaths[idx], true);
}
updateProgress();
} catch (IOException e) {
throw new HiveException(e);
}
}
}
}
public void initializeBucketPaths(int filesIdx, String taskId, boolean isNativeTable,
boolean isSkewedStoredAsSubDirectories) {
if (isNativeTable) {
String extension = Utilities.getFileExtension(jc, isCompressed, hiveOutputFormat);
String taskWithExt = extension == null ? taskId : taskId + extension;
if (!isMmTable && !isDirectInsert) {
if (!bDynParts && !isSkewedStoredAsSubDirectories) {
finalPaths[filesIdx] = new Path(parent, taskWithExt);
} else {
finalPaths[filesIdx] = new Path(buildTmpPath(), taskWithExt);
}
outPaths[filesIdx] = new Path(buildTaskOutputTempPath(), Utilities.toTempPath(taskId));
} else {
String taskIdPath = taskId;
if (conf.isMerge()) {
// Make sure we don't collide with the source files.
// MM tables don't support concat so we don't expect the merge of merged files.
taskIdPath += ".merged";
}
if (extension != null) {
taskIdPath += extension;
}
Path finalPath;
if (isDirectInsert) {
finalPath = buildTmpPath();
} else {
finalPath = new Path(buildTmpPath(), taskIdPath);
}
// In the cases that have multi-stage insert, e.g. a "hive.skewjoin.key"-based skew join,
// it can happen that we want multiple commits into the same directory from different
// tasks (not just task instances). In non-MM case, Utilities.renameOrMoveFiles ensures
// unique names. We could do the same here, but this will still cause the old file to be
// deleted because it has not been committed in /this/ FSOP. We are going to fail to be
// safe. Potentially, we could implement some partial commit between stages, if this
// affects some less obscure scenario.
try {
FileSystem fpfs = finalPath.getFileSystem(hconf);
if (!isDirectInsert && fpfs.exists(finalPath)) {
throw new RuntimeException(finalPath + " already exists");
}
} catch (IOException e) {
throw new RuntimeException(e);
}
finalPaths[filesIdx] = finalPath;
outPaths[filesIdx] = finalPath;
}
LOG.info("Final Path: FS " + finalPaths[filesIdx]);
if (!isMmTable && !isDirectInsert) {
LOG.info("Writing to temp file: FS " + outPaths[filesIdx]);
}
} else {
finalPaths[filesIdx] = outPaths[filesIdx] = specPath;
}
}
public Path buildTmpPath() {
String pathStr = tmpPathRoot.toString();
if (subdirBeforeTxn != null) {
pathStr += Path.SEPARATOR + subdirBeforeTxn;
}
if (subdirForTxn != null) {
pathStr += Path.SEPARATOR + subdirForTxn;
}
if (subdirAfterTxn != null) {
pathStr += Path.SEPARATOR + subdirAfterTxn;
}
return new Path(pathStr);
}
public Path buildTaskOutputTempPath() {
if (taskOutputTempPathRoot == null) {
return null;
}
assert subdirForTxn == null;
String pathStr = taskOutputTempPathRoot.toString();
if (subdirBeforeTxn != null) {
pathStr += Path.SEPARATOR + subdirBeforeTxn;
}
if (subdirAfterTxn != null) {
pathStr += Path.SEPARATOR + subdirAfterTxn;
}
return new Path(pathStr);
}
public void addToStat(String statType, long amount) {
stat.addToStat(statType, amount);
}
public Collection<String> getStoredStats() {
return stat.getStoredStats();
}
/**
* This method is intended for use with ACID unbucketed tables, where the DELETE ops behave as
* though they are bucketed, but without an explicit pre-specified bucket count. The bucketNum
* is read out of the middle value of the ROW__ID variable and this is written out from a single
* FileSink, in ways similar to the multi file spray, but without knowing the total number of
* buckets ahead of time.
*
* ROW__ID (1,2[0],3) => bucket_00002
* ROW__ID (1,3[0],4) => bucket_00003 etc
*
* A new FSP is created for each partition, so this only requires the bucket numbering and that
* is mapped in directly as an index.
*
* This relies on ReduceSinkOperator to shuffle update/delete rows by
* UDFToInteger(RecordIdentifier), i.e. by writerId in ROW__ID.
* {@link org.apache.hadoop.hive.ql.parse.SemanticAnalyzer#getPartitionColsFromBucketColsForUpdateDelete(Operator, boolean)}
*/
public int createDynamicBucket(int bucketNum) {
// this assumes all paths are bucket names (which means no lookup is needed)
int writerOffset = bucketNum;
if (updaters.length <= writerOffset) {
this.updaters = Arrays.copyOf(updaters, writerOffset + 1);
this.outPaths = Arrays.copyOf(outPaths, writerOffset + 1);
this.finalPaths = Arrays.copyOf(finalPaths, writerOffset + 1);
}
if (this.finalPaths[writerOffset] == null) {
if (conf.isDirectInsert()) {
this.outPathsCommitted = Arrays.copyOf(outPathsCommitted, writerOffset + 1);
this.finalPaths[writerOffset] = buildTmpPath();
this.outPaths[writerOffset] = buildTmpPath();
} else {
// uninitialized bucket
String bucketName =
Utilities.replaceTaskIdFromFilename(Utilities.getTaskId(hconf), bucketNum);
this.finalPaths[writerOffset] = new Path(bDynParts ? buildTmpPath() : parent, bucketName);
this.outPaths[writerOffset] = new Path(buildTaskOutputTempPath(), bucketName);
}
}
return writerOffset;
}
} // class FSPaths
private static final long serialVersionUID = 1L;
protected transient FileSystem fs;
protected transient Serializer serializer;
protected final transient LongWritable row_count = new LongWritable();
/**
* The evaluators for the multiFile sprayer. If the table under consideration has 1000 buckets,
* it is not a good idea to start so many reducers - if the maximum number of reducers is 100,
* each reducer can write 10 files - this way we effectively get 1000 files.
*/
private transient ExprNodeEvaluator[] partitionEval;
protected transient int totalFiles;
private transient int numFiles;
protected transient boolean multiFileSpray;
protected transient final Map<Integer, Integer> bucketMap = new HashMap<Integer, Integer>();
private transient boolean isBucketed = false;
private transient int bucketId;
private transient ObjectInspector[] partitionObjectInspectors;
protected transient HivePartitioner<HiveKey, Object> prtner;
protected transient final HiveKey key = new HiveKey();
private transient Configuration hconf;
protected transient FSPaths fsp;
protected transient boolean bDynParts;
private transient SubStructObjectInspector subSetOI;
private transient int timeOut; // JT timeout in msec.
private transient long lastProgressReport = System.currentTimeMillis();
protected transient boolean autoDelete = false;
protected transient JobConf jc;
Class<? extends Writable> outputClass;
String taskId, originalTaskId;
protected boolean filesCreated = false;
protected BitSet filesCreatedPerBucket = new BitSet();
protected boolean isCompactionTable = false;
private void initializeSpecPath() {
// For a query of the type:
// insert overwrite table T1
// select * from (subq1 union all subq2)u;
// subQ1 and subQ2 write to directories Parent/Child_1 and
// Parent/Child_2 respectively, and union is removed.
// The movetask that follows subQ1 and subQ2 tasks moves the directory
// 'Parent'
// However, if the above query contains dynamic partitions, subQ1 and
// subQ2 have to write to directories: Parent/DynamicPartition/Child_1
// and Parent/DynamicPartition/Child_1 respectively.
// The movetask that follows subQ1 and subQ2 tasks still moves the directory
// 'Parent'
boolean isLinked = conf.isLinkedFileSink();
if (!isLinked) {
// Simple case - no union.
specPath = conf.getDirName();
unionPath = null;
} else {
isUnionDp = (dpCtx != null);
if (conf.isDirectInsert()) {
specPath = conf.getParentDir();
unionPath = null;
} else if (conf.isMmTable() || isUnionDp) {
// MM tables need custom handling for union suffix; DP tables use parent too.
specPath = conf.getParentDir();
unionPath = conf.getDirName().getName();
} else {
// For now, keep the old logic for non-MM non-DP union case. Should probably be unified.
specPath = conf.getDirName();
unionPath = null;
}
}
if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) {
Utilities.FILE_OP_LOGGER.trace("Setting up FSOP " + System.identityHashCode(this) + " ("
+ conf.isLinkedFileSink() + ") with " + taskId + " and " + specPath + " + " + unionPath);
}
}
/** Kryo ctor. */
protected FileSinkOperator() {
super();
}
public FileSinkOperator(CompilationOpContext ctx) {
super(ctx);
}
@Override
protected void initializeOp(Configuration hconf) throws HiveException {
super.initializeOp(hconf);
try {
this.hconf = hconf;
filesCreated = false;
isTemporary = conf.isTemporary();
multiFileSpray = conf.isMultiFileSpray();
this.isBucketed = hconf.getInt(hive_metastoreConstants.BUCKET_COUNT, 0) > 0;
this.isCompactionTable = conf.isCompactionTable();
totalFiles = conf.getTotalFiles();
numFiles = conf.getNumFiles();
dpCtx = conf.getDynPartCtx();
lbCtx = conf.getLbCtx();
fsp = prevFsp = null;
valToPaths = new HashMap<String, FSPaths>();
taskId = originalTaskId = Utilities.getTaskId(hconf);
initializeSpecPath();
fs = specPath.getFileSystem(hconf);
jc = new JobConf(hconf);
setWriteOperation(jc, getConf().getTableInfo().getTableName(), getConf().getWriteOperation());
try {
createHiveOutputFormat(jc);
} catch (HiveException ex) {
logOutputFormatError(jc, ex);
throw ex;
}
isCompressed = conf.getCompressed();
if (conf.isLinkedFileSink() && conf.isDirectInsert()) {
parent = Utilities.toTempPath(conf.getFinalDirName());
} else {
parent = Utilities.toTempPath(conf.getDirName());
}
statsFromRecordWriter = new boolean[numFiles];
AbstractSerDe serde = conf.getTableInfo().getSerDeClass().newInstance();
serde.initialize(unsetNestedColumnPaths(jc), conf.getTableInfo().getProperties(), null);
serializer = serde;
outputClass = serializer.getSerializedClass();
destTablePath = conf.getDestPath();
isInsertOverwrite = conf.getInsertOverwrite();
counterGroup = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVECOUNTERGROUP);
LOG.info("Using serializer : " + serializer + " and formatter : " + hiveOutputFormat
+ (isCompressed ? " with compression" : ""));
// Timeout is chosen to make sure that even if one iteration takes more than
// half of the script.timeout but less than script.timeout, we will still
// be able to report progress.
timeOut = hconf.getInt("mapred.healthChecker.script.timeout", 600000) / 2;
if (multiFileSpray) {
partitionEval = new ExprNodeEvaluator[conf.getPartitionCols().size()];
int i = 0;
for (ExprNodeDesc e : conf.getPartitionCols()) {
partitionEval[i++] = ExprNodeEvaluatorFactory.get(e);
}
partitionObjectInspectors = initEvaluators(partitionEval, outputObjInspector);
prtner = (HivePartitioner<HiveKey, Object>) ReflectionUtils.newInstance(
jc.getPartitionerClass(), null);
}
if (dpCtx != null && !inspectPartitionValues()) {
dpSetup();
}
if (lbCtx != null) {
lbSetup();
}
if (!bDynParts) {
fsp = new FSPaths(specPath, conf.isMmTable(), conf.isDirectInsert(), conf.getInsertOverwrite(),
conf.getAcidOperation());
fsp.subdirAfterTxn = combinePathFragments(generateListBucketingDirName(null), unionPath);
if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) {
Utilities.FILE_OP_LOGGER.trace("creating new paths " + System.identityHashCode(fsp)
+ " from ctor; childSpec " + unionPath + ": tmpPath " + fsp.buildTmpPath()
+ ", task path " + fsp.buildTaskOutputTempPath());
}
// Create all the files - this is required because empty files need to be created for
// empty buckets
// createBucketFiles(fsp);
if (!this.isSkewedStoredAsSubDirectories) {
valToPaths.put("", fsp); // special entry for non-DP case
}
}
final StoragePolicyValue tmpStorage = StoragePolicyValue.lookup(HiveConf
.getVar(hconf, HIVE_TEMPORARY_TABLE_STORAGE));
if (isTemporary && fsp != null
&& tmpStorage != StoragePolicyValue.DEFAULT) {
assert !conf.isMmTable(); // Not supported for temp tables.
final Path outputPath = fsp.buildTaskOutputTempPath();
StoragePolicyShim shim = ShimLoader.getHadoopShims()
.getStoragePolicyShim(fs);
if (shim != null) {
// directory creation is otherwise within the writers
fs.mkdirs(outputPath);
shim.setStoragePolicy(outputPath, tmpStorage);
}
}
if (conf.getWriteType() == AcidUtils.Operation.UPDATE ||
conf.getWriteType() == AcidUtils.Operation.DELETE) {
// ROW__ID is always in the first field
recIdField = ((StructObjectInspector)outputObjInspector).getAllStructFieldRefs().get(0);
recIdInspector = (StructObjectInspector)recIdField.getFieldObjectInspector();
// bucket is the second field in the record id
bucketField = recIdInspector.getAllStructFieldRefs().get(1);
bucketInspector = (IntObjectInspector)bucketField.getFieldObjectInspector();
}
numRows = 0;
cntr = 1;
logEveryNRows = HiveConf.getLongVar(hconf, HiveConf.ConfVars.HIVE_LOG_N_RECORDS);
statsMap.put(getCounterName(Counter.RECORDS_OUT), row_count);
// Setup hashcode
hashFunc = conf.getTableInfo().getBucketingVersion() == 2 ?
ObjectInspectorUtils::getBucketHashCode :
ObjectInspectorUtils::getBucketHashCodeOld;
//Counter for number of rows that are associated with a destination table in FileSinkOperator.
//This count is used to get total number of rows in an insert query.
if (conf.getTableInfo() != null && conf.getTableInfo().getTableName() != null) {
statsMap.put(TOTAL_TABLE_ROWS_WRITTEN, row_count);
}
} catch (HiveException e) {
throw e;
} catch (Exception e) {
throw new HiveException(e);
}
}
/**
* Whether partition values are stored in the data files too (as opposed to just being represented in the partition
* folder name), along with non-partition column values. Therefore if true, the object inspector should not
* disregard/remove the partition columns.
*
* @return whether partition values should be part of the object inspector too
*/
private boolean inspectPartitionValues() {
return Optional.ofNullable(conf).map(FileSinkDesc::getTableInfo)
.map(TableDesc::getProperties)
.map(props -> props.getProperty(hive_metastoreConstants.META_TABLE_STORAGE))
.map(handler -> {
try {
return HiveUtils.getStorageHandler(hconf, handler);
} catch (HiveException e) {
return null;
}
})
.map(HiveStorageHandler::alwaysUnpartitioned)
.orElse(Boolean.FALSE);
}
public String getCounterName(Counter counter) {
String suffix = Integer.toString(conf.getDestTableId());
String fullName = conf.getTableInfo().getTableName();
if (fullName != null) {
suffix = suffix + "_" + fullName.toLowerCase();
}
return counter + "_" + suffix;
}
private void logOutputFormatError(Configuration hconf, HiveException ex) {
StringBuilder errorWriter = new StringBuilder();
errorWriter.append("Failed to create output format; configuration: ");
// redact sensitive information before logging
HiveConfUtil.dumpConfig(hconf, errorWriter);
Properties tdp = null;
if (this.conf.getTableInfo() != null
&& (tdp = this.conf.getTableInfo().getProperties()) != null) {
errorWriter.append(";\n table properties: { ");
for (Map.Entry<Object, Object> e : tdp.entrySet()) {
errorWriter.append(e.getKey() + ": " + e.getValue() + ", ");
}
errorWriter.append('}');
}
LOG.error(errorWriter.toString(), ex);
}
/**
* Initialize list bucketing information
*/
private void lbSetup() {
this.isSkewedStoredAsSubDirectories = ((lbCtx == null) ? false : lbCtx.isSkewedStoredAsDir());
}
/**
* Set up for dynamic partitioning including a new ObjectInspector for the output row.
*/
private void dpSetup() {
this.bDynParts = false;
this.numDynParts = dpCtx.getNumDPCols();
this.dpColNames = dpCtx.getDPColNames();
this.maxPartitions = dpCtx.getMaxPartitionsPerNode();
assert numDynParts == dpColNames.size()
: "number of dynamic partitions should be the same as the size of DP mapping";
if (dpColNames != null && dpColNames.size() > 0) {
this.bDynParts = true;
assert inputObjInspectors.length == 1 : "FileSinkOperator should have 1 parent, but it has "
+ inputObjInspectors.length;
StructObjectInspector soi = (StructObjectInspector) inputObjInspectors[0];
this.dpStartCol = Utilities.getDPColOffset(conf);
this.subSetOI = new SubStructObjectInspector(soi, 0, this.dpStartCol);
this.dpVals = new ArrayList<String>(numDynParts);
this.dpWritables = new ArrayList<Object>(numDynParts);
}
}
/**
* There was an issue with the query-based MINOR compaction (HIVE-23763), that the row distribution between the FileSinkOperators
* was not correlated correctly with the bucket numbers. So it could happen that rows from different buckets ended up in the same
* FileSinkOperator and got written out into one file. This is not correct, one bucket file must contain rows from the same bucket.
* Therefore the FileSinkOperator got extended with this method to be able to handle rows from different buckets.
* In this case it will create separate files from each bucket. This logic is similar to the one in the createDynamicBucket method.
* @param fsp
* @throws HiveException
*/
protected void createBucketFilesForCompaction(FSPaths fsp) throws HiveException {
try {
if (fsp.outPaths.length < bucketId + 1) {
fsp.updaters = Arrays.copyOf(fsp.updaters, bucketId + 1);
fsp.outPaths = Arrays.copyOf(fsp.outPaths, bucketId + 1);
fsp.finalPaths = Arrays.copyOf(fsp.finalPaths, bucketId + 1);
fsp.outWriters = Arrays.copyOf(fsp.outWriters, bucketId + 1);
statsFromRecordWriter = Arrays.copyOf(statsFromRecordWriter, bucketId + 1);
}
createBucketForFileIdx(fsp, bucketId);
} catch (Exception e) {
throw new HiveException(e);
}
filesCreatedPerBucket.set(bucketId);
}
protected void createBucketFiles(FSPaths fsp) throws HiveException {
try {
int filesIdx = 0;
Set<Integer> seenBuckets = new HashSet<Integer>();
for (int idx = 0; idx < totalFiles; idx++) {
if (this.getExecContext() != null && this.getExecContext().getFileId() != null) {
LOG.info("replace taskId from execContext");
taskId = Utilities.replaceTaskIdFromFilename(taskId, this.getExecContext().getFileId());
LOG.info("new taskId: FS " + taskId);
assert !multiFileSpray;
assert totalFiles == 1;
}
int bucketNum = 0;
if (multiFileSpray) {
key.setHashCode(idx);
// Does this hashcode belong to this reducer
int numReducers = totalFiles / numFiles;
if (numReducers > 1) {
int currReducer = Integer.parseInt(Utilities.getTaskIdFromFilename(Utilities
.getTaskId(hconf)));
int reducerIdx = prtner.getPartition(key, null, numReducers);
if (currReducer != reducerIdx) {
continue;
}
}
bucketNum = prtner.getBucket(key, null, totalFiles);
if (seenBuckets.contains(bucketNum)) {
continue;
}
seenBuckets.add(bucketNum);
bucketMap.put(bucketNum, filesIdx);
taskId = Utilities.replaceTaskIdFromFilename(Utilities.getTaskId(hconf), bucketNum);
}
createBucketForFileIdx(fsp, filesIdx);
filesIdx++;
}
assert filesIdx == numFiles;
} catch (Exception e) {
throw new HiveException(e);
}
filesCreated = true;
}
protected void createBucketForFileIdx(FSPaths fsp, int filesIdx)
throws HiveException {
try {
if (isCompactionTable) {
fsp.initializeBucketPaths(filesIdx, AcidUtils.BUCKET_PREFIX + String.format(AcidUtils.BUCKET_DIGITS, bucketId),
isNativeTable(), isSkewedStoredAsSubDirectories);
} else {
fsp.initializeBucketPaths(filesIdx, taskId, isNativeTable(), isSkewedStoredAsSubDirectories);
}
if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) {
Utilities.FILE_OP_LOGGER.trace("createBucketForFileIdx " + filesIdx + ": final path " + fsp.finalPaths[filesIdx]
+ "; out path " + fsp.outPaths[filesIdx] +" (spec path " + specPath + ", tmp path "
+ fsp.buildTmpPath() + ", task " + taskId + ")");
}
LOG.info("New Final Path: FS " + fsp.finalPaths[filesIdx]);
if (isNativeTable() && !conf.isMmTable() && !conf.isDirectInsert()) {
// in recent hadoop versions, use deleteOnExit to clean tmp files.
autoDelete = fs.deleteOnExit(fsp.outPaths[filesIdx]);
}
updateDPCounters(fsp, filesIdx);
Utilities.copyTableJobPropertiesToConf(conf.getTableInfo(), jc);
// only create bucket files only if no dynamic partitions,
// buckets of dynamic partitions will be created for each newly created partition
//todo IOW integration. Full Acid uses the else if block to create Acid's RecordUpdater (HiveFileFormatUtils)
// and that will set writingBase(conf.getInsertOverwrite())
// If MM wants to create a new base for IOW (instead of delta dir), it should specify it here
if (conf.getWriteType() == AcidUtils.Operation.NOT_ACID || conf.isMmTable() || isCompactionTable) {
Path outPath = fsp.outPaths[filesIdx];
if (conf.isMmTable()
&& !FileUtils.mkdir(fs, outPath.getParent(), hconf)) {
LOG.warn("Unable to create directory with inheritPerms: " + outPath);
}
fsp.outWriters[filesIdx] = HiveFileFormatUtils.getHiveRecordWriter(jc, conf.getTableInfo(),
outputClass, conf, outPath, reporter);
// If the record writer provides stats, get it from there instead of the serde
statsFromRecordWriter[filesIdx] = fsp.outWriters[filesIdx] instanceof
StatsProvidingRecordWriter;
// increment the CREATED_FILES counter
} else if (conf.getWriteType() == AcidUtils.Operation.INSERT) {
Path outPath = fsp.outPaths[filesIdx];
if (conf.isDirectInsert()
&& !FileUtils.mkdir(fs, outPath.getParent(), hconf)) {
LOG.warn("Unable to create directory: " + outPath);
}
// Only set up the updater for insert. For update and delete we don't know unitl we see
// the row.
ObjectInspector inspector = bDynParts ? subSetOI : outputObjInspector;
int acidBucketNum = Integer.parseInt(Utilities.getTaskIdFromFilename(taskId));
Integer attemptId = getAttemptIdFromTaskId(taskId);
fsp.updaters[filesIdx] = HiveFileFormatUtils.getAcidRecordUpdater(jc, conf.getTableInfo(),
acidBucketNum, conf, fsp.outPaths[filesIdx], inspector, reporter, -1, attemptId); // outPath.getParent()
}
if (reporter != null) {
reporter.incrCounter(counterGroup, Operator.HIVE_COUNTER_CREATED_FILES, 1);
}
} catch (IOException e) {
throw new HiveException(e);
}
}
private void updateDPCounters(final FSPaths fsp, final int filesIdx) {
// There are 2 cases where we increment CREATED_DYNAMIC_PARTITIONS counters
// 1) Insert overwrite (all partitions are newly created)
// 2) Insert into table which creates new partitions (some new partitions)
if (bDynParts && destTablePath != null && fsp.dpDirForCounters != null) {
Path destPartPath = new Path(destTablePath, fsp.dpDirForCounters);
// For MM tables, directory structure is
// <table-dir>/<partition-dir>/<delta-dir>/
// For Non-MM tables, directory structure is
// <table-dir>/<staging-dir>/<partition-dir>
// if UNION ALL insert, for non-mm tables subquery creates another subdirectory at the end for each union queries
// <table-dir>/<staging-dir>/<partition-dir>/<union-dir>
// for non-MM tables, the final destination partition directory is created during move task via rename
// for MM tables and ACID insert, the final destination partition directory is created by the tasks themselves
try {
if (conf.isMmTable() || conf.isDirectInsert()) {
createDpDir(destPartPath);
} else {
// outPath will be
// non-union case: <table-dir>/<staging-dir>/<partition-dir>/<taskid>
// union case: <table-dir>/<staging-dir>/<partition-dir>/<union-dir>/<taskid>
Path dpStagingDir = fsp.outPaths[filesIdx].getParent();
if (isUnionDp) {
dpStagingDir = dpStagingDir.getParent();
}
if (isInsertOverwrite) {
createDpDir(dpStagingDir);
} else {
createDpDirCheckSrc(dpStagingDir, destPartPath);
}
}