You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/06/05 15:24:44 UTC

[GitHub] [hudi] codope commented on a diff in pull request #3391: [HUDI-83] Fix Timestamp/Date type read by Hive3

codope commented on code in PR #3391:
URL: https://github.com/apache/hudi/pull/3391#discussion_r889703023


##########
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.hudi.hadoop.avro;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hudi.hadoop.HoodieColumnProjectionUtils;
+import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
+import org.apache.parquet.avro.AvroReadSupport;
+import org.apache.parquet.avro.AvroSchemaConverter;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetInputSplit;
+import org.apache.parquet.hadoop.ParquetRecordReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter;
+
+public class HoodieAvroParquetReader extends RecordReader<Void, ArrayWritable> {
+
+  private final ParquetRecordReader<GenericData.Record> parquetRecordReader;
+
+  public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf) throws IOException {
+    AvroReadSupport avroReadSupport = new AvroReadSupport<>();
+    // if exists read columns, we need to filter columns.
+    List<String> readColNames = Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(conf));
+    if (!readColNames.isEmpty()) {
+      // get base schema
+      ParquetMetadata fileFooter =

Review Comment:
   Most query engines already read file footer, and now with this additional call we'll take a perfomance hit even if it's just for certain cases (when supportTimestamp is true). We should try to rely on native engine readers as much as possible.



##########
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetInputFormat.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.hudi.hadoop.avro;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.parquet.hadoop.ParquetInputFormat;
+import org.apache.parquet.hadoop.util.ContextUtil;
+
+import java.io.IOException;
+
+public class HoodieAvroParquetInputFormat extends ParquetInputFormat<ArrayWritable> {

Review Comment:
   Is another custom input format necessary? Why can't we do this as a conditional in `HoodieParquetInputFormat`? A javadoc explaining why we need this would be helpful



##########
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetInputFormat.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.hudi.hadoop.avro;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.parquet.hadoop.ParquetInputFormat;
+import org.apache.parquet.hadoop.util.ContextUtil;
+
+import java.io.IOException;
+
+public class HoodieAvroParquetInputFormat extends ParquetInputFormat<ArrayWritable> {

Review Comment:
   How will this work with Presto/Trino? Do we need to use the annotation `UseRecordReaderFromInputFormat`?



##########
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.hudi.hadoop.avro;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hudi.hadoop.HoodieColumnProjectionUtils;
+import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
+import org.apache.parquet.avro.AvroReadSupport;
+import org.apache.parquet.avro.AvroSchemaConverter;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetInputSplit;
+import org.apache.parquet.hadoop.ParquetRecordReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter;
+
+public class HoodieAvroParquetReader extends RecordReader<Void, ArrayWritable> {

Review Comment:
   please add javadoc



##########
hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java:
##########
@@ -76,8 +76,10 @@ public class HiveSyncConfig extends HoodieSyncConfig {
   @Parameter(names = {"--help", "-h"}, help = true)
   public Boolean help = false;
 
-  @Parameter(names = {"--support-timestamp"}, description = "'INT64' with original type TIMESTAMP_MICROS is converted to hive 'timestamp' type."
-      + "Disabled by default for backward compatibility.")
+  @Parameter(names = {"--support-timestamp"},

Review Comment:
   Do we need similar config in Presto/Trino as well? 



##########
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.hudi.hadoop.avro;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hudi.hadoop.HoodieColumnProjectionUtils;
+import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
+import org.apache.parquet.avro.AvroReadSupport;
+import org.apache.parquet.avro.AvroSchemaConverter;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetInputSplit;
+import org.apache.parquet.hadoop.ParquetRecordReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter;
+
+public class HoodieAvroParquetReader extends RecordReader<Void, ArrayWritable> {
+
+  private final ParquetRecordReader<GenericData.Record> parquetRecordReader;
+
+  public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf) throws IOException {
+    AvroReadSupport avroReadSupport = new AvroReadSupport<>();
+    // if exists read columns, we need to filter columns.
+    List<String> readColNames = Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(conf));
+    if (!readColNames.isEmpty()) {
+      // get base schema
+      ParquetMetadata fileFooter =
+              ParquetFileReader.readFooter(conf, ((ParquetInputSplit) inputSplit).getPath(), ParquetMetadataConverter.NO_FILTER);
+      MessageType messageType = fileFooter.getFileMetaData().getSchema();
+      Schema baseSchema = new AvroSchemaConverter(conf).convert(messageType);
+      // filter schema for reading
+      final Schema filterSchema = Schema.createRecord(baseSchema.getName(),
+              baseSchema.getDoc(), baseSchema.getNamespace(), baseSchema.isError(),
+              baseSchema.getFields().stream()
+                      .filter(f -> readColNames.contains(f.name()))
+                      .map(f -> new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal()))
+              .collect(Collectors.toList()));
+      avroReadSupport.setAvroReadSchema(conf, filterSchema);
+      avroReadSupport.setRequestedProjection(conf, filterSchema);
+    }
+    parquetRecordReader = new ParquetRecordReader<>(avroReadSupport, getFilter(conf));
+  }
+
+  @Override
+  public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
+    parquetRecordReader.initialize(split, context);
+  }
+
+  @Override
+  public boolean nextKeyValue() throws IOException, InterruptedException {
+    return parquetRecordReader.nextKeyValue();
+  }
+
+  @Override
+  public Void getCurrentKey() throws IOException, InterruptedException {
+    return parquetRecordReader.getCurrentKey();
+  }
+
+  @Override
+  public ArrayWritable getCurrentValue() throws IOException, InterruptedException {
+    GenericRecord record = parquetRecordReader.getCurrentValue();
+    return (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(record, record.getSchema(), true);

Review Comment:
   Should it be `true` always or should we guard by config?



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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