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: us...@infra.apache.org