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



##########
File path: 
hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+@SuppressWarnings("Duplicates")

Review comment:
       Added

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.io.storage;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileContext;
+import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+import org.apache.hadoop.io.Writable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * HoodieHFileWriter writes IndexedRecords into an HFile. The record's key is 
used as the key and the
+ * AVRO encoded record bytes are saved as the value.
+ *
+ * Limitations (compared to columnar formats like Parquet or ORC):
+ *  1. Records should be added in order of keys
+ *  2. There are no column stats
+ */
+public class HoodieHFileWriter<T extends HoodieRecordPayload, R extends 
IndexedRecord>
+    implements HoodieFileWriter<R> {
+  private static AtomicLong recordIndex = new AtomicLong(1);
+  private static final Logger LOG = 
LogManager.getLogger(HoodieHFileWriter.class);
+
+  public static final String KEY_SCHEMA = "schema";
+  public static final String KEY_BLOOM_FILTER_META_BLOCK = "bloomFilter";
+  public static final String KEY_BLOOM_FILTER_TYPE_CODE = 
"bloomFilterTypeCode";
+  public static final String KEY_MIN_RECORD = "minRecordKey";
+  public static final String KEY_MAX_RECORD = "maxRecordKey";
+
+  private final Path file;
+  private HoodieHFileConfig hfileConfig;
+  private final HoodieWrapperFileSystem fs;
+  private final long maxFileSize;
+  private final String instantTime;
+  private final SparkTaskContextSupplier sparkTaskContextSupplier;
+  private HFile.Writer writer;
+  private String minRecordKey;
+  private String maxRecordKey;
+
+  public HoodieHFileWriter(String instantTime, Path file, HoodieHFileConfig 
hfileConfig, Schema schema,
+      SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException {
+
+    Configuration conf = FSUtils.registerFileSystem(file, 
hfileConfig.getHadoopConf());
+    this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, conf);
+    this.fs = (HoodieWrapperFileSystem) this.file.getFileSystem(conf);
+    this.hfileConfig = hfileConfig;
+
+    // We cannot accurately measure the snappy compressed output file size. We 
are choosing a
+    // conservative 10%
+    // TODO - compute this compression ratio dynamically by looking at the 
bytes written to the
+    // stream and the actual file size reported by HDFS
+    // this.maxFileSize = hfileConfig.getMaxFileSize()
+    //    + Math.round(hfileConfig.getMaxFileSize() * 
hfileConfig.getCompressionRatio());
+    this.maxFileSize = hfileConfig.getMaxFileSize();
+    this.instantTime = instantTime;
+    this.sparkTaskContextSupplier = sparkTaskContextSupplier;
+
+    HFileContext context = new 
HFileContextBuilder().withBlockSize(hfileConfig.getBlockSize())
+          .withCompression(hfileConfig.getCompressionAlgorithm())
+          .build();
+    CacheConfig cacheConfig = new CacheConfig(conf);
+    this.writer = HFile.getWriterFactory(conf, cacheConfig).withPath(this.fs, 
this.file).withFileContext(context).create();
+
+    writer.appendFileInfo(KEY_SCHEMA.getBytes(), schema.toString().getBytes());
+  }
+
+  @Override
+  public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws 
IOException {
+    String seqId =
+        HoodieRecord.generateSequenceId(instantTime, 
sparkTaskContextSupplier.getPartitionIdSupplier().get(), 
recordIndex.getAndIncrement());
+    HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, 
record.getRecordKey(), record.getPartitionPath(),
+        file.getName());
+    HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, 
instantTime, seqId);
+
+    writeAvro(record.getRecordKey(), (IndexedRecord)avroRecord);
+  }
+
+  @Override
+  public boolean canWrite() {
+    return fs.getBytesWritten(file) < maxFileSize;
+  }
+
+  @Override
+  public void writeAvro(String recordKey, IndexedRecord object) throws 
IOException {
+    byte[] value = HoodieAvroUtils.avroToBytes((GenericRecord)object);
+    KeyValue kv = new KeyValue(recordKey.getBytes(), null, null, value);
+    writer.append(kv);
+
+    if (hfileConfig.useBloomFilter()) {

Review comment:
       ParquetWriter uses HoodieAvroWriteSupport which has the bloom filter 
code. I did not want to create another class for such a few lines of code. 
Also, it may be better for HFile to disable bloom filter in some cases (e.g. 
RFC-15 HFile will not need bloom filter within the HFile).  

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.io.storage;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hudi.common.bloom.BloomFilter;
+
+public class HoodieHFileConfig {
+
+  private Compression.Algorithm compressionAlgorithm;
+  private int blockSize;
+  private long maxFileSize;
+  private Configuration hadoopConf;
+  private BloomFilter bloomFilter;
+
+  public HoodieHFileConfig(Compression.Algorithm compressionAlgorithm, int 
blockSize, long maxFileSize,

Review comment:
       Can you please provide the source to your benchmarking so I can add the 
configs? The source code link in HUDI-432 is broken.

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.HoodieAvroUtils;
+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.HoodieHFileReader;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileContext;
+import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+import org.apache.hadoop.hbase.util.Pair;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nonnull;
+
+/**
+ * HoodieHFileDataBlock contains a list of records stored inside an HFile 
format. It is used with the HFile
+ * base file format.
+ */
+public class HoodieHFileDataBlock extends HoodieDataBlock {
+  private static final Logger LOG = 
LogManager.getLogger(HoodieHFileDataBlock.class);
+  private static Compression.Algorithm compressionAlgorithm = 
Compression.Algorithm.GZ;
+  private static int blockSize = 1 * 1024 * 1024;
+
+  public HoodieHFileDataBlock(@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 HoodieHFileDataBlock(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 HoodieHFileDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull 
Map<HeaderMetadataType, String> header) {
+    super(records, header, new HashMap<>());
+  }
+
+  @Override
+  public HoodieLogBlockType getBlockType() {
+    return HoodieLogBlockType.HFILE_DATA_BLOCK;
+  }
+
+  @Override
+  protected byte[] serializeRecords() throws IOException {
+    HFileContext context = new 
HFileContextBuilder().withBlockSize(blockSize).withCompression(compressionAlgorithm)
+        .build();
+    Configuration conf = new Configuration();
+    CacheConfig cacheConfig = new CacheConfig(conf);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    FSDataOutputStream ostream = new FSDataOutputStream(baos, null);
+
+    HFile.Writer writer = HFile.getWriterFactory(conf, cacheConfig)
+        .withOutputStream(ostream).withFileContext(context).create();
+
+    // Serialize records into bytes
+    Map<String, byte[]> recordMap = new TreeMap<>();
+    Iterator<IndexedRecord> itr = records.iterator();
+    boolean useIntegerKey = false;
+    int key = 0;
+    int keySize = 0;
+    Field keyField = 
records.get(0).getSchema().getField(HoodieRecord.RECORD_KEY_METADATA_FIELD);
+    if (keyField == null) {

Review comment:
       HFile requires written key-values to be sorted by the key. When adding 
the serialized GenericRecord, the serialized bytes are the "value" and the key 
is to be extracted from the record. 
   
   A HUDI GenericRecord usually has the "_hoodie_record_key" field which can be 
used as the unique key. But in unit tests, the schema might not have 
_hoodie_record_key. So to support unit tests, a sequential integer is used to 
order the writes into HFile. 

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.HoodieAvroUtils;
+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.HoodieHFileReader;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileContext;
+import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+import org.apache.hadoop.hbase.util.Pair;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nonnull;
+
+/**
+ * HoodieHFileDataBlock contains a list of records stored inside an HFile 
format. It is used with the HFile
+ * base file format.
+ */
+public class HoodieHFileDataBlock extends HoodieDataBlock {
+  private static final Logger LOG = 
LogManager.getLogger(HoodieHFileDataBlock.class);
+  private static Compression.Algorithm compressionAlgorithm = 
Compression.Algorithm.GZ;
+  private static int blockSize = 1 * 1024 * 1024;
+
+  public HoodieHFileDataBlock(@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 HoodieHFileDataBlock(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 HoodieHFileDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull 
Map<HeaderMetadataType, String> header) {
+    super(records, header, new HashMap<>());
+  }
+
+  @Override
+  public HoodieLogBlockType getBlockType() {
+    return HoodieLogBlockType.HFILE_DATA_BLOCK;
+  }
+
+  @Override
+  protected byte[] serializeRecords() throws IOException {
+    HFileContext context = new 
HFileContextBuilder().withBlockSize(blockSize).withCompression(compressionAlgorithm)
+        .build();
+    Configuration conf = new Configuration();
+    CacheConfig cacheConfig = new CacheConfig(conf);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    FSDataOutputStream ostream = new FSDataOutputStream(baos, null);
+
+    HFile.Writer writer = HFile.getWriterFactory(conf, cacheConfig)
+        .withOutputStream(ostream).withFileContext(context).create();
+
+    // Serialize records into bytes
+    Map<String, byte[]> recordMap = new TreeMap<>();
+    Iterator<IndexedRecord> itr = records.iterator();
+    boolean useIntegerKey = false;
+    int key = 0;
+    int keySize = 0;
+    Field keyField = 
records.get(0).getSchema().getField(HoodieRecord.RECORD_KEY_METADATA_FIELD);
+    if (keyField == null) {
+      // Missing key metadata field so we should use an integer sequence key
+      useIntegerKey = true;
+      keySize = (int) Math.ceil(Math.log(records.size())) + 1;
+    }
+    while (itr.hasNext()) {
+      IndexedRecord record = itr.next();
+      String recordKey;
+      if (useIntegerKey) {
+        recordKey = String.format("%" + keySize + "s", key++);
+      } else {
+        recordKey = record.get(keyField.pos()).toString();
+      }
+      byte[] recordBytes = HoodieAvroUtils.avroToBytes(record);
+      recordMap.put(recordKey, recordBytes);
+    }
+
+    // Write the records
+    recordMap.forEach((recordKey, recordBytes) -> {

Review comment:
       Because we need to write to HFile in sorted-by-key order. The loop in 
121 is looping over the record Iterator which does not guarantee any order on 
the records read.
   




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


Reply via email to