1
0

[HUDI-431] Adding support for Parquet in MOR LogBlocks (#4333)

- Adding support for Parquet in MOR tables Log blocks

Co-authored-by: Sivabalan Narayanan <n.siva.b@gmail.com>
This commit is contained in:
Alexey Kudinkin
2022-02-02 11:35:05 -08:00
committed by GitHub
parent caef3d5c58
commit a68e1dc2db
45 changed files with 1613 additions and 670 deletions

View File

@@ -108,7 +108,7 @@ public class TestHoodieLogFileCommand extends CLIFunctionalTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, INSTANT_TIME); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, INSTANT_TIME);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = new HoodieAvroDataBlock(records, header); dataBlock = new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
} }
} }
@@ -188,7 +188,7 @@ public class TestHoodieLogFileCommand extends CLIFunctionalTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, INSTANT_TIME); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, INSTANT_TIME);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
} finally { } finally {
if (writer != null) { if (writer != null) {

View File

@@ -83,14 +83,17 @@ public class HoodieStorageConfig extends HoodieConfig {
.withDocumentation("Lower values increase the size of metadata tracked within HFile, but can offer potentially " .withDocumentation("Lower values increase the size of metadata tracked within HFile, but can offer potentially "
+ "faster lookup times."); + "faster lookup times.");
// used to size log files public static final ConfigProperty<String> LOGFILE_DATA_BLOCK_FORMAT = ConfigProperty
.key("hoodie.logfile.data.block.format")
.noDefaultValue()
.withDocumentation("Format of the data block within delta logs. Following formats are currently supported \"avro\", \"hfile\", \"parquet\"");
public static final ConfigProperty<String> LOGFILE_MAX_SIZE = ConfigProperty public static final ConfigProperty<String> LOGFILE_MAX_SIZE = ConfigProperty
.key("hoodie.logfile.max.size") .key("hoodie.logfile.max.size")
.defaultValue(String.valueOf(1024 * 1024 * 1024)) // 1 GB .defaultValue(String.valueOf(1024 * 1024 * 1024)) // 1 GB
.withDocumentation("LogFile max size. This is the maximum size allowed for a log file " .withDocumentation("LogFile max size. This is the maximum size allowed for a log file "
+ "before it is rolled over to the next version."); + "before it is rolled over to the next version.");
// used to size data blocks in log file
public static final ConfigProperty<String> LOGFILE_DATA_BLOCK_MAX_SIZE = ConfigProperty public static final ConfigProperty<String> LOGFILE_DATA_BLOCK_MAX_SIZE = ConfigProperty
.key("hoodie.logfile.data.block.max.size") .key("hoodie.logfile.data.block.max.size")
.defaultValue(String.valueOf(256 * 1024 * 1024)) .defaultValue(String.valueOf(256 * 1024 * 1024))

View File

@@ -37,9 +37,11 @@ import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.model.WriteConcurrencyMode;
import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.marker.MarkerType;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.metrics.HoodieMetricsConfig; import org.apache.hudi.config.metrics.HoodieMetricsConfig;
@@ -1506,6 +1508,11 @@ public class HoodieWriteConfig extends HoodieConfig {
return getString(HoodieStorageConfig.PARQUET_OUTPUT_TIMESTAMP_TYPE); return getString(HoodieStorageConfig.PARQUET_OUTPUT_TIMESTAMP_TYPE);
} }
public Option<HoodieLogBlock.HoodieLogBlockType> getLogDataBlockFormat() {
return Option.ofNullable(getString(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT))
.map(HoodieLogBlock.HoodieLogBlockType::fromId);
}
public long getLogFileMaxSize() { public long getLogFileMaxSize() {
return getLong(HoodieStorageConfig.LOGFILE_MAX_SIZE); return getLong(HoodieStorageConfig.LOGFILE_MAX_SIZE);
} }

View File

@@ -38,10 +38,12 @@ import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.log.AppendResult; import org.apache.hudi.common.table.log.AppendResult;
import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; 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;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock;
import org.apache.hudi.common.table.view.TableFileSystemView.SliceView; import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
@@ -49,6 +51,7 @@ import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.SizeEstimator; import org.apache.hudi.common.util.SizeEstimator;
import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieAppendException; import org.apache.hudi.exception.HoodieAppendException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.HoodieTable;
@@ -360,13 +363,13 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchemaWithMetaFields.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchemaWithMetaFields.toString());
List<HoodieLogBlock> blocks = new ArrayList<>(2); List<HoodieLogBlock> blocks = new ArrayList<>(2);
if (recordList.size() > 0) { if (recordList.size() > 0) {
if (config.populateMetaFields()) { String keyField = config.populateMetaFields()
blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header)); ? HoodieRecord.RECORD_KEY_METADATA_FIELD
} else { : hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp();
final String keyField = hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp();
blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header, keyField)); blocks.add(getBlock(config, pickLogDataBlockFormat(), recordList, header, keyField));
}
} }
if (keysToDelete.size() > 0) { if (keysToDelete.size() > 0) {
blocks.add(new HoodieDeleteBlock(keysToDelete.toArray(new HoodieKey[keysToDelete.size()]), header)); blocks.add(new HoodieDeleteBlock(keysToDelete.toArray(new HoodieKey[keysToDelete.size()]), header));
} }
@@ -497,4 +500,40 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
numberOfRecords = 0; numberOfRecords = 0;
} }
} }
private HoodieLogBlock.HoodieLogBlockType pickLogDataBlockFormat() {
Option<HoodieLogBlock.HoodieLogBlockType> logBlockTypeOpt = config.getLogDataBlockFormat();
if (logBlockTypeOpt.isPresent()) {
return logBlockTypeOpt.get();
}
// Fallback to deduce data-block type based on the base file format
switch (hoodieTable.getBaseFileFormat()) {
case PARQUET:
case ORC:
return HoodieLogBlock.HoodieLogBlockType.AVRO_DATA_BLOCK;
case HFILE:
return HoodieLogBlock.HoodieLogBlockType.HFILE_DATA_BLOCK;
default:
throw new HoodieException("Base file format " + hoodieTable.getBaseFileFormat()
+ " does not have associated log block type");
}
}
private static HoodieLogBlock getBlock(HoodieWriteConfig writeConfig,
HoodieLogBlock.HoodieLogBlockType logDataBlockFormat,
List<IndexedRecord> recordList,
Map<HeaderMetadataType, String> header,
String keyField) {
switch (logDataBlockFormat) {
case AVRO_DATA_BLOCK:
return new HoodieAvroDataBlock(recordList, header, keyField);
case HFILE_DATA_BLOCK:
return new HoodieHFileDataBlock(recordList, header, writeConfig.getHFileCompressionAlgorithm());
case PARQUET_DATA_BLOCK:
return new HoodieParquetDataBlock(recordList, header, keyField, writeConfig.getParquetCompressionCodec());
default:
throw new HoodieException("Data block format " + logDataBlockFormat + " not implemented");
}
}
} }

View File

@@ -51,13 +51,23 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
private final TaskContextSupplier taskContextSupplier; private final TaskContextSupplier taskContextSupplier;
private final boolean populateMetaFields; private final boolean populateMetaFields;
public HoodieParquetWriter(String instantTime, Path file, HoodieAvroParquetConfig parquetConfig, public HoodieParquetWriter(String instantTime,
Schema schema, TaskContextSupplier taskContextSupplier, boolean populateMetaFields) throws IOException { Path file,
HoodieAvroParquetConfig parquetConfig,
Schema schema,
TaskContextSupplier taskContextSupplier,
boolean populateMetaFields) throws IOException {
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()), super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(), ParquetFileWriter.Mode.CREATE,
parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(), parquetConfig.getWriteSupport(),
parquetConfig.dictionaryEnabled(), DEFAULT_IS_VALIDATING_ENABLED, parquetConfig.getCompressionCodecName(),
DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf())); parquetConfig.getBlockSize(),
parquetConfig.getPageSize(),
parquetConfig.getPageSize(),
parquetConfig.dictionaryEnabled(),
DEFAULT_IS_VALIDATING_ENABLED,
DEFAULT_WRITER_VERSION,
FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()); this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
this.fs = this.fs =
(HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf())); (HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));

View File

@@ -18,7 +18,11 @@
package org.apache.hudi.table; package org.apache.hudi.table;
import org.apache.avro.Schema;
import org.apache.avro.specific.SpecificRecordBase; import org.apache.avro.specific.SpecificRecordBase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCleanerPlan;
@@ -44,7 +48,6 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -69,11 +72,6 @@ import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
import org.apache.hudi.table.marker.WriteMarkers; import org.apache.hudi.table.marker.WriteMarkers;
import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.hudi.table.marker.WriteMarkersFactory;
import org.apache.avro.Schema;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.table.storage.HoodieLayoutFactory; import org.apache.hudi.table.storage.HoodieLayoutFactory;
import org.apache.hudi.table.storage.HoodieStorageLayout; import org.apache.hudi.table.storage.HoodieStorageLayout;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
@@ -719,19 +717,6 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
return metaClient.getTableConfig().getLogFileFormat(); return metaClient.getTableConfig().getLogFileFormat();
} }
public HoodieLogBlockType getLogDataBlockFormat() {
switch (getBaseFileFormat()) {
case PARQUET:
case ORC:
return HoodieLogBlockType.AVRO_DATA_BLOCK;
case HFILE:
return HoodieLogBlockType.HFILE_DATA_BLOCK;
default:
throw new HoodieException("Base file format " + getBaseFileFormat()
+ " does not have associated log block format");
}
}
public String getBaseFileExtension() { public String getBaseFileExtension() {
return getBaseFileFormat().getFileExtension(); return getBaseFileFormat().getFileExtension();
} }

View File

@@ -182,7 +182,7 @@ public class HoodieWriteableTestTable extends HoodieMetadataTestTable {
LOG.warn("Failed to convert record " + r.toString(), e); LOG.warn("Failed to convert record " + r.toString(), e);
return null; return null;
} }
}).collect(Collectors.toList()), header)); }).collect(Collectors.toList()), header, HoodieRecord.RECORD_KEY_METADATA_FIELD));
return Pair.of(partitionPath, logWriter.getLogFile()); return Pair.of(partitionPath, logWriter.getLogFile());
} }
} }

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.testutils; package org.apache.hudi.testutils;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterFactory;
@@ -39,6 +40,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
@@ -132,12 +134,12 @@ public class HoodieFlinkWriteableTestTable extends HoodieWriteableTestTable {
try { try {
GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get(); GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get();
HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), ""); HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), "");
return (org.apache.avro.generic.IndexedRecord) val; return (IndexedRecord) val;
} catch (java.io.IOException e) { } catch (IOException e) {
LOG.warn("Failed to convert record " + r.toString(), e); LOG.warn("Failed to convert record " + r.toString(), e);
return null; return null;
} }
}).collect(Collectors.toList()), header)); }).collect(Collectors.toList()), header, HoodieRecord.RECORD_KEY_METADATA_FIELD));
return Pair.of(partitionPath, logWriter.getLogFile()); return Pair.of(partitionPath, logWriter.getLogFile());
} }
} }

View File

@@ -18,17 +18,6 @@
package org.apache.hudi.avro; package org.apache.hudi.avro;
import org.apache.hudi.common.config.SerializableSchema;
import org.apache.hudi.common.model.HoodieOperation;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.SchemaCompatibilityException;
import org.apache.avro.Conversions.DecimalConversion; import org.apache.avro.Conversions.DecimalConversion;
import org.apache.avro.JsonProperties; import org.apache.avro.JsonProperties;
import org.apache.avro.LogicalTypes; import org.apache.avro.LogicalTypes;
@@ -50,15 +39,22 @@ import org.apache.avro.io.EncoderFactory;
import org.apache.avro.io.JsonDecoder; import org.apache.avro.io.JsonDecoder;
import org.apache.avro.io.JsonEncoder; import org.apache.avro.io.JsonEncoder;
import org.apache.avro.specific.SpecificRecordBase; import org.apache.avro.specific.SpecificRecordBase;
import org.apache.hudi.common.config.SerializableSchema;
import org.apache.hudi.common.model.HoodieOperation;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.SchemaCompatibilityException;
import java.io.ByteArrayInputStream; import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.math.BigDecimal; import java.math.BigDecimal;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.sql.Timestamp; import java.sql.Timestamp;
import java.time.LocalDate; import java.time.LocalDate;
import java.util.ArrayList; import java.util.ArrayList;
@@ -67,8 +63,6 @@ import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;
/** /**
* Helper class to do common stuff across Avro. * Helper class to do common stuff across Avro.
@@ -343,7 +337,7 @@ public class HoodieAvroUtils {
} }
/** /**
* Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the new * Given an Avro record with a given schema, rewrites it into the new schema while setting fields only from the new
* schema. * schema.
* NOTE: Here, the assumption is that you cannot go from an evolved schema (schema with (N) fields) * NOTE: Here, the assumption is that you cannot go from an evolved schema (schema with (N) fields)
* to an older schema (schema with (N-1) fields). All fields present in the older record schema MUST be present in the * to an older schema (schema with (N-1) fields). All fields present in the older record schema MUST be present in the
@@ -377,6 +371,16 @@ public class HoodieAvroUtils {
return newRecord; return newRecord;
} }
/**
* Converts list of {@link GenericRecord} provided into the {@link GenericRecord} adhering to the
* provided {@code newSchema}.
*
* To better understand conversion rules please check {@link #rewriteRecord(GenericRecord, Schema)}
*/
public static List<GenericRecord> rewriteRecords(List<GenericRecord> records, Schema newSchema) {
return records.stream().map(r -> rewriteRecord(r, newSchema)).collect(Collectors.toList());
}
private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field f) { private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field f) {
// cache the result of oldRecord.get() to save CPU expensive hash lookup // cache the result of oldRecord.get() to save CPU expensive hash lookup
Schema oldSchema = oldRecord.getSchema(); Schema oldSchema = oldRecord.getSchema();
@@ -392,33 +396,6 @@ public class HoodieAvroUtils {
} }
} }
public static byte[] compress(String text) {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
OutputStream out = new DeflaterOutputStream(baos);
out.write(text.getBytes(StandardCharsets.UTF_8));
out.close();
} catch (IOException e) {
throw new HoodieIOException("IOException while compressing text " + text, e);
}
return baos.toByteArray();
}
public static String decompress(byte[] bytes) {
InputStream in = new InflaterInputStream(new ByteArrayInputStream(bytes));
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
byte[] buffer = new byte[8192];
int len;
while ((len = in.read(buffer)) > 0) {
baos.write(buffer, 0, len);
}
return new String(baos.toByteArray(), StandardCharsets.UTF_8);
} catch (IOException e) {
throw new HoodieIOException("IOException while decompressing text", e);
}
}
/** /**
* Generate a reader schema off the provided writeSchema, to just project out the provided columns. * Generate a reader schema off the provided writeSchema, to just project out the provided columns.
*/ */

View File

@@ -18,11 +18,10 @@
package org.apache.hudi.avro; package org.apache.hudi.avro;
import org.apache.avro.Schema;
import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter; import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
import org.apache.avro.Schema;
import org.apache.parquet.avro.AvroWriteSupport; import org.apache.parquet.avro.AvroWriteSupport;
import org.apache.parquet.hadoop.api.WriteSupport; import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.MessageType;

View File

@@ -135,6 +135,17 @@ public class FSUtils {
return providedPath; return providedPath;
} }
/**
* Makes path qualified w/ {@link FileSystem}'s URI
*
* @param fs instance of {@link FileSystem} path belongs to
* @param path path to be qualified
* @return qualified path, prefixed w/ the URI of the target FS object provided
*/
public static Path makeQualified(FileSystem fs, Path path) {
return path.makeQualified(fs.getUri(), fs.getWorkingDirectory());
}
/** /**
* A write token uniquely identifies an attempt at one of the IOHandle operations (Merge/Create/Append). * A write token uniquely identifies an attempt at one of the IOHandle operations (Merge/Create/Append).
*/ */

View File

@@ -136,7 +136,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
} }
} }
private static Path convertPathWithScheme(Path oldPath, String newScheme) { public static Path convertPathWithScheme(Path oldPath, String newScheme) {
URI oldURI = oldPath.toUri(); URI oldURI = oldPath.toUri();
URI newURI; URI newURI;
try { try {

View File

@@ -19,10 +19,11 @@
package org.apache.hudi.common.fs.inline; package org.apache.hudi.common.fs.inline;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.util.ValidationUtils;
import java.io.File; import java.io.File;
import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
/** /**
* Utils to parse InLineFileSystem paths. * Utils to parse InLineFileSystem paths.
* Inline FS format: * Inline FS format:
@@ -61,10 +62,10 @@ public class InLineFSUtils {
/** /**
* InlineFS Path format: * InlineFS Path format:
* "inlinefs://path/to/outer/file/outer_file_schema/?start_offset=start_offset>&length=<length>" * "inlinefs://path/to/outer/file/outer_file_scheme/?start_offset=start_offset>&length=<length>"
* <p> * <p>
* Outer File Path format: * Outer File Path format:
* "outer_file_schema://path/to/outer/file" * "outer_file_scheme://path/to/outer/file"
* <p> * <p>
* Example * Example
* Input: "inlinefs://file1/s3a/?start_offset=20&length=40". * Input: "inlinefs://file1/s3a/?start_offset=20&length=40".
@@ -74,40 +75,48 @@ public class InLineFSUtils {
* @return Outer file Path from the InLineFS Path * @return Outer file Path from the InLineFS Path
*/ */
public static Path getOuterFilePathFromInlinePath(Path inlineFSPath) { public static Path getOuterFilePathFromInlinePath(Path inlineFSPath) {
final String scheme = inlineFSPath.getParent().getName(); assertInlineFSPath(inlineFSPath);
final String outerFileScheme = inlineFSPath.getParent().getName();
final Path basePath = inlineFSPath.getParent().getParent(); final Path basePath = inlineFSPath.getParent().getParent();
ValidationUtils.checkArgument(basePath.toString().contains(SCHEME_SEPARATOR), checkArgument(basePath.toString().contains(SCHEME_SEPARATOR),
"Invalid InLineFSPath: " + inlineFSPath); "Invalid InLineFS path: " + inlineFSPath);
final String pathExceptScheme = basePath.toString().substring(basePath.toString().indexOf(SCHEME_SEPARATOR) + 1); final String pathExceptScheme = basePath.toString().substring(basePath.toString().indexOf(SCHEME_SEPARATOR) + 1);
final String fullPath = scheme + SCHEME_SEPARATOR final String fullPath = outerFileScheme + SCHEME_SEPARATOR
+ (scheme.equals(LOCAL_FILESYSTEM_SCHEME) ? PATH_SEPARATOR : "") + (outerFileScheme.equals(LOCAL_FILESYSTEM_SCHEME) ? PATH_SEPARATOR : "")
+ pathExceptScheme; + pathExceptScheme;
return new Path(fullPath); return new Path(fullPath);
} }
/** /**
* Eg input : "inlinefs://file1/s3a/?start_offset=20&length=40". * Returns start offset w/in the base for the block identified by the given InlineFS path
* output: 20
* *
* @param inlinePath * input: "inlinefs://file1/s3a/?start_offset=20&length=40".
* @return * output: 20
*/ */
public static int startOffset(Path inlinePath) { public static int startOffset(Path inlineFSPath) {
String[] slices = inlinePath.toString().split("[?&=]"); assertInlineFSPath(inlineFSPath);
String[] slices = inlineFSPath.toString().split("[?&=]");
return Integer.parseInt(slices[slices.length - 3]); return Integer.parseInt(slices[slices.length - 3]);
} }
/** /**
* Eg input : "inlinefs:/file1/s3a/?start_offset=20&length=40". * Returns length of the block (embedded w/in the base file) identified by the given InlineFS path
* Output: 40
* *
* @param inlinePath * input: "inlinefs:/file1/s3a/?start_offset=20&length=40".
* @return * output: 40
*/ */
public static int length(Path inlinePath) { public static int length(Path inlinePath) {
assertInlineFSPath(inlinePath);
String[] slices = inlinePath.toString().split("[?&=]"); String[] slices = inlinePath.toString().split("[?&=]");
return Integer.parseInt(slices[slices.length - 1]); return Integer.parseInt(slices[slices.length - 1]);
} }
private static void assertInlineFSPath(Path inlinePath) {
String scheme = inlinePath.toUri().getScheme();
checkArgument(InLineFileSystem.SCHEME.equals(scheme));
}
} }

View File

@@ -57,6 +57,7 @@ public class InLineFileSystem extends FileSystem {
return URI.create(getScheme()); return URI.create(getScheme());
} }
@Override
public String getScheme() { public String getScheme() {
return SCHEME; return SCHEME;
} }
@@ -129,5 +130,4 @@ public class InLineFileSystem extends FileSystem {
public boolean mkdirs(Path path, FsPermission fsPermission) throws IOException { public boolean mkdirs(Path path, FsPermission fsPermission) throws IOException {
throw new UnsupportedOperationException("Can't set working directory"); throw new UnsupportedOperationException("Can't set working directory");
} }
} }

View File

@@ -18,11 +18,10 @@
package org.apache.hudi.common.model; package org.apache.hudi.common.model;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.fs.FSUtils;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
@@ -60,7 +59,7 @@ public class HoodieLogFile implements Serializable {
public HoodieLogFile(Path logPath) { public HoodieLogFile(Path logPath) {
this.fileStatus = null; this.fileStatus = null;
this.pathStr = logPath.toString(); this.pathStr = logPath.toString();
this.fileLen = 0; this.fileLen = -1;
} }
public HoodieLogFile(Path logPath, Long fileLen) { public HoodieLogFile(Path logPath, Long fileLen) {

View File

@@ -18,6 +18,11 @@
package org.apache.hudi.common.table; package org.apache.hudi.common.table;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex; import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
import org.apache.hudi.common.bootstrap.index.NoOpBootstrapIndex; import org.apache.hudi.common.bootstrap.index.NoOpBootstrapIndex;
import org.apache.hudi.common.config.ConfigClassProperty; import org.apache.hudi.common.config.ConfigClassProperty;
@@ -36,12 +41,6 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;

View File

@@ -30,6 +30,7 @@ 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.HoodieDeleteBlock;
import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock; 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;
import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock;
import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.hudi.common.util.SpillableMapUtils;
@@ -230,6 +231,7 @@ public abstract class AbstractHoodieLogRecordReader {
switch (logBlock.getBlockType()) { switch (logBlock.getBlockType()) {
case HFILE_DATA_BLOCK: case HFILE_DATA_BLOCK:
case AVRO_DATA_BLOCK: case AVRO_DATA_BLOCK:
case PARQUET_DATA_BLOCK:
LOG.info("Reading a data block from file " + logFile.getPath() + " at instant " LOG.info("Reading a data block from file " + logFile.getPath() + " at instant "
+ logBlock.getLogBlockHeader().get(INSTANT_TIME)); + logBlock.getLogBlockHeader().get(INSTANT_TIME));
if (isNewInstantBlock(logBlock) && !readBlocksLazily) { if (isNewInstantBlock(logBlock) && !readBlocksLazily) {
@@ -426,6 +428,9 @@ public abstract class AbstractHoodieLogRecordReader {
case HFILE_DATA_BLOCK: case HFILE_DATA_BLOCK:
processDataBlock((HoodieHFileDataBlock) lastBlock, keys); processDataBlock((HoodieHFileDataBlock) lastBlock, keys);
break; break;
case PARQUET_DATA_BLOCK:
processDataBlock((HoodieParquetDataBlock) lastBlock, keys);
break;
case DELETE_BLOCK: case DELETE_BLOCK:
Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(this::processNextDeletedKey); Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(this::processNextDeletedKey);
break; break;

View File

@@ -18,6 +18,13 @@
package org.apache.hudi.common.table.log; package org.apache.hudi.common.table.log;
import org.apache.avro.Schema;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BufferedFSInputStream;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.fs.SchemeAwareFSDataInputStream; import org.apache.hudi.common.fs.SchemeAwareFSDataInputStream;
import org.apache.hudi.common.fs.TimedFSDataInputStream; import org.apache.hudi.common.fs.TimedFSDataInputStream;
@@ -31,21 +38,15 @@ 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;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.CorruptedLogFileException; import org.apache.hudi.exception.CorruptedLogFileException;
import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.BufferedFSInputStream;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import javax.annotation.Nullable;
import java.io.EOFException; import java.io.EOFException;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
@@ -53,6 +54,9 @@ import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
/** /**
* Scans a log file and provides block level iterator on the log file Loads the entire block contents in memory Can emit * Scans a log file and provides block level iterator on the log file Loads the entire block contents in memory Can emit
* either a DataBlock, CommandBlock, DeleteBlock or CorruptBlock (if one is found). * either a DataBlock, CommandBlock, DeleteBlock or CorruptBlock (if one is found).
@@ -63,6 +67,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
private static final int BLOCK_SCAN_READ_BUFFER_SIZE = 1024 * 1024; // 1 MB private static final int BLOCK_SCAN_READ_BUFFER_SIZE = 1024 * 1024; // 1 MB
private static final Logger LOG = LogManager.getLogger(HoodieLogFileReader.class); private static final Logger LOG = LogManager.getLogger(HoodieLogFileReader.class);
private final Configuration hadoopConf;
private final FSDataInputStream inputStream; private final FSDataInputStream inputStream;
private final HoodieLogFile logFile; private final HoodieLogFile logFile;
private final byte[] magicBuffer = new byte[6]; private final byte[] magicBuffer = new byte[6];
@@ -72,7 +77,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
private long reverseLogFilePosition; private long reverseLogFilePosition;
private long lastReverseLogFilePosition; private long lastReverseLogFilePosition;
private boolean reverseReader; private boolean reverseReader;
private boolean enableInlineReading; private boolean enableRecordLookups;
private boolean closed = false; private boolean closed = false;
private transient Thread shutdownThread = null; private transient Thread shutdownThread = null;
@@ -88,76 +93,26 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
} }
public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
boolean readBlockLazily, boolean reverseReader, boolean enableInlineReading, boolean readBlockLazily, boolean reverseReader, boolean enableRecordLookups,
String keyField) throws IOException { String keyField) throws IOException {
FSDataInputStream fsDataInputStream = fs.open(logFile.getPath(), bufferSize); this.hadoopConf = fs.getConf();
this.logFile = logFile; // NOTE: We repackage {@code HoodieLogFile} here to make sure that the provided path
this.inputStream = getFSDataInputStream(fsDataInputStream, fs, bufferSize); // is prefixed with an appropriate scheme given that we're not propagating the FS
// further
this.logFile = new HoodieLogFile(FSUtils.makeQualified(fs, logFile.getPath()), logFile.getFileSize());
this.inputStream = getFSDataInputStream(fs, this.logFile, bufferSize);
this.readerSchema = readerSchema; this.readerSchema = readerSchema;
this.readBlockLazily = readBlockLazily; this.readBlockLazily = readBlockLazily;
this.reverseReader = reverseReader; this.reverseReader = reverseReader;
this.enableInlineReading = enableInlineReading; this.enableRecordLookups = enableRecordLookups;
this.keyField = keyField; this.keyField = keyField;
if (this.reverseReader) { if (this.reverseReader) {
this.reverseLogFilePosition = this.lastReverseLogFilePosition = logFile.getFileSize(); this.reverseLogFilePosition = this.lastReverseLogFilePosition = this.logFile.getFileSize();
} }
addShutDownHook(); addShutDownHook();
} }
public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) throws IOException {
this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, false, false);
}
/**
* Fetch the right {@link FSDataInputStream} to be used by wrapping with required input streams.
* @param fsDataInputStream original instance of {@link FSDataInputStream}.
* @param fs instance of {@link FileSystem} in use.
* @param bufferSize buffer size to be used.
* @return the right {@link FSDataInputStream} as required.
*/
private FSDataInputStream getFSDataInputStream(FSDataInputStream fsDataInputStream, FileSystem fs, int bufferSize) {
if (FSUtils.isGCSFileSystem(fs)) {
// in GCS FS, we might need to interceptor seek offsets as we might get EOF exception
return new SchemeAwareFSDataInputStream(getFSDataInputStreamForGCS(fsDataInputStream, bufferSize), true);
}
if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) {
return new TimedFSDataInputStream(logFile.getPath(), new FSDataInputStream(
new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(), bufferSize)));
}
// fsDataInputStream.getWrappedStream() maybe a BufferedFSInputStream
// need to wrap in another BufferedFSInputStream the make bufferSize work?
return fsDataInputStream;
}
/**
* GCS FileSystem needs some special handling for seek and hence this method assists to fetch the right {@link FSDataInputStream} to be
* used by wrapping with required input streams.
* @param fsDataInputStream original instance of {@link FSDataInputStream}.
* @param bufferSize buffer size to be used.
* @return the right {@link FSDataInputStream} as required.
*/
private FSDataInputStream getFSDataInputStreamForGCS(FSDataInputStream fsDataInputStream, int bufferSize) {
// incase of GCS FS, there are two flows.
// a. fsDataInputStream.getWrappedStream() instanceof FSInputStream
// b. fsDataInputStream.getWrappedStream() not an instanceof FSInputStream, but an instance of FSDataInputStream.
// (a) is handled in the first if block and (b) is handled in the second if block. If not, we fallback to original fsDataInputStream
if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) {
return new TimedFSDataInputStream(logFile.getPath(), new FSDataInputStream(
new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(), bufferSize)));
}
if (fsDataInputStream.getWrappedStream() instanceof FSDataInputStream
&& ((FSDataInputStream) fsDataInputStream.getWrappedStream()).getWrappedStream() instanceof FSInputStream) {
FSInputStream inputStream = (FSInputStream)((FSDataInputStream) fsDataInputStream.getWrappedStream()).getWrappedStream();
return new TimedFSDataInputStream(logFile.getPath(),
new FSDataInputStream(new BufferedFSInputStream(inputStream, bufferSize)));
}
return fsDataInputStream;
}
@Override @Override
public HoodieLogFile getLogFile() { public HoodieLogFile getLogFile() {
return logFile; return logFile;
@@ -181,15 +136,10 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
// TODO : convert content and block length to long by using ByteBuffer, raw byte [] allows // TODO : convert content and block length to long by using ByteBuffer, raw byte [] allows
// for max of Integer size // for max of Integer size
private HoodieLogBlock readBlock() throws IOException { private HoodieLogBlock readBlock() throws IOException {
int blockSize;
int blocksize;
int type;
HoodieLogBlockType blockType = null;
Map<HeaderMetadataType, String> header = null;
try { try {
// 1 Read the total size of the block // 1 Read the total size of the block
blocksize = (int) inputStream.readLong(); blockSize = (int) inputStream.readLong();
} catch (EOFException | CorruptedLogFileException e) { } catch (EOFException | CorruptedLogFileException e) {
// An exception reading any of the above indicates a corrupt block // An exception reading any of the above indicates a corrupt block
// Create a corrupt block by finding the next MAGIC marker or EOF // Create a corrupt block by finding the next MAGIC marker or EOF
@@ -197,9 +147,9 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
} }
// We may have had a crash which could have written this block partially // We may have had a crash which could have written this block partially
// Skip blocksize in the stream and we should either find a sync marker (start of the next // Skip blockSize in the stream and we should either find a sync marker (start of the next
// block) or EOF. If we did not find either of it, then this block is a corrupted block. // block) or EOF. If we did not find either of it, then this block is a corrupted block.
boolean isCorrupted = isBlockCorrupt(blocksize); boolean isCorrupted = isBlockCorrupted(blockSize);
if (isCorrupted) { if (isCorrupted) {
return createCorruptBlock(); return createCorruptBlock();
} }
@@ -208,71 +158,85 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
HoodieLogFormat.LogFormatVersion nextBlockVersion = readVersion(); HoodieLogFormat.LogFormatVersion nextBlockVersion = readVersion();
// 3. Read the block type for a log block // 3. Read the block type for a log block
if (nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION) { HoodieLogBlockType blockType = tryReadBlockType(nextBlockVersion);
type = inputStream.readInt();
ValidationUtils.checkArgument(type < HoodieLogBlockType.values().length, "Invalid block byte type found " + type);
blockType = HoodieLogBlockType.values()[type];
}
// 4. Read the header for a log block, if present // 4. Read the header for a log block, if present
if (nextBlockVersion.hasHeader()) {
header = HoodieLogBlock.getLogMetadata(inputStream);
}
int contentLength = blocksize; Map<HeaderMetadataType, String> header =
nextBlockVersion.hasHeader() ? HoodieLogBlock.getLogMetadata(inputStream) : null;
// 5. Read the content length for the content // 5. Read the content length for the content
if (nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION) { // Fallback to full-block size if no content-length
contentLength = (int) inputStream.readLong(); // TODO replace w/ hasContentLength
} int contentLength =
nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION ? (int) inputStream.readLong() : blockSize;
// 6. Read the content or skip content based on IO vs Memory trade-off by client // 6. Read the content or skip content based on IO vs Memory trade-off by client
// TODO - have a max block size and reuse this buffer in the ByteBuffer
// (hard to guess max block size for now)
long contentPosition = inputStream.getPos(); long contentPosition = inputStream.getPos();
byte[] content = HoodieLogBlock.readOrSkipContent(inputStream, contentLength, readBlockLazily); boolean shouldReadLazily = readBlockLazily && nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION;
Option<byte[]> content = HoodieLogBlock.tryReadContent(inputStream, contentLength, shouldReadLazily);
// 7. Read footer if any // 7. Read footer if any
Map<HeaderMetadataType, String> footer = null; Map<HeaderMetadataType, String> footer =
if (nextBlockVersion.hasFooter()) { nextBlockVersion.hasFooter() ? HoodieLogBlock.getLogMetadata(inputStream) : null;
footer = HoodieLogBlock.getLogMetadata(inputStream);
}
// 8. Read log block length, if present. This acts as a reverse pointer when traversing a // 8. Read log block length, if present. This acts as a reverse pointer when traversing a
// log file in reverse // log file in reverse
@SuppressWarnings("unused")
long logBlockLength = 0;
if (nextBlockVersion.hasLogBlockLength()) { if (nextBlockVersion.hasLogBlockLength()) {
logBlockLength = inputStream.readLong(); inputStream.readLong();
} }
// 9. Read the log block end position in the log file // 9. Read the log block end position in the log file
long blockEndPos = inputStream.getPos(); long blockEndPos = inputStream.getPos();
HoodieLogBlock.HoodieLogBlockContentLocation logBlockContentLoc =
new HoodieLogBlock.HoodieLogBlockContentLocation(hadoopConf, logFile, contentPosition, contentLength, blockEndPos);
switch (Objects.requireNonNull(blockType)) { switch (Objects.requireNonNull(blockType)) {
// based on type read the block
case AVRO_DATA_BLOCK: case AVRO_DATA_BLOCK:
if (nextBlockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) { if (nextBlockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) {
return HoodieAvroDataBlock.getBlock(content, readerSchema); return HoodieAvroDataBlock.getBlock(content.get(), readerSchema);
} else { } else {
return new HoodieAvroDataBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily, return new HoodieAvroDataBlock(inputStream, content, readBlockLazily, logBlockContentLoc,
contentPosition, contentLength, blockEndPos, readerSchema, header, footer, keyField); Option.ofNullable(readerSchema), header, footer, keyField);
} }
case HFILE_DATA_BLOCK: case HFILE_DATA_BLOCK:
return new HoodieHFileDataBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily, checkState(nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION,
contentPosition, contentLength, blockEndPos, readerSchema, String.format("HFile block could not be of version (%d)", HoodieLogFormatVersion.DEFAULT_VERSION));
header, footer, enableInlineReading, keyField);
return new HoodieHFileDataBlock(inputStream, content, readBlockLazily, logBlockContentLoc,
Option.ofNullable(readerSchema), header, footer, enableRecordLookups);
case PARQUET_DATA_BLOCK:
checkState(nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION,
String.format("Parquet block could not be of version (%d)", HoodieLogFormatVersion.DEFAULT_VERSION));
return new HoodieParquetDataBlock(inputStream, content, readBlockLazily, logBlockContentLoc,
Option.ofNullable(readerSchema), header, footer, keyField);
case DELETE_BLOCK: case DELETE_BLOCK:
return HoodieDeleteBlock.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily, return new HoodieDeleteBlock(content, inputStream, readBlockLazily, Option.of(logBlockContentLoc), header, footer);
contentPosition, contentLength, blockEndPos, header, footer);
case COMMAND_BLOCK: case COMMAND_BLOCK:
return HoodieCommandBlock.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily, return new HoodieCommandBlock(content, inputStream, readBlockLazily, Option.of(logBlockContentLoc), header, footer);
contentPosition, contentLength, blockEndPos, header, footer);
default: default:
throw new HoodieNotSupportedException("Unsupported Block " + blockType); throw new HoodieNotSupportedException("Unsupported Block " + blockType);
} }
} }
@Nullable
private HoodieLogBlockType tryReadBlockType(HoodieLogFormat.LogFormatVersion blockVersion) throws IOException {
if (blockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) {
return null;
}
int type = inputStream.readInt();
checkArgument(type < HoodieLogBlockType.values().length, "Invalid block byte type found " + type);
return HoodieLogBlockType.values()[type];
}
private HoodieLogBlock createCorruptBlock() throws IOException { private HoodieLogBlock createCorruptBlock() throws IOException {
LOG.info("Log " + logFile + " has a corrupted block at " + inputStream.getPos()); LOG.info("Log " + logFile + " has a corrupted block at " + inputStream.getPos());
long currentPos = inputStream.getPos(); long currentPos = inputStream.getPos();
@@ -282,12 +246,13 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
LOG.info("Next available block in " + logFile + " starts at " + nextBlockOffset); LOG.info("Next available block in " + logFile + " starts at " + nextBlockOffset);
int corruptedBlockSize = (int) (nextBlockOffset - currentPos); int corruptedBlockSize = (int) (nextBlockOffset - currentPos);
long contentPosition = inputStream.getPos(); long contentPosition = inputStream.getPos();
byte[] corruptedBytes = HoodieLogBlock.readOrSkipContent(inputStream, corruptedBlockSize, readBlockLazily); Option<byte[]> corruptedBytes = HoodieLogBlock.tryReadContent(inputStream, corruptedBlockSize, readBlockLazily);
return HoodieCorruptBlock.getBlock(logFile, inputStream, Option.ofNullable(corruptedBytes), readBlockLazily, HoodieLogBlock.HoodieLogBlockContentLocation logBlockContentLoc =
contentPosition, corruptedBlockSize, nextBlockOffset, new HashMap<>(), new HashMap<>()); new HoodieLogBlock.HoodieLogBlockContentLocation(hadoopConf, logFile, contentPosition, corruptedBlockSize, nextBlockOffset);
return new HoodieCorruptBlock(corruptedBytes, inputStream, readBlockLazily, Option.of(logBlockContentLoc), new HashMap<>(), new HashMap<>());
} }
private boolean isBlockCorrupt(int blocksize) throws IOException { private boolean isBlockCorrupted(int blocksize) throws IOException {
long currentPos = inputStream.getPos(); long currentPos = inputStream.getPos();
try { try {
inputStream.seek(currentPos + blocksize); inputStream.seek(currentPos + blocksize);
@@ -481,4 +446,59 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
public void remove() { public void remove() {
throw new UnsupportedOperationException("Remove not supported for HoodieLogFileReader"); throw new UnsupportedOperationException("Remove not supported for HoodieLogFileReader");
} }
/**
* Fetch the right {@link FSDataInputStream} to be used by wrapping with required input streams.
* @param fs instance of {@link FileSystem} in use.
* @param bufferSize buffer size to be used.
* @return the right {@link FSDataInputStream} as required.
*/
private static FSDataInputStream getFSDataInputStream(FileSystem fs,
HoodieLogFile logFile,
int bufferSize) throws IOException {
FSDataInputStream fsDataInputStream = fs.open(logFile.getPath(), bufferSize);
if (FSUtils.isGCSFileSystem(fs)) {
// in GCS FS, we might need to interceptor seek offsets as we might get EOF exception
return new SchemeAwareFSDataInputStream(getFSDataInputStreamForGCS(fsDataInputStream, logFile, bufferSize), true);
}
if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) {
return new TimedFSDataInputStream(logFile.getPath(), new FSDataInputStream(
new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(), bufferSize)));
}
// fsDataInputStream.getWrappedStream() maybe a BufferedFSInputStream
// need to wrap in another BufferedFSInputStream the make bufferSize work?
return fsDataInputStream;
}
/**
* GCS FileSystem needs some special handling for seek and hence this method assists to fetch the right {@link FSDataInputStream} to be
* used by wrapping with required input streams.
* @param fsDataInputStream original instance of {@link FSDataInputStream}.
* @param bufferSize buffer size to be used.
* @return the right {@link FSDataInputStream} as required.
*/
private static FSDataInputStream getFSDataInputStreamForGCS(FSDataInputStream fsDataInputStream,
HoodieLogFile logFile,
int bufferSize) {
// incase of GCS FS, there are two flows.
// a. fsDataInputStream.getWrappedStream() instanceof FSInputStream
// b. fsDataInputStream.getWrappedStream() not an instanceof FSInputStream, but an instance of FSDataInputStream.
// (a) is handled in the first if block and (b) is handled in the second if block. If not, we fallback to original fsDataInputStream
if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) {
return new TimedFSDataInputStream(logFile.getPath(), new FSDataInputStream(
new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(), bufferSize)));
}
if (fsDataInputStream.getWrappedStream() instanceof FSDataInputStream
&& ((FSDataInputStream) fsDataInputStream.getWrappedStream()).getWrappedStream() instanceof FSInputStream) {
FSInputStream inputStream = (FSInputStream)((FSDataInputStream) fsDataInputStream.getWrappedStream()).getWrappedStream();
return new TimedFSDataInputStream(logFile.getPath(),
new FSDataInputStream(new BufferedFSInputStream(inputStream, bufferSize)));
}
return fsDataInputStream;
}
} }

View File

@@ -60,13 +60,6 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
private static final String APPEND_UNAVAILABLE_EXCEPTION_MESSAGE = "not sufficiently replicated yet"; private static final String APPEND_UNAVAILABLE_EXCEPTION_MESSAGE = "not sufficiently replicated yet";
/**
* @param fs
* @param logFile
* @param bufferSize
* @param replication
* @param sizeThreshold
*/
HoodieLogFormatWriter(FileSystem fs, HoodieLogFile logFile, Integer bufferSize, Short replication, Long sizeThreshold, String rolloverLogWriteToken) { HoodieLogFormatWriter(FileSystem fs, HoodieLogFile logFile, Integer bufferSize, Short replication, Long sizeThreshold, String rolloverLogWriteToken) {
this.fs = fs; this.fs = fs;
this.logFile = logFile; this.logFile = logFile;

View File

@@ -18,13 +18,6 @@
package org.apache.hudi.common.table.log.block; package org.apache.hudi.common.table.log.block;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.SizeAwareDataInputStream;
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.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericDatumWriter;
@@ -36,59 +29,64 @@ import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.Encoder; import org.apache.avro.io.Encoder;
import org.apache.avro.io.EncoderFactory; import org.apache.avro.io.EncoderFactory;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hudi.common.fs.SizeAwareDataInputStream;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import javax.annotation.Nonnull;
import java.io.ByteArrayInputStream; import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.DataInputStream; import java.io.DataInputStream;
import java.io.DataOutputStream; import java.io.DataOutputStream;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;
import javax.annotation.Nonnull; import static org.apache.hudi.common.util.ValidationUtils.checkState;
/** /**
* HoodieAvroDataBlock contains a list of records serialized using Avro. It is used with the Parquet base file format. * HoodieAvroDataBlock contains a list of records serialized using Avro. It is used with the Parquet base file format.
*/ */
public class HoodieAvroDataBlock extends HoodieDataBlock { public class HoodieAvroDataBlock extends HoodieDataBlock {
private ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>(); private final ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
private ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>(); private final ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
public HoodieAvroDataBlock(@Nonnull Map<HeaderMetadataType, String> logBlockHeader, public HoodieAvroDataBlock(FSDataInputStream inputStream,
@Nonnull Map<HeaderMetadataType, String> logBlockFooter, Option<byte[]> content,
@Nonnull Option<HoodieLogBlockContentLocation> blockContentLocation, @Nonnull Option<byte[]> content, boolean readBlockLazily,
FSDataInputStream inputStream, boolean readBlockLazily) { HoodieLogBlockContentLocation logBlockContentLocation,
super(logBlockHeader, logBlockFooter, blockContentLocation, content, inputStream, readBlockLazily); Option<Schema> readerSchema,
Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer,
String keyField) {
super(content, inputStream, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, keyField, false);
} }
public HoodieAvroDataBlock(HoodieLogFile logFile, FSDataInputStream inputStream, Option<byte[]> content, public HoodieAvroDataBlock(@Nonnull List<IndexedRecord> records,
boolean readBlockLazily, long position, long blockSize, long blockEndpos, Schema readerSchema, @Nonnull Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> header, Map<HeaderMetadataType, String> footer, String keyField) { @Nonnull String keyField
super(content, inputStream, readBlockLazily, ) {
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), readerSchema, header,
footer, keyField);
}
public HoodieAvroDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull Map<HeaderMetadataType,
String> header, String keyField) {
super(records, header, new HashMap<>(), keyField); super(records, header, new HashMap<>(), keyField);
} }
public HoodieAvroDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull Map<HeaderMetadataType, String> header) {
super(records, header, new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD);
}
@Override @Override
public HoodieLogBlockType getBlockType() { public HoodieLogBlockType getBlockType() {
return HoodieLogBlockType.AVRO_DATA_BLOCK; return HoodieLogBlockType.AVRO_DATA_BLOCK;
} }
@Override @Override
protected byte[] serializeRecords() throws IOException { protected byte[] serializeRecords(List<IndexedRecord> records) throws IOException {
Schema schema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); Schema schema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
GenericDatumWriter<IndexedRecord> writer = new GenericDatumWriter<>(schema); GenericDatumWriter<IndexedRecord> writer = new GenericDatumWriter<>(schema);
ByteArrayOutputStream baos = new ByteArrayOutputStream(); ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -118,7 +116,6 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
output.writeInt(size); output.writeInt(size);
// Write the content // Write the content
output.write(temp.toByteArray()); output.write(temp.toByteArray());
itr.remove();
} catch (IOException e) { } catch (IOException e) {
throw new HoodieIOException("IOException converting HoodieAvroDataBlock to bytes", e); throw new HoodieIOException("IOException converting HoodieAvroDataBlock to bytes", e);
} }
@@ -130,9 +127,11 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
// TODO (na) - Break down content into smaller chunks of byte [] to be GC as they are used // TODO (na) - Break down content into smaller chunks of byte [] to be GC as they are used
// TODO (na) - Implement a recordItr instead of recordList // TODO (na) - Implement a recordItr instead of recordList
@Override @Override
protected void deserializeRecords() throws IOException { protected List<IndexedRecord> deserializeRecords(byte[] content) throws IOException {
checkState(readerSchema != null, "Reader's schema has to be non-null");
SizeAwareDataInputStream dis = SizeAwareDataInputStream dis =
new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(getContent().get()))); new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
// 1. Read version for this data block // 1. Read version for this data block
int version = dis.readInt(); int version = dis.readInt();
@@ -141,12 +140,8 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
// Get schema from the header // Get schema from the header
Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
// If readerSchema was not present, use writerSchema GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
if (schema == null) {
schema = writerSchema;
}
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, schema);
// 2. Get the total records // 2. Get the total records
int totalRecords = 0; int totalRecords = 0;
if (logBlockVersion.hasRecordCount()) { if (logBlockVersion.hasRecordCount()) {
@@ -157,17 +152,17 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
// 3. Read the content // 3. Read the content
for (int i = 0; i < totalRecords; i++) { for (int i = 0; i < totalRecords; i++) {
int recordLength = dis.readInt(); int recordLength = dis.readInt();
BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(getContent().get(), dis.getNumberOfBytesRead(), BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(content, dis.getNumberOfBytesRead(),
recordLength, decoderCache.get()); recordLength, decoderCache.get());
decoderCache.set(decoder); decoderCache.set(decoder);
IndexedRecord record = reader.read(null, decoder); IndexedRecord record = reader.read(null, decoder);
records.add(record); records.add(record);
dis.skipBytes(recordLength); dis.skipBytes(recordLength);
} }
dis.close(); dis.close();
this.records = records;
// Free up content to be GC'd, deflate return records;
deflate();
} }
//---------------------------------------------------------------------------------------- //----------------------------------------------------------------------------------------
@@ -183,9 +178,7 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
*/ */
@Deprecated @Deprecated
public HoodieAvroDataBlock(List<IndexedRecord> records, Schema schema) { public HoodieAvroDataBlock(List<IndexedRecord> records, Schema schema) {
super(new HashMap<>(), new HashMap<>(), Option.empty(), Option.empty(), null, false); super(records, Collections.singletonMap(HeaderMetadataType.SCHEMA, schema.toString()), new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD);
this.records = records;
this.schema = schema;
} }
/** /**
@@ -201,7 +194,7 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
int schemaLength = dis.readInt(); int schemaLength = dis.readInt();
byte[] compressedSchema = new byte[schemaLength]; byte[] compressedSchema = new byte[schemaLength];
dis.readFully(compressedSchema, 0, schemaLength); dis.readFully(compressedSchema, 0, schemaLength);
Schema writerSchema = new Schema.Parser().parse(HoodieAvroUtils.decompress(compressedSchema)); Schema writerSchema = new Schema.Parser().parse(decompress(compressedSchema));
if (readerSchema == null) { if (readerSchema == null) {
readerSchema = writerSchema; readerSchema = writerSchema;
@@ -224,6 +217,33 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
return new HoodieAvroDataBlock(records, readerSchema); return new HoodieAvroDataBlock(records, readerSchema);
} }
private static byte[] compress(String text) {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
OutputStream out = new DeflaterOutputStream(baos);
out.write(text.getBytes(StandardCharsets.UTF_8));
out.close();
} catch (IOException e) {
throw new HoodieIOException("IOException while compressing text " + text, e);
}
return baos.toByteArray();
}
private static String decompress(byte[] bytes) {
InputStream in = new InflaterInputStream(new ByteArrayInputStream(bytes));
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
byte[] buffer = new byte[8192];
int len;
while ((len = in.read(buffer)) > 0) {
baos.write(buffer, 0, len);
}
return new String(baos.toByteArray(), StandardCharsets.UTF_8);
} catch (IOException e) {
throw new HoodieIOException("IOException while decompressing text", e);
}
}
@Deprecated @Deprecated
public byte[] getBytes(Schema schema) throws IOException { public byte[] getBytes(Schema schema) throws IOException {
@@ -232,10 +252,12 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
DataOutputStream output = new DataOutputStream(baos); DataOutputStream output = new DataOutputStream(baos);
// 2. Compress and Write schema out // 2. Compress and Write schema out
byte[] schemaContent = HoodieAvroUtils.compress(schema.toString()); byte[] schemaContent = compress(schema.toString());
output.writeInt(schemaContent.length); output.writeInt(schemaContent.length);
output.write(schemaContent); output.write(schemaContent);
List<IndexedRecord> records = getRecords();
// 3. Write total number of records // 3. Write total number of records
output.writeInt(records.size()); output.writeInt(records.size());

View File

@@ -18,7 +18,6 @@
package org.apache.hudi.common.table.log.block; package org.apache.hudi.common.table.log.block;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
@@ -44,9 +43,9 @@ public class HoodieCommandBlock extends HoodieLogBlock {
this(Option.empty(), null, false, Option.empty(), header, new HashMap<>()); this(Option.empty(), null, false, Option.empty(), header, new HashMap<>());
} }
private HoodieCommandBlock(Option<byte[]> content, FSDataInputStream inputStream, boolean readBlockLazily, public HoodieCommandBlock(Option<byte[]> content, FSDataInputStream inputStream, boolean readBlockLazily,
Option<HoodieLogBlockContentLocation> blockContentLocation, Map<HeaderMetadataType, String> header, Option<HoodieLogBlockContentLocation> blockContentLocation, Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer) { Map<HeaderMetadataType, String> footer) {
super(header, footer, blockContentLocation, content, inputStream, readBlockLazily); super(header, footer, blockContentLocation, content, inputStream, readBlockLazily);
this.type = this.type =
HoodieCommandBlockTypeEnum.values()[Integer.parseInt(header.get(HeaderMetadataType.COMMAND_BLOCK_TYPE))]; HoodieCommandBlockTypeEnum.values()[Integer.parseInt(header.get(HeaderMetadataType.COMMAND_BLOCK_TYPE))];
@@ -65,12 +64,4 @@ public class HoodieCommandBlock extends HoodieLogBlock {
public byte[] getContentBytes() { public byte[] getContentBytes() {
return new byte[0]; return new byte[0];
} }
public static HoodieLogBlock getBlock(HoodieLogFile logFile, FSDataInputStream inputStream, Option<byte[]> content,
boolean readBlockLazily, long position, long blockSize, long blockEndPos, Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer) {
return new HoodieCommandBlock(content, inputStream, readBlockLazily,
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer);
}
} }

View File

@@ -18,7 +18,6 @@
package org.apache.hudi.common.table.log.block; package org.apache.hudi.common.table.log.block;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
@@ -32,15 +31,14 @@ import java.util.Map;
*/ */
public class HoodieCorruptBlock extends HoodieLogBlock { public class HoodieCorruptBlock extends HoodieLogBlock {
private HoodieCorruptBlock(Option<byte[]> corruptedBytes, FSDataInputStream inputStream, boolean readBlockLazily, public HoodieCorruptBlock(Option<byte[]> corruptedBytes, FSDataInputStream inputStream, boolean readBlockLazily,
Option<HoodieLogBlockContentLocation> blockContentLocation, Map<HeaderMetadataType, String> header, Option<HoodieLogBlockContentLocation> blockContentLocation, Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer) { Map<HeaderMetadataType, String> footer) {
super(header, footer, blockContentLocation, corruptedBytes, inputStream, readBlockLazily); super(header, footer, blockContentLocation, corruptedBytes, inputStream, readBlockLazily);
} }
@Override @Override
public byte[] getContentBytes() throws IOException { public byte[] getContentBytes() throws IOException {
if (!getContent().isPresent() && readBlockLazily) { if (!getContent().isPresent() && readBlockLazily) {
// read content from disk // read content from disk
inflate(); inflate();
@@ -53,11 +51,4 @@ public class HoodieCorruptBlock extends HoodieLogBlock {
return HoodieLogBlockType.CORRUPT_BLOCK; return HoodieLogBlockType.CORRUPT_BLOCK;
} }
public static HoodieLogBlock getBlock(HoodieLogFile logFile, FSDataInputStream inputStream,
Option<byte[]> corruptedBytes, boolean readBlockLazily, long position, long blockSize, long blockEndPos,
Map<HeaderMetadataType, String> header, Map<HeaderMetadataType, String> footer) {
return new HoodieCorruptBlock(corruptedBytes, inputStream, readBlockLazily,
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer);
}
} }

View File

@@ -18,25 +18,24 @@
package org.apache.hudi.common.table.log.block; package org.apache.hudi.common.table.log.block;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hudi.common.util.Option;
import javax.annotation.Nonnull; import org.apache.hudi.exception.HoodieIOException;
import java.io.IOException; import java.io.IOException;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
/** /**
* DataBlock contains a list of records serialized using formats compatible with the base file format. * DataBlock contains a list of records serialized using formats compatible with the base file format.
* For each base file format there is a corresponding DataBlock format. * For each base file format there is a corresponding DataBlock format.
* * <p>
* The Datablock contains: * The Datablock contains:
* 1. Data Block version * 1. Data Block version
* 2. Total number of records in the block * 2. Total number of records in the block
@@ -44,125 +43,151 @@ import java.util.Map;
*/ */
public abstract class HoodieDataBlock extends HoodieLogBlock { public abstract class HoodieDataBlock extends HoodieLogBlock {
protected List<IndexedRecord> records; // TODO rebase records/content to leverage Either to warrant
protected Schema schema; // that they are mutex (used by read/write flows respectively)
protected String keyField; private Option<List<IndexedRecord>> records;
public HoodieDataBlock(@Nonnull Map<HeaderMetadataType, String> logBlockHeader, /**
@Nonnull Map<HeaderMetadataType, String> logBlockFooter, * Key field's name w/in the record's schema
@Nonnull Option<HoodieLogBlockContentLocation> blockContentLocation, @Nonnull Option<byte[]> content, */
FSDataInputStream inputStream, boolean readBlockLazily) { private final String keyFieldName;
super(logBlockHeader, logBlockFooter, blockContentLocation, content, inputStream, readBlockLazily);
this.keyField = HoodieRecord.RECORD_KEY_METADATA_FIELD;
}
public HoodieDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull Map<HeaderMetadataType, String> header, private final boolean enablePointLookups;
@Nonnull Map<HeaderMetadataType, String> footer, String keyField) {
this(header, footer, Option.empty(), Option.empty(), null, false);
this.records = records;
this.schema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
this.keyField = keyField;
}
protected HoodieDataBlock(Option<byte[]> content, @Nonnull FSDataInputStream inputStream, boolean readBlockLazily, protected final Schema readerSchema;
Option<HoodieLogBlockContentLocation> blockContentLocation, Schema readerSchema,
@Nonnull Map<HeaderMetadataType, String> headers, @Nonnull Map<HeaderMetadataType, /**
String> footer, String keyField) { * NOTE: This ctor is used on the write-path (ie when records ought to be written into the log)
this(headers, footer, blockContentLocation, content, inputStream, readBlockLazily); */
this.schema = readerSchema; public HoodieDataBlock(List<IndexedRecord> records,
this.keyField = keyField; Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer,
String keyFieldName) {
super(header, footer, Option.empty(), Option.empty(), null, false);
this.records = Option.of(records);
this.keyFieldName = keyFieldName;
// If no reader-schema has been provided assume writer-schema as one
this.readerSchema = getWriterSchema(super.getLogBlockHeader());
this.enablePointLookups = false;
} }
/** /**
* Util method to get a data block for the requested type. * NOTE: This ctor is used on the write-path (ie when records ought to be written into the log)
*
* @param logDataBlockFormat - Data block type
* @param recordList - List of records that goes in the data block
* @param header - data block header
* @return Data block of the requested type.
*/ */
public static HoodieLogBlock getBlock(HoodieLogBlockType logDataBlockFormat, List<IndexedRecord> recordList, protected HoodieDataBlock(Option<byte[]> content,
Map<HeaderMetadataType, String> header) { FSDataInputStream inputStream,
return getBlock(logDataBlockFormat, recordList, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); boolean readBlockLazily,
} Option<HoodieLogBlockContentLocation> blockContentLocation,
Option<Schema> readerSchema,
/** Map<HeaderMetadataType, String> headers,
* Util method to get a data block for the requested type. Map<HeaderMetadataType, String> footer,
* String keyFieldName,
* @param logDataBlockFormat - Data block type boolean enablePointLookups) {
* @param recordList - List of records that goes in the data block super(headers, footer, blockContentLocation, content, inputStream, readBlockLazily);
* @param header - data block header this.records = Option.empty();
* @param keyField - FieldId to get the key from the records this.keyFieldName = keyFieldName;
* @return Data block of the requested type. // If no reader-schema has been provided assume writer-schema as one
*/ this.readerSchema = readerSchema.orElseGet(() -> getWriterSchema(super.getLogBlockHeader()));
public static HoodieLogBlock getBlock(HoodieLogBlockType logDataBlockFormat, List<IndexedRecord> recordList, this.enablePointLookups = enablePointLookups;
Map<HeaderMetadataType, String> header, String keyField) {
switch (logDataBlockFormat) {
case AVRO_DATA_BLOCK:
return new HoodieAvroDataBlock(recordList, header, keyField);
case HFILE_DATA_BLOCK:
return new HoodieHFileDataBlock(recordList, header, keyField);
default:
throw new HoodieException("Data block format " + logDataBlockFormat + " not implemented");
}
} }
@Override @Override
public byte[] getContentBytes() throws IOException { public byte[] getContentBytes() throws IOException {
// In case this method is called before realizing records from content // In case this method is called before realizing records from content
if (getContent().isPresent()) { Option<byte[]> content = getContent();
return getContent().get();
} else if (readBlockLazily && !getContent().isPresent() && records == null) { checkState(content.isPresent() || records.isPresent(), "Block is in invalid state");
// read block lazily
createRecordsFromContentBytes(); if (content.isPresent()) {
return content.get();
} }
return serializeRecords(); return serializeRecords(records.get());
} }
public abstract HoodieLogBlockType getBlockType(); protected static Schema getWriterSchema(Map<HeaderMetadataType, String> logBlockHeader) {
return new Schema.Parser().parse(logBlockHeader.get(HeaderMetadataType.SCHEMA));
}
public List<IndexedRecord> getRecords() { /**
if (records == null) { * Returns all the records contained w/in this block
*/
public final List<IndexedRecord> getRecords() {
if (!records.isPresent()) {
try { try {
// in case records are absent, read content lazily and then convert to IndexedRecords // in case records are absent, read content lazily and then convert to IndexedRecords
createRecordsFromContentBytes(); records = Option.of(readRecordsFromBlockPayload());
} catch (IOException io) { } catch (IOException io) {
throw new HoodieIOException("Unable to convert content bytes to records", io); throw new HoodieIOException("Unable to convert content bytes to records", io);
} }
} }
return records; return records.get();
}
public Schema getSchema() {
return readerSchema;
} }
/** /**
* Batch get of keys of interest. Implementation can choose to either do full scan and return matched entries or * Batch get of keys of interest. Implementation can choose to either do full scan and return matched entries or
* do a seek based parsing and return matched entries. * do a seek based parsing and return matched entries.
*
* @param keys keys of interest. * @param keys keys of interest.
* @return List of IndexedRecords for the keys of interest. * @return List of IndexedRecords for the keys of interest.
* @throws IOException * @throws IOException in case of failures encountered when reading/parsing records
*/ */
public List<IndexedRecord> getRecords(List<String> keys) throws IOException { public final List<IndexedRecord> getRecords(List<String> keys) throws IOException {
throw new UnsupportedOperationException("On demand batch get based on interested keys not supported"); boolean fullScan = keys.isEmpty();
} if (enablePointLookups && !fullScan) {
return lookupRecords(keys);
public Schema getSchema() {
// if getSchema was invoked before converting byte [] to records
if (records == null) {
getRecords();
} }
return schema;
// Otherwise, we fetch all the records and filter out all the records, but the
// ones requested
List<IndexedRecord> allRecords = getRecords();
if (fullScan) {
return allRecords;
}
HashSet<String> keySet = new HashSet<>(keys);
return allRecords.stream()
.filter(record -> keySet.contains(getRecordKey(record).orElse(null)))
.collect(Collectors.toList());
} }
protected void createRecordsFromContentBytes() throws IOException { protected List<IndexedRecord> readRecordsFromBlockPayload() throws IOException {
if (readBlockLazily && !getContent().isPresent()) { if (readBlockLazily && !getContent().isPresent()) {
// read log block contents from disk // read log block contents from disk
inflate(); inflate();
} }
deserializeRecords(); try {
return deserializeRecords(getContent().get());
} finally {
// Free up content to be GC'd by deflating the block
deflate();
}
} }
protected abstract byte[] serializeRecords() throws IOException; protected List<IndexedRecord> lookupRecords(List<String> keys) throws IOException {
throw new UnsupportedOperationException(
String.format("Point lookups are not supported by this Data block type (%s)", getBlockType())
);
}
protected abstract void deserializeRecords() throws IOException; protected abstract byte[] serializeRecords(List<IndexedRecord> records) throws IOException;
protected abstract List<IndexedRecord> deserializeRecords(byte[] content) throws IOException;
public abstract HoodieLogBlockType getBlockType();
protected Option<Schema.Field> getKeyField(Schema schema) {
return Option.ofNullable(schema.getField(keyFieldName));
}
protected Option<String> getRecordKey(IndexedRecord record) {
return getKeyField(record.getSchema())
.map(keyField -> record.get(keyField.pos()))
.map(Object::toString);
}
} }

View File

@@ -20,7 +20,6 @@ package org.apache.hudi.common.table.log.block;
import org.apache.hudi.common.fs.SizeAwareDataInputStream; import org.apache.hudi.common.fs.SizeAwareDataInputStream;
import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SerializationUtils; import org.apache.hudi.common.util.SerializationUtils;
import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIOException;
@@ -47,7 +46,7 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
this.keysToDelete = keysToDelete; this.keysToDelete = keysToDelete;
} }
private HoodieDeleteBlock(Option<byte[]> content, FSDataInputStream inputStream, boolean readBlockLazily, public HoodieDeleteBlock(Option<byte[]> content, FSDataInputStream inputStream, boolean readBlockLazily,
Option<HoodieLogBlockContentLocation> blockContentLocation, Map<HeaderMetadataType, String> header, Option<HoodieLogBlockContentLocation> blockContentLocation, Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer) { Map<HeaderMetadataType, String> footer) {
super(header, footer, blockContentLocation, content, inputStream, readBlockLazily); super(header, footer, blockContentLocation, content, inputStream, readBlockLazily);
@@ -55,11 +54,12 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
@Override @Override
public byte[] getContentBytes() throws IOException { public byte[] getContentBytes() throws IOException {
Option<byte[]> content = getContent();
// In case this method is called before realizing keys from content // In case this method is called before realizing keys from content
if (getContent().isPresent()) { if (content.isPresent()) {
return getContent().get(); return content.get();
} else if (readBlockLazily && !getContent().isPresent() && keysToDelete == null) { } else if (readBlockLazily && keysToDelete == null) {
// read block lazily // read block lazily
getKeysToDelete(); getKeysToDelete();
} }
@@ -100,11 +100,4 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
return HoodieLogBlockType.DELETE_BLOCK; return HoodieLogBlockType.DELETE_BLOCK;
} }
public static HoodieLogBlock getBlock(HoodieLogFile logFile, FSDataInputStream inputStream, Option<byte[]> content,
boolean readBlockLazily, long position, long blockSize, long blockEndPos, Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer) throws IOException {
return new HoodieDeleteBlock(content, inputStream, readBlockLazily,
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer);
}
} }

View File

@@ -18,19 +18,7 @@
package org.apache.hudi.common.table.log.block; package org.apache.hudi.common.table.log.block;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.inline.InLineFSUtils;
import org.apache.hudi.common.fs.inline.InLineFileSystem;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.io.storage.HoodieHBaseKVComparator;
import org.apache.hudi.io.storage.HoodieHFileReader;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
@@ -43,11 +31,18 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.inline.InLineFSUtils;
import org.apache.hudi.common.fs.inline.InLineFileSystem;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.io.storage.HoodieHBaseKVComparator;
import org.apache.hudi.io.storage.HoodieHFileReader;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import javax.annotation.Nonnull;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
@@ -58,29 +53,36 @@ import java.util.Map;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
/** /**
* HoodieHFileDataBlock contains a list of records stored inside an HFile format. It is used with the HFile * HoodieHFileDataBlock contains a list of records stored inside an HFile format. It is used with the HFile
* base file format. * base file format.
*/ */
public class HoodieHFileDataBlock extends HoodieDataBlock { public class HoodieHFileDataBlock extends HoodieDataBlock {
private static final Logger LOG = LogManager.getLogger(HoodieHFileDataBlock.class); private static final Logger LOG = LogManager.getLogger(HoodieHFileDataBlock.class);
private static Compression.Algorithm compressionAlgorithm = Compression.Algorithm.GZ;
private static int blockSize = 1 * 1024 * 1024;
private boolean enableInlineReading = false;
public HoodieHFileDataBlock(HoodieLogFile logFile, FSDataInputStream inputStream, Option<byte[]> content, private static final int DEFAULT_BLOCK_SIZE = 1024 * 1024;
boolean readBlockLazily, long position, long blockSize, long blockEndpos,
Schema readerSchema, Map<HeaderMetadataType, String> header, private final Option<Compression.Algorithm> compressionAlgorithm;
Map<HeaderMetadataType, String> footer, boolean enableInlineReading, String keyField) {
super(content, inputStream, readBlockLazily, public HoodieHFileDataBlock(FSDataInputStream inputStream,
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), Option<byte[]> content,
readerSchema, header, footer, keyField); boolean readBlockLazily,
this.enableInlineReading = enableInlineReading; HoodieLogBlockContentLocation logBlockContentLocation,
Option<Schema> readerSchema,
Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer,
boolean enablePointLookups) {
super(content, inputStream, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, HoodieHFileReader.KEY_FIELD_NAME, enablePointLookups);
this.compressionAlgorithm = Option.empty();
} }
public HoodieHFileDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull Map<HeaderMetadataType, String> header, public HoodieHFileDataBlock(List<IndexedRecord> records,
String keyField) { Map<HeaderMetadataType, String> header,
super(records, header, new HashMap<>(), keyField); Compression.Algorithm compressionAlgorithm) {
super(records, header, new HashMap<>(), HoodieHFileReader.KEY_FIELD_NAME);
this.compressionAlgorithm = Option.of(compressionAlgorithm);
} }
@Override @Override
@@ -89,43 +91,45 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
} }
@Override @Override
protected byte[] serializeRecords() throws IOException { protected byte[] serializeRecords(List<IndexedRecord> records) throws IOException {
HFileContext context = new HFileContextBuilder().withBlockSize(blockSize).withCompression(compressionAlgorithm) HFileContext context = new HFileContextBuilder()
.withBlockSize(DEFAULT_BLOCK_SIZE)
.withCompression(compressionAlgorithm.get())
.build(); .build();
Configuration conf = new Configuration(); Configuration conf = new Configuration();
CacheConfig cacheConfig = new CacheConfig(conf); CacheConfig cacheConfig = new CacheConfig(conf);
ByteArrayOutputStream baos = new ByteArrayOutputStream(); ByteArrayOutputStream baos = new ByteArrayOutputStream();
FSDataOutputStream ostream = new FSDataOutputStream(baos, null); FSDataOutputStream ostream = new FSDataOutputStream(baos, null);
HFile.Writer writer = HFile.getWriterFactory(conf, cacheConfig) // Use simple incrementing counter as a key
.withOutputStream(ostream).withFileContext(context).withComparator(new HoodieHBaseKVComparator()).create(); boolean useIntegerKey = !getRecordKey(records.get(0)).isPresent();
// This is set here to avoid re-computing this in the loop
int keyWidth = useIntegerKey ? (int) Math.ceil(Math.log(records.size())) + 1 : -1;
// Serialize records into bytes // Serialize records into bytes
Map<String, byte[]> sortedRecordsMap = new TreeMap<>(); Map<String, byte[]> sortedRecordsMap = new TreeMap<>();
Iterator<IndexedRecord> itr = records.iterator(); Iterator<IndexedRecord> itr = records.iterator();
boolean useIntegerKey = false;
int key = 0; int id = 0;
int keySize = 0;
final Field keyFieldSchema = records.get(0).getSchema().getField(HoodieHFileReader.KEY_FIELD_NAME);
if (keyFieldSchema == 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()) { while (itr.hasNext()) {
IndexedRecord record = itr.next(); IndexedRecord record = itr.next();
String recordKey; String recordKey;
if (useIntegerKey) { if (useIntegerKey) {
recordKey = String.format("%" + keySize + "s", key++); recordKey = String.format("%" + keyWidth + "s", id++);
} else { } else {
recordKey = record.get(keyFieldSchema.pos()).toString(); recordKey = getRecordKey(record).get();
} }
final byte[] recordBytes = serializeRecord(record, Option.ofNullable(keyFieldSchema));
final byte[] recordBytes = serializeRecord(record);
ValidationUtils.checkState(!sortedRecordsMap.containsKey(recordKey), ValidationUtils.checkState(!sortedRecordsMap.containsKey(recordKey),
"Writing multiple records with same key not supported for " + this.getClass().getName()); "Writing multiple records with same key not supported for " + this.getClass().getName());
sortedRecordsMap.put(recordKey, recordBytes); sortedRecordsMap.put(recordKey, recordBytes);
} }
HFile.Writer writer = HFile.getWriterFactory(conf, cacheConfig)
.withOutputStream(ostream).withFileContext(context).withComparator(new HoodieHBaseKVComparator()).create();
// Write the records // Write the records
sortedRecordsMap.forEach((recordKey, recordBytes) -> { sortedRecordsMap.forEach((recordKey, recordBytes) -> {
try { try {
@@ -144,79 +148,52 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
} }
@Override @Override
protected void createRecordsFromContentBytes() throws IOException { protected List<IndexedRecord> deserializeRecords(byte[] content) throws IOException {
if (enableInlineReading) { checkState(readerSchema != null, "Reader's schema has to be non-null");
getRecords(Collections.emptyList());
} else { // Get schema from the header
super.createRecordsFromContentBytes(); Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
// Read the content
HoodieHFileReader<IndexedRecord> reader = new HoodieHFileReader<>(content);
List<Pair<String, IndexedRecord>> records = reader.readAllRecords(writerSchema, readerSchema);
return records.stream().map(Pair::getSecond).collect(Collectors.toList());
}
// TODO abstract this w/in HoodieDataBlock
@Override
protected List<IndexedRecord> lookupRecords(List<String> keys) throws IOException {
HoodieLogBlockContentLocation blockContentLoc = getBlockContentLocation().get();
// NOTE: It's important to extend Hadoop configuration here to make sure configuration
// is appropriately carried over
Configuration inlineConf = new Configuration(blockContentLoc.getHadoopConf());
inlineConf.set("fs." + InLineFileSystem.SCHEME + ".impl", InLineFileSystem.class.getName());
Path inlinePath = InLineFSUtils.getInlineFilePath(
blockContentLoc.getLogFile().getPath(),
blockContentLoc.getLogFile().getPath().getFileSystem(inlineConf).getScheme(),
blockContentLoc.getContentPositionInLogFile(),
blockContentLoc.getBlockSize());
// HFile read will be efficient if keys are sorted, since on storage, records are sorted by key. This will avoid unnecessary seeks.
Collections.sort(keys);
try (HoodieHFileReader<IndexedRecord> reader =
new HoodieHFileReader<>(inlineConf, inlinePath, new CacheConfig(inlineConf), inlinePath.getFileSystem(inlineConf))) {
// Get writer's schema from the header
List<Pair<String, IndexedRecord>> logRecords = reader.readRecords(keys, readerSchema);
return logRecords.stream().map(Pair::getSecond).collect(Collectors.toList());
} }
} }
@Override private byte[] serializeRecord(IndexedRecord record) {
public List<IndexedRecord> getRecords(List<String> keys) throws IOException { Option<Schema.Field> keyField = getKeyField(record.getSchema());
readWithInlineFS(keys); // Reset key value w/in the record to avoid duplicating the key w/in payload
return records;
}
/**
* Serialize the record to byte buffer.
*
* @param record - Record to serialize
* @param keyField - Key field in the schema
* @return Serialized byte buffer for the record
*/
private byte[] serializeRecord(final IndexedRecord record, final Option<Field> keyField) {
if (keyField.isPresent()) { if (keyField.isPresent()) {
record.put(keyField.get().pos(), StringUtils.EMPTY_STRING); record.put(keyField.get().pos(), StringUtils.EMPTY_STRING);
} }
return HoodieAvroUtils.indexedRecordToBytes(record); return HoodieAvroUtils.indexedRecordToBytes(record);
} }
private void readWithInlineFS(List<String> keys) throws IOException {
boolean enableFullScan = keys.isEmpty();
// 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;
}
Configuration conf = new Configuration();
CacheConfig cacheConf = new CacheConfig(conf);
Configuration inlineConf = new Configuration();
inlineConf.set("fs." + InLineFileSystem.SCHEME + ".impl", InLineFileSystem.class.getName());
Path inlinePath = InLineFSUtils.getInlineFilePath(
getBlockContentLocation().get().getLogFile().getPath(),
getBlockContentLocation().get().getLogFile().getPath().getFileSystem(conf).getScheme(),
getBlockContentLocation().get().getContentPositionInLogFile(),
getBlockContentLocation().get().getBlockSize());
if (!enableFullScan) {
// HFile read will be efficient if keys are sorted, since on storage, records are sorted by key. This will avoid unnecessary seeks.
Collections.sort(keys);
}
HoodieHFileReader reader = new HoodieHFileReader(inlineConf, inlinePath, cacheConf, inlinePath.getFileSystem(inlineConf));
List<org.apache.hadoop.hbase.util.Pair<String, IndexedRecord>> logRecords = enableFullScan ? reader.readAllRecords(writerSchema, schema) :
reader.readRecords(keys, schema);
reader.close();
this.records = logRecords.stream().map(t -> t.getSecond()).collect(Collectors.toList());
}
@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

@@ -18,15 +18,18 @@
package org.apache.hudi.common.table.log.block; package org.apache.hudi.common.table.log.block;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.TypeUtils;
import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIOException;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import javax.annotation.Nonnull; import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.DataInputStream; import java.io.DataInputStream;
@@ -36,6 +39,8 @@ import java.io.IOException;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
/** /**
* Abstract class defining a block in HoodieLogFile. * Abstract class defining a block in HoodieLogFile.
*/ */
@@ -58,14 +63,17 @@ public abstract class HoodieLogBlock {
// TODO : change this to just InputStream so this works for any FileSystem // TODO : change this to just InputStream so this works for any FileSystem
// create handlers to return specific type of inputstream based on FS // create handlers to return specific type of inputstream based on FS
// input stream corresponding to the log file where this logBlock belongs // input stream corresponding to the log file where this logBlock belongs
protected FSDataInputStream inputStream; private final FSDataInputStream inputStream;
// Toggle flag, whether to read blocks lazily (I/O intensive) or not (Memory intensive) // Toggle flag, whether to read blocks lazily (I/O intensive) or not (Memory intensive)
protected boolean readBlockLazily; protected boolean readBlockLazily;
public HoodieLogBlock(@Nonnull Map<HeaderMetadataType, String> logBlockHeader, public HoodieLogBlock(
@Nonnull Map<HeaderMetadataType, String> logBlockHeader,
@Nonnull Map<HeaderMetadataType, String> logBlockFooter, @Nonnull Map<HeaderMetadataType, String> logBlockFooter,
@Nonnull Option<HoodieLogBlockContentLocation> blockContentLocation, @Nonnull Option<byte[]> content, @Nonnull Option<HoodieLogBlockContentLocation> blockContentLocation,
FSDataInputStream inputStream, boolean readBlockLazily) { @Nonnull Option<byte[]> content,
@Nullable FSDataInputStream inputStream,
boolean readBlockLazily) {
this.logBlockHeader = logBlockHeader; this.logBlockHeader = logBlockHeader;
this.logBlockFooter = logBlockFooter; this.logBlockFooter = logBlockFooter;
this.blockContentLocation = blockContentLocation; this.blockContentLocation = blockContentLocation;
@@ -109,7 +117,25 @@ public abstract class HoodieLogBlock {
* Type of the log block WARNING: This enum is serialized as the ordinal. Only add new enums at the end. * Type of the log block WARNING: This enum is serialized as the ordinal. Only add new enums at the end.
*/ */
public enum HoodieLogBlockType { public enum HoodieLogBlockType {
COMMAND_BLOCK, DELETE_BLOCK, CORRUPT_BLOCK, AVRO_DATA_BLOCK, HFILE_DATA_BLOCK COMMAND_BLOCK(":command"),
DELETE_BLOCK(":delete"),
CORRUPT_BLOCK(":corrupted"),
AVRO_DATA_BLOCK("avro"),
HFILE_DATA_BLOCK("hfile"),
PARQUET_DATA_BLOCK("parquet");
private static final Map<String, HoodieLogBlockType> ID_TO_ENUM_MAP =
TypeUtils.getValueToEnumMap(HoodieLogBlockType.class, e -> e.id);
private final String id;
HoodieLogBlockType(String id) {
this.id = id;
}
public static HoodieLogBlockType fromId(String id) {
return ID_TO_ENUM_MAP.get(id);
}
} }
/** /**
@@ -132,7 +158,8 @@ public abstract class HoodieLogBlock {
* intensive CompactedScanner, the location helps to lazily read contents from the log file * intensive CompactedScanner, the location helps to lazily read contents from the log file
*/ */
public static final class HoodieLogBlockContentLocation { public static final class HoodieLogBlockContentLocation {
// Hadoop Config required to access the file
private final Configuration hadoopConf;
// The logFile that contains this block // The logFile that contains this block
private final HoodieLogFile logFile; private final HoodieLogFile logFile;
// The filePosition in the logFile for the contents of this block // The filePosition in the logFile for the contents of this block
@@ -142,14 +169,22 @@ public abstract class HoodieLogBlock {
// The final position where the complete block ends // The final position where the complete block ends
private final long blockEndPos; private final long blockEndPos;
HoodieLogBlockContentLocation(HoodieLogFile logFile, long contentPositionInLogFile, long blockSize, public HoodieLogBlockContentLocation(Configuration hadoopConf,
long blockEndPos) { HoodieLogFile logFile,
long contentPositionInLogFile,
long blockSize,
long blockEndPos) {
this.hadoopConf = hadoopConf;
this.logFile = logFile; this.logFile = logFile;
this.contentPositionInLogFile = contentPositionInLogFile; this.contentPositionInLogFile = contentPositionInLogFile;
this.blockSize = blockSize; this.blockSize = blockSize;
this.blockEndPos = blockEndPos; this.blockEndPos = blockEndPos;
} }
public Configuration getHadoopConf() {
return hadoopConf;
}
public HoodieLogFile getLogFile() { public HoodieLogFile getLogFile() {
return logFile; return logFile;
} }
@@ -210,24 +245,27 @@ public abstract class HoodieLogBlock {
* Read or Skip block content of a log block in the log file. Depends on lazy reading enabled in * Read or Skip block content of a log block in the log file. Depends on lazy reading enabled in
* {@link HoodieMergedLogRecordScanner} * {@link HoodieMergedLogRecordScanner}
*/ */
public static byte[] readOrSkipContent(FSDataInputStream inputStream, Integer contentLength, boolean readBlockLazily) public static Option<byte[]> tryReadContent(FSDataInputStream inputStream, Integer contentLength, boolean readLazily)
throws IOException { throws IOException {
byte[] content = null; if (readLazily) {
if (!readBlockLazily) {
// Read the contents in memory
content = new byte[contentLength];
inputStream.readFully(content, 0, contentLength);
} else {
// Seek to the end of the content block // Seek to the end of the content block
inputStream.seek(inputStream.getPos() + contentLength); inputStream.seek(inputStream.getPos() + contentLength);
return Option.empty();
} }
return content;
// TODO re-use buffer if stream is backed by buffer
// Read the contents in memory
byte[] content = new byte[contentLength];
inputStream.readFully(content, 0, contentLength);
return Option.of(content);
} }
/** /**
* When lazyReading of blocks is turned on, inflate the content of a log block from disk. * When lazyReading of blocks is turned on, inflate the content of a log block from disk.
*/ */
protected void inflate() throws HoodieIOException { protected void inflate() throws HoodieIOException {
checkState(!content.isPresent(), "Block has already been inflated");
checkState(inputStream != null, "Block should have input-stream provided");
try { try {
content = Option.of(new byte[(int) this.getBlockContentLocation().get().getBlockSize()]); content = Option.of(new byte[(int) this.getBlockContentLocation().get().getBlockSize()]);

View File

@@ -0,0 +1,171 @@
/*
* 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.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.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hudi.common.fs.inline.InLineFSUtils;
import org.apache.hudi.common.fs.inline.InLineFileSystem;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ParquetReaderIterator;
import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
import org.apache.hudi.io.storage.HoodieParquetStreamWriter;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.hadoop.ParquetReader;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.parquet.hadoop.util.HadoopInputFile;
import org.apache.parquet.io.InputFile;
import javax.annotation.Nonnull;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
/**
* HoodieParquetDataBlock contains a list of records serialized using Parquet.
*/
public class HoodieParquetDataBlock extends HoodieDataBlock {
private final Option<CompressionCodecName> compressionCodecName;
public HoodieParquetDataBlock(FSDataInputStream inputStream,
Option<byte[]> content,
boolean readBlockLazily,
HoodieLogBlockContentLocation logBlockContentLocation,
Option<Schema> readerSchema,
Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer,
String keyField) {
super(content, inputStream, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, keyField, false);
this.compressionCodecName = Option.empty();
}
public HoodieParquetDataBlock(
@Nonnull List<IndexedRecord> records,
@Nonnull Map<HeaderMetadataType, String> header,
@Nonnull String keyField,
@Nonnull CompressionCodecName compressionCodecName
) {
super(records, header, new HashMap<>(), keyField);
this.compressionCodecName = Option.of(compressionCodecName);
}
@Override
public HoodieLogBlockType getBlockType() {
return HoodieLogBlockType.PARQUET_DATA_BLOCK;
}
@Override
protected byte[] serializeRecords(List<IndexedRecord> records) throws IOException {
if (records.size() == 0) {
return new byte[0];
}
Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
new AvroSchemaConverter().convert(writerSchema), writerSchema, Option.empty());
HoodieAvroParquetConfig avroParquetConfig =
new HoodieAvroParquetConfig(
writeSupport,
compressionCodecName.get(),
ParquetWriter.DEFAULT_BLOCK_SIZE,
ParquetWriter.DEFAULT_PAGE_SIZE,
1024 * 1024 * 1024,
new Configuration(),
Double.parseDouble(String.valueOf(0.1)));//HoodieStorageConfig.PARQUET_COMPRESSION_RATIO.defaultValue()));
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try (FSDataOutputStream outputStream = new FSDataOutputStream(baos)) {
try (HoodieParquetStreamWriter<IndexedRecord> parquetWriter = new HoodieParquetStreamWriter<>(outputStream, avroParquetConfig)) {
for (IndexedRecord record : records) {
String recordKey = getRecordKey(record).orElse(null);
parquetWriter.writeAvro(recordKey, record);
}
outputStream.flush();
}
}
return baos.toByteArray();
}
public static Iterator<IndexedRecord> getProjectedParquetRecordsIterator(Configuration conf,
Schema readerSchema,
InputFile inputFile) throws IOException {
AvroReadSupport.setAvroReadSchema(conf, readerSchema);
AvroReadSupport.setRequestedProjection(conf, readerSchema);
ParquetReader<IndexedRecord> reader =
AvroParquetReader.<IndexedRecord>builder(inputFile).withConf(conf).build();
return new ParquetReaderIterator<>(reader);
}
/**
* NOTE: We're overriding the whole reading sequence to make sure we properly respect
* the requested Reader's schema and only fetch the columns that have been explicitly
* requested by the caller (providing projected Reader's schema)
*/
@Override
protected List<IndexedRecord> readRecordsFromBlockPayload() throws IOException {
HoodieLogBlockContentLocation blockContentLoc = getBlockContentLocation().get();
// NOTE: It's important to extend Hadoop configuration here to make sure configuration
// is appropriately carried over
Configuration inlineConf = new Configuration(blockContentLoc.getHadoopConf());
inlineConf.set("fs." + InLineFileSystem.SCHEME + ".impl", InLineFileSystem.class.getName());
Path inlineLogFilePath = InLineFSUtils.getInlineFilePath(
blockContentLoc.getLogFile().getPath(),
blockContentLoc.getLogFile().getPath().getFileSystem(inlineConf).getScheme(),
blockContentLoc.getContentPositionInLogFile(),
blockContentLoc.getBlockSize());
ArrayList<IndexedRecord> records = new ArrayList<>();
getProjectedParquetRecordsIterator(
inlineConf,
readerSchema,
HadoopInputFile.fromPath(inlineLogFilePath, inlineConf)
)
.forEachRemaining(records::add);
return records;
}
@Override
protected List<IndexedRecord> deserializeRecords(byte[] content) throws IOException {
throw new UnsupportedOperationException("Should not be invoked");
}
}

View File

@@ -108,14 +108,31 @@ public final class Option<T> implements Serializable {
} }
} }
/**
* Returns this {@link Option} if not empty, otherwise evaluates the provided supplier
* and returns the alternative
*/
public Option<T> or(Supplier<? extends Option<T>> other) {
return val != null ? this : other.get();
}
/**
* Identical to {@code Optional.orElse}
*/
public T orElse(T other) { public T orElse(T other) {
return val != null ? val : other; return val != null ? val : other;
} }
/**
* Identical to {@code Optional.orElseGet}
*/
public T orElseGet(Supplier<? extends T> other) { public T orElseGet(Supplier<? extends T> other) {
return val != null ? val : other.get(); return val != null ? val : other.get();
} }
/**
* Identical to {@code Optional.orElseThrow}
*/
public <X extends Throwable> T orElseThrow(Supplier<? extends X> exceptionSupplier) throws X { public <X extends Throwable> T orElseThrow(Supplier<? extends X> exceptionSupplier) throws X {
if (val != null) { if (val != null) {
return val; return val;

View File

@@ -0,0 +1,138 @@
/*
* 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.util.io;
import javax.annotation.Nonnull;
import java.io.InputStream;
import java.nio.ByteBuffer;
/**
* Instance of {@link InputStream} backed by {@link ByteBuffer}, implementing following
* functionality (on top of what's required by {@link InputStream})
*
* <ol>
* <li>Seeking: enables random access by allowing to seek to an arbitrary position w/in the stream</li>
* <li>(Thread-safe) Copying: enables to copy from the underlying buffer not modifying the state of the stream</li>
* </ol>
*
* NOTE: Generally methods of this class are NOT thread-safe, unless specified otherwise
*/
public class ByteBufferBackedInputStream extends InputStream {
private final ByteBuffer buffer;
private final int bufferOffset;
public ByteBufferBackedInputStream(ByteBuffer buf) {
this.buffer = buf.duplicate();
// We're marking current buffer position, so that we will be able
// to reset it later on appropriately (to support seek operations)
this.buffer.mark();
this.bufferOffset = buffer.position();
}
public ByteBufferBackedInputStream(byte[] array) {
this(array, 0, array.length);
}
public ByteBufferBackedInputStream(byte[] array, int offset, int length) {
this(ByteBuffer.wrap(array, offset, length));
}
@Override
public int read() {
if (!buffer.hasRemaining()) {
throw new IllegalArgumentException("Reading past backed buffer boundary");
}
return buffer.get() & 0xFF;
}
@Override
public int read(@Nonnull byte[] bytes, int offset, int length) {
if (!buffer.hasRemaining()) {
throw new IllegalArgumentException("Reading past backed buffer boundary");
}
// Determine total number of bytes available to read
int available = Math.min(length, buffer.remaining());
// Copy bytes into the target buffer
buffer.get(bytes, offset, available);
return available;
}
/**
* Returns current position of the stream
*/
public int getPosition() {
return buffer.position() - bufferOffset;
}
/**
* Seeks to a position w/in the stream
*
* NOTE: Position is relative to the start of the stream (ie its absolute w/in this stream),
* with following invariant being assumed:
* <p>0 <= pos <= length (of the stream)</p>
*
* This method is NOT thread-safe
*
* @param pos target position to seek to w/in the holding buffer
*/
public void seek(long pos) {
buffer.reset(); // to mark
int offset = buffer.position();
// NOTE: That the new pos is still relative to buffer's offset
int newPos = offset + (int) pos;
if (newPos > buffer.limit() || newPos < offset) {
throw new IllegalArgumentException(
String.format("Can't seek past the backing buffer (limit %d, offset %d, new %d)", buffer.limit(), offset, newPos)
);
}
buffer.position(newPos);
}
/**
* Copies at most {@code length} bytes starting from position {@code pos} into the target
* buffer with provided {@code offset}. Returns number of bytes copied from the backing buffer
*
* NOTE: This does not change the current position of the stream and is thread-safe
*
* @param pos absolute position w/in stream to read from
* @param targetBuffer target buffer to copy into
* @param offset target buffer offset to copy at
* @param length length of the sequence to copy
* @return number of bytes copied
*/
public int copyFrom(long pos, byte[] targetBuffer, int offset, int length) {
int bufferPos = bufferOffset + (int) pos;
if (bufferPos > buffer.limit()) {
throw new IllegalArgumentException(
String.format("Can't read past the backing buffer boundary (offset %d, length %d)", pos, buffer.limit() - bufferOffset)
);
} else if (length > targetBuffer.length) {
throw new IllegalArgumentException(
String.format("Target buffer is too small (length %d, buffer size %d)", length, targetBuffer.length)
);
}
// Determine total number of bytes available to read
int available = Math.min(length, buffer.limit() - bufferPos);
// Get current buffer position in the backing array
System.arraycopy(buffer.array(), bufferPos, targetBuffer, offset, available);
return available;
}
}

View File

@@ -27,7 +27,7 @@ import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
public interface HoodieFileReader<R extends IndexedRecord> { public interface HoodieFileReader<R extends IndexedRecord> extends AutoCloseable {
public String[] readMinMaxRecordKeys(); public String[] readMinMaxRecordKeys();

View File

@@ -18,7 +18,6 @@
package org.apache.hudi.io.storage; package org.apache.hudi.io.storage;
import java.io.ByteArrayInputStream;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
@@ -51,6 +50,7 @@ import org.apache.hudi.common.bloom.BloomFilterFactory;
import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.io.ByteBufferBackedInputStream;
import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIOException;
@@ -77,11 +77,11 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf); this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
} }
public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig, FileSystem inlineFs) throws IOException { public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig, FileSystem fs) throws IOException {
this.conf = configuration; this.conf = configuration;
this.path = path; this.path = path;
this.fsDataInputStream = inlineFs.open(path); this.fsDataInputStream = fs.open(path);
this.reader = HFile.createReader(inlineFs, path, cacheConfig, configuration); this.reader = HFile.createReader(fs, path, cacheConfig, configuration);
} }
public HoodieHFileReader(byte[] content) throws IOException { public HoodieHFileReader(byte[] content) throws IOException {
@@ -332,28 +332,14 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
} }
} }
static class SeekableByteArrayInputStream extends ByteArrayInputStream implements Seekable, PositionedReadable { static class SeekableByteArrayInputStream extends ByteBufferBackedInputStream implements Seekable, PositionedReadable {
public SeekableByteArrayInputStream(byte[] buf) { public SeekableByteArrayInputStream(byte[] buf) {
super(buf); super(buf);
} }
@Override @Override
public long getPos() throws IOException { public long getPos() throws IOException {
return pos; return getPosition();
}
@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 @Override
@@ -363,19 +349,7 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
@Override @Override
public int read(long position, byte[] buffer, int offset, int length) throws IOException { public int read(long position, byte[] buffer, int offset, int length) throws IOException {
return copyFrom(position, buffer, offset, length);
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 @Override

View File

@@ -34,9 +34,9 @@ import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.ParquetReader;
public class HoodieParquetReader<R extends IndexedRecord> implements HoodieFileReader { public class HoodieParquetReader<R extends IndexedRecord> implements HoodieFileReader<R> {
private Path path; private final Path path;
private Configuration conf; private final Configuration conf;
private final BaseFileUtils parquetUtils; private final BaseFileUtils parquetUtils;
public HoodieParquetReader(Configuration configuration, Path path) { public HoodieParquetReader(Configuration configuration, Path path) {
@@ -45,6 +45,7 @@ public class HoodieParquetReader<R extends IndexedRecord> implements HoodieFileR
this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET); this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET);
} }
@Override
public String[] readMinMaxRecordKeys() { public String[] readMinMaxRecordKeys() {
return parquetUtils.readMinMaxRecordKeys(conf, path); return parquetUtils.readMinMaxRecordKeys(conf, path);
} }
@@ -55,15 +56,15 @@ public class HoodieParquetReader<R extends IndexedRecord> implements HoodieFileR
} }
@Override @Override
public Set<String> filterRowKeys(Set candidateRowKeys) { public Set<String> filterRowKeys(Set<String> candidateRowKeys) {
return parquetUtils.filterRowKeys(conf, path, candidateRowKeys); return parquetUtils.filterRowKeys(conf, path, candidateRowKeys);
} }
@Override @Override
public Iterator<R> getRecordIterator(Schema schema) throws IOException { public Iterator<R> getRecordIterator(Schema schema) throws IOException {
AvroReadSupport.setAvroReadSchema(conf, schema); AvroReadSupport.setAvroReadSchema(conf, schema);
ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(path).withConf(conf).build(); ParquetReader<R> reader = AvroParquetReader.<R>builder(path).withConf(conf).build();
return new ParquetReaderIterator(reader); return new ParquetReaderIterator<>(reader);
} }
@Override @Override

View File

@@ -0,0 +1,88 @@
/*
* 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.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hudi.parquet.io.OutputStreamBackedOutputFile;
import org.apache.parquet.hadoop.ParquetFileWriter;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.io.OutputFile;
import java.io.IOException;
// TODO(HUDI-3035) unify w/ HoodieParquetWriter
public class HoodieParquetStreamWriter<R extends IndexedRecord> implements AutoCloseable {
private final ParquetWriter<R> writer;
private final HoodieAvroWriteSupport writeSupport;
public HoodieParquetStreamWriter(FSDataOutputStream outputStream,
HoodieAvroParquetConfig parquetConfig) throws IOException {
this.writeSupport = parquetConfig.getWriteSupport();
this.writer = new Builder<R>(new OutputStreamBackedOutputFile(outputStream), writeSupport)
.withWriteMode(ParquetFileWriter.Mode.CREATE)
.withCompressionCodec(parquetConfig.getCompressionCodecName())
.withRowGroupSize(parquetConfig.getBlockSize())
.withPageSize(parquetConfig.getPageSize())
.withDictionaryPageSize(parquetConfig.getPageSize())
.withDictionaryEncoding(parquetConfig.dictionaryEnabled())
.withWriterVersion(ParquetWriter.DEFAULT_WRITER_VERSION)
.withConf(parquetConfig.getHadoopConf())
.build();
}
public void writeAvro(String key, R object) throws IOException {
writer.write(object);
writeSupport.add(key);
}
@Override
public void close() throws IOException {
writer.close();
}
private static class Builder<T> extends ParquetWriter.Builder<T, Builder<T>> {
private final WriteSupport<T> writeSupport;
private Builder(Path file, WriteSupport<T> writeSupport) {
super(file);
this.writeSupport = writeSupport;
}
private Builder(OutputFile file, WriteSupport<T> writeSupport) {
super(file);
this.writeSupport = writeSupport;
}
@Override
protected Builder<T> self() {
return this;
}
@Override
protected WriteSupport<T> getWriteSupport(Configuration conf) {
return writeSupport;
}
}
}

View File

@@ -0,0 +1,63 @@
/**
* 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.parquet.io;
import org.apache.hudi.common.util.io.ByteBufferBackedInputStream;
import org.apache.parquet.io.DelegatingSeekableInputStream;
import org.apache.parquet.io.InputFile;
import org.apache.parquet.io.SeekableInputStream;
/**
* Implementation of {@link InputFile} backed by {@code byte[]} buffer
*/
public class ByteBufferBackedInputFile implements InputFile {
private final byte[] buffer;
private final int offset;
private final int length;
public ByteBufferBackedInputFile(byte[] buffer, int offset, int length) {
this.buffer = buffer;
this.offset = offset;
this.length = length;
}
public ByteBufferBackedInputFile(byte[] buffer) {
this(buffer, 0, buffer.length);
}
@Override
public long getLength() {
return length;
}
@Override
public SeekableInputStream newStream() {
return new DelegatingSeekableInputStream(new ByteBufferBackedInputStream(buffer, offset, length)) {
@Override
public long getPos() {
return ((ByteBufferBackedInputStream) getStream()).getPosition();
}
@Override
public void seek(long newPos) {
((ByteBufferBackedInputStream) getStream()).seek(newPos);
}
};
}
}

View File

@@ -0,0 +1,94 @@
/**
* 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.parquet.io;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.parquet.io.OutputFile;
import org.apache.parquet.io.PositionOutputStream;
import javax.annotation.Nonnull;
import java.io.IOException;
/**
* Implementation of the {@link OutputFile} backed by {@link java.io.OutputStream}
*/
public class OutputStreamBackedOutputFile implements OutputFile {
private static final long DEFAULT_BLOCK_SIZE = 1024L * 1024L;
private final FSDataOutputStream outputStream;
public OutputStreamBackedOutputFile(FSDataOutputStream outputStream) {
this.outputStream = outputStream;
}
@Override
public PositionOutputStream create(long blockSizeHint) {
return new PositionOutputStreamAdapter(outputStream);
}
@Override
public PositionOutputStream createOrOverwrite(long blockSizeHint) {
return create(blockSizeHint);
}
@Override
public boolean supportsBlockSize() {
return false;
}
@Override
public long defaultBlockSize() {
return DEFAULT_BLOCK_SIZE;
}
private static class PositionOutputStreamAdapter extends PositionOutputStream {
private final FSDataOutputStream delegate;
PositionOutputStreamAdapter(FSDataOutputStream delegate) {
this.delegate = delegate;
}
@Override
public long getPos() throws IOException {
return delegate.getPos();
}
@Override
public void write(int b) throws IOException {
delegate.write(b);
}
@Override
public void write(@Nonnull byte[] buffer, int off, int len) throws IOException {
delegate.write(buffer, off, len);
}
@Override
public void flush() throws IOException {
delegate.flush();
}
@Override
public void close() {
// We're deliberately not closing the delegate stream here to allow caller
// to explicitly manage its lifecycle
}
}
}

View File

@@ -21,10 +21,12 @@ package org.apache.hudi.common.functional;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieArchivedLogFile; import org.apache.hudi.common.model.HoodieArchivedLogFile;
@@ -46,7 +48,9 @@ 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;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock;
import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.common.testutils.FileCreateUtils;
import org.apache.hudi.common.testutils.HadoopMapRedUtils;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.testutils.SchemaTestUtil;
@@ -54,8 +58,9 @@ import org.apache.hudi.common.testutils.minicluster.MiniClusterUtil;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.exception.CorruptedLogFileException; import org.apache.hudi.exception.CorruptedLogFileException;
import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIOException;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.parquet.hadoop.util.counters.BenchmarkCounter;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
@@ -96,11 +101,12 @@ import static org.junit.jupiter.params.provider.Arguments.arguments;
@SuppressWarnings("Duplicates") @SuppressWarnings("Duplicates")
public class TestHoodieLogFormat extends HoodieCommonTestHarness { public class TestHoodieLogFormat extends HoodieCommonTestHarness {
private static final HoodieLogBlockType DEFAULT_DATA_BLOCK_TYPE = HoodieLogBlockType.AVRO_DATA_BLOCK;
private static String BASE_OUTPUT_PATH = "/tmp/"; private static String BASE_OUTPUT_PATH = "/tmp/";
private FileSystem fs; private FileSystem fs;
private Path partitionPath; private Path partitionPath;
private int bufferSize = 4096; private int bufferSize = 4096;
private HoodieLogBlockType dataBlockType = HoodieLogBlockType.AVRO_DATA_BLOCK;
@BeforeAll @BeforeAll
public static void setUpClass() throws IOException, InterruptedException { public static void setUpClass() throws IOException, InterruptedException {
@@ -139,7 +145,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
} }
@ParameterizedTest @ParameterizedTest
@EnumSource(names = { "AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK" }) @EnumSource(names = {"AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK", "PARQUET_DATA_BLOCK"})
public void testBasicAppend(HoodieLogBlockType dataBlockType) throws IOException, InterruptedException, URISyntaxException { public void testBasicAppend(HoodieLogBlockType dataBlockType) throws IOException, InterruptedException, URISyntaxException {
Writer writer = Writer writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
@@ -171,7 +177,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
// Write out a block // Write out a block
AppendResult firstAppend = writer.appendBlock(dataBlock); AppendResult firstAppend = writer.appendBlock(dataBlock);
// Get the size of the block // Get the size of the block
@@ -186,7 +192,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build(); .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build();
records = SchemaTestUtil.generateTestRecords(0, 100); records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
AppendResult secondAppend = writer.appendBlock(dataBlock); AppendResult secondAppend = writer.appendBlock(dataBlock);
assertEquals(firstAppend.logFile(), secondAppend.logFile()); assertEquals(firstAppend.logFile(), secondAppend.logFile());
@@ -198,7 +204,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
// Write one more block, which should not go to the new log file. // Write one more block, which should not go to the new log file.
records = SchemaTestUtil.generateTestRecords(0, 100); records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
AppendResult rolloverAppend = writer.appendBlock(dataBlock); AppendResult rolloverAppend = writer.appendBlock(dataBlock);
assertNotEquals(secondAppend.logFile(), rolloverAppend.logFile()); assertNotEquals(secondAppend.logFile(), rolloverAppend.logFile());
@@ -245,7 +251,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
Writer writer2 = builder2.build(); Writer writer2 = builder2.build();
writer2.appendBlock(dataBlock); writer2.appendBlock(dataBlock);
@@ -257,8 +263,9 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
assertEquals(logFile1.getLogVersion(), logFile2.getLogVersion() - 1, "Log Files must have different versions"); assertEquals(logFile1.getLogVersion(), logFile2.getLogVersion() - 1, "Log Files must have different versions");
} }
@Test @ParameterizedTest
public void testMultipleAppend() throws IOException, URISyntaxException, InterruptedException { @EnumSource(names = {"AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK", "PARQUET_DATA_BLOCK"})
public void testMultipleAppend(HoodieLogBlockType dataBlockType) throws IOException, URISyntaxException, InterruptedException {
Writer writer = Writer writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
@@ -266,7 +273,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header); HoodieDataBlock dataBlock = getDataBlock(dataBlockType, records, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize(); long size1 = writer.getCurrentSize();
writer.close(); writer.close();
@@ -276,7 +283,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100); records = SchemaTestUtil.generateTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header); dataBlock = getDataBlock(dataBlockType, records, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
long size2 = writer.getCurrentSize(); long size2 = writer.getCurrentSize();
assertTrue(size2 > size1, "We just wrote a new block - size2 should be > size1"); assertTrue(size2 > size1, "We just wrote a new block - size2 should be > size1");
@@ -290,7 +297,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100); records = SchemaTestUtil.generateTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header); dataBlock = getDataBlock(dataBlockType, records, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
long size3 = writer.getCurrentSize(); long size3 = writer.getCurrentSize();
assertTrue(size3 > size2, "We just wrote a new block - size3 should be > size2"); assertTrue(size3 > size2, "We just wrote a new block - size3 should be > size2");
@@ -309,26 +316,27 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
* This is actually a test on concurrent append and not recovery lease. Commenting this out. * This is actually a test on concurrent append and not recovery lease. Commenting this out.
* https://issues.apache.org/jira/browse/HUDI-117 * https://issues.apache.org/jira/browse/HUDI-117
*/ */
/** /**
* @Test public void testLeaseRecovery() throws IOException, URISyntaxException, InterruptedException { Writer writer * @Test public void testLeaseRecovery() throws IOException, URISyntaxException, InterruptedException { Writer writer
* = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) * = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
* .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") * .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
* .overBaseCommit("100").withFs(fs).build(); List<IndexedRecord> records = * .overBaseCommit("100").withFs(fs).build(); List<IndexedRecord> records =
* SchemaTestUtil.generateTestRecords(0, 100); Map<HoodieLogBlock.HeaderMetadataType, String> header = * SchemaTestUtil.generateTestRecords(0, 100); Map<HoodieLogBlock.HeaderMetadataType, String> header =
* Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); * Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
* header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieAvroDataBlock * header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieAvroDataBlock
* dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size1 = * dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size1 =
* writer.getCurrentSize(); // do not close this writer - this simulates a data note appending to a log dying * writer.getCurrentSize(); // do not close this writer - this simulates a data note appending to a log dying
* without closing the file // writer.close(); * without closing the file // writer.close();
* * <p>
* writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) * writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
* .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") * .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100")
* .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); * .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100);
* header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new * header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new
* HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size2 = * HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size2 =
* writer.getCurrentSize(); assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1); * writer.getCurrentSize(); assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1);
* assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match", * assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match",
* size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); } * size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); }
*/ */
@Test @Test
@@ -344,7 +352,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
for (int i = 0; i < 2; i++) { for (int i = 0; i < 2; i++) {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath) Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath)
@@ -371,14 +379,14 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
assertTrue(reader.hasNext(), "We wrote a block, we should be able to read it"); assertTrue(reader.hasNext(), "We wrote a block, we should be able to read it");
HoodieLogBlock nextBlock = reader.next(); HoodieLogBlock nextBlock = reader.next();
assertEquals(dataBlockType, nextBlock.getBlockType(), "The next block should be a data block"); assertEquals(DEFAULT_DATA_BLOCK_TYPE, nextBlock.getBlockType(), "The next block should be a data block");
HoodieDataBlock dataBlockRead = (HoodieDataBlock) nextBlock; HoodieDataBlock dataBlockRead = (HoodieDataBlock) nextBlock;
assertEquals(copyOfRecords.size(), dataBlockRead.getRecords().size(), assertEquals(copyOfRecords.size(), dataBlockRead.getRecords().size(),
"Read records size should be equal to the written records size"); "Read records size should be equal to the written records size");
@@ -400,10 +408,10 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
byte[] dataBlockContentBytes = getDataBlock(records, header).getContentBytes(); byte[] dataBlockContentBytes = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header).getContentBytes();
HoodieDataBlock reusableDataBlock = new HoodieAvroDataBlock(null, null, HoodieLogBlock.HoodieLogBlockContentLocation logBlockContentLoc = new HoodieLogBlock.HoodieLogBlockContentLocation(new Configuration(), null, 0, dataBlockContentBytes.length, 0);
Option.ofNullable(dataBlockContentBytes), false, 0, dataBlockContentBytes.length, HoodieDataBlock reusableDataBlock = new HoodieAvroDataBlock(null, Option.ofNullable(dataBlockContentBytes), false,
0, getSimpleSchema(), header, new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD); logBlockContentLoc, Option.ofNullable(getSimpleSchema()), header, new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD);
long writtenSize = 0; long writtenSize = 0;
int logBlockWrittenNum = 0; int logBlockWrittenNum = 0;
while (writtenSize < Integer.MAX_VALUE) { while (writtenSize < Integer.MAX_VALUE) {
@@ -418,7 +426,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
true, true); true, true);
assertTrue(reader.hasNext(), "We wrote a block, we should be able to read it"); assertTrue(reader.hasNext(), "We wrote a block, we should be able to read it");
HoodieLogBlock nextBlock = reader.next(); HoodieLogBlock nextBlock = reader.next();
assertEquals(dataBlockType, nextBlock.getBlockType(), "The next block should be a data block"); assertEquals(DEFAULT_DATA_BLOCK_TYPE, nextBlock.getBlockType(), "The next block should be a data block");
HoodieDataBlock dataBlockRead = (HoodieDataBlock) nextBlock; HoodieDataBlock dataBlockRead = (HoodieDataBlock) nextBlock;
assertEquals(copyOfRecords.size(), dataBlockRead.getRecords().size(), assertEquals(copyOfRecords.size(), dataBlockRead.getRecords().size(),
"Read records size should be equal to the written records size"); "Read records size should be equal to the written records size");
@@ -447,11 +455,16 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
oversizeWriter.close(); oversizeWriter.close();
} }
@Test @ParameterizedTest
public void testBasicAppendAndRead() throws IOException, URISyntaxException, InterruptedException { @EnumSource(names = {"AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK", "PARQUET_DATA_BLOCK"})
Writer writer = public void testBasicAppendAndRead(HoodieLogBlockType dataBlockType) throws IOException, URISyntaxException, InterruptedException {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) Writer writer = HoodieLogFormat.newWriterBuilder()
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); .onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1")
.overBaseCommit("100")
.withFs(fs)
.build();
List<IndexedRecord> records1 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> records1 = SchemaTestUtil.generateTestRecords(0, 100);
Schema schema = getSimpleSchema(); Schema schema = getSimpleSchema();
List<IndexedRecord> copyOfRecords1 = records1.stream() List<IndexedRecord> copyOfRecords1 = records1.stream()
@@ -459,30 +472,39 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(dataBlockType, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
writer = writer = HoodieLogFormat.newWriterBuilder()
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) .onParentPath(partitionPath)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); .withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1")
.overBaseCommit("100")
.withFs(fs)
.build();
List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100);
List<IndexedRecord> copyOfRecords2 = records2.stream() List<IndexedRecord> copyOfRecords2 = records2.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records2, header); dataBlock = getDataBlock(dataBlockType, records2, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
// Close and Open again and append 100 more records // Close and Open again and append 100 more records
writer = writer = HoodieLogFormat.newWriterBuilder()
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) .onParentPath(partitionPath)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); .withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1")
.overBaseCommit("100")
.withFs(fs)
.build();
List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100);
List<IndexedRecord> copyOfRecords3 = records3.stream() List<IndexedRecord> copyOfRecords3 = records3.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records3, header); dataBlock = getDataBlock(dataBlockType, records3, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -538,7 +560,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
allRecords.add(copyOfRecords1); allRecords.add(copyOfRecords1);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
} }
writer.close(); writer.close();
@@ -580,7 +602,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -602,11 +624,11 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
// Append a proper block that is of the missing length of the corrupted block // Append a proper block that is of the missing length of the corrupted block
writer = writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 10); records = SchemaTestUtil.generateTestRecords(0, 10);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -644,7 +666,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100); records = SchemaTestUtil.generateTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -674,7 +696,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -702,7 +724,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 10); records = SchemaTestUtil.generateTestRecords(0, 10);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -741,7 +763,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
// Write 2 // Write 2
@@ -749,7 +771,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> copyOfRecords2 = records2.stream() List<IndexedRecord> copyOfRecords2 = records2.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records2, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -804,14 +826,14 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
// Write 2 // Write 2
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records2, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
// Rollback the last write // Rollback the last write
@@ -827,7 +849,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> copyOfRecords3 = records3.stream() List<IndexedRecord> copyOfRecords3 = records3.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records3, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records3, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -880,7 +902,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -914,7 +936,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records3, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records3, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -968,7 +990,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
// Write 2 // Write 2
@@ -976,7 +998,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<IndexedRecord> copyOfRecords2 = records2.stream() List<IndexedRecord> copyOfRecords2 = records2.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
dataBlock = getDataBlock(records2, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
copyOfRecords1.addAll(copyOfRecords2); copyOfRecords1.addAll(copyOfRecords2);
@@ -1089,13 +1111,13 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
// Write 2 // Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records2, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
// Delete 50 keys // Delete 50 keys
@@ -1173,7 +1195,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
// Delete 50 keys // Delete 50 keys
@@ -1232,7 +1254,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
FileCreateUtils.createDeltaCommit(basePath, "100", fs); FileCreateUtils.createDeltaCommit(basePath, "100", fs);
@@ -1290,7 +1312,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
@@ -1354,7 +1376,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
@@ -1473,7 +1495,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records.subList(0, numRecordsInLog1), header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records.subList(0, numRecordsInLog1), header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
// Get the size of the block // Get the size of the block
long size = writer.getCurrentSize(); long size = writer.getCurrentSize();
@@ -1487,7 +1509,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header2 = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header2 = new HashMap<>();
header2.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header2.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header2.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header2.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock2 = getDataBlock(records2.subList(0, numRecordsInLog2), header2); HoodieDataBlock dataBlock2 = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2.subList(0, numRecordsInLog2), header2);
writer2.appendBlock(dataBlock2); writer2.appendBlock(dataBlock2);
// Get the size of the block // Get the size of the block
writer2.close(); writer2.close();
@@ -1574,7 +1596,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -1584,7 +1606,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100);
List<IndexedRecord> copyOfRecords2 = records2.stream() List<IndexedRecord> copyOfRecords2 = records2.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
dataBlock = getDataBlock(records2, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -1595,7 +1617,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100);
List<IndexedRecord> copyOfRecords3 = records3.stream() List<IndexedRecord> copyOfRecords3 = records3.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
dataBlock = getDataBlock(records3, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records3, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -1646,7 +1668,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -1674,7 +1696,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100); records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -1708,7 +1730,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header); HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -1716,7 +1738,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records2, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -1725,7 +1747,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records3, header); dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records3, header);
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
writer.close(); writer.close();
@@ -1786,8 +1808,66 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
} }
} }
private HoodieDataBlock getDataBlock(List<IndexedRecord> records, Map<HeaderMetadataType, String> header) { @ParameterizedTest
return getDataBlock(dataBlockType, records, header); @EnumSource(names = {"AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK", "PARQUET_DATA_BLOCK"})
public void testDataBlockFormatAppendAndReadWithProjectedSchema(
HoodieLogBlockType dataBlockType
) throws IOException, URISyntaxException, InterruptedException {
Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1")
.overBaseCommit("100")
.withFs(fs)
.build();
List<GenericRecord> records = SchemaTestUtil.generateTestGenericRecords(0, 1000);
Schema schema = getSimpleSchema();
Map<HoodieLogBlock.HeaderMetadataType, String> header =
new HashMap<HoodieLogBlock.HeaderMetadataType, String>() {{
put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
}};
// Init Benchmark to report number of bytes actually read from the Block
BenchmarkCounter.initCounterFromReporter(HadoopMapRedUtils.createTestReporter(), fs.getConf());
// NOTE: Have to use this ugly hack since List generic is not covariant in its type param
HoodieDataBlock dataBlock = getDataBlock(dataBlockType, (List<IndexedRecord>)(List) records, header);
writer.appendBlock(dataBlock);
writer.close();
Schema projectedSchema = HoodieAvroUtils.generateProjectionSchema(schema, Collections.singletonList("name"));
List<GenericRecord> projectedRecords = HoodieAvroUtils.rewriteRecords(records, projectedSchema);
try (Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), projectedSchema, true, false)) {
assertTrue(reader.hasNext(), "First block should be available");
HoodieLogBlock nextBlock = reader.next();
HoodieDataBlock dataBlockRead = (HoodieDataBlock) nextBlock;
Map<HoodieLogBlockType, Integer> expectedReadBytes =
new HashMap<HoodieLogBlockType, Integer>() {{
put(HoodieLogBlockType.AVRO_DATA_BLOCK, 0); // not supported
put(HoodieLogBlockType.HFILE_DATA_BLOCK, 0); // not supported
put(HoodieLogBlockType.PARQUET_DATA_BLOCK, 2605);
}};
assertEquals(projectedRecords.size(), dataBlockRead.getRecords().size(),
"Read records size should be equal to the written records size");
assertEquals(projectedRecords, dataBlockRead.getRecords(),
"Both records lists should be the same. (ordering guaranteed)");
assertEquals(dataBlockRead.getSchema(), projectedSchema);
int bytesRead = (int) BenchmarkCounter.getBytesRead();
assertEquals(expectedReadBytes.get(dataBlockType), bytesRead, "Read bytes have to match");
}
} }
private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List<IndexedRecord> records, private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List<IndexedRecord> records,
@@ -1796,7 +1876,9 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
case AVRO_DATA_BLOCK: case AVRO_DATA_BLOCK:
return new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); return new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
case HFILE_DATA_BLOCK: case HFILE_DATA_BLOCK:
return new HoodieHFileDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); return new HoodieHFileDataBlock(records, header, Compression.Algorithm.GZ);
case PARQUET_DATA_BLOCK:
return new HoodieParquetDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD, CompressionCodecName.GZIP);
default: default:
throw new RuntimeException("Unknown data block type " + dataBlockType); throw new RuntimeException("Unknown data block type " + dataBlockType);
} }

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.common.functional; package org.apache.hudi.common.functional;
import org.apache.hudi.common.model.HoodieArchivedLogFile; import org.apache.hudi.common.model.HoodieArchivedLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
@@ -104,7 +105,7 @@ public class TestHoodieLogFormatAppendFailure {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(2); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(2);
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath) Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath)
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive") .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive")

View File

@@ -0,0 +1,87 @@
/*
* 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.testutils;
import org.apache.hadoop.mapred.Counters;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hudi.common.util.Option;
import java.util.concurrent.ConcurrentHashMap;
public class HadoopMapRedUtils {
/**
* Creates instance of {@link Reporter} to collect reported counters
*/
public static Reporter createTestReporter() {
class TestReporter implements Reporter {
private final ConcurrentHashMap<String, Counters.Counter> counters =
new ConcurrentHashMap<>();
@Override
public void setStatus(String status) {
// not-supported
}
@Override
public Counters.Counter getCounter(Enum<?> name) {
return counters.computeIfAbsent(name.name(), (ignored) -> new Counters.Counter());
}
@Override
public Counters.Counter getCounter(String group, String name) {
return counters.computeIfAbsent(getKey(group, name), (ignored) -> new Counters.Counter());
}
@Override
public void incrCounter(Enum<?> key, long amount) {
Option.ofNullable(counters.get(key))
.ifPresent(c -> c.increment(amount));
}
@Override
public void incrCounter(String group, String counter, long amount) {
Option.ofNullable(counters.get(getKey(group, counter)))
.ifPresent(c -> c.increment(amount));
}
@Override
public InputSplit getInputSplit() throws UnsupportedOperationException {
throw new UnsupportedOperationException("not supported");
}
@Override
public float getProgress() {
return -1;
}
@Override
public void progress() {
// not-supported
}
private String getKey(String group, String name) {
return String.format("%s:%s", group, name);
}
}
return new TestReporter();
}
}

View File

@@ -71,6 +71,10 @@ public final class SchemaTestUtil {
return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit); return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit);
} }
public static List<GenericRecord> generateTestGenericRecords(int from, int limit) throws IOException, URISyntaxException {
return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit);
}
public static List<String> generateTestJsonRecords(int from, int limit) throws IOException, URISyntaxException { public static List<String> generateTestJsonRecords(int from, int limit) throws IOException, URISyntaxException {
Path dataPath = initializeSampleDataPath(); Path dataPath = initializeSampleDataPath();
@@ -81,9 +85,9 @@ public final class SchemaTestUtil {
} }
} }
private static List<IndexedRecord> toRecords(Schema writerSchema, Schema readerSchema, int from, int limit) private static <T extends IndexedRecord> List<T> toRecords(Schema writerSchema, Schema readerSchema, int from, int limit)
throws IOException, URISyntaxException { throws IOException, URISyntaxException {
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema); GenericDatumReader<T> reader = new GenericDatumReader<>(writerSchema, readerSchema);
Path dataPath = initializeSampleDataPath(); Path dataPath = initializeSampleDataPath();
try (Stream<String> stream = Files.lines(dataPath)) { try (Stream<String> stream = Files.lines(dataPath)) {

View File

@@ -0,0 +1,113 @@
/*
* 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.util.io;
import org.junit.jupiter.api.Test;
import java.nio.ByteBuffer;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
public class TestByteBufferBackedInputStream {
@Test
public void testConstructor() {
byte[] bytes = { 0xD, 0xE, 0xA, 0xD, 0xD, 0xE, 0xE, 0xD };
ByteBuffer byteBuf = ByteBuffer.wrap(bytes, 0, 1);
ByteBuffer byteBufClone = byteBuf.duplicate();
// ByteBuffer ctor
ByteBufferBackedInputStream first = new ByteBufferBackedInputStream(byteBuf);
assertEquals(first.read(), 0xD);
assertThrows(IllegalArgumentException.class, first::read);
// Make sure that the original buffer stays intact
assertEquals(byteBufClone, byteBuf);
// byte[] ctor
ByteBufferBackedInputStream second = new ByteBufferBackedInputStream(bytes);
assertEquals(second.read(), 0xD);
// byte[] ctor (w/ offset)
ByteBufferBackedInputStream third = new ByteBufferBackedInputStream(bytes, 1, 1);
assertEquals(third.read(), 0xE);
assertThrows(IllegalArgumentException.class, third::read);
}
@Test
public void testRead() {
byte[] sourceBytes = { 0xD, 0xE, 0xA, 0xD, 0xD, 0xE, 0xE, 0xD };
ByteBufferBackedInputStream stream = new ByteBufferBackedInputStream(sourceBytes);
int firstByte = stream.read();
assertEquals(firstByte, 0xD);
byte[] readBytes = new byte[4];
int read = stream.read(readBytes, 1, 3);
assertEquals(3, read);
assertArrayEquals(new byte[]{0, 0xE, 0xA, 0xD}, readBytes);
assertEquals(4, stream.getPosition());
}
@Test
public void testSeek() {
byte[] sourceBytes = { 0xD, 0xE, 0xA, 0xD, 0xD, 0xA, 0xE, 0xD };
ByteBufferBackedInputStream stream = new ByteBufferBackedInputStream(sourceBytes, 1, 7);
// Seek to 2 byte in the stream (3 in the original buffer)
stream.seek(1);
int firstRead = stream.read();
assertEquals(0xA, firstRead);
// Seek to 5 byte in the stream (6 in the original buffer)
stream.seek(5);
int secondRead = stream.read();
assertEquals(0xE, secondRead);
// Try to seek past the stream boundary
assertThrows(IllegalArgumentException.class, () -> stream.seek(8));
}
@Test
public void testCopyFrom() {
byte[] sourceBytes = { 0xD, 0xE, 0xA, 0xD, 0xD, 0xA, 0xE, 0xD };
ByteBufferBackedInputStream stream = new ByteBufferBackedInputStream(sourceBytes);
int firstByte = stream.read();
assertEquals(firstByte, 0xD);
// Copy 5 byes from the stream (while keeping stream's state intact)
byte[] targetBytes = new byte[5];
stream.copyFrom(2, targetBytes, 0, targetBytes.length);
assertArrayEquals(new byte[] { 0xA, 0xD, 0xD, 0xA, 0xE }, targetBytes);
// Continue reading the stream from where we left of (before copying)
int secondByte = stream.read();
assertEquals(secondByte, 0xE);
}
}

View File

@@ -163,6 +163,12 @@ public class TestHoodieRealtimeRecordReader {
HoodieLogBlock.HoodieLogBlockType.HFILE_DATA_BLOCK); HoodieLogBlock.HoodieLogBlockType.HFILE_DATA_BLOCK);
} }
@Test
public void testParquetInlineReader() throws Exception {
testReaderInternal(ExternalSpillableMap.DiskMapType.BITCASK, false, false,
HoodieLogBlock.HoodieLogBlockType.PARQUET_DATA_BLOCK);
}
private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType, private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType,
boolean isCompressionEnabled, boolean isCompressionEnabled,
boolean partitioned) throws Exception { boolean partitioned) throws Exception {

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.hadoop.testutils;
import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.fs.RawLocalFileSystem;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieFileFormat;
@@ -33,6 +34,7 @@ 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.HoodieDataBlock;
import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock; 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;
import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock;
import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
@@ -47,6 +49,7 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConf;
import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.avro.AvroParquetWriter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
@@ -363,9 +366,14 @@ public class InputFormatTestUtil {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchema.toString());
HoodieDataBlock dataBlock = (logBlockType == HoodieLogBlock.HoodieLogBlockType.HFILE_DATA_BLOCK) HoodieDataBlock dataBlock = null;
? new HoodieHFileDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD) : if (logBlockType == HoodieLogBlock.HoodieLogBlockType.HFILE_DATA_BLOCK) {
new HoodieAvroDataBlock(records, header); dataBlock = new HoodieHFileDataBlock(records, header, Compression.Algorithm.GZ);
} else if (logBlockType == HoodieLogBlock.HoodieLogBlockType.PARQUET_DATA_BLOCK) {
dataBlock = new HoodieParquetDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD, CompressionCodecName.GZIP);
} else {
dataBlock = new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
}
writer.appendBlock(dataBlock); writer.appendBlock(dataBlock);
return writer; return writer;
} }

View File

@@ -28,6 +28,7 @@ import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieDeltaWriteStat; import org.apache.hudi.common.model.HoodieDeltaWriteStat;
import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.HoodieWriteStat;
@@ -428,7 +429,7 @@ public class HiveTestUtil {
Map<HeaderMetadataType, String> header = new HashMap<>(2); Map<HeaderMetadataType, String> header = new HashMap<>(2);
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, dataFile.getCommitTime()); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, dataFile.getCommitTime());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
logWriter.appendBlock(dataBlock); logWriter.appendBlock(dataBlock);
logWriter.close(); logWriter.close();
return logWriter.getLogFile(); return logWriter.getLogFile();