ListColumnWriter.java
/*
* Licensed 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 com.facebook.presto.orc.writer;
import com.facebook.presto.common.block.Block;
import com.facebook.presto.common.block.ColumnarArray;
import com.facebook.presto.orc.ColumnWriterOptions;
import com.facebook.presto.orc.DwrfDataEncryptor;
import com.facebook.presto.orc.OrcEncoding;
import com.facebook.presto.orc.checkpoint.StreamCheckpoint;
import com.facebook.presto.orc.metadata.ColumnEncoding;
import com.facebook.presto.orc.metadata.CompressedMetadataWriter;
import com.facebook.presto.orc.metadata.MetadataWriter;
import com.facebook.presto.orc.metadata.RowGroupIndex;
import com.facebook.presto.orc.metadata.Stream;
import com.facebook.presto.orc.metadata.Stream.StreamKind;
import com.facebook.presto.orc.metadata.statistics.ColumnStatistics;
import com.facebook.presto.orc.stream.LongOutputStream;
import com.facebook.presto.orc.stream.PresentOutputStream;
import com.facebook.presto.orc.stream.StreamDataOutput;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.airlift.slice.Slice;
import org.openjdk.jol.info.ClassLayout;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import static com.facebook.presto.common.block.ColumnarArray.toColumnarArray;
import static com.facebook.presto.orc.OrcEncoding.DWRF;
import static com.facebook.presto.orc.metadata.ColumnEncoding.ColumnEncodingKind.DIRECT;
import static com.facebook.presto.orc.metadata.ColumnEncoding.ColumnEncodingKind.DIRECT_V2;
import static com.facebook.presto.orc.metadata.CompressionKind.NONE;
import static com.facebook.presto.orc.stream.LongOutputStream.createLengthOutputStream;
import static com.facebook.presto.orc.writer.ColumnWriterUtils.buildRowGroupIndexes;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static java.util.Objects.requireNonNull;
public class ListColumnWriter
implements ColumnWriter
{
private static final int INSTANCE_SIZE = ClassLayout.parseClass(ListColumnWriter.class).instanceSize();
private final int column;
private final int sequence;
private final boolean compressed;
private final ColumnEncoding columnEncoding;
private final LongOutputStream lengthStream;
private final PresentOutputStream presentStream;
private final CompressedMetadataWriter metadataWriter;
private final ColumnWriter elementWriter;
private final List<ColumnStatistics> rowGroupColumnStatistics = new ArrayList<>();
private long columnStatisticsRetainedSizeInBytes;
private int nonNullValueCount;
private long rawSize;
private boolean closed;
public ListColumnWriter(
int column,
int sequence,
ColumnWriterOptions columnWriterOptions,
Optional<DwrfDataEncryptor> dwrfEncryptor,
OrcEncoding orcEncoding,
ColumnWriter elementWriter,
MetadataWriter metadataWriter)
{
checkArgument(column >= 0, "column is negative");
checkArgument(sequence >= 0, "sequence is negative");
requireNonNull(columnWriterOptions, "columnWriterOptions is null");
this.column = column;
this.sequence = sequence;
this.compressed = columnWriterOptions.getCompressionKind() != NONE;
this.columnEncoding = new ColumnEncoding(orcEncoding == DWRF ? DIRECT : DIRECT_V2, 0);
this.elementWriter = requireNonNull(elementWriter, "elementWriter is null");
this.lengthStream = createLengthOutputStream(columnWriterOptions, dwrfEncryptor, orcEncoding);
this.presentStream = new PresentOutputStream(columnWriterOptions, dwrfEncryptor);
this.metadataWriter = new CompressedMetadataWriter(metadataWriter, columnWriterOptions, dwrfEncryptor);
}
@Override
public List<ColumnWriter> getNestedColumnWriters()
{
return ImmutableList.<ColumnWriter>builder()
.add(elementWriter)
.addAll(elementWriter.getNestedColumnWriters())
.build();
}
@Override
public Map<Integer, ColumnEncoding> getColumnEncodings()
{
ImmutableMap.Builder<Integer, ColumnEncoding> encodings = ImmutableMap.builder();
encodings.put(column, columnEncoding);
encodings.putAll(elementWriter.getColumnEncodings());
return encodings.build();
}
@Override
public void beginRowGroup()
{
lengthStream.recordCheckpoint();
presentStream.recordCheckpoint();
elementWriter.beginRowGroup();
}
@Override
public long writeBlock(Block block)
{
checkState(!closed);
checkArgument(block.getPositionCount() > 0, "Block is empty");
ColumnarArray columnarArray = toColumnarArray(block);
return writeColumnarArray(columnarArray);
}
private long writeColumnarArray(ColumnarArray columnarArray)
{
// write nulls and lengths
int blockNonNullValueCount = 0;
long childRawSize = 0;
for (int position = 0; position < columnarArray.getPositionCount(); position++) {
boolean present = !columnarArray.isNull(position);
presentStream.writeBoolean(present);
if (present) {
blockNonNullValueCount++;
lengthStream.writeLong(columnarArray.getLength(position));
}
}
// write element values
Block elementsBlock = columnarArray.getElementsBlock();
if (elementsBlock.getPositionCount() > 0) {
childRawSize += elementWriter.writeBlock(elementsBlock);
}
nonNullValueCount += blockNonNullValueCount;
long rawSize = (columnarArray.getPositionCount() - blockNonNullValueCount) * NULL_SIZE + childRawSize;
this.rawSize += rawSize;
return rawSize;
}
@Override
public Map<Integer, ColumnStatistics> finishRowGroup()
{
checkState(!closed);
ColumnStatistics statistics = new ColumnStatistics((long) nonNullValueCount, null, rawSize, null);
rowGroupColumnStatistics.add(statistics);
columnStatisticsRetainedSizeInBytes += statistics.getRetainedSizeInBytes();
nonNullValueCount = 0;
rawSize = 0;
ImmutableMap.Builder<Integer, ColumnStatistics> columnStatistics = ImmutableMap.builder();
columnStatistics.put(column, statistics);
columnStatistics.putAll(elementWriter.finishRowGroup());
return columnStatistics.build();
}
@Override
public void close()
{
closed = true;
elementWriter.close();
lengthStream.close();
presentStream.close();
}
@Override
public Map<Integer, ColumnStatistics> getColumnStripeStatistics()
{
checkState(closed);
ImmutableMap.Builder<Integer, ColumnStatistics> columnStatistics = ImmutableMap.builder();
columnStatistics.put(column, ColumnStatistics.mergeColumnStatistics(rowGroupColumnStatistics));
columnStatistics.putAll(elementWriter.getColumnStripeStatistics());
return columnStatistics.build();
}
@Override
public List<StreamDataOutput> getIndexStreams(Optional<List<? extends StreamCheckpoint>> prependCheckpoints)
throws IOException
{
checkState(closed);
List<RowGroupIndex> rowGroupIndexes = buildRowGroupIndexes(compressed, rowGroupColumnStatistics, prependCheckpoints, presentStream, lengthStream);
Slice slice = metadataWriter.writeRowIndexes(rowGroupIndexes);
Stream stream = new Stream(column, sequence, StreamKind.ROW_INDEX, slice.length(), false);
ImmutableList.Builder<StreamDataOutput> indexStreams = ImmutableList.builder();
indexStreams.add(new StreamDataOutput(slice, stream));
indexStreams.addAll(elementWriter.getIndexStreams(Optional.empty()));
return indexStreams.build();
}
@Override
public List<StreamDataOutput> getDataStreams()
{
checkState(closed);
ImmutableList.Builder<StreamDataOutput> outputDataStreams = ImmutableList.builder();
presentStream.getStreamDataOutput(column, sequence).ifPresent(outputDataStreams::add);
outputDataStreams.add(lengthStream.getStreamDataOutput(column, sequence));
outputDataStreams.addAll(elementWriter.getDataStreams());
return outputDataStreams.build();
}
@Override
public long getBufferedBytes()
{
return lengthStream.getBufferedBytes() + presentStream.getBufferedBytes() + elementWriter.getBufferedBytes();
}
@Override
public long getRetainedBytes()
{
return INSTANCE_SIZE + lengthStream.getRetainedBytes() + presentStream.getRetainedBytes() + elementWriter.getRetainedBytes() + columnStatisticsRetainedSizeInBytes;
}
@Override
public void reset()
{
closed = false;
lengthStream.reset();
presentStream.reset();
elementWriter.reset();
rowGroupColumnStatistics.clear();
columnStatisticsRetainedSizeInBytes = 0;
nonNullValueCount = 0;
rawSize = 0;
}
}