1
0

[HUDI-960] Implementation of the HFile base and log file format. (#1804)

* [HUDI-960] Implementation of the HFile base and log file format.

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


Other changes: 
 - Remove databricks/avro from pom
 - Fix HoodieClientTestUtils from not using scala imports/reflection based conversion etc
 - Breaking up limitFileSize(), per parquet and hfile base files
 - Added three new configs for HoodieHFileConfig - prefetchBlocksOnOpen, cacheDataInL1, dropBehindCacheCompaction
 - Throw UnsupportedException in HFileReader.getRecordKeys()
 - Updated HoodieCopyOnWriteTable to create the correct merge handle (HoodieSortedMergeHandle for HFile and HoodieMergeHandle otherwise)

* Fixing checkstyle

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Prashant Wason
2020-08-31 08:05:59 -07:00
committed by GitHub
parent 6df8f88d86
commit 6461927eac
54 changed files with 2224 additions and 295 deletions

View File

@@ -35,6 +35,7 @@ import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.avro.io.BinaryDecoder;
import org.apache.avro.io.BinaryEncoder;
import org.apache.avro.io.DatumWriter;
@@ -84,7 +85,11 @@ public class HoodieAvroUtils {
* Convert a given avro record to bytes.
*/
public static byte[] avroToBytes(GenericRecord record) {
GenericDatumWriter<GenericRecord> writer = new GenericDatumWriter<>(record.getSchema());
return indexedRecordToBytes(record);
}
public static <T extends IndexedRecord> byte[] indexedRecordToBytes(T record) {
GenericDatumWriter<T> writer = new GenericDatumWriter<>(record.getSchema());
try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, reuseEncoder.get());
reuseEncoder.set(encoder);
@@ -115,9 +120,16 @@ public class HoodieAvroUtils {
* Convert serialized bytes back into avro record.
*/
public static GenericRecord bytesToAvro(byte[] bytes, Schema schema) throws IOException {
return bytesToAvro(bytes, schema, schema);
}
/**
* Convert serialized bytes back into avro record.
*/
public static GenericRecord bytesToAvro(byte[] bytes, Schema writerSchema, Schema readerSchema) throws IOException {
BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(bytes, reuseDecoder.get());
reuseDecoder.set(decoder);
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<>(schema);
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
return reader.read(null, decoder);
}

View File

@@ -27,6 +27,7 @@ import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
import org.apache.hudi.common.table.log.block.HoodieDataBlock;
import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.SpillableMapUtils;
@@ -145,6 +146,7 @@ public abstract class AbstractHoodieLogRecordScanner {
break;
}
switch (r.getBlockType()) {
case HFILE_DATA_BLOCK:
case AVRO_DATA_BLOCK:
LOG.info("Reading a data block from file " + logFile.getPath());
if (isNewInstantBlock(r) && !readBlocksLazily) {
@@ -305,6 +307,9 @@ public abstract class AbstractHoodieLogRecordScanner {
case AVRO_DATA_BLOCK:
processDataBlock((HoodieAvroDataBlock) lastBlock);
break;
case HFILE_DATA_BLOCK:
processDataBlock((HoodieHFileDataBlock) lastBlock);
break;
case DELETE_BLOCK:
Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(this::processNextDeletedKey);
break;

View File

@@ -24,6 +24,7 @@ import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
import org.apache.hudi.common.table.log.block.HoodieCorruptBlock;
import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
@@ -179,6 +180,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
// 8. Read log block length, if present. This acts as a reverse pointer when traversing a
// log file in reverse
@SuppressWarnings("unused")
long logBlockLength = 0;
if (nextBlockVersion.hasLogBlockLength()) {
logBlockLength = inputStream.readLong();
@@ -196,6 +198,9 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
return new HoodieAvroDataBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, readerSchema, header, footer);
}
case HFILE_DATA_BLOCK:
return new HoodieHFileDataBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, readerSchema, header, footer);
case DELETE_BLOCK:
return HoodieDeleteBlock.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, header, footer);

View File

@@ -77,6 +77,8 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
switch (logDataBlockFormat) {
case AVRO_DATA_BLOCK:
return new HoodieAvroDataBlock(recordList, header);
case HFILE_DATA_BLOCK:
return new HoodieHFileDataBlock(recordList, header);
default:
throw new HoodieException("Data block format " + logDataBlockFormat + " not implemented");
}

View File

@@ -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<>();
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.indexedRecordToBytes(record);
sortedRecordsMap.put(recordKey, recordBytes);
}
// Write the records
sortedRecordsMap.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());
List<Pair<String, IndexedRecord>> records = reader.readAllRecords(writerSchema, schema);
this.records = records.stream().map(t -> t.getSecond()).collect(Collectors.toList());
// Free up content to be GC'd, deflate
deflate();
}
}

View File

@@ -110,7 +110,7 @@ public abstract class HoodieLogBlock {
* 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
}
/**

View File

@@ -25,6 +25,7 @@ import java.util.Set;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.util.Option;
public interface HoodieFileReader<R extends IndexedRecord> {
@@ -34,7 +35,19 @@ public interface HoodieFileReader<R extends IndexedRecord> {
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());
}
default Option<R> getRecordByKey(String key, Schema readerSchema) throws IOException {
throw new UnsupportedOperationException();
}
default Option<R> getRecordByKey(String key) throws IOException {
return getRecordByKey(key, getSchema());
}
Schema getSchema();

View File

@@ -24,10 +24,12 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import java.io.IOException;
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;
public class HoodieFileReaderFactory {
@@ -37,6 +39,10 @@ public class HoodieFileReaderFactory {
if (PARQUET.getFileExtension().equals(extension)) {
return newParquetFileReader(conf, path);
}
if (HFILE.getFileExtension().equals(extension)) {
return newHFileFileReader(conf, path);
}
throw new UnsupportedOperationException(extension + " format not supported yet.");
}
@@ -44,4 +50,10 @@ public class HoodieFileReaderFactory {
Configuration conf, Path path) throws IOException {
return new HoodieParquetReader<>(conf, path);
}
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileReader<R> newHFileFileReader(
Configuration conf, Path path) throws IOException {
CacheConfig cacheConfig = new CacheConfig(conf);
return new HoodieHFileReader<>(conf, path, cacheConfig);
}
}

View File

@@ -0,0 +1,300 @@
/*
* 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;
public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader {
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() {
try {
Map<byte[], byte[]> 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) {
// Current implementation reads all records and filters them. In certain cases, it many be better to:
// 1. Scan a limited subset of keys (min/max range of candidateRowKeys)
// 2. Lookup keys individually (if the size of candidateRowKeys is much less than the total keys in file)
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 = getRecordFromCell(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(KEY_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)getRecordFromCell(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)getRecordFromCell(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, true);
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 = getRecordFromCell(c, getSchema(), readerSchema);
return Option.of(record);
}
return Option.empty();
}
private R getRecordFromCell(Cell c, Schema writerSchema, Schema readerSchema) throws IOException {
byte[] value = Arrays.copyOfRange(c.getValueArray(), c.getValueOffset(), c.getValueOffset() + c.getValueLength());
return (R)HoodieAvroUtils.bytesToAvro(value, writerSchema, readerSchema);
}
@Override
public long getTotalRecords() {
return reader.getEntries();
}
@Override
public void close() {
try {
reader.close();
reader = null;
} catch (IOException e) {
e.printStackTrace();
}
}
static class SeekableByteArrayInputStream extends ByteArrayInputStream implements Seekable, PositionedReadable {
public SeekableByteArrayInputStream(byte[] buf) {
super(buf);
}
@Override
public long getPos() throws IOException {
return pos;
}
@Override
public void seek(long pos) throws IOException {
if (mark != 0) {
throw new IllegalStateException();
}
reset();
long skipped = skip(pos);
if (skipped != pos) {
throw new IOException();
}
}
@Override
public boolean seekToNewSource(long targetPos) throws IOException {
return false;
}
@Override
public int read(long position, byte[] buffer, int offset, int length) throws IOException {
if (position >= buf.length) {
throw new IllegalArgumentException();
}
if (position + length > buf.length) {
throw new IllegalArgumentException();
}
if (length > buffer.length) {
throw new IllegalArgumentException();
}
System.arraycopy(buf, (int) position, buffer, offset, length);
return length;
}
@Override
public void readFully(long position, byte[] buffer) throws IOException {
read(position, buffer, 0, buffer.length);
}
@Override
public void readFully(long position, byte[] buffer, int offset, int length) throws IOException {
read(position, buffer, offset, length);
}
}
}