You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2021/04/14 17:52:42 UTC

[GitHub] [drill] vdiravka commented on a change in pull request #2143: DRILL-7825: Unknown logical type in Parquet

vdiravka commented on a change in pull request #2143:
URL: https://github.com/apache/drill/pull/2143#discussion_r613456253



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
##########
@@ -328,24 +329,26 @@ protected PrimitiveConverter getConverterForType(String name, PrimitiveType type
         }
       }
       case FIXED_LEN_BYTE_ARRAY:
-        switch (type.getOriginalType()) {
-          case DECIMAL: {
-            ParquetReaderUtility.checkDecimalTypeEnabled(options);
-            return getVarDecimalConverter(name, type);
-          }
-          case INTERVAL: {
-            IntervalWriter writer = type.isRepetition(Repetition.REPEATED)
-                ? getWriter(name, (m, f) -> m.list(f).interval(), l -> l.list().interval())
-                : getWriter(name, (m, f) -> m.interval(f), l -> l.interval());
-            return new DrillFixedLengthByteArrayToInterval(writer);
-          }
-          default: {
-            VarBinaryWriter writer = type.isRepetition(Repetition.REPEATED)
-                ? getWriter(name, (m, f) -> m.list(f).varBinary(), l -> l.list().varBinary())
-                : getWriter(name, (m, f) -> m.varBinary(f), l -> l.varBinary());
-            return new DrillFixedBinaryToVarbinaryConverter(writer, type.getTypeLength(), mutator.getManagedBuffer());
+        // TODO: to follow the latest parquet code, rewrite it by using LogicalTypeAnnotation instead of OriginalType
+        OriginalType originalType = type.getOriginalType();
+        if( originalType != null) {
+          switch (type.getOriginalType()) {
+            case DECIMAL: {
+              ParquetReaderUtility.checkDecimalTypeEnabled(options);
+              return getVarDecimalConverter(name, type);
+            }
+            case INTERVAL: {
+              IntervalWriter writer = type.isRepetition(Repetition.REPEATED)
+                      ? getWriter(name, (m, f) -> m.list(f).interval(), l -> l.list().interval())
+                      : getWriter(name, (m, f) -> m.interval(f), l -> l.interval());
+              return new DrillFixedLengthByteArrayToInterval(writer);
+            }
           }
         }
+        VarBinaryWriter writer = type.isRepetition(Repetition.REPEATED)
+                ? getWriter(name, (m, f) -> m.list(f).varBinary(), l -> l.list().varBinary())
+                : getWriter(name, MapWriter::varBinary, ListWriter::varBinary);
+        return new DrillFixedBinaryToVarbinaryConverter(writer, type.getTypeLength(), mutator.getManagedBuffer());

Review comment:
       ok, will do

##########
File path: pom.xml
##########
@@ -47,9 +47,9 @@
     <junit.version>4.12</junit.version>
     <slf4j.version>1.7.26</slf4j.version>
     <shaded.guava.version>28.2-jre</shaded.guava.version>
-    <guava.version>19.0</guava.version>
+    <guava.version>19.0</guava.version> <!--todo: 28.2-jre guava can be used here-->

Review comment:
       ok. Yes, I know about newer version. But the newer version can bring some new other issues for sure.
   The plan to check `<guava.version>28.2-jre</guava.version>` at first. If it is fine then to drop `<shaded.guava.version>` at all. And then to update guava version to the newest one

##########
File path: exec/jdbc-all/pom.xml
##########
@@ -575,7 +575,7 @@
 
         <build>
           <plugins>
-            <plugin>
+            <plugin> <!-- TODO: this plugin has common things with default profile. Factor out this common things to avoid duplicate code -->

Review comment:
       ok :)

##########
File path: exec/java-exec/src/main/java/org/apache/parquet/hadoop/ParquetColumnChunkPageWriteStore.java
##########
@@ -260,14 +260,16 @@ public long getMemSize() {
     }
 
     /**
-     * Writes a number of pages within corresponding column chunk
+     * Writes a number of pages within corresponding column chunk <br>
+     * // TODO: the Bloom Filter can be useful in filtering entire row groups,
+     *     see <a href="https://issues.apache.org/jira/browse/DRILL-7895">DRILL-7895</a>

Review comment:
       Yes, you are right. The best way to remove Drill's copy of this class, we have `todo` about in `ParquetRecordWriter#256`, but we can't to it before [PARQUET-1006](https://issues.apache.org/jira/browse/PARQUET-1006) resolving. Adding the latest functionality from Parquet version of this class requires some deeper involving into it (it requires proper instantiating of `ParquetColumnChunkPageWriteStore`) and doesn't related to `UUID` logical type. That's why DRILL-7895 is created

##########
File path: exec/java-exec/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
##########
@@ -0,0 +1,1633 @@
+/*
+ * 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.format.Util.writeFileCryptoMetaData;
+import static org.apache.parquet.format.Util.writeFileMetaData;
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.MAX_STATS_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.zip.CRC32;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.Version;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.EncodingStats;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.ColumnEncryptionProperties;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.crypto.ModuleCipherFactory;
+import org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.apache.parquet.hadoop.ParquetOutputFormat.JobSummaryLevel;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.Util;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.StrictKeyValueMetadataMergeStrategy;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.metadata.GlobalMetaData;
+import org.apache.parquet.hadoop.metadata.KeyValueMetadataMergeStrategy;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.hadoop.util.HadoopStreams;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.ColumnIndexBuilder;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndexBuilder;
+import org.apache.parquet.internal.hadoop.metadata.IndexReference;
+import org.apache.parquet.io.InputFile;
+import org.apache.parquet.io.OutputFile;
+import org.apache.parquet.io.SeekableInputStream;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.PositionOutputStream;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.TypeUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Internal implementation of the Parquet file writer as a block container<br>
+ * Note: this is temporary Drill-Parquet class needed to write empty parquet files. Details in
+ * <a href="https://issues.apache.org/jira/browse/PARQUET-2026">PARQUET-2026</a>
+ */
+public class ParquetFileWriter {

Review comment:
       I wanted and tried to create a class extended from an original one and override `endBlock()` method. But unfortunately there are several private fields, which can be used only in scope of original class. So extending can't help. Anyway it is temporary solution, since the ticket to revisit dropping empty files is created and even if it will be rejected I'll create other ticket to make `endBlock()` be possible to be overridden. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org