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 2020/07/06 21:28:20 UTC

[GitHub] [hudi] prashantwason opened a new pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

prashantwason opened a new pull request #1804:
URL: https://github.com/apache/hudi/pull/1804


   1. Includes HFileWriter and HFileReader
   2. Includes HFileInputFormat for both snapshot and realtime input format for Hive
   3. Unit test for new code
   4. IT for using HFile format and querying using Hive (Presto and SparkSQL are not supported)
   
   Advantage:
   HFile file format saves data as binary key-value pairs. This implementation chooses the following values:
   1. Key = Hoodie Record Key (as bytes)
   2. Value = Avro encoded GenericRecord (as bytes)
   
   HFile allows efficient lookup of a record by key or range of keys. Hence, this base file format is well suited to applications like RFC-15, RFC-08 which will benefit from the ability to lookup records by key or search in a range of keys without having to read the entire data/log format.
   
   Limitations:
   HFile storage format has certain limitations when used as a general purpose data storage format.
   1. Does not have a implemented reader for Presto and SparkSQL
   2. Is not a columnar file format and hence may lead to lower compression levels and greater IO on query side due to lack of column pruning
   
   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contributing.html before opening a pull request.*
   
   ## What is the purpose of the pull request
   
   *(For example: This pull request adds quick-start document.)*
   
   ## Brief change log
   
   *(for example:)*
     - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.


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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r475936227



##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+/**
+ * Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
+ *
+ * The implementation performs a merge-sort by comparing the key of the record being written to the list of
+ * keys in newRecordKeys (sorted in-memory).
+ */
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
+
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Called by compactor code path.
+   */
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,

Review comment:
       So, no code calls this atm ? @prashantwason 
   
   The following code in `HoodieMergeOnReadTableCompactor` 
   
   ```
   if (oldDataFileOpt.isPresent()) {
         result = hoodieCopyOnWriteTable.handleUpdate(instantTime, operation.getPartitionPath(),
                 operation.getFileId(), scanner.getRecords(),
             oldDataFileOpt.get());
       }
   ```
   ends up calling the following. 
   
   ```
    protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId,
         Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) {
       return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords,
               partitionPath, fileId, dataFileToBeMerged, sparkTaskContextSupplier);
     }
   ```




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r464720436



##########
File path: hudi-client/pom.xml
##########
@@ -102,6 +102,12 @@
       <artifactId>spark-sql_${scala.binary.version}</artifactId>
     </dependency>
 
+    <!-- spark-avro -->
+    <dependency>

Review comment:
       This package is being used only in the unit tests for the function createConverterToSQL to convert Avro record to a spark sql row. Is there another way?
   
   I have added the test-scope to this dependency.  




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r461108736



##########
File path: hudi-client/pom.xml
##########
@@ -102,6 +102,12 @@
       <artifactId>spark-sql_${scala.binary.version}</artifactId>
     </dependency>
 
+    <!-- spark-avro -->
+    <dependency>

Review comment:
       we have moved to spark-avro in apache spark IIRC. we should not use com.databricks anymore?

##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+/**
+ * Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
+ *
+ * The implementation performs a merge-sort by comparing the key of the record being written to the list of
+ * keys in newRecordKeys (sorted in-memory).
+ */
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();

Review comment:
       for now, I guess it's okay to assume the records will fit into memory? eventually we need to make this sorting spillable (using rocksDB for eg) for RFC-08 indexing work

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

Review comment:
       I guess you are holding the entire byte[] corresponding to the HFile in memory and then logging this as  the payload? 

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

Review comment:
       This seems fine for metadata logging . eventually, we may want to think about writing directly to the underlying `fs` . This needs some changes on log format writing per se. 

##########
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) -> {
+      try {
+        KeyValue kv = new KeyValue(recordKey.getBytes(), null, null, recordBytes);
+        writer.append(kv);
+      } catch (IOException e) {
+        throw new HoodieIOException("IOException serializing records", e);
+      }
+    });
+
+    writer.close();
+    ostream.flush();
+    ostream.close();
+
+    return baos.toByteArray();
+  }
+
+  @Override
+  protected void deserializeRecords() throws IOException {
+    // Get schema from the header
+    Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
+
+    // If readerSchema was not present, use writerSchema
+    if (schema == null) {
+      schema = writerSchema;
+    }
+
+    // Read the content
+    HoodieHFileReader reader = new HoodieHFileReader<>(getContent().get());

Review comment:
       Can we explore use of `InlineFileSystem` to read HFile as-is. This was you can actually do point lookup on teh log blocks, just like you do on the base file. it will be significantly faster




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r464708877



##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+/**
+ * Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
+ *
+ * The implementation performs a merge-sort by comparing the key of the record being written to the list of
+ * keys in newRecordKeys (sorted in-memory).
+ */
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();

Review comment:
       The assumption here is that the record keys (for the records being updated) will fit into memory. These record keys are extracted from the keySet of keyToNewRecords which is already a ExternalSpillableMap.
   
    




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r458355188



##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java
##########
@@ -34,7 +35,17 @@
 
   public Set<String> filterRowKeys(Set<String> candidateRowKeys);
 
-  public Iterator<R> getRecordIterator(Schema schema) throws IOException;
+  public Iterator<R> getRecordIterator(Schema readerSchema) throws IOException;
+
+  default Iterator<R> getRecordIterator() throws IOException {
+    return getRecordIterator(getSchema());
+  }
+
+  public Option<R> getRecordByKey(String key, Schema readerSchema) throws IOException;

Review comment:
       We can introduce that config but how will it be used? 
   
   I feel the lookup by key is only useful for internal features (like RFC-15 or RFC-08) rather than a generic API for HUDI. HUDI record keys tend to be UUDIs which are large and looking them up is not a common usecase.
   
   
    

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
+        content.length, new CacheConfig(conf), conf);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;

Review comment:
       Updated.

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
+        content.length, new CacheConfig(conf), conf);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
+          new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
+    } catch (IOException e) {
+      throw new HoodieException("Could not read min/max record key out of file information block correctly from path", e);
+    }
+  }
+
+  @Override
+  public Schema getSchema() {
+    if (schema == null) {
+      try {
+        Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
+        schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
+      } catch (IOException e) {
+        throw new HoodieException("Could not read schema of file from path", e);
+      }
+    }
+
+    return schema;
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      ByteBuffer serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
+      byte[] filterBytes = new byte[serializedFilter.remaining()];
+      serializedFilter.get(filterBytes); // read the bytes that were written
+      return BloomFilterFactory.fromString(new String(filterBytes),
+          new String(fileInfo.get(KEY_BLOOM_FILTER_TYPE_CODE.getBytes())));
+    } catch (IOException e) {
+      throw new HoodieException("Could not read bloom filter from " + path, e);
+    }
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set candidateRowKeys) {

Review comment:
       I have added a comment to capture the potential for optimization.




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r477078000



##########
File path: hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java
##########
@@ -115,6 +115,35 @@ public void testParquetDemo() throws Exception {
     testIncrementalHiveQueryAfterCompaction();
   }
 
+  @Test
+  public void testHFileDemo() throws Exception {

Review comment:
       Ok.

##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+/**
+ * Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
+ *
+ * The implementation performs a merge-sort by comparing the key of the record being written to the list of
+ * keys in newRecordKeys (sorted in-memory).
+ */
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
+
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Called by compactor code path.
+   */
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+      Map<String, HoodieRecord<T>> keyToNewRecordsOrig, String partitionPath, String fileId,
+      HoodieBaseFile dataFileToBeMerged, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged,
+        sparkTaskContextSupplier);
+
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
+   */
+  @Override
+  public void write(GenericRecord oldRecord) {
+    String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+
+    // To maintain overall sorted order across updates and inserts, write any new inserts whose keys are less than
+    // the oldRecord's key.
+    while (!newRecordKeysSorted.isEmpty() && newRecordKeysSorted.peek().compareTo(key) <= 0) {
+      String keyToPreWrite = newRecordKeysSorted.remove();

Review comment:
       This is a streaming sort-merge. The logic is as follows:
   
   1. Hold the keys of records which have changed in memory (in a PriorityQueue). This is in addition to the Map whic his already there in HoodieMergeHandle.
   2. For each write()
      - if the key of the record being written < head of PriorityQueue  => Write out this record
      - if the key of the record being written > head of PriorityQueue  => Write out all records with smaller keys
   
   Do you have some other algorithm in mind?

##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+/**
+ * Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
+ *
+ * The implementation performs a merge-sort by comparing the key of the record being written to the list of
+ * keys in newRecordKeys (sorted in-memory).
+ */
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
+
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Called by compactor code path.
+   */
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,

Review comment:
       We will need to change the HoodieMergeOnReadTableCompactor code to use HoodieSortedMergeHandle for HFile cases. 
   
   

##########
File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
##########
@@ -81,6 +87,7 @@ public Builder fromProperties(Properties props) {
 
     public Builder limitFileSize(long maxFileSize) {
       props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
+      props.setProperty(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));

Review comment:
       I overloaded for two reasons:
   1. changing this function name will be backward incompatible
   2. Each HUDI Write Config can only have one type of base files. So limitFileSIze is the limit of the base file size.
   
    How about changing the key from HFILE_FILE_MAX_BYTES / PARQUET_FILE_MAX_BYTES to BASE_FILE_MAX_BYTES ?




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r477551807



##########
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:
       @vinothchandar Does not matter where the sorting is performed. It will definitely be referable in the partitioner.
   
   I traced the entire data path of writes from HoodieWriteClient.upsert(). There are several code paths each using their own partitioner. So could not locate a single place where to implement/ensure the sorting. 
   
   Is there a base class for all Partitioners within HUDI? Maybe we can update that to accept a boolean sort parameter.




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r477471316



##########
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:
       Added the three settings. Please review.




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r475940393



##########
File path: hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * HoodieInputFormat for HUDI datasets which store data in HFile base file format.
+ */
+@UseFileSplitsFromInputFormat
+public class HoodieHFileInputFormat extends FileInputFormat<NullWritable, ArrayWritable> implements Configurable {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileInputFormat.class);
+
+  protected Configuration conf;
+
+  protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) {
+    return HoodieInputFormatUtils.filterInstantsTimeline(timeline);
+  }
+
+  @Override
+  public FileStatus[] listStatus(JobConf job) throws IOException {

Review comment:
       While this is true, we can use more helpers and avoid copying to a large degree? 




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r475936227



##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+/**
+ * Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
+ *
+ * The implementation performs a merge-sort by comparing the key of the record being written to the list of
+ * keys in newRecordKeys (sorted in-memory).
+ */
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
+
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Called by compactor code path.
+   */
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,

Review comment:
       So, no code calls this atm ?
   
   The following code in `HoodieMergeOnReadTableCompactor` 
   
   ```
   if (oldDataFileOpt.isPresent()) {
         result = hoodieCopyOnWriteTable.handleUpdate(instantTime, operation.getPartitionPath(),
                 operation.getFileId(), scanner.getRecords(),
             oldDataFileOpt.get());
       }
   ```
   ends up calling the following. 
   
   ```
    protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId,
         Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) {
       return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords,
               partitionPath, fileId, dataFileToBeMerged, sparkTaskContextSupplier);
     }
   ```




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r464709761



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

Review comment:
       Yes. This is similar to how the Avro Log Block is being written right now with the records being serialized into ByteArrayOutputStream in-memory. 




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



[GitHub] [hudi] vinothchandar merged pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar merged pull request #1804:
URL: https://github.com/apache/hudi/pull/1804


   


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



[GitHub] [hudi] prashantwason commented on pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#issuecomment-682213033


   > @prashantwason if you broadly agree, I will make the change and land this, so you can focus on rfc-15 more :)
   Sure @vinothchandar . Thanks for all the help. Lets get this rolling soon.
   


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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r458355532



##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);

Review comment:
       This constructor is creating an HFile.Reader from a byte array (bytes from a HFile Data Block saved in a log file). HFile.createReader constructor requires a FSDataInputStreamWrapper which requires a IO stream implementing "Seekable" interface. 
   
   In other words, this is required for creating a HFile.reader out of an in-memory byte array and is not related to the internals of the HFile reading logic.

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
+        content.length, new CacheConfig(conf), conf);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
+          new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
+    } catch (IOException e) {
+      throw new HoodieException("Could not read min/max record key out of file information block correctly from path", e);
+    }
+  }
+
+  @Override
+  public Schema getSchema() {
+    if (schema == null) {
+      try {
+        Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
+        schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
+      } catch (IOException e) {
+        throw new HoodieException("Could not read schema of file from path", e);
+      }
+    }
+
+    return schema;
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      ByteBuffer serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
+      byte[] filterBytes = new byte[serializedFilter.remaining()];
+      serializedFilter.get(filterBytes); // read the bytes that were written
+      return BloomFilterFactory.fromString(new String(filterBytes),
+          new String(fileInfo.get(KEY_BLOOM_FILTER_TYPE_CODE.getBytes())));
+    } catch (IOException e) {
+      throw new HoodieException("Could not read bloom filter from " + path, e);
+    }
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set candidateRowKeys) {
+    try {
+      List<Pair<String, R>> allRecords = readAllRecords();
+      Set<String> rowKeys = new HashSet<>();
+      allRecords.forEach(t -> {
+        if (candidateRowKeys.contains(t.getFirst())) {
+          rowKeys.add(t.getFirst());
+        }
+      });
+      return rowKeys;
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to read row keys from " + path, e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException {
+    List<Pair<String, R>> recordList = new LinkedList<>();
+    try {
+      HFileScanner scanner = reader.getScanner(false, false);

Review comment:
       The usecase here is very limited - read and return all records once. So I feel the parameters should be specific to optimizing this usecase.
   
   Caching the blocks does not have any value as we are reading the blocks once and wont be reading again.
   
      * @param pread
      *          Use positional read rather than seek+read if true (pread is better
      *          for random reads, seek+read is better scanning).
   
   pread is false based on the above code comment in org.apache.hadoop.hbase.io.hfile.HFileReaderV3

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
+        content.length, new CacheConfig(conf), conf);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
+          new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
+    } catch (IOException e) {
+      throw new HoodieException("Could not read min/max record key out of file information block correctly from path", e);
+    }
+  }
+
+  @Override
+  public Schema getSchema() {
+    if (schema == null) {
+      try {
+        Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
+        schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
+      } catch (IOException e) {
+        throw new HoodieException("Could not read schema of file from path", e);
+      }
+    }
+
+    return schema;
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      ByteBuffer serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
+      byte[] filterBytes = new byte[serializedFilter.remaining()];
+      serializedFilter.get(filterBytes); // read the bytes that were written
+      return BloomFilterFactory.fromString(new String(filterBytes),
+          new String(fileInfo.get(KEY_BLOOM_FILTER_TYPE_CODE.getBytes())));
+    } catch (IOException e) {
+      throw new HoodieException("Could not read bloom filter from " + path, e);
+    }
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set candidateRowKeys) {
+    try {
+      List<Pair<String, R>> allRecords = readAllRecords();
+      Set<String> rowKeys = new HashSet<>();
+      allRecords.forEach(t -> {
+        if (candidateRowKeys.contains(t.getFirst())) {
+          rowKeys.add(t.getFirst());
+        }
+      });
+      return rowKeys;
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to read row keys from " + path, e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException {
+    List<Pair<String, R>> recordList = new LinkedList<>();
+    try {
+      HFileScanner scanner = reader.getScanner(false, false);
+      if (scanner.seekTo()) {
+        do {
+          Cell c = scanner.getKeyValue();
+          byte[] keyBytes = Arrays.copyOfRange(c.getRowArray(), c.getRowOffset(), c.getRowOffset() + c.getRowLength());
+          R record = readNextRecord(c, writerSchema, readerSchema);
+          recordList.add(new Pair<>(new String(keyBytes), record));
+        } while (scanner.next());
+      }
+
+      return recordList;
+    } catch (IOException e) {
+      throw new HoodieException("Error reading hfile " + path + " as a dataframe", e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords() throws IOException {
+    Schema schema = new Schema.Parser().parse(new String(reader.loadFileInfo().get("schema".getBytes())));
+    return readAllRecords(schema, schema);
+  }
+
+  @Override
+  public Iterator getRecordIterator(Schema readerSchema) throws IOException {
+    final HFileScanner scanner = reader.getScanner(false, false);
+    return new Iterator<R>() {
+      private R next = null;
+      private boolean eof = false;
+
+      @Override
+      public boolean hasNext() {
+        try {
+          // To handle when hasNext() is called multiple times for idempotency and/or the first time
+          if (this.next == null && !this.eof) {
+            if (!scanner.isSeeked() && scanner.seekTo()) {
+                this.next = (R)readNextRecord(scanner.getKeyValue(), getSchema(), readerSchema);
+            }
+          }
+          return this.next != null;
+        } catch (IOException io) {
+          throw new HoodieIOException("unable to read next record from hfile ", io);
+        }
+      }
+
+      @Override
+      public R next() {
+        try {
+          // To handle case when next() is called before hasNext()
+          if (this.next == null) {
+            if (!hasNext()) {
+              throw new HoodieIOException("No more records left to read from hfile");
+            }
+          }
+          R retVal = this.next;
+          if (scanner.next()) {
+            this.next = (R)readNextRecord(scanner.getKeyValue(), getSchema(), readerSchema);

Review comment:
       As I understand it, the iterator interface contract is:
   hasNext: Does another record exist?
   next: Return the next record.
   
   Whether the next record is retrieved in hasNext or next is upto the implementation. If we retrieve the next record in hasNext, we need to additionally deal with idempotency - hasNext called multiple times before next called. 
   
   There could be a better implementation for sure but I have copied this from ParquetReaderIterator and looked ok to me.

##########
File path: hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * HoodieInputFormat for HUDI datasets which store data in HFile base file format.
+ */
+@UseFileSplitsFromInputFormat
+public class HoodieHFileInputFormat extends FileInputFormat<NullWritable, ArrayWritable> implements Configurable {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileInputFormat.class);
+
+  protected Configuration conf;
+
+  protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) {
+    return HoodieInputFormatUtils.filterInstantsTimeline(timeline);
+  }
+
+  @Override
+  public FileStatus[] listStatus(JobConf job) throws IOException {

Review comment:
       The code reuse is already being done as much as possible. 
   
   The way InputFormats are implemented in HUDI does not allow any further optimization:
   1. HoodieParquetInputFormat extends MapredParquetInputFormat
   2.  HoodieHFileInputFormat extends FileInputFormat
   3. HoodieXXXXInputFormat will need to extend its own BaseInputFormat.
   
   Hence, its not possible to define a base class for all InputFormats for maximum code reuse.  




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r477486286



##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java
##########
@@ -77,4 +79,9 @@ public long getTotalRecords() {
     // TODO Auto-generated method stub
     return 0;
   }
+
+  @Override
+  public Option getRecordByKey(String key, Schema schema) throws IOException {
+    throw new HoodieException("HoodieParquetReader does not support reading records by key");

Review comment:
       Done.




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



[GitHub] [hudi] vinothchandar commented on pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#issuecomment-683837050


   I will keep working on 1256. landing this to unblock other PRs for RFC-15 @prashantwason 


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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r475932748



##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java
##########
@@ -55,7 +56,7 @@
   private long recordsWritten = 0;
   private long insertRecordsWritten = 0;
   private long recordsDeleted = 0;
-  private Iterator<HoodieRecord<T>> recordIterator;
+  private Map<String, HoodieRecord<T>> recordMap;

Review comment:
       okay understood this better. looks good for now 




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



[GitHub] [hudi] vinothchandar commented on pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#issuecomment-680017703


   ```
   [ERROR] Failures: 
   [ERROR]   ITTestHoodieDemo.testParquetDemo:115->testHiveAfterSecondBatchAfterCompaction:360->ITTestBase.assertStdOutContains:287->ITTestBase.saveUpLogs:255->ITTestBase.executeCommandStringInDocker:206->ITTestBase.executeCommandInDocker:185 Command ([cat, /tmp/root/hive.log, |, grep, -i, exception, -A, 10, -B, 5]) expected to succeed. Exit (1) ==> expected: <0> but was: <1>
   [INFO] 
   ```
   
   Looks like this command failed, probably because there were no exceptions here?


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



[GitHub] [hudi] nsivabalan commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r477545069



##########
File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
##########
@@ -94,6 +100,16 @@ public Builder parquetPageSize(int pageSize) {
       return this;
     }
 
+    public Builder hfileMaxFileSize(long maxFileSize) {
+      props.setProperty(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));

Review comment:
       @vinothchandar : wrt you comment on having two diff configs. I see similar configs at other places too. like bloom index parallelism, we have one config per index type. Initially I thought we will have any one config which will be used by any index type that is being initialized. But I saw that every index has its own set of configs and don't share any. 

##########
File path: hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieMergeOnReadTableCompactor.java
##########
@@ -135,15 +135,14 @@
 
     // Compacting is very similar to applying updates to existing file
     Iterator<List<WriteStatus>> result;
-    // If the dataFile is present, there is a base parquet file present, perform updates else perform inserts into a
-    // new base parquet file.
+    // If the dataFile is present, perform updates else perform inserts into a new base file.
     if (oldDataFileOpt.isPresent()) {
       result = hoodieCopyOnWriteTable.handleUpdate(instantTime, operation.getPartitionPath(),
               operation.getFileId(), scanner.getRecords(),
           oldDataFileOpt.get());
     } else {
       result = hoodieCopyOnWriteTable.handleInsert(instantTime, operation.getPartitionPath(), operation.getFileId(),
-          scanner.iterator());
+          scanner.getRecords());

Review comment:
       if we have two overloaded methods, then we could use table. requireSortedRecords() and call appropriate methods either w/ iterator or with records. 

##########
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:
       These are the 3 configs. May I know where these are added 
   ```
       conf.set(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, "true");
       conf.set("CACHE_DATA_IN_L1","true");
       conf.set("hbase.hfile.drop.behind.compaction", "false");
   ```

##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java
##########
@@ -90,9 +91,10 @@ public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTa
    * Called by the compactor code path.
    */
   public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
-      String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordIterator, SparkTaskContextSupplier sparkTaskContextSupplier) {
+      String partitionPath, String fileId, Map<String, HoodieRecord<T>> recordMap,

Review comment:
       Not sure if we can leak the type of base file to compactor. But did you think about having two overloaded methods here. So for parquet compaction path, iterator will be passed in, where as for hfile compaction, record map will be passed in. 

##########
File path: hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * HoodieInputFormat for HUDI datasets which store data in HFile base file format.
+ */
+@UseFileSplitsFromInputFormat
+public class HoodieHFileInputFormat extends FileInputFormat<NullWritable, ArrayWritable> implements Configurable {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileInputFormat.class);
+
+  protected Configuration conf;
+
+  protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) {
+    return HoodieInputFormatUtils.filterInstantsTimeline(timeline);
+  }
+
+  @Override
+  public FileStatus[] listStatus(JobConf job) throws IOException {

Review comment:
       yes, thats what I meant, by using some helper class.

##########
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[]> sortedRecordsMap = new TreeMap<>();

Review comment:
       minor. why call this as sortedRecordsMap. I don't see any sorting actually




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r475893386



##########
File path: hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java
##########
@@ -45,22 +45,39 @@
 import org.apache.avro.generic.GenericRecord;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.parquet.avro.AvroSchemaConverter;
 import org.apache.parquet.hadoop.ParquetWriter;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
 import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructType;
+
+import com.databricks.spark.avro.SchemaConverters;
+
+import scala.Function1;

Review comment:
       @prashantwason it's best we don't import scala methods into java. com.databricks/avro is simply deprecated. We have eequivalent functionality in spark-avro itself. In general we have to fix this test code that reads HFile as a DataSet<Row> in a different way .  




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r464714469



##########
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) -> {
+      try {
+        KeyValue kv = new KeyValue(recordKey.getBytes(), null, null, recordBytes);
+        writer.append(kv);
+      } catch (IOException e) {
+        throw new HoodieIOException("IOException serializing records", e);
+      }
+    });
+
+    writer.close();
+    ostream.flush();
+    ostream.close();
+
+    return baos.toByteArray();
+  }
+
+  @Override
+  protected void deserializeRecords() throws IOException {
+    // Get schema from the header
+    Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
+
+    // If readerSchema was not present, use writerSchema
+    if (schema == null) {
+      schema = writerSchema;
+    }
+
+    // Read the content
+    HoodieHFileReader reader = new HoodieHFileReader<>(getContent().get());

Review comment:
       The log record scanning infrastructure (AbstractHoodieLogRecordScanner and derived classes) do not support this optimization.
   
   AbstractHoodieLogRecordScanner reads each data block from the log file into memory and then calls HoodieLogblock.getRecords(). Since the contents of the entire log block are already in memory, it is simpler to create a HoodieHFileReader on the byte[].
   
   But you have a valid point. If the AbstractHoodieLogRecordScanner could be changed to provide the offset of the data block instead of reading it fully (i.e. it can provide a InLineFsDataInputStream) then HFileReader can do point lookups easily. But this may or may not improve performance because of the following reasons:
   1. In RFC-15 we need to read all the records anyways since we are creating an updated in-memory state of the change to metadata. So point look does not help much here.
   2. In RFC-08, point lookups should help if the number of key lookups are relatively smaller than total keys in the block. (within HFile full file scan may be faster in some cases). 
   
    Good idea but probably can be implemented separately.




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r458319202



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieLogBlock.java
##########
@@ -110,7 +110,7 @@ public long getLogBlockLength() {
    * Type of the log block WARNING: This enum is serialized as the ordinal. Only add new enums at the end.
    */
   public enum HoodieLogBlockType {
-    COMMAND_BLOCK, DELETE_BLOCK, CORRUPT_BLOCK, AVRO_DATA_BLOCK
+    COMMAND_BLOCK, DELETE_BLOCK, CORRUPT_BLOCK, AVRO_DATA_BLOCK, HFILE_DATA_BLOCK

Review comment:
       Yes, a separate DELETE block is not required for HFile. The delete functionality is implemented independent of the data blocks which only save record updates.
   
   DELETE_BLOCK saves record keys which have been deleted since. While reading the log blocks (HoodieMergedLogRecordScanner), if a DELETE block is encountered then we save a EmptyPayload which represents a delete marker for the record. Such records wont be written out (compaction) or processed (RealtimeRecordReader) thereby representing a delete.
   
   >> So, we might have to fetch all values and resolve to the latest one to find if the value represents delete or active.
   Deleted records are never saved. Only deleted keys are saved within the DELETE block. 




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r475939558



##########
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) -> {
+      try {
+        KeyValue kv = new KeyValue(recordKey.getBytes(), null, null, recordBytes);
+        writer.append(kv);
+      } catch (IOException e) {
+        throw new HoodieIOException("IOException serializing records", e);
+      }
+    });
+
+    writer.close();
+    ostream.flush();
+    ostream.close();
+
+    return baos.toByteArray();
+  }
+
+  @Override
+  protected void deserializeRecords() throws IOException {
+    // Get schema from the header
+    Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
+
+    // If readerSchema was not present, use writerSchema
+    if (schema == null) {
+      schema = writerSchema;
+    }
+
+    // Read the content
+    HoodieHFileReader reader = new HoodieHFileReader<>(getContent().get());

Review comment:
       understood. I had a different view of what RFC-15 would do. 
   
   we will track this for RFC-08 cc @n3nash 




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r475940809



##########
File path: hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieHFileRealtimeInputFormat.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.realtime;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.hadoop.HoodieHFileInputFormat;
+import org.apache.hudi.hadoop.UseFileSplitsFromInputFormat;
+import org.apache.hudi.hadoop.UseRecordReaderFromInputFormat;
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * HoodieRealtimeInputFormat for HUDI datasets which store data in HFile base file format.
+ */
+@UseRecordReaderFromInputFormat
+@UseFileSplitsFromInputFormat
+public class HoodieHFileRealtimeInputFormat extends HoodieHFileInputFormat {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileRealtimeInputFormat.class);
+
+  @Override
+  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+    Stream<FileSplit> fileSplits = Arrays.stream(super.getSplits(job, numSplits)).map(is -> (FileSplit) is);
+    return HoodieRealtimeInputFormatUtils.getRealtimeSplits(job, fileSplits);
+  }
+
+  @Override
+  public FileStatus[] listStatus(JobConf job) throws IOException {
+    // Call the HoodieInputFormat::listStatus to obtain all latest hfiles, based on commit timeline.
+    return super.listStatus(job);
+  }
+
+  @Override
+  protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) {
+    // no specific filtering for Realtime format
+    return timeline;
+  }
+
+  @Override
+  public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSplit split, final JobConf jobConf,
+      final Reporter reporter) throws IOException {
+    // Hive on Spark invokes multiple getRecordReaders from different threads in the same spark task (and hence the

Review comment:
       we need to share code somehow. This same large comment need not be in multiple places




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r477491521



##########
File path: hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java
##########
@@ -115,6 +115,35 @@ public void testParquetDemo() throws Exception {
     testIncrementalHiveQueryAfterCompaction();
   }
 
+  @Test
+  public void testHFileDemo() throws Exception {

Review comment:
       Filed https://issues.apache.org/jira/browse/HUDI-1229




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r478483860



##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+/**
+ * Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
+ *
+ * The implementation performs a merge-sort by comparing the key of the record being written to the list of
+ * keys in newRecordKeys (sorted in-memory).
+ */
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
+
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Called by compactor code path.
+   */
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+      Map<String, HoodieRecord<T>> keyToNewRecordsOrig, String partitionPath, String fileId,
+      HoodieBaseFile dataFileToBeMerged, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged,
+        sparkTaskContextSupplier);
+
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
+   */
+  @Override
+  public void write(GenericRecord oldRecord) {
+    String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+
+    // To maintain overall sorted order across updates and inserts, write any new inserts whose keys are less than
+    // the oldRecord's key.
+    while (!newRecordKeysSorted.isEmpty() && newRecordKeysSorted.peek().compareTo(key) <= 0) {
+      String keyToPreWrite = newRecordKeysSorted.remove();

Review comment:
       I am thinking we don't need the map in HoodieMergeHandle or the priorityQueue. The record which have changeed i.e. the input iterator is already sorted. lets call it `inputItr` 
   
   So , we can just compare the recordBeingWritten with inputItr.next() and write out the smallest one, if equal, we call the payload to merge. 
   
   This will avoid any kind of memory overhead 

##########
File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
##########
@@ -94,6 +100,16 @@ public Builder parquetPageSize(int pageSize) {
       return this;
     }
 
+    public Builder hfileMaxFileSize(long maxFileSize) {
+      props.setProperty(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));

Review comment:
       not following. sorry. are you suggesting having a single config or two? 
   So, we need to have a config per usage of HFile. so we can control the base file size for data, metadata, record index separately. 
   
   We cannot have a generic base.file.size or hfile.size config here, at this level IMO. cc @prashantwason  

##########
File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
##########
@@ -94,6 +100,16 @@ public Builder parquetPageSize(int pageSize) {
       return this;
     }
 
+    public Builder hfileMaxFileSize(long maxFileSize) {
+      props.setProperty(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));

Review comment:
       >I see that you split this function into file specific functions. That doable but with more base file formats added, it may be cumbersome and verbose to keep adding the .limitXXXFileSize for specific formats.
   
   @prashantwason I think we need to eventually have a config "per use" of base file - data, metadata, index - since people may want to control them differently. So, in that sense, this has to kind of change.
   yes the change is backwards compatible to RDD clients (which I thought was okay, since its just uber. if you prefer to not have that, lmk. IMO, its about time, we cleaned these up, given we are moving to having way more base files/tables in the mix) 

##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java
##########
@@ -90,9 +91,10 @@ public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTa
    * Called by the compactor code path.
    */
   public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
-      String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordIterator, SparkTaskContextSupplier sparkTaskContextSupplier) {
+      String partitionPath, String fileId, Map<String, HoodieRecord<T>> recordMap,

Review comment:
       Ideally not. the more the compactor can function wihtout knowing the base file type specifics, the better

##########
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[]> sortedRecordsMap = new TreeMap<>();

Review comment:
       its a tree map. its sorted/ordered 

##########
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[]> sortedRecordsMap = new TreeMap<>();

Review comment:
       >Does not matter where the sorting is performed. It will definitely be referable in the partitioner.
    
   @prashantwason for some reason I cannot find your comment overlaid here. (are you using the review feature?) . Anywasy, what I meant was just `rdd.repartitionAndSort...` for the AppendHandle path as well. There is no generic partitioner in Hudi, since the ones we have are all serving different purposes 




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



[GitHub] [hudi] vinothchandar commented on pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#issuecomment-657946663


   @prashantwason will review this tomorrow or wed. works? Please let me know if you are blocked by this.


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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r458355367



##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),

Review comment:
       Exception will be thrown here.

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
+        content.length, new CacheConfig(conf), conf);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
+          new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
+    } catch (IOException e) {
+      throw new HoodieException("Could not read min/max record key out of file information block correctly from path", e);
+    }
+  }
+
+  @Override
+  public Schema getSchema() {
+    if (schema == null) {
+      try {
+        Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
+        schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
+      } catch (IOException e) {
+        throw new HoodieException("Could not read schema of file from path", e);
+      }
+    }
+
+    return schema;
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      ByteBuffer serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
+      byte[] filterBytes = new byte[serializedFilter.remaining()];
+      serializedFilter.get(filterBytes); // read the bytes that were written
+      return BloomFilterFactory.fromString(new String(filterBytes),
+          new String(fileInfo.get(KEY_BLOOM_FILTER_TYPE_CODE.getBytes())));
+    } catch (IOException e) {
+      throw new HoodieException("Could not read bloom filter from " + path, e);
+    }
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set candidateRowKeys) {
+    try {
+      List<Pair<String, R>> allRecords = readAllRecords();
+      Set<String> rowKeys = new HashSet<>();
+      allRecords.forEach(t -> {
+        if (candidateRowKeys.contains(t.getFirst())) {
+          rowKeys.add(t.getFirst());
+        }
+      });
+      return rowKeys;
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to read row keys from " + path, e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException {
+    List<Pair<String, R>> recordList = new LinkedList<>();
+    try {
+      HFileScanner scanner = reader.getScanner(false, false);
+      if (scanner.seekTo()) {
+        do {
+          Cell c = scanner.getKeyValue();
+          byte[] keyBytes = Arrays.copyOfRange(c.getRowArray(), c.getRowOffset(), c.getRowOffset() + c.getRowLength());
+          R record = readNextRecord(c, writerSchema, readerSchema);
+          recordList.add(new Pair<>(new String(keyBytes), record));
+        } while (scanner.next());
+      }
+
+      return recordList;
+    } catch (IOException e) {
+      throw new HoodieException("Error reading hfile " + path + " as a dataframe", e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords() throws IOException {
+    Schema schema = new Schema.Parser().parse(new String(reader.loadFileInfo().get("schema".getBytes())));
+    return readAllRecords(schema, schema);
+  }
+
+  @Override
+  public Iterator getRecordIterator(Schema readerSchema) throws IOException {
+    final HFileScanner scanner = reader.getScanner(false, false);
+    return new Iterator<R>() {
+      private R next = null;
+      private boolean eof = false;
+
+      @Override
+      public boolean hasNext() {
+        try {
+          // To handle when hasNext() is called multiple times for idempotency and/or the first time
+          if (this.next == null && !this.eof) {
+            if (!scanner.isSeeked() && scanner.seekTo()) {
+                this.next = (R)readNextRecord(scanner.getKeyValue(), getSchema(), readerSchema);
+            }
+          }
+          return this.next != null;
+        } catch (IOException io) {
+          throw new HoodieIOException("unable to read next record from hfile ", io);
+        }
+      }
+
+      @Override
+      public R next() {
+        try {
+          // To handle case when next() is called before hasNext()
+          if (this.next == null) {
+            if (!hasNext()) {
+              throw new HoodieIOException("No more records left to read from hfile");
+            }
+          }
+          R retVal = this.next;
+          if (scanner.next()) {
+            this.next = (R)readNextRecord(scanner.getKeyValue(), getSchema(), readerSchema);
+          } else {
+            this.next = null;
+            this.eof = true;
+          }
+          return retVal;
+        } catch (IOException io) {
+          throw new HoodieIOException("unable to read next record from parquet file ", io);
+        }
+      }
+    };
+  }
+
+  @Override
+  public Option getRecordByKey(String key, Schema readerSchema) throws IOException {
+    HFileScanner scanner = reader.getScanner(false, false);
+    KeyValue kv = new KeyValue(key.getBytes(), null, null, null);
+    if (scanner.seekTo(kv) == 0) {
+      Cell c = scanner.getKeyValue();
+      byte[] keyBytes = Arrays.copyOfRange(c.getRowArray(), c.getRowOffset(), c.getRowOffset() + c.getRowLength());
+      R record = readNextRecord(c, getSchema(), readerSchema);
+      return Option.of(record);
+    }
+
+    return Option.empty();
+  }
+
+  private R readNextRecord(Cell c, Schema writerSchema, Schema readerSchema) throws IOException {

Review comment:
       getRecordFromCell is good.

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
+        content.length, new CacheConfig(conf), conf);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
+          new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
+    } catch (IOException e) {
+      throw new HoodieException("Could not read min/max record key out of file information block correctly from path", e);
+    }
+  }
+
+  @Override
+  public Schema getSchema() {
+    if (schema == null) {
+      try {
+        Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
+        schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
+      } catch (IOException e) {
+        throw new HoodieException("Could not read schema of file from path", e);
+      }
+    }
+
+    return schema;
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      ByteBuffer serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
+      byte[] filterBytes = new byte[serializedFilter.remaining()];
+      serializedFilter.get(filterBytes); // read the bytes that were written
+      return BloomFilterFactory.fromString(new String(filterBytes),
+          new String(fileInfo.get(KEY_BLOOM_FILTER_TYPE_CODE.getBytes())));
+    } catch (IOException e) {
+      throw new HoodieException("Could not read bloom filter from " + path, e);
+    }
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set candidateRowKeys) {
+    try {
+      List<Pair<String, R>> allRecords = readAllRecords();
+      Set<String> rowKeys = new HashSet<>();
+      allRecords.forEach(t -> {
+        if (candidateRowKeys.contains(t.getFirst())) {
+          rowKeys.add(t.getFirst());
+        }
+      });
+      return rowKeys;
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to read row keys from " + path, e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException {
+    List<Pair<String, R>> recordList = new LinkedList<>();
+    try {
+      HFileScanner scanner = reader.getScanner(false, false);
+      if (scanner.seekTo()) {
+        do {
+          Cell c = scanner.getKeyValue();
+          byte[] keyBytes = Arrays.copyOfRange(c.getRowArray(), c.getRowOffset(), c.getRowOffset() + c.getRowLength());
+          R record = readNextRecord(c, writerSchema, readerSchema);
+          recordList.add(new Pair<>(new String(keyBytes), record));
+        } while (scanner.next());
+      }
+
+      return recordList;
+    } catch (IOException e) {
+      throw new HoodieException("Error reading hfile " + path + " as a dataframe", e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords() throws IOException {
+    Schema schema = new Schema.Parser().parse(new String(reader.loadFileInfo().get("schema".getBytes())));

Review comment:
       There is already a constant defined. Corrected.




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



[GitHub] [hudi] nsivabalan commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r456376140



##########
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:
       when I was benchmarking Hfile, found few other cache config params to be useful. 
   prefetch on open, cache L1 and drop behind compaction. More details can be found [here](https://issues.apache.org/jira/browse/HUDI-432). Can we add these 3 configs to this class as well

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieLogBlock.java
##########
@@ -110,7 +110,7 @@ public long getLogBlockLength() {
    * Type of the log block WARNING: This enum is serialized as the ordinal. Only add new enums at the end.
    */
   public enum HoodieLogBlockType {
-    COMMAND_BLOCK, DELETE_BLOCK, CORRUPT_BLOCK, AVRO_DATA_BLOCK
+    COMMAND_BLOCK, DELETE_BLOCK, CORRUPT_BLOCK, AVRO_DATA_BLOCK, HFILE_DATA_BLOCK

Review comment:
       correct me if I am wrong. we may not have a separate delete block for hfile, since everything is a key, value in bytes. So, we might have to fetch all values and resolve to the latest one to find if the value represents delete or active. 

##########
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:
       java docs

##########
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:
       can we move all bloom filter related code to a separate class. So that this class is in line w/ ParquetWriter as well. I am not suggesting to introduce an interface, just another class. 

##########
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:
       can you help me understand whats the usecase for this ?

##########
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:
       why another loop rather than writing it in line 121?

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java
##########
@@ -34,7 +35,17 @@
 
   public Set<String> filterRowKeys(Set<String> candidateRowKeys);
 
-  public Iterator<R> getRecordIterator(Schema schema) throws IOException;
+  public Iterator<R> getRecordIterator(Schema readerSchema) throws IOException;
+
+  default Iterator<R> getRecordIterator() throws IOException {
+    return getRecordIterator(getSchema());
+  }
+
+  public Option<R> getRecordByKey(String key, Schema readerSchema) throws IOException;

Review comment:
       do you think we introduce a config to say whether a particular file format supports single key look up. Not all file formats might support look up by Key. So, for those, we can keep this as unsupported and for Hfile kind of formats, we can have impl. 

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
+        content.length, new CacheConfig(conf), conf);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
+          new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
+    } catch (IOException e) {
+      throw new HoodieException("Could not read min/max record key out of file information block correctly from path", e);
+    }
+  }
+
+  @Override
+  public Schema getSchema() {
+    if (schema == null) {
+      try {
+        Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
+        schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
+      } catch (IOException e) {
+        throw new HoodieException("Could not read schema of file from path", e);
+      }
+    }
+
+    return schema;
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      ByteBuffer serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
+      byte[] filterBytes = new byte[serializedFilter.remaining()];
+      serializedFilter.get(filterBytes); // read the bytes that were written
+      return BloomFilterFactory.fromString(new String(filterBytes),
+          new String(fileInfo.get(KEY_BLOOM_FILTER_TYPE_CODE.getBytes())));
+    } catch (IOException e) {
+      throw new HoodieException("Could not read bloom filter from " + path, e);
+    }
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set candidateRowKeys) {

Review comment:
       not required in this patch. but we should think if we can decide whether to read entire set and filter or just do random seeks based on candidate set size to be filtered for.

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
+        content.length, new CacheConfig(conf), conf);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
+          new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
+    } catch (IOException e) {
+      throw new HoodieException("Could not read min/max record key out of file information block correctly from path", e);
+    }
+  }
+
+  @Override
+  public Schema getSchema() {
+    if (schema == null) {
+      try {
+        Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
+        schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
+      } catch (IOException e) {
+        throw new HoodieException("Could not read schema of file from path", e);
+      }
+    }
+
+    return schema;
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      ByteBuffer serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
+      byte[] filterBytes = new byte[serializedFilter.remaining()];
+      serializedFilter.get(filterBytes); // read the bytes that were written
+      return BloomFilterFactory.fromString(new String(filterBytes),
+          new String(fileInfo.get(KEY_BLOOM_FILTER_TYPE_CODE.getBytes())));
+    } catch (IOException e) {
+      throw new HoodieException("Could not read bloom filter from " + path, e);
+    }
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set candidateRowKeys) {
+    try {
+      List<Pair<String, R>> allRecords = readAllRecords();
+      Set<String> rowKeys = new HashSet<>();
+      allRecords.forEach(t -> {
+        if (candidateRowKeys.contains(t.getFirst())) {
+          rowKeys.add(t.getFirst());
+        }
+      });
+      return rowKeys;
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to read row keys from " + path, e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException {
+    List<Pair<String, R>> recordList = new LinkedList<>();
+    try {
+      HFileScanner scanner = reader.getScanner(false, false);

Review comment:
       why cacheBlocks and pread args are hard coded? shouldn't we configurize it. 

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);

Review comment:
       why do we need this seekable. HFile reader by default support look up by key right. Can you help me understand 

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
+        content.length, new CacheConfig(conf), conf);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
+          new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
+    } catch (IOException e) {
+      throw new HoodieException("Could not read min/max record key out of file information block correctly from path", e);
+    }
+  }
+
+  @Override
+  public Schema getSchema() {
+    if (schema == null) {
+      try {
+        Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
+        schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
+      } catch (IOException e) {
+        throw new HoodieException("Could not read schema of file from path", e);
+      }
+    }
+
+    return schema;
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      ByteBuffer serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
+      byte[] filterBytes = new byte[serializedFilter.remaining()];
+      serializedFilter.get(filterBytes); // read the bytes that were written
+      return BloomFilterFactory.fromString(new String(filterBytes),
+          new String(fileInfo.get(KEY_BLOOM_FILTER_TYPE_CODE.getBytes())));
+    } catch (IOException e) {
+      throw new HoodieException("Could not read bloom filter from " + path, e);
+    }
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set candidateRowKeys) {
+    try {
+      List<Pair<String, R>> allRecords = readAllRecords();
+      Set<String> rowKeys = new HashSet<>();
+      allRecords.forEach(t -> {
+        if (candidateRowKeys.contains(t.getFirst())) {
+          rowKeys.add(t.getFirst());
+        }
+      });
+      return rowKeys;
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to read row keys from " + path, e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException {
+    List<Pair<String, R>> recordList = new LinkedList<>();
+    try {
+      HFileScanner scanner = reader.getScanner(false, false);
+      if (scanner.seekTo()) {
+        do {
+          Cell c = scanner.getKeyValue();
+          byte[] keyBytes = Arrays.copyOfRange(c.getRowArray(), c.getRowOffset(), c.getRowOffset() + c.getRowLength());
+          R record = readNextRecord(c, writerSchema, readerSchema);
+          recordList.add(new Pair<>(new String(keyBytes), record));
+        } while (scanner.next());
+      }
+
+      return recordList;
+    } catch (IOException e) {
+      throw new HoodieException("Error reading hfile " + path + " as a dataframe", e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords() throws IOException {
+    Schema schema = new Schema.Parser().parse(new String(reader.loadFileInfo().get("schema".getBytes())));

Review comment:
       minor. why using hard coded string. we should use constant for "schema".

##########
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")
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieSortedMergeHandle.class);
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
+
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Called by compactor code path.
+   */
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+      Map<String, HoodieRecord<T>> keyToNewRecordsOrig, String partitionPath, String fileId,
+      HoodieBaseFile dataFileToBeMerged, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged,
+        sparkTaskContextSupplier);
+
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
+   */
+  @Override
+  public void write(GenericRecord oldRecord) {
+    String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+
+    // To maintain overall sorted order across updates and inserts, write any new inserts whose keys are less than
+    // the oldRecord's key.
+    while (!newRecordKeysSorted.isEmpty() && newRecordKeysSorted.peek().compareTo(key) <= 0) {
+      String keyToPreWrite = newRecordKeysSorted.remove();
+      if (keyToPreWrite.equals(key)) {
+        // will be handled as an update later
+        break;
+      }
+
+      // This is a new insert
+      HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(keyToPreWrite));
+      if (writtenRecordKeys.contains(keyToPreWrite)) {
+        throw new HoodieUpsertException("Insert/Update not in sorted order");
+      }
+      try {
+        if (useWriterSchema) {
+          writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
+        } else {
+          writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(originalSchema));
+        }
+        insertRecordsWritten++;
+        writtenRecordKeys.add(keyToPreWrite);
+      } catch (IOException e) {
+        throw new HoodieUpsertException("Failed to write records", e);
+      }
+    }
+
+    super.write(oldRecord);
+  }
+
+  @Override
+  public WriteStatus close() {
+    // write out any pending records (this can happen when inserts are turned into updates)
+    newRecordKeysSorted.stream().sorted().forEach(key -> {

Review comment:
       can you remind me why sorting is required here? my understanding is that newRecordKeysSorted is already sorted. 

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
+        content.length, new CacheConfig(conf), conf);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
+          new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
+    } catch (IOException e) {
+      throw new HoodieException("Could not read min/max record key out of file information block correctly from path", e);
+    }
+  }
+
+  @Override
+  public Schema getSchema() {
+    if (schema == null) {
+      try {
+        Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
+        schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
+      } catch (IOException e) {
+        throw new HoodieException("Could not read schema of file from path", e);
+      }
+    }
+
+    return schema;
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      ByteBuffer serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
+      byte[] filterBytes = new byte[serializedFilter.remaining()];
+      serializedFilter.get(filterBytes); // read the bytes that were written
+      return BloomFilterFactory.fromString(new String(filterBytes),
+          new String(fileInfo.get(KEY_BLOOM_FILTER_TYPE_CODE.getBytes())));
+    } catch (IOException e) {
+      throw new HoodieException("Could not read bloom filter from " + path, e);
+    }
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set candidateRowKeys) {
+    try {
+      List<Pair<String, R>> allRecords = readAllRecords();
+      Set<String> rowKeys = new HashSet<>();
+      allRecords.forEach(t -> {
+        if (candidateRowKeys.contains(t.getFirst())) {
+          rowKeys.add(t.getFirst());
+        }
+      });
+      return rowKeys;
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to read row keys from " + path, e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException {
+    List<Pair<String, R>> recordList = new LinkedList<>();
+    try {
+      HFileScanner scanner = reader.getScanner(false, false);
+      if (scanner.seekTo()) {
+        do {
+          Cell c = scanner.getKeyValue();
+          byte[] keyBytes = Arrays.copyOfRange(c.getRowArray(), c.getRowOffset(), c.getRowOffset() + c.getRowLength());
+          R record = readNextRecord(c, writerSchema, readerSchema);
+          recordList.add(new Pair<>(new String(keyBytes), record));
+        } while (scanner.next());
+      }
+
+      return recordList;
+    } catch (IOException e) {
+      throw new HoodieException("Error reading hfile " + path + " as a dataframe", e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords() throws IOException {
+    Schema schema = new Schema.Parser().parse(new String(reader.loadFileInfo().get("schema".getBytes())));
+    return readAllRecords(schema, schema);
+  }
+
+  @Override
+  public Iterator getRecordIterator(Schema readerSchema) throws IOException {
+    final HFileScanner scanner = reader.getScanner(false, false);
+    return new Iterator<R>() {
+      private R next = null;
+      private boolean eof = false;
+
+      @Override
+      public boolean hasNext() {
+        try {
+          // To handle when hasNext() is called multiple times for idempotency and/or the first time
+          if (this.next == null && !this.eof) {
+            if (!scanner.isSeeked() && scanner.seekTo()) {
+                this.next = (R)readNextRecord(scanner.getKeyValue(), getSchema(), readerSchema);
+            }
+          }
+          return this.next != null;
+        } catch (IOException io) {
+          throw new HoodieIOException("unable to read next record from hfile ", io);
+        }
+      }
+
+      @Override
+      public R next() {
+        try {
+          // To handle case when next() is called before hasNext()
+          if (this.next == null) {
+            if (!hasNext()) {
+              throw new HoodieIOException("No more records left to read from hfile");
+            }
+          }
+          R retVal = this.next;
+          if (scanner.next()) {
+            this.next = (R)readNextRecord(scanner.getKeyValue(), getSchema(), readerSchema);
+          } else {
+            this.next = null;
+            this.eof = true;
+          }
+          return retVal;
+        } catch (IOException io) {
+          throw new HoodieIOException("unable to read next record from parquet file ", io);
+        }
+      }
+    };
+  }
+
+  @Override
+  public Option getRecordByKey(String key, Schema readerSchema) throws IOException {
+    HFileScanner scanner = reader.getScanner(false, false);
+    KeyValue kv = new KeyValue(key.getBytes(), null, null, null);
+    if (scanner.seekTo(kv) == 0) {
+      Cell c = scanner.getKeyValue();
+      byte[] keyBytes = Arrays.copyOfRange(c.getRowArray(), c.getRowOffset(), c.getRowOffset() + c.getRowLength());
+      R record = readNextRecord(c, getSchema(), readerSchema);
+      return Option.of(record);
+    }
+
+    return Option.empty();
+  }
+
+  private R readNextRecord(Cell c, Schema writerSchema, Schema readerSchema) throws IOException {

Review comment:
       curious. why named it as readNextRecord. felt "getRecord" or "getRecordFromCell" would be more apt. 

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
+        content.length, new CacheConfig(conf), conf);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;

Review comment:
       minor. why define this outside the try block

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),

Review comment:
       is this where exception will be thrown if file does not exists or do we check file exists check elsewhere prior to this step ?

##########
File path: hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieHFileRealtimeInputFormat.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.realtime;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.hadoop.HoodieHFileInputFormat;
+import org.apache.hudi.hadoop.UseFileSplitsFromInputFormat;
+import org.apache.hudi.hadoop.UseRecordReaderFromInputFormat;
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * HoodieRealtimeInputFormat for HUDI datasets which store data in HFile base file format.
+ */
+@UseRecordReaderFromInputFormat
+@UseFileSplitsFromInputFormat
+public class HoodieHFileRealtimeInputFormat extends HoodieHFileInputFormat {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileRealtimeInputFormat.class);
+
+  @Override
+  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {

Review comment:
       similar comment as above. can we see if we can re-use code if possible. 

##########
File path: hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * HoodieInputFormat for HUDI datasets which store data in HFile base file format.
+ */
+@UseFileSplitsFromInputFormat
+public class HoodieHFileInputFormat extends FileInputFormat<NullWritable, ArrayWritable> implements Configurable {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileInputFormat.class);
+
+  protected Configuration conf;
+
+  protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) {
+    return HoodieInputFormatUtils.filterInstantsTimeline(timeline);
+  }
+
+  @Override
+  public FileStatus[] listStatus(JobConf job) throws IOException {

Review comment:
       I see lot of similarities between this and HoodieParquetInputFormat. Is there a way to re-use the code in anyway. 

##########
File path: hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieHFileRealtimeInputFormat.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.realtime;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.hadoop.HoodieHFileInputFormat;
+import org.apache.hudi.hadoop.UseFileSplitsFromInputFormat;
+import org.apache.hudi.hadoop.UseRecordReaderFromInputFormat;
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * HoodieRealtimeInputFormat for HUDI datasets which store data in HFile base file format.
+ */
+@UseRecordReaderFromInputFormat
+@UseFileSplitsFromInputFormat
+public class HoodieHFileRealtimeInputFormat extends HoodieHFileInputFormat {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileRealtimeInputFormat.class);
+
+  @Override
+  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+    Stream<FileSplit> fileSplits = Arrays.stream(super.getSplits(job, numSplits)).map(is -> (FileSplit) is);
+    return HoodieRealtimeInputFormatUtils.getRealtimeSplits(job, fileSplits);
+  }
+
+  @Override
+  public FileStatus[] listStatus(JobConf job) throws IOException {
+    // Call the HoodieInputFormat::listStatus to obtain all latest parquet files, based on commit

Review comment:
       minor: fix java docs. // parquet -> hfile

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
+  private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  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";
+
+  public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
+    this.conf = configuration;
+    this.path = path;
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+  }
+
+  public HoodieHFileReader(byte[] content) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path("hoodie");
+    SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
+    FSDataInputStream fsdis = new FSDataInputStream(bis);
+    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
+        content.length, new CacheConfig(conf), conf);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
+          new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
+    } catch (IOException e) {
+      throw new HoodieException("Could not read min/max record key out of file information block correctly from path", e);
+    }
+  }
+
+  @Override
+  public Schema getSchema() {
+    if (schema == null) {
+      try {
+        Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
+        schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
+      } catch (IOException e) {
+        throw new HoodieException("Could not read schema of file from path", e);
+      }
+    }
+
+    return schema;
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      ByteBuffer serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
+      byte[] filterBytes = new byte[serializedFilter.remaining()];
+      serializedFilter.get(filterBytes); // read the bytes that were written
+      return BloomFilterFactory.fromString(new String(filterBytes),
+          new String(fileInfo.get(KEY_BLOOM_FILTER_TYPE_CODE.getBytes())));
+    } catch (IOException e) {
+      throw new HoodieException("Could not read bloom filter from " + path, e);
+    }
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set candidateRowKeys) {
+    try {
+      List<Pair<String, R>> allRecords = readAllRecords();
+      Set<String> rowKeys = new HashSet<>();
+      allRecords.forEach(t -> {
+        if (candidateRowKeys.contains(t.getFirst())) {
+          rowKeys.add(t.getFirst());
+        }
+      });
+      return rowKeys;
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to read row keys from " + path, e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException {
+    List<Pair<String, R>> recordList = new LinkedList<>();
+    try {
+      HFileScanner scanner = reader.getScanner(false, false);
+      if (scanner.seekTo()) {
+        do {
+          Cell c = scanner.getKeyValue();
+          byte[] keyBytes = Arrays.copyOfRange(c.getRowArray(), c.getRowOffset(), c.getRowOffset() + c.getRowLength());
+          R record = readNextRecord(c, writerSchema, readerSchema);
+          recordList.add(new Pair<>(new String(keyBytes), record));
+        } while (scanner.next());
+      }
+
+      return recordList;
+    } catch (IOException e) {
+      throw new HoodieException("Error reading hfile " + path + " as a dataframe", e);
+    }
+  }
+
+  public List<Pair<String, R>> readAllRecords() throws IOException {
+    Schema schema = new Schema.Parser().parse(new String(reader.loadFileInfo().get("schema".getBytes())));
+    return readAllRecords(schema, schema);
+  }
+
+  @Override
+  public Iterator getRecordIterator(Schema readerSchema) throws IOException {
+    final HFileScanner scanner = reader.getScanner(false, false);
+    return new Iterator<R>() {
+      private R next = null;
+      private boolean eof = false;
+
+      @Override
+      public boolean hasNext() {
+        try {
+          // To handle when hasNext() is called multiple times for idempotency and/or the first time
+          if (this.next == null && !this.eof) {
+            if (!scanner.isSeeked() && scanner.seekTo()) {
+                this.next = (R)readNextRecord(scanner.getKeyValue(), getSchema(), readerSchema);
+            }
+          }
+          return this.next != null;
+        } catch (IOException io) {
+          throw new HoodieIOException("unable to read next record from hfile ", io);
+        }
+      }
+
+      @Override
+      public R next() {
+        try {
+          // To handle case when next() is called before hasNext()
+          if (this.next == null) {
+            if (!hasNext()) {
+              throw new HoodieIOException("No more records left to read from hfile");
+            }
+          }
+          R retVal = this.next;
+          if (scanner.next()) {
+            this.next = (R)readNextRecord(scanner.getKeyValue(), getSchema(), readerSchema);

Review comment:
       why move to next element here itself. shouldn't we rely on hasNext() everytime to move to to next record as per the Iterator interface contract?




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r475937184



##########
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:
       I think its here. https://github.com/nsivabalan/hudi/commit/70ee9947335c1ed0c92a285225ab728adaa7c5ab ? @nsivabalan  ? 




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



[GitHub] [hudi] prashantwason commented on pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#issuecomment-658325538


   @vinothchandar Am not currently blocked on this. Take a fine-combed look so we can make this a model for adding new file formats to HUDI. 
   
   Please suggest better testing approaches / extra documentation / code comments.


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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r478655201



##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+/**
+ * Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
+ *
+ * The implementation performs a merge-sort by comparing the key of the record being written to the list of
+ * keys in newRecordKeys (sorted in-memory).
+ */
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
+
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Called by compactor code path.
+   */
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+      Map<String, HoodieRecord<T>> keyToNewRecordsOrig, String partitionPath, String fileId,
+      HoodieBaseFile dataFileToBeMerged, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged,
+        sparkTaskContextSupplier);
+
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
+   */
+  @Override
+  public void write(GenericRecord oldRecord) {
+    String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+
+    // To maintain overall sorted order across updates and inserts, write any new inserts whose keys are less than
+    // the oldRecord's key.
+    while (!newRecordKeysSorted.isEmpty() && newRecordKeysSorted.peek().compareTo(key) <= 0) {
+      String keyToPreWrite = newRecordKeysSorted.remove();

Review comment:
       If the inputItr is sorted then yes all this overhead can be removed. 




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r475935700



##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+/**
+ * Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
+ *
+ * The implementation performs a merge-sort by comparing the key of the record being written to the list of
+ * keys in newRecordKeys (sorted in-memory).
+ */
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
+
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Called by compactor code path.
+   */
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+      Map<String, HoodieRecord<T>> keyToNewRecordsOrig, String partitionPath, String fileId,
+      HoodieBaseFile dataFileToBeMerged, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged,
+        sparkTaskContextSupplier);
+
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
+   */
+  @Override
+  public void write(GenericRecord oldRecord) {
+    String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+
+    // To maintain overall sorted order across updates and inserts, write any new inserts whose keys are less than
+    // the oldRecord's key.
+    while (!newRecordKeysSorted.isEmpty() && newRecordKeysSorted.peek().compareTo(key) <= 0) {
+      String keyToPreWrite = newRecordKeysSorted.remove();

Review comment:
       cc @prashantwason can you please chime in. I feel we can avoid the queue altogether and just sort merge? 




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r477536873



##########
File path: hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieHFileInputFormat.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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;
+
+import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
+import org.apache.hudi.common.testutils.HoodieTestUtils;
+import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
+import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapreduce.Job;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestHoodieHFileInputFormat {

Review comment:
       This is the same test as TestHoodieParquetInputFormat.java 
   
   I could not combine the two as the test calls various methods on the InputFormat (HoodieParquetInputFormat or HoodieHFileInputFormat) and we currently do not have a common base class. So this can be combined once we re-work the input formats.




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



[GitHub] [hudi] vinothchandar commented on pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#issuecomment-680381474


   actually compaction is failing. 
   
   ```
   INFO: 20/08/25 01:10:42 ERROR HoodieCompactor: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 4 times, most recent failure: Lost task 0.3 in stage 0.0 (TID 3, 172.18.0.13, executor 0): java.io.InvalidClassException: org.apache.hudi.config.HoodieWriteConfig; local class incompatible: stream classdesc serialVersionUID = 5557714552053876810, local class serialVersionUID = 2074255769914985087
   Aug 25, 2020 1:10:42 AM org.apache.hudi.cli.utils.InputStreamConsumer run
   INFO: 	at java.io.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:699)
   ```
   
   thats the real issue. looking 


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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r475815871



##########
File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
##########
@@ -81,6 +87,7 @@ public Builder fromProperties(Properties props) {
 
     public Builder limitFileSize(long maxFileSize) {
       props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
+      props.setProperty(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));

Review comment:
       not sure if its a good idea to overload two configs like this. we may need to break this builder method up separately.

##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java
##########
@@ -55,7 +56,7 @@
   private long recordsWritten = 0;
   private long insertRecordsWritten = 0;
   private long recordsDeleted = 0;
-  private Iterator<HoodieRecord<T>> recordIterator;
+  private Map<String, HoodieRecord<T>> recordMap;

Review comment:
       need to ensure that having this be a map wont affect normal inserts.

##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+/**
+ * Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
+ *
+ * The implementation performs a merge-sort by comparing the key of the record being written to the list of
+ * keys in newRecordKeys (sorted in-memory).
+ */
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
+
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Called by compactor code path.
+   */
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+      Map<String, HoodieRecord<T>> keyToNewRecordsOrig, String partitionPath, String fileId,
+      HoodieBaseFile dataFileToBeMerged, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged,
+        sparkTaskContextSupplier);
+
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
+   */
+  @Override
+  public void write(GenericRecord oldRecord) {
+    String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+
+    // To maintain overall sorted order across updates and inserts, write any new inserts whose keys are less than
+    // the oldRecord's key.
+    while (!newRecordKeysSorted.isEmpty() && newRecordKeysSorted.peek().compareTo(key) <= 0) {
+      String keyToPreWrite = newRecordKeysSorted.remove();

Review comment:
       instead, we can just do a streaming sort-merge? 

##########
File path: hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java
##########
@@ -115,6 +115,35 @@ public void testParquetDemo() throws Exception {
     testIncrementalHiveQueryAfterCompaction();
   }
 
+  @Test
+  public void testHFileDemo() throws Exception {

Review comment:
       this is good. but will add to our integration test runtime by a lot. we should disable this and enabled later. need a JIRA

##########
File path: hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieMergeOnReadTableCompactor.java
##########
@@ -135,15 +135,14 @@
 
     // Compacting is very similar to applying updates to existing file
     Iterator<List<WriteStatus>> result;
-    // If the dataFile is present, there is a base parquet file present, perform updates else perform inserts into a
-    // new base parquet file.
+    // If the dataFile is present, perform updates else perform inserts into a new base file.
     if (oldDataFileOpt.isPresent()) {
       result = hoodieCopyOnWriteTable.handleUpdate(instantTime, operation.getPartitionPath(),
               operation.getFileId(), scanner.getRecords(),
           oldDataFileOpt.get());
     } else {
       result = hoodieCopyOnWriteTable.handleInsert(instantTime, operation.getPartitionPath(), operation.getFileId(),
-          scanner.iterator());
+          scanner.getRecords());

Review comment:
       this will pay additional merge costs

##########
File path: hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieHFileInputFormat.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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;
+
+import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
+import org.apache.hudi.common.testutils.HoodieTestUtils;
+import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
+import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapreduce.Job;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestHoodieHFileInputFormat {

Review comment:
       is this copy-pasting some other test? room for code re-use? 

##########
File path: hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.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 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;
+
+    // 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()

Review comment:
       are these configs used at all now? 

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java
##########
@@ -77,4 +79,9 @@ public long getTotalRecords() {
     // TODO Auto-generated method stub
     return 0;
   }
+
+  @Override
+  public Option getRecordByKey(String key, Schema schema) throws IOException {
+    throw new HoodieException("HoodieParquetReader does not support reading records by key");

Review comment:
       throw UnsupportedException

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java
##########
@@ -34,7 +35,17 @@
 
   public Set<String> filterRowKeys(Set<String> candidateRowKeys);
 
-  public Iterator<R> getRecordIterator(Schema schema) throws IOException;
+  public Iterator<R> getRecordIterator(Schema readerSchema) throws IOException;
+
+  default Iterator<R> getRecordIterator() throws IOException {
+    return getRecordIterator(getSchema());
+  }
+
+  public Option<R> getRecordByKey(String key, Schema readerSchema) throws IOException;

Review comment:
       I think. throwing `UnSupportedException` is actually better. nothing can fundamentally change whether a format supports this or not 

##########
File path: hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java
##########
@@ -83,68 +79,6 @@ protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline tim
     return timeline;
   }
 
-  /**
-   * Add a field to the existing fields projected.
-   */
-  private static Configuration addProjectionField(Configuration conf, String fieldName, int fieldIndex) {

Review comment:
       is this dead code? 




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r475939229



##########
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:
       @prashantwason we can sort the keys being sent into the data block, using spark partitioning itself? why do we have to sort the records again explicitly? 




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



[GitHub] [hudi] vinothchandar commented on pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#issuecomment-683803209


   https://issues.apache.org/jira/browse/HUDI-1256 this tracks the follow ups from here. 


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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r458355792



##########
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")
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieSortedMergeHandle.class);
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
+
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Called by compactor code path.
+   */
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+      Map<String, HoodieRecord<T>> keyToNewRecordsOrig, String partitionPath, String fileId,
+      HoodieBaseFile dataFileToBeMerged, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged,
+        sparkTaskContextSupplier);
+
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
+   */
+  @Override
+  public void write(GenericRecord oldRecord) {
+    String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+
+    // To maintain overall sorted order across updates and inserts, write any new inserts whose keys are less than
+    // the oldRecord's key.
+    while (!newRecordKeysSorted.isEmpty() && newRecordKeysSorted.peek().compareTo(key) <= 0) {
+      String keyToPreWrite = newRecordKeysSorted.remove();
+      if (keyToPreWrite.equals(key)) {
+        // will be handled as an update later
+        break;
+      }
+
+      // This is a new insert
+      HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(keyToPreWrite));
+      if (writtenRecordKeys.contains(keyToPreWrite)) {
+        throw new HoodieUpsertException("Insert/Update not in sorted order");
+      }
+      try {
+        if (useWriterSchema) {
+          writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
+        } else {
+          writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(originalSchema));
+        }
+        insertRecordsWritten++;
+        writtenRecordKeys.add(keyToPreWrite);
+      } catch (IOException e) {
+        throw new HoodieUpsertException("Failed to write records", e);
+      }
+    }
+
+    super.write(oldRecord);
+  }
+
+  @Override
+  public WriteStatus close() {
+    // write out any pending records (this can happen when inserts are turned into updates)
+    newRecordKeysSorted.stream().sorted().forEach(key -> {

Review comment:
       Yes, sorting is not required here. Leftover from earlier where I was not using PriorityQueue for newRecordKeysSorted.




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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r477552841



##########
File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
##########
@@ -81,6 +87,7 @@ public Builder fromProperties(Properties props) {
 
     public Builder limitFileSize(long maxFileSize) {
       props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
+      props.setProperty(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));

Review comment:
       @vinothchandar I see that you split this function into file specific functions. That doable but with more base file formats added, it may be cumbersome and verbose to keep adding the .limitXXXFileSize for specific formats. 
   
   I am ok either way.




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r475907308



##########
File path: hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java
##########
@@ -45,22 +45,39 @@
 import org.apache.avro.generic.GenericRecord;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.parquet.avro.AvroSchemaConverter;
 import org.apache.parquet.hadoop.ParquetWriter;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
 import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructType;
+
+import com.databricks.spark.avro.SchemaConverters;
+
+import scala.Function1;

Review comment:
       Figured out a way to fix it for now. tests seem happy 




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



[GitHub] [hudi] prashantwason edited a comment on pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason edited a comment on pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#issuecomment-682213033


   > @prashantwason if you broadly agree, I will make the change and land this, so you can focus on rfc-15 more :)
   Sure @vinothchandar . Thanks for all the help. Lets get this rolling soon.
   
   I will look into the comments too but you can do the needful.


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



[GitHub] [hudi] prashantwason commented on a change in pull request #1804: [HUDI-960] Implementation of the HFile base and log file format.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r477534697



##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+/**
+ * Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
+ *
+ * The implementation performs a merge-sort by comparing the key of the record being written to the list of
+ * keys in newRecordKeys (sorted in-memory).
+ */
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+
+  private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
+
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
+    newRecordKeysSorted.addAll(keyToNewRecords.keySet());
+  }
+
+  /**
+   * Called by compactor code path.
+   */
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,

Review comment:
       Updated.




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