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 2021/11/13 08:13:54 UTC

[GitHub] [hudi] prashantwason commented on a change in pull request #3930: [HUDI-431] [WIP] Adding parquet data block with inline read support

prashantwason commented on a change in pull request #3930:
URL: https://github.com/apache/hudi/pull/3930#discussion_r748696391



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
##########
@@ -702,16 +703,21 @@ public HoodieFileFormat getLogFileFormat() {
     return metaClient.getTableConfig().getLogFileFormat();
   }
 
-  public HoodieLogBlockType getLogDataBlockFormat() {
-    switch (getBaseFileFormat()) {

Review comment:
       I think this config is not required. 
   
   For a dataset with a PARQUET file format, an inline log block format of PARQUET makes sense. Mix and match of base file format and inline log format does not seem to make writing data or query engine side faster.
   
   So we can have a config - useInlineFiles() - when enabled, the  inline file format will be format and it will have the same format as the base file format. This needs to be fixed for the dataset so can be from table config.

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.common.table.log.block;
+
+import org.apache.hudi.avro.HoodieAvroWriteSupport;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.bloom.BloomFilterTypeCode;
+import org.apache.hudi.common.fs.inline.InLineFSUtils;
+import org.apache.hudi.common.fs.inline.InLineFileSystem;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
+import org.apache.hudi.io.storage.HoodieParquetReader;
+import org.apache.hudi.io.storage.HoodieParquetStreamWriter;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.avro.AvroSchemaConverter;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import javax.annotation.Nonnull;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * HoodieParquetDataBlock contains a list of records serialized using Parquet.
+ */
+public class HoodieParquetDataBlock extends HoodieDataBlock {
+
+  public HoodieParquetDataBlock(@Nonnull Map<HeaderMetadataType, String> logBlockHeader,
+                                @Nonnull Map<HeaderMetadataType, String> logBlockFooter,
+                                @Nonnull Option<HoodieLogBlockContentLocation> blockContentLocation, @Nonnull Option<byte[]> content,
+                                FSDataInputStream inputStream, boolean readBlockLazily) {
+    super(logBlockHeader, logBlockFooter, blockContentLocation, content, inputStream, readBlockLazily);
+  }
+
+  public HoodieParquetDataBlock(HoodieLogFile logFile, FSDataInputStream inputStream, Option<byte[]> content,
+                                boolean readBlockLazily, long position, long blockSize, long blockEndpos, Schema readerSchema,
+                                Map<HeaderMetadataType, String> header, Map<HeaderMetadataType, String> footer) {
+    super(content, inputStream, readBlockLazily,
+        Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), readerSchema, header,
+        footer);
+  }
+
+  public HoodieParquetDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull Map<HeaderMetadataType, String> header) {
+    super(records, header, new HashMap<>());
+  }
+
+  @Override
+  public HoodieLogBlockType getBlockType() {
+    return HoodieLogBlockType.PARQUET_DATA_BLOCK;
+  }
+
+  @Override
+  protected byte[] serializeRecords() throws IOException {
+    // TODO: Need to decide from where to fetch all config values required below. We can't re-use index config as the purpose is different.

Review comment:
       I think the configs here should match the configs for the dataset base file format. 
   
   Under what case would the user want separate configs for base file and inline log blocks?




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