drill-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From vdiravka <...@git.apache.org>
Subject [GitHub] drill pull request #846: DRILL-5544: Out of heap running CTAS against text d...
Date Tue, 06 Jun 2017 16:10:02 GMT
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/846#discussion_r119836892
  
    --- Diff: exec/java-exec/src/main/java/org/apache/parquet/hadoop/ParquetColumnChunkPageWriteStore.java
---
    @@ -0,0 +1,269 @@
    +/*
    + * 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.parquet.hadoop;
    +
    +import static org.apache.parquet.column.statistics.Statistics.getStatsBasedOnType;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.Sets;
    +import org.apache.drill.exec.store.parquet.ParquetDirectByteBufferAllocator;
    +import org.apache.parquet.bytes.BytesInput;
    +import org.apache.parquet.bytes.CapacityByteArrayOutputStream;
    +import org.apache.parquet.column.ColumnDescriptor;
    +import org.apache.parquet.column.Encoding;
    +import org.apache.parquet.column.page.DictionaryPage;
    +import org.apache.parquet.column.page.PageWriteStore;
    +import org.apache.parquet.column.page.PageWriter;
    +import org.apache.parquet.column.statistics.Statistics;
    +import org.apache.parquet.format.converter.ParquetMetadataConverter;
    +import org.apache.parquet.hadoop.CodecFactory.BytesCompressor;
    +import org.apache.parquet.io.ParquetEncodingException;
    +import org.apache.parquet.schema.MessageType;
    +import org.apache.parquet.bytes.ByteBufferAllocator;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This is a copy of ColumnChunkPageWriteStore from parquet library except of OutputStream
that is used here.
    + * Using of CapacityByteArrayOutputStream allows to use different ByteBuffer allocators.
    + * It will be no need in this class once PARQUET-1006 is resolved.
    + */
    +public class ParquetColumnChunkPageWriteStore implements PageWriteStore, Closeable {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetDirectByteBufferAllocator.class);
    +
    +  private static ParquetMetadataConverter parquetMetadataConverter = new ParquetMetadataConverter();
    +
    +  private final Map<ColumnDescriptor, ColumnChunkPageWriter> writers = Maps.newHashMap();
    +  private final MessageType schema;
    +
    +  public ParquetColumnChunkPageWriteStore(BytesCompressor compressor,
    +                                          MessageType schema,
    +                                          int initialSlabSize,
    +                                          int maxCapacityHint,
    +                                          ByteBufferAllocator allocator) {
    +    this.schema = schema;
    +    for (ColumnDescriptor path : schema.getColumns()) {
    +      writers.put(path,  new ColumnChunkPageWriter(path, compressor, initialSlabSize,
maxCapacityHint, allocator));
    +    }
    +  }
    +
    +  @Override
    +  public PageWriter getPageWriter(ColumnDescriptor path) {
    +    return writers.get(path);
    +  }
    +
    +  public void flushToFileWriter(ParquetFileWriter writer) throws IOException {
    +    for (ColumnDescriptor path : schema.getColumns()) {
    +      ColumnChunkPageWriter pageWriter = writers.get(path);
    +      pageWriter.writeToFileWriter(writer);
    +    }
    +  }
    +
    +  @Override
    +  public void close() {
    +    for (ColumnChunkPageWriter pageWriter : writers.values()) {
    +      pageWriter.close();
    +    }
    +  }
    +
    +  private static final class ColumnChunkPageWriter implements PageWriter, Closeable {
    +
    +    private final ColumnDescriptor path;
    +    private final BytesCompressor compressor;
    +
    +    private final CapacityByteArrayOutputStream buf;
    +    private DictionaryPage dictionaryPage;
    +
    +    private long uncompressedLength;
    +    private long compressedLength;
    +    private long totalValueCount;
    +    private int pageCount;
    +
    +    // repetition and definition level encodings are used only for v1 pages and don't
change
    +    private Set<Encoding> rlEncodings = Sets.newHashSet();
    +    private Set<Encoding> dlEncodings = Sets.newHashSet();
    +    private List<Encoding> dataEncodings = Lists.newArrayList();
    +
    +    private Statistics totalStatistics;
    +
    +    private ColumnChunkPageWriter(ColumnDescriptor path,
    +                                  BytesCompressor compressor,
    +                                  int initialSlabSize,
    +                                  int maxCapacityHint,
    +                                  ByteBufferAllocator allocator) {
    +      this.path = path;
    +      this.compressor = compressor;
    +      this.buf = new CapacityByteArrayOutputStream(initialSlabSize, maxCapacityHint,
allocator);
    +      this.totalStatistics = getStatsBasedOnType(this.path.getType());
    +    }
    +
    +    @Override
    +    public void writePage(BytesInput bytes,
    +                          int valueCount,
    +                          Statistics statistics,
    +                          Encoding rlEncoding,
    +                          Encoding dlEncoding,
    +                          Encoding valuesEncoding) throws IOException {
    +      long uncompressedSize = bytes.size();
    +      if (uncompressedSize > Integer.MAX_VALUE) {
    --- End diff --
    
    We should throw an exception here, because parquet creates bad metadata if the uncompressed
or compressed size of a page exceeds Integer.MAX_VALUE.
    But with Drill we can't get there with default options for parquet (PARQUET_BLOCK_SIZE,
PARQUET_PAGE_SIZE, MINIMUM_RECORD_COUNT_FOR_CHECK).
    The page size is controlled by the method [ColumnWriterV1.accountForValueWritten()](https://github.com/apache/parquet-mr/blob/70f28810a5547219e18ffc3465f519c454fee6e5/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java#L95).
According to it once page size is bigger than the page size option value (1MB - default in
Drill) the page will be written.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

Mime
View raw message