[ 
https://issues.apache.org/jira/browse/DRILL-7825?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17320536#comment-17320536
 ] 

ASF GitHub Bot commented on DRILL-7825:
---------------------------------------

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



##########
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:
       Don't you want to implement this TODO? :) Looks like it may be a simple 
visitor (by the way outer switch case may be rewritten in a similar way).
   ```suggestion
           
LogicalTypeAnnotation.LogicalTypeAnnotationVisitor<PrimitiveConverter> 
typeAnnotationVisitor = new 
LogicalTypeAnnotation.LogicalTypeAnnotationVisitor<PrimitiveConverter>() {
             @Override
             public Optional<PrimitiveConverter> 
visit(LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType) {
               ParquetReaderUtility.checkDecimalTypeEnabled(options);
               return Optional.of(getVarDecimalConverter(name, type));
             }
   
             @Override
             public Optional<PrimitiveConverter> 
visit(LogicalTypeAnnotation.IntervalLogicalTypeAnnotation intervalLogicalType) {
               IntervalWriter writer = type.isRepetition(Repetition.REPEATED)
                   ? getWriter(name, (m, f) -> m.list(f).interval(), l -> 
l.list().interval())
                   : getWriter(name, MapWriter::interval, ListWriter::interval);
               return Optional.of(new 
DrillFixedLengthByteArrayToInterval(writer));
             }
           };
   
           LogicalTypeAnnotation logicalTypeAnnotation = 
type.getLogicalTypeAnnotation();
           if (logicalTypeAnnotation != null) {
             logicalTypeAnnotation.accept(typeAnnotationVisitor).orElseGet(() 
-> {
               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());
             });
           }
   ```

##########
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:
       Is it possible somehow to reuse the original ParquetFileWriter to avoid 
so large code duplication?

##########
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:
       This class was created as a copy of the ColumnChunkPageWriteStore class 
from the parquet library (see DRILL-5544 for details)
   
   Since it is a copy, it is better to sync it with the original version 
instead of adding TODO with adding some specific features from it...

##########
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:
       Could you please implement this TODO? It doesn't look to be complicated.

##########
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:
       Let's create a ticket instead of adding the comment. Also, there are 
newer versions of Guava.




-- 
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:
[email protected]


> Error: SYSTEM ERROR: RuntimeException: Unknown logical type <LogicalType 
> UUID:UUIDType()>
> -----------------------------------------------------------------------------------------
>
>                 Key: DRILL-7825
>                 URL: https://issues.apache.org/jira/browse/DRILL-7825
>             Project: Apache Drill
>          Issue Type: Bug
>          Components: Storage - Parquet
>    Affects Versions: 1.17.0
>         Environment: Windows 10 single local node.
>            Reporter: ian
>            Assignee: Vitalii Diravka
>            Priority: Critical
>             Fix For: 1.19.0
>
>         Attachments: uuid-simple-fixed-length-array.parquet, uuid.parquet
>
>
> Parquet logical type UUID fails on read.  Only workaround is to store as 
> text, a 125% penalty. 
> Here is the schema dump for the attached test parquet file.  I can read the 
> file okay from R and natively through C++.
> {code:java}
> 3961 $ parquet-dump-schema uuid.parquet
> required group field_id=0 schema {
>  required fixed_len_byte_array(16) field_id=1 uuid_req1 (UUID);
>  optional fixed_len_byte_array(16) field_id=2 uuid_opt1 (UUID);
>  required fixed_len_byte_array(16) field_id=3 uuid_req2 (UUID);
> }{code}
> UPDATE:  I tested with a simple fixed binary column, and received the 
> following error.
> See second attached uuid-simple-fixed-length-array.parquet.
>  
> {code:java}
> org.apache.drill.common.exceptions.UserRemoteException: INTERNAL_ERROR ERROR: 
> Error in drill parquet reader (complex).
> Message: Failure in setting up reader
> Parquet Metadata: null
> Fragment: 0:0
> Please, refer to logs for more information.
> [Error Id: f6fdd477-c208-4a3d-8476-e366921e5787 on PWXAA:31010]
>  at 
> org.apache.drill.exec.rpc.user.QueryResultHandler.resultArrived(QueryResultHandler.java:125)
>  at org.apache.drill.exec.rpc.user.UserClient.handle(UserClient.java:422)
>  at org.apache.drill.exec.rpc.user.UserClient.handle(UserClient.java:96)
>  at org.apache.drill.exec.rpc.RpcBus$InboundHandler.decode(RpcBus.java:273)
>  at org.apache.drill.exec.rpc.RpcBus$InboundHandler.decode(RpcBus.java:243)
>  at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
>  at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:287)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
>  at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:102)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
>  at 
> io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:312)
>  at 
> io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:286)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
>  at 
> io.netty.channel.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:86)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:335)
>  at 
> io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1294)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:356)
>  at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:342)
>  at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:911)
>  at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>  at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:645)
>  at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:580)
>  at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:497)
>  at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:459)
>  at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:131)
>  
>  
> {code}
> I'm new.. I put this as MAJOR from reading the severity definitions, but 
> gladly defer to those who know better how to classify.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to