/
OrcFlowFileWriter.java
2650 lines (2428 loc) · 107 KB
/
OrcFlowFileWriter.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.io.orc;
import static com.google.common.base.Preconditions.checkArgument;
import java.io.IOException;
import java.io.OutputStream;
import java.lang.management.ManagementFactory;
import java.nio.ByteBuffer;
import java.sql.Timestamp;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import java.util.TimeZone;
import java.util.TreeMap;
import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.common.JavaUtils;
import org.apache.hadoop.hive.common.type.HiveDecimal;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
import org.apache.hadoop.hive.ql.io.IOConstants;
import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO;
import org.apache.hadoop.hive.ql.io.orc.CompressionCodec.Modifier;
import org.apache.hadoop.hive.ql.io.orc.OrcFile.CompressionStrategy;
import org.apache.hadoop.hive.ql.io.orc.OrcFile.EncodingStrategy;
import org.apache.hadoop.hive.ql.io.orc.OrcProto.RowIndexEntry;
import org.apache.hadoop.hive.ql.io.orc.OrcProto.StripeStatistics;
import org.apache.hadoop.hive.ql.io.orc.OrcProto.Type;
import org.apache.hadoop.hive.ql.io.orc.OrcProto.UserMetadataItem;
import org.apache.hadoop.hive.ql.util.JavaDataModel;
import org.apache.hadoop.hive.serde2.io.DateWritable;
import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.StructField;
import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.Text;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.collect.Lists;
import com.google.common.primitives.Longs;
import com.google.protobuf.ByteString;
import com.google.protobuf.CodedOutputStream;
import org.apache.nifi.stream.io.ByteCountingOutputStream;
/**
* An ORC file writer. The file is divided into stripes, which is the natural
* unit of work when reading. Each stripe is buffered in memory until the
* memory reaches the stripe size and then it is written out broken down by
* columns. Each column is written by a TreeWriter that is specific to that
* type of column. TreeWriters may have children TreeWriters that handle the
* sub-types. Each of the TreeWriters writes the column's data as a set of
* streams.
* <p>
* This class is synchronized so that multi-threaded access is ok. In
* particular, because the MemoryManager is shared between writers, this class
* assumes that checkMemory may be called from a separate thread.
*/
public class OrcFlowFileWriter implements Writer, MemoryManager.Callback {
private static final Log LOG = LogFactory.getLog(WriterImpl.class);
private static final int HDFS_BUFFER_SIZE = 256 * 1024;
private static final int MIN_ROW_INDEX_STRIDE = 1000;
// threshold above which buffer size will be automatically resized
private static final int COLUMN_COUNT_THRESHOLD = 1000;
private final Path path;
private final long defaultStripeSize;
private long adjustedStripeSize;
private final int rowIndexStride;
private final CompressionKind compress;
private final CompressionCodec codec;
private final boolean addBlockPadding;
private final int bufferSize;
private final long blockSize;
private final float paddingTolerance;
// the streams that make up the current stripe
private final Map<StreamName, BufferedStream> streams = new TreeMap<>();
private final OutputStream flowFileOutputStream;
private ByteCountingOutputStream rawWriter = null;
// the compressed metadata information outStream
private OutStream writer = null;
// a protobuf outStream around streamFactory
private CodedOutputStream protobufWriter = null;
private long headerLength;
private int columnCount;
private long rowCount = 0;
private long rowsInStripe = 0;
private long rawDataSize = 0;
private int rowsInIndex = 0;
private int stripesAtLastFlush = -1;
private final List<OrcProto.StripeInformation> stripes = new ArrayList<>();
private final Map<String, ByteString> userMetadata = new TreeMap<>();
private final StreamFactory streamFactory = new StreamFactory();
private final TreeWriter treeWriter;
private final boolean buildIndex;
private final MemoryManager memoryManager;
private final OrcFile.Version version;
private final Configuration conf;
private final OrcFile.WriterCallback callback;
private final OrcFile.WriterContext callbackContext;
private final OrcFile.EncodingStrategy encodingStrategy;
private final OrcFile.CompressionStrategy compressionStrategy;
private final boolean[] bloomFilterColumns;
private final double bloomFilterFpp;
public OrcFlowFileWriter(OutputStream flowFileOutputStream,
Path path,
Configuration conf,
ObjectInspector inspector,
long stripeSize,
CompressionKind compress,
int bufferSize,
int rowIndexStride,
MemoryManager memoryManager,
boolean addBlockPadding,
OrcFile.Version version,
OrcFile.WriterCallback callback,
EncodingStrategy encodingStrategy,
CompressionStrategy compressionStrategy,
float paddingTolerance,
long blockSizeValue,
String bloomFilterColumnNames,
double bloomFilterFpp) throws IOException {
this.flowFileOutputStream = flowFileOutputStream;
this.path = path;
this.conf = conf;
this.callback = callback;
callbackContext = (callback != null) ? () -> OrcFlowFileWriter.this : null;
this.adjustedStripeSize = stripeSize;
this.defaultStripeSize = stripeSize;
this.version = version;
this.encodingStrategy = encodingStrategy;
this.compressionStrategy = compressionStrategy;
this.addBlockPadding = addBlockPadding;
this.blockSize = blockSizeValue;
this.paddingTolerance = paddingTolerance;
this.compress = compress;
this.rowIndexStride = rowIndexStride;
this.memoryManager = memoryManager;
buildIndex = rowIndexStride > 0;
codec = createCodec(compress);
String allColumns = conf.get(IOConstants.COLUMNS);
if (allColumns == null) {
allColumns = getColumnNamesFromInspector(inspector);
}
this.bufferSize = getEstimatedBufferSize(allColumns, bufferSize);
if (version == OrcFile.Version.V_0_11) {
/* do not write bloom filters for ORC v11 */
this.bloomFilterColumns =
OrcUtils.includeColumns(null, allColumns, inspector);
} else {
this.bloomFilterColumns =
OrcUtils.includeColumns(bloomFilterColumnNames, allColumns, inspector);
}
this.bloomFilterFpp = bloomFilterFpp;
treeWriter = createTreeWriter(inspector, streamFactory, false);
if (buildIndex && rowIndexStride < MIN_ROW_INDEX_STRIDE) {
throw new IllegalArgumentException("Row stride must be at least " +
MIN_ROW_INDEX_STRIDE);
}
// ensure that we are able to handle callbacks before we register ourselves
memoryManager.addWriter(path, stripeSize, this);
}
private String getColumnNamesFromInspector(ObjectInspector inspector) {
List<String> fieldNames = Lists.newArrayList();
Joiner joiner = Joiner.on(",");
if (inspector instanceof StructObjectInspector) {
StructObjectInspector soi = (StructObjectInspector) inspector;
List<? extends StructField> fields = soi.getAllStructFieldRefs();
fieldNames.addAll(fields.stream().map((Function<StructField, String>) StructField::getFieldName).collect(Collectors.toList()));
}
return joiner.join(fieldNames);
}
@VisibleForTesting
int getEstimatedBufferSize(int bs) {
return getEstimatedBufferSize(conf.get(IOConstants.COLUMNS), bs);
}
int getEstimatedBufferSize(String colNames, int bs) {
long availableMem = getMemoryAvailableForORC();
if (colNames != null) {
final int numCols = colNames.split(",").length;
if (numCols > COLUMN_COUNT_THRESHOLD) {
// In BufferedStream, there are 3 outstream buffers (compressed,
// uncompressed and overflow) and list of previously compressed buffers.
// Since overflow buffer is rarely used, lets consider only 2 allocation.
// Also, initially, the list of compression buffers will be empty.
final int outStreamBuffers = codec == null ? 1 : 2;
// max possible streams per column is 5. For string columns, there is
// ROW_INDEX, PRESENT, DATA, LENGTH, DICTIONARY_DATA streams.
final int maxStreams = 5;
// Lets assume 10% memory for holding dictionary in memory and other
// object allocations
final long miscAllocation = (long) (0.1f * availableMem);
// compute the available memory
final long remainingMem = availableMem - miscAllocation;
int estBufferSize = (int) (remainingMem / (maxStreams * outStreamBuffers * numCols));
estBufferSize = getClosestBufferSize(estBufferSize, bs);
if (estBufferSize > bs) {
estBufferSize = bs;
}
LOG.info("WIDE TABLE - Number of columns: " + numCols + " Chosen compression buffer size: " + estBufferSize);
return estBufferSize;
}
}
return bs;
}
private int getClosestBufferSize(int estBufferSize, int bs) {
final int kb4 = 4 * 1024;
final int kb8 = 8 * 1024;
final int kb16 = 16 * 1024;
final int kb32 = 32 * 1024;
final int kb64 = 64 * 1024;
final int kb128 = 128 * 1024;
final int kb256 = 256 * 1024;
if (estBufferSize <= kb4) {
return kb4;
} else if (estBufferSize > kb4 && estBufferSize <= kb8) {
return kb8;
} else if (estBufferSize > kb8 && estBufferSize <= kb16) {
return kb16;
} else if (estBufferSize > kb16 && estBufferSize <= kb32) {
return kb32;
} else if (estBufferSize > kb32 && estBufferSize <= kb64) {
return kb64;
} else if (estBufferSize > kb64 && estBufferSize <= kb128) {
return kb128;
} else {
return kb256;
}
}
// the assumption is only one ORC writer open at a time, which holds true for
// most of the cases. HIVE-6455 forces single writer case.
private long getMemoryAvailableForORC() {
HiveConf.ConfVars poolVar = HiveConf.ConfVars.HIVE_ORC_FILE_MEMORY_POOL;
double maxLoad = conf.getFloat(poolVar.varname, poolVar.defaultFloatVal);
long totalMemoryPool = Math.round(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax() * maxLoad);
return totalMemoryPool;
}
public static CompressionCodec createCodec(CompressionKind kind) {
switch (kind) {
case NONE:
return null;
case ZLIB:
return new ZlibCodec();
case SNAPPY:
return new SnappyCodec();
case LZO:
try {
Class<? extends CompressionCodec> lzo =
(Class<? extends CompressionCodec>)
JavaUtils.loadClass("org.apache.hadoop.hive.ql.io.orc.LzoCodec");
return lzo.newInstance();
} catch (ClassNotFoundException e) {
throw new IllegalArgumentException("LZO is not available.", e);
} catch (InstantiationException e) {
throw new IllegalArgumentException("Problem initializing LZO", e);
} catch (IllegalAccessException e) {
throw new IllegalArgumentException("Insufficient access to LZO", e);
}
default:
throw new IllegalArgumentException("Unknown compression codec: " +
kind);
}
}
@Override
public synchronized boolean checkMemory(double newScale) throws IOException {
long limit = Math.round(adjustedStripeSize * newScale);
long size = estimateStripeSize();
if (LOG.isDebugEnabled()) {
LOG.debug("ORC writer " + path + " size = " + size + " limit = " +
limit);
}
if (size > limit) {
flushStripe();
return true;
}
return false;
}
/**
* This class is used to hold the contents of streams as they are buffered.
* The TreeWriters write to the outStream and the codec compresses the
* data as buffers fill up and stores them in the output list. When the
* stripe is being written, the whole stream is written to the file.
*/
private class BufferedStream implements OutStream.OutputReceiver {
private final OutStream outStream;
private final List<ByteBuffer> output = new ArrayList<ByteBuffer>();
BufferedStream(String name, int bufferSize,
CompressionCodec codec) throws IOException {
outStream = new OutStream(name, bufferSize, codec, this);
}
/**
* Receive a buffer from the compression codec.
*
* @param buffer the buffer to save
* @throws IOException if an error occurs while receiving a buffer
*/
@Override
public void output(ByteBuffer buffer) {
output.add(buffer);
}
/**
* Get the number of bytes in buffers that are allocated to this stream.
*
* @return number of bytes in buffers
*/
public long getBufferSize() {
long result = 0;
for (ByteBuffer buf : output) {
result += buf.capacity();
}
return outStream.getBufferSize() + result;
}
/**
* Flush the stream to the codec.
*
* @throws IOException if an error occurs while flushing the stream
*/
public void flush() throws IOException {
outStream.flush();
}
/**
* Clear all of the buffers.
*
* @throws IOException if an error occurs while clearing the buffers
*/
public void clear() throws IOException {
outStream.clear();
output.clear();
}
/**
* Check the state of suppress flag in output stream
*
* @return value of suppress flag
*/
public boolean isSuppressed() {
return outStream.isSuppressed();
}
/**
* Get the number of bytes that will be written to the output. Assumes
* the stream has already been flushed.
*
* @return the number of bytes
*/
public long getOutputSize() {
long result = 0;
for (ByteBuffer buffer : output) {
result += buffer.remaining();
}
return result;
}
/**
* Write the saved compressed buffers to the OutputStream.
*
* @param out the stream to write to
* @throws IOException if an error occurs during write
*/
void spillTo(OutputStream out) throws IOException {
for (ByteBuffer buffer : output) {
out.write(buffer.array(), buffer.arrayOffset() + buffer.position(),
buffer.remaining());
}
}
@Override
public String toString() {
return outStream.toString();
}
}
/**
* An output receiver that writes the ByteBuffers to the output stream
* as they are received.
*/
private class DirectStream implements OutStream.OutputReceiver {
private final OutputStream output;
DirectStream(OutputStream output) {
this.output = output;
}
@Override
public void output(ByteBuffer buffer) throws IOException {
output.write(buffer.array(), buffer.arrayOffset() + buffer.position(),
buffer.remaining());
}
}
private static class RowIndexPositionRecorder implements PositionRecorder {
private final OrcProto.RowIndexEntry.Builder builder;
RowIndexPositionRecorder(OrcProto.RowIndexEntry.Builder builder) {
this.builder = builder;
}
@Override
public void addPosition(long position) {
builder.addPositions(position);
}
}
/**
* Interface from the Writer to the TreeWriters. This limits the visibility
* that the TreeWriters have into the Writer.
*/
private class StreamFactory {
/**
* Create a stream to store part of a column.
*
* @param column the column id for the stream
* @param kind the kind of stream
* @return The output outStream that the section needs to be written to.
* @throws IOException if an error occurs while creating the stream
*/
public OutStream createStream(int column,
OrcProto.Stream.Kind kind
) throws IOException {
final StreamName name = new StreamName(column, kind);
final EnumSet<CompressionCodec.Modifier> modifiers;
switch (kind) {
case BLOOM_FILTER:
case DATA:
case DICTIONARY_DATA:
if (getCompressionStrategy() == CompressionStrategy.SPEED) {
modifiers = EnumSet.of(Modifier.FAST, Modifier.TEXT);
} else {
modifiers = EnumSet.of(Modifier.DEFAULT, Modifier.TEXT);
}
break;
case LENGTH:
case DICTIONARY_COUNT:
case PRESENT:
case ROW_INDEX:
case SECONDARY:
// easily compressed using the fastest modes
modifiers = EnumSet.of(Modifier.FASTEST, Modifier.BINARY);
break;
default:
LOG.warn("Missing ORC compression modifiers for " + kind);
modifiers = null;
break;
}
BufferedStream result = streams.get(name);
if (result == null) {
result = new BufferedStream(name.toString(), bufferSize,
codec == null ? codec : codec.modify(modifiers));
streams.put(name, result);
}
return result.outStream;
}
/**
* Get the next column id.
*
* @return a number from 0 to the number of columns - 1
*/
public int getNextColumnId() {
return columnCount++;
}
/**
* Get the current column id. After creating all tree writers this count should tell how many
* columns (including columns within nested complex objects) are created in total.
*
* @return current column id
*/
public int getCurrentColumnId() {
return columnCount;
}
/**
* Get the stride rate of the row index.
*/
public int getRowIndexStride() {
return rowIndexStride;
}
/**
* Should be building the row index.
*
* @return true if we are building the index
*/
public boolean buildIndex() {
return buildIndex;
}
/**
* Is the ORC file compressed?
*
* @return are the streams compressed
*/
public boolean isCompressed() {
return codec != null;
}
/**
* Get the encoding strategy to use.
*
* @return encoding strategy
*/
public EncodingStrategy getEncodingStrategy() {
return encodingStrategy;
}
/**
* Get the compression strategy to use.
*
* @return compression strategy
*/
public CompressionStrategy getCompressionStrategy() {
return compressionStrategy;
}
/**
* Get the bloom filter columns
*
* @return bloom filter columns
*/
public boolean[] getBloomFilterColumns() {
return bloomFilterColumns;
}
/**
* Get bloom filter false positive percentage.
*
* @return fpp
*/
public double getBloomFilterFPP() {
return bloomFilterFpp;
}
/**
* Get the writer's configuration.
*
* @return configuration
*/
public Configuration getConfiguration() {
return conf;
}
/**
* Get the version of the file to write.
*/
public OrcFile.Version getVersion() {
return version;
}
}
/**
* The parent class of all of the writers for each column. Each column
* is written by an instance of this class. The compound types (struct,
* list, map, and union) have children tree writers that write the children
* types.
*/
private abstract static class TreeWriter {
protected final int id;
protected final ObjectInspector inspector;
private final BitFieldWriter isPresent;
private final boolean isCompressed;
protected final ColumnStatisticsImpl indexStatistics;
protected final ColumnStatisticsImpl stripeColStatistics;
private final ColumnStatisticsImpl fileStatistics;
protected TreeWriter[] childrenWriters;
protected final RowIndexPositionRecorder rowIndexPosition;
private final OrcProto.RowIndex.Builder rowIndex;
private final OrcProto.RowIndexEntry.Builder rowIndexEntry;
private final PositionedOutputStream rowIndexStream;
private final PositionedOutputStream bloomFilterStream;
protected final BloomFilterIO bloomFilter;
protected final boolean createBloomFilter;
private final OrcProto.BloomFilterIndex.Builder bloomFilterIndex;
private final OrcProto.BloomFilter.Builder bloomFilterEntry;
private boolean foundNulls;
private OutStream isPresentOutStream;
private final List<StripeStatistics.Builder> stripeStatsBuilders;
/**
* Create a tree writer.
*
* @param columnId the column id of the column to write
* @param inspector the object inspector to use
* @param streamFactory limited access to the Writer's data.
* @param nullable can the value be null?
* @throws IOException if an error occurs during creation
*/
TreeWriter(int columnId, ObjectInspector inspector,
StreamFactory streamFactory,
boolean nullable) throws IOException {
this.isCompressed = streamFactory.isCompressed();
this.id = columnId;
this.inspector = inspector;
if (nullable) {
isPresentOutStream = streamFactory.createStream(id,
OrcProto.Stream.Kind.PRESENT);
isPresent = new BitFieldWriter(isPresentOutStream, 1);
} else {
isPresent = null;
}
this.foundNulls = false;
createBloomFilter = streamFactory.getBloomFilterColumns()[columnId];
indexStatistics = ColumnStatisticsImpl.create(inspector);
stripeColStatistics = ColumnStatisticsImpl.create(inspector);
fileStatistics = ColumnStatisticsImpl.create(inspector);
childrenWriters = new TreeWriter[0];
rowIndex = OrcProto.RowIndex.newBuilder();
rowIndexEntry = OrcProto.RowIndexEntry.newBuilder();
rowIndexPosition = new RowIndexPositionRecorder(rowIndexEntry);
stripeStatsBuilders = Lists.newArrayList();
if (streamFactory.buildIndex()) {
rowIndexStream = streamFactory.createStream(id, OrcProto.Stream.Kind.ROW_INDEX);
} else {
rowIndexStream = null;
}
if (createBloomFilter) {
bloomFilterEntry = OrcProto.BloomFilter.newBuilder();
bloomFilterIndex = OrcProto.BloomFilterIndex.newBuilder();
bloomFilterStream = streamFactory.createStream(id, OrcProto.Stream.Kind.BLOOM_FILTER);
bloomFilter = new BloomFilterIO(streamFactory.getRowIndexStride(),
streamFactory.getBloomFilterFPP());
} else {
bloomFilterEntry = null;
bloomFilterIndex = null;
bloomFilterStream = null;
bloomFilter = null;
}
}
protected OrcProto.RowIndex.Builder getRowIndex() {
return rowIndex;
}
protected ColumnStatisticsImpl getStripeStatistics() {
return stripeColStatistics;
}
protected ColumnStatisticsImpl getFileStatistics() {
return fileStatistics;
}
protected OrcProto.RowIndexEntry.Builder getRowIndexEntry() {
return rowIndexEntry;
}
IntegerWriter createIntegerWriter(PositionedOutputStream output,
boolean signed, boolean isDirectV2,
StreamFactory writer) {
if (isDirectV2) {
boolean alignedBitpacking = false;
if (writer.getEncodingStrategy().equals(EncodingStrategy.SPEED)) {
alignedBitpacking = true;
}
return new RunLengthIntegerWriterV2(output, signed, alignedBitpacking);
} else {
return new RunLengthIntegerWriter(output, signed);
}
}
boolean isNewWriteFormat(StreamFactory writer) {
return writer.getVersion() != OrcFile.Version.V_0_11;
}
/**
* Add a new value to the column.
*
* @param obj The value to write to the column
* @throws IOException if an error occurs during add
*/
void write(Object obj) throws IOException {
if (obj != null) {
indexStatistics.increment();
} else {
indexStatistics.setNull();
}
if (isPresent != null) {
isPresent.write(obj == null ? 0 : 1);
if (obj == null) {
foundNulls = true;
}
}
}
private void removeIsPresentPositions() {
for (int i = 0; i < rowIndex.getEntryCount(); ++i) {
RowIndexEntry.Builder entry = rowIndex.getEntryBuilder(i);
List<Long> positions = entry.getPositionsList();
// bit streams use 3 positions if uncompressed, 4 if compressed
positions = positions.subList(isCompressed ? 4 : 3, positions.size());
entry.clearPositions();
entry.addAllPositions(positions);
}
}
/**
* Write the stripe out to the file.
*
* @param builder the stripe footer that contains the information about the
* layout of the stripe. The TreeWriter is required to update
* the footer with its information.
* @param requiredIndexEntries the number of index entries that are
* required. this is to check to make sure the
* row index is well formed.
* @throws IOException if an error occurs during write
*/
void writeStripe(OrcProto.StripeFooter.Builder builder,
int requiredIndexEntries) throws IOException {
if (isPresent != null) {
isPresent.flush();
// if no nulls are found in a stream, then suppress the stream
if (!foundNulls) {
isPresentOutStream.suppress();
// since isPresent bitstream is suppressed, update the index to
// remove the positions of the isPresent stream
if (rowIndexStream != null) {
removeIsPresentPositions();
}
}
}
// merge stripe-level column statistics to file statistics and write it to
// stripe statistics
OrcProto.StripeStatistics.Builder stripeStatsBuilder = OrcProto.StripeStatistics.newBuilder();
writeStripeStatistics(stripeStatsBuilder, this);
stripeStatsBuilders.add(stripeStatsBuilder);
// reset the flag for next stripe
foundNulls = false;
builder.addColumns(getEncoding());
builder.setWriterTimezone(TimeZone.getDefault().getID());
if (rowIndexStream != null) {
if (rowIndex.getEntryCount() != requiredIndexEntries) {
throw new IllegalArgumentException("Column has wrong number of " +
"index entries found: " + rowIndex.getEntryCount() + " expected: " +
requiredIndexEntries);
}
rowIndex.build().writeTo(rowIndexStream);
rowIndexStream.flush();
}
rowIndex.clear();
rowIndexEntry.clear();
// write the bloom filter to out stream
if (bloomFilterStream != null) {
bloomFilterIndex.build().writeTo(bloomFilterStream);
bloomFilterStream.flush();
bloomFilterIndex.clear();
bloomFilterEntry.clear();
}
}
private void writeStripeStatistics(OrcProto.StripeStatistics.Builder builder,
TreeWriter treeWriter) {
treeWriter.fileStatistics.merge(treeWriter.stripeColStatistics);
builder.addColStats(treeWriter.stripeColStatistics.serialize().build());
treeWriter.stripeColStatistics.reset();
for (TreeWriter child : treeWriter.getChildrenWriters()) {
writeStripeStatistics(builder, child);
}
}
TreeWriter[] getChildrenWriters() {
return childrenWriters;
}
/**
* Get the encoding for this column.
*
* @return the information about the encoding of this column
*/
OrcProto.ColumnEncoding getEncoding() {
return OrcProto.ColumnEncoding.newBuilder().setKind(
OrcProto.ColumnEncoding.Kind.DIRECT).build();
}
/**
* Create a row index entry with the previous location and the current
* index statistics. Also merges the index statistics into the file
* statistics before they are cleared. Finally, it records the start of the
* next index and ensures all of the children columns also create an entry.
*
* @throws IOException if an error occurs during create
*/
void createRowIndexEntry() throws IOException {
stripeColStatistics.merge(indexStatistics);
rowIndexEntry.setStatistics(indexStatistics.serialize());
indexStatistics.reset();
rowIndex.addEntry(rowIndexEntry);
rowIndexEntry.clear();
addBloomFilterEntry();
recordPosition(rowIndexPosition);
for (TreeWriter child : childrenWriters) {
child.createRowIndexEntry();
}
}
void addBloomFilterEntry() {
if (createBloomFilter) {
bloomFilterEntry.setNumHashFunctions(bloomFilter.getNumHashFunctions());
bloomFilterEntry.addAllBitset(Longs.asList(bloomFilter.getBitSet()));
bloomFilterIndex.addBloomFilter(bloomFilterEntry.build());
bloomFilter.reset();
bloomFilterEntry.clear();
}
}
/**
* Record the current position in each of this column's streams.
*
* @param recorder where should the locations be recorded
* @throws IOException if an error occurs during position recording
*/
void recordPosition(PositionRecorder recorder) throws IOException {
if (isPresent != null) {
isPresent.getPosition(recorder);
}
}
/**
* Estimate how much memory the writer is consuming excluding the streams.
*
* @return the number of bytes.
*/
long estimateMemory() {
long result = 0;
for (TreeWriter child : childrenWriters) {
result += child.estimateMemory();
}
return result;
}
/**
* Handle the top level object write.
*
* This default method is used for all types except structs, which are the
* typical case. VectorizedRowBatch assumes the top level object is a
* struct, so we use the first column for all other types.
* @param batch the batch to write from
* @param offset the row to start on
* @param length the number of rows to write
* @throws IOException if an error occurs during write
*/
void writeRootBatch(VectorizedRowBatch batch, int offset,
int length) throws IOException {
writeBatch(batch.cols[0], offset, length);
}
/**
* Write the values from the given vector from offset for length elements.
* @param vector the vector to write from
* @param offset the first value from the vector to write
* @param length the number of values from the vector to write
* @throws IOException if an error occurs during write
*/
void writeBatch(ColumnVector vector, int offset,
int length) throws IOException {
if (vector.noNulls) {
if (isPresent != null) {
for (int i = 0; i < length; ++i) {
isPresent.write(1);
indexStatistics.increment();
}
}
} else {
if (vector.isRepeating) {
boolean isNull = vector.isNull[0];
if (isPresent != null) {
for (int i = 0; i < length; ++i) {
isPresent.write(isNull ? 0 : 1);
}
}
if (isNull) {
foundNulls = true;
indexStatistics.setNull();
} else {
indexStatistics.increment();
}
} else {
// count the number of non-null values
int nonNullCount = 0;
for(int i = 0; i < length; ++i) {
boolean isNull = vector.isNull[i + offset];
if (!isNull) {
nonNullCount += 1;
}
if (isPresent != null) {
isPresent.write(isNull ? 0 : 1);
}
}
indexStatistics.increment();
if (nonNullCount != length) {
foundNulls = true;
indexStatistics.setNull();
}
}
}
}
}
private static class BooleanTreeWriter extends TreeWriter {
private final BitFieldWriter writer;
BooleanTreeWriter(int columnId,
ObjectInspector inspector,
StreamFactory writer,
boolean nullable) throws IOException {
super(columnId, inspector, writer, nullable);
PositionedOutputStream out = writer.createStream(id,
OrcProto.Stream.Kind.DATA);
this.writer = new BitFieldWriter(out, 1);
recordPosition(rowIndexPosition);
}
@Override
void write(Object obj) throws IOException {
super.write(obj);
if (obj != null) {
boolean val = ((BooleanObjectInspector) inspector).get(obj);
indexStatistics.updateBoolean(val);
writer.write(val ? 1 : 0);
}
}
@Override
void writeStripe(OrcProto.StripeFooter.Builder builder,
int requiredIndexEntries) throws IOException {