@Override protected ByteArrayOutputStream createValueOutputStream() { return new ByteArrayOutputStream(SNAPPY, COMPRESSION_BLOCK_SIZE); }
@Override protected void writeValue(ByteArrayOutputStream outputStream, Slice value) { outputStream.writeSlice(value); }
@Override public void close() { checkState(!closed); closed = true; lengthStream.close(); dataStream.close(); presentStream.close(); }
dictionaryLengthStream.writeLong(length); Slice value = dictionaryElements.getSlice(sortedDictionaryIndex, 0, length); dictionaryDataStream.writeSlice(value); dictionaryDataStream.close(); dictionaryLengthStream.close();
@Override public List<StreamDataOutput> getDataStreams() { checkState(closed); ImmutableList.Builder<StreamDataOutput> outputDataStreams = ImmutableList.builder(); presentStream.getStreamDataOutput(column).ifPresent(outputDataStreams::add); outputDataStreams.add(lengthStream.getStreamDataOutput(column)); outputDataStreams.add(dataStream.getStreamDataOutput(column)); return outputDataStreams.build(); }
@Override public long getRetainedBytes() { long retainedBytes = INSTANCE_SIZE + lengthStream.getRetainedBytes() + dataStream.getRetainedBytes() + presentStream.getRetainedBytes(); for (ColumnStatistics statistics : rowGroupColumnStatistics) { retainedBytes += statistics.getRetainedSizeInBytes(); } return retainedBytes; }
@Override public void reset() { checkState(closed); closed = false; lengthStream.reset(); dataStream.reset(); presentStream.reset(); rowGroupColumnStatistics.clear(); statisticsBuilder = statisticsBuilderSupplier.get(); } }
@Override public void beginRowGroup() { checkState(!closed); presentStream.recordCheckpoint(); lengthStream.recordCheckpoint(); dataStream.recordCheckpoint(); }
@Override public long getBufferedBytes() { return lengthStream.getBufferedBytes() + dataStream.getBufferedBytes() + presentStream.getBufferedBytes(); }
@Override public List<StreamDataOutput> getIndexStreams(CompressedMetadataWriter metadataWriter) throws IOException { checkState(closed); ImmutableList.Builder<RowGroupIndex> rowGroupIndexes = ImmutableList.builder(); List<LongStreamCheckpoint> lengthCheckpoints = lengthStream.getCheckpoints(); List<ByteArrayStreamCheckpoint> dataCheckpoints = dataStream.getCheckpoints(); Optional<List<BooleanStreamCheckpoint>> presentCheckpoints = presentStream.getCheckpoints(); for (int i = 0; i < rowGroupColumnStatistics.size(); i++) { int groupId = i; ColumnStatistics columnStatistics = rowGroupColumnStatistics.get(groupId); LongStreamCheckpoint lengthCheckpoint = lengthCheckpoints.get(groupId); ByteArrayStreamCheckpoint dataCheckpoint = dataCheckpoints.get(groupId); Optional<BooleanStreamCheckpoint> presentCheckpoint = presentCheckpoints.map(checkpoints -> checkpoints.get(groupId)); List<Integer> positions = createSliceColumnPositionList(compressed, lengthCheckpoint, dataCheckpoint, presentCheckpoint); rowGroupIndexes.add(new RowGroupIndex(positions, columnStatistics)); } Slice slice = metadataWriter.writeRowIndexes(rowGroupIndexes.build()); Stream stream = new Stream(column, StreamKind.ROW_INDEX, slice.length(), false); return ImmutableList.of(new StreamDataOutput(slice, stream)); }
dictionaryLengthStream.writeLong(length); Slice value = dictionaryElements.getSlice(sortedDictionaryIndex, 0, length); dictionaryDataStream.writeSlice(value); dictionaryDataStream.close(); dictionaryLengthStream.close();
@Override public List<StreamDataOutput> getDataStreams() { checkState(closed); if (directEncoded) { return directColumnWriter.getDataStreams(); } // actually write data ImmutableList.Builder<StreamDataOutput> outputDataStreams = ImmutableList.builder(); presentStream.getStreamDataOutput(column).ifPresent(outputDataStreams::add); outputDataStreams.add(dataStream.getStreamDataOutput(column)); outputDataStreams.add(dictionaryLengthStream.getStreamDataOutput(column)); outputDataStreams.add(dictionaryDataStream.getStreamDataOutput(column)); return outputDataStreams.build(); }
@Override public long getRetainedBytes() { long retainedBytes = INSTANCE_SIZE + values.sizeOf() + dataStream.getRetainedBytes() + presentStream.getRetainedBytes() + dictionaryDataStream.getRetainedBytes() + dictionaryLengthStream.getRetainedBytes() + dictionary.getRetainedSizeInBytes() + (directColumnWriter == null ? 0 : directColumnWriter.getRetainedBytes()); for (DictionaryRowGroup rowGroup : rowGroups) { retainedBytes += rowGroup.getColumnStatistics().getRetainedSizeInBytes(); } return retainedBytes; }
@Override public void reset() { checkState(closed); closed = false; dataStream.reset(); presentStream.reset(); dictionaryDataStream.reset(); dictionaryLengthStream.reset(); rowGroups.clear(); rowGroupValueCount = 0; statisticsBuilder = newStringStatisticsBuilder(); columnEncoding = null; dictionary.clear(); rawBytes = 0; totalValueCount = 0; totalNonNullValueCount = 0; if (directEncoded) { directEncoded = false; directColumnWriter.reset(); } }
@Override public void beginRowGroup() { checkState(!closed); presentStream.recordCheckpoint(); lengthStream.recordCheckpoint(); dataStream.recordCheckpoint(); }
@Override public long getBufferedBytes() { return lengthStream.getBufferedBytes() + dataStream.getBufferedBytes() + presentStream.getBufferedBytes(); }
@Override public List<StreamDataOutput> getIndexStreams(CompressedMetadataWriter metadataWriter) throws IOException { checkState(closed); ImmutableList.Builder<RowGroupIndex> rowGroupIndexes = ImmutableList.builder(); List<LongStreamCheckpoint> lengthCheckpoints = lengthStream.getCheckpoints(); List<ByteArrayStreamCheckpoint> dataCheckpoints = dataStream.getCheckpoints(); Optional<List<BooleanStreamCheckpoint>> presentCheckpoints = presentStream.getCheckpoints(); for (int i = 0; i < rowGroupColumnStatistics.size(); i++) { int groupId = i; ColumnStatistics columnStatistics = rowGroupColumnStatistics.get(groupId); LongStreamCheckpoint lengthCheckpoint = lengthCheckpoints.get(groupId); ByteArrayStreamCheckpoint dataCheckpoint = dataCheckpoints.get(groupId); Optional<BooleanStreamCheckpoint> presentCheckpoint = presentCheckpoints.map(checkpoints -> checkpoints.get(groupId)); List<Integer> positions = createSliceColumnPositionList(compressed, lengthCheckpoint, dataCheckpoint, presentCheckpoint); rowGroupIndexes.add(new RowGroupIndex(positions, columnStatistics)); } Slice slice = metadataWriter.writeRowIndexes(rowGroupIndexes.build()); Stream stream = new Stream(column, StreamKind.ROW_INDEX, slice.length(), false); return ImmutableList.of(new StreamDataOutput(slice, stream)); }
public SliceDirectColumnWriter(int column, Type type, CompressionKind compression, int bufferSize, OrcEncoding orcEncoding, Supplier<SliceColumnStatisticsBuilder> statisticsBuilderSupplier) { checkArgument(column >= 0, "column is negative"); this.column = column; this.type = requireNonNull(type, "type is null"); this.compressed = requireNonNull(compression, "compression is null") != NONE; this.columnEncoding = new ColumnEncoding(orcEncoding == DWRF ? DIRECT : DIRECT_V2, 0); this.lengthStream = createLengthOutputStream(compression, bufferSize, orcEncoding); this.dataStream = new ByteArrayOutputStream(compression, bufferSize); this.presentStream = new PresentOutputStream(compression, bufferSize); this.statisticsBuilderSupplier = statisticsBuilderSupplier; statisticsBuilder = statisticsBuilderSupplier.get(); }
@Override public void writeBlock(Block block) { checkState(!closed); checkArgument(block.getPositionCount() > 0, "Block is empty"); // record nulls for (int position = 0; position < block.getPositionCount(); position++) { presentStream.writeBoolean(!block.isNull(position)); } // record values for (int position = 0; position < block.getPositionCount(); position++) { if (!block.isNull(position)) { Slice value = type.getSlice(block, position); lengthStream.writeLong(value.length()); dataStream.writeSlice(value); statisticsBuilder.addValue(value); } } }
@Override public List<StreamDataOutput> getDataStreams() { checkState(closed); ImmutableList.Builder<StreamDataOutput> outputDataStreams = ImmutableList.builder(); presentStream.getStreamDataOutput(column).ifPresent(outputDataStreams::add); outputDataStreams.add(lengthStream.getStreamDataOutput(column)); outputDataStreams.add(dataStream.getStreamDataOutput(column)); return outputDataStreams.build(); }