diff --git a/docs/_data/topnav.yml b/docs/_data/topnav.yml
index b144d9a83..190573a3e 100644
--- a/docs/_data/topnav.yml
+++ b/docs/_data/topnav.yml
@@ -23,7 +23,7 @@ topnav_dropdowns:
url: /api_docs.html
output: web
- title: Code Structure
- url: /code_structure.html
+ url: /code_and_design.html
output: web
- title: Roadmap
url: /roadmap.html
diff --git a/docs/code_and_design.md b/docs/code_and_design.md
new file mode 100644
index 000000000..3baaa97b4
--- /dev/null
+++ b/docs/code_and_design.md
@@ -0,0 +1,38 @@
+---
+title: Code Structure
+keywords: usecases
+sidebar: mydoc_sidebar
+permalink: code_and_design.html
+---
+
+## Code & Project Structure
+
+ * hoodie-client : Spark client library to take a bunch of inserts + updates and apply them to a Hoodie table
+ * hoodie-common : Common code shared between different artifacts of Hoodie
+
+ ## HoodieLogFormat
+
+ The following diagram depicts the LogFormat for Hoodie MergeOnRead. Each logfile consists of one or more log blocks.
+ Each logblock follows the format shown below.
+
+ | Field | Description |
+ |-------------- |------------------|
+ | MAGIC | A magic header that marks the start of a block |
+ | VERSION | The version of the LogFormat, this helps define how to switch between different log format as it evolves |
+ | TYPE | The type of the log block |
+ | HEADER LENGTH | The length of the headers, 0 if no headers |
+ | HEADER | Metadata needed for a log block. For eg. INSTANT_TIME, TARGET_INSTANT_TIME, SCHEMA etc. |
+ | CONTENT LENGTH | The length of the content of the log block |
+ | CONTENT | The content of the log block, for example, for a DATA_BLOCK, the content is (number of records + actual records) in byte [] |
+ | FOOTER LENGTH | The length of the footers, 0 if no footers |
+ | FOOTER | Metadata needed for a log block. For eg. index entries, a bloom filter for records in a DATA_BLOCK etc. |
+ | LOGBLOCK LENGTH | The total number of bytes written for a log block, typically the SUM(everything_above). This is a LONG. This acts as a reverse pointer to be able to traverse the log in reverse.|
+
+
+ {% include image.html file="hoodie_log_format_v2.png" alt="hoodie_log_format_v2.png" %}
+
+
+
+
+
+
diff --git a/docs/code_structure.md b/docs/code_structure.md
deleted file mode 100644
index 2550c905e..000000000
--- a/docs/code_structure.md
+++ /dev/null
@@ -1,16 +0,0 @@
----
-title: Code Structure
-keywords: usecases
-sidebar: mydoc_sidebar
-permalink: code_structure.html
----
-
-## Code & Project Structure
-
- * hoodie-client : Spark client library to take a bunch of inserts + updates and apply them to a Hoodie table
- * hoodie-common : Common code shared between different artifacts of Hoodie
-
-
-
-
-
diff --git a/docs/configurations.md b/docs/configurations.md
index 328c5a6e7..2ff607707 100644
--- a/docs/configurations.md
+++ b/docs/configurations.md
@@ -74,6 +74,14 @@ summary: "Here we list all possible configurations and what they mean"
Should hoodie dynamically compute the insertSplitSize based on the last 24 commit's metadata. Turned off by default.
- [approxRecordSize](#approxRecordSize) ()
The average record size. If specified, hoodie will use this and not compute dynamically based on the last 24 commit's metadata. No value set as default. This is critical in computing the insert parallelism and bin-packing inserts into small files. See above.
+ - [withCompactionLazyBlockReadEnabled](#withCompactionLazyBlockReadEnabled) (true)
+ When a CompactedLogScanner merges all log files, this config helps to choose whether the logblocks should be read lazily or not. Choose true to use I/O intensive lazy block reading (low memory usage) or false for Memory intensive immediate block read (high memory usage)
+ - [withMaxNumDeltaCommitsBeforeCompaction](#withMaxNumDeltaCommitsBeforeCompaction) (maxNumDeltaCommitsBeforeCompaction = 10)
+ Number of max delta commits to keep before triggering an inline compaction
+ - [withCompactionReverseLogReadEnabled](#withCompactionReverseLogReadEnabled) (false)
+ HoodieLogFormatReader reads a logfile in the forward direction starting from pos=0 to pos=file_length. If this config is set to true, the Reader reads the logfile in reverse direction, from pos=file_length to pos=0
+ - [withMaxMemorySizePerCompactionInBytes](#withMaxMemorySizePerCompactionInBytes) (maxMemorySizePerCompactionInBytes = 1GB)
+ HoodieCompactedLogScanner reads logblocks, converts records to HoodieRecords and then merges these log blocks and records. At any point, the number of entries in a log block can be less than or equal to the number of entries in the corresponding parquet file. This can lead to OOM in the Scanner. Hence, a spillable map helps alleviate the memory pressure. Use this config to set the max allowable inMemory footprint of the spillable map.
- [withMetricsConfig](#withMetricsConfig) (HoodieMetricsConfig)
Hoodie publishes metrics on every commit, clean, rollback etc.
diff --git a/docs/images/hoodie_log_format_v2.png b/docs/images/hoodie_log_format_v2.png
new file mode 100644
index 000000000..c392e2677
Binary files /dev/null and b/docs/images/hoodie_log_format_v2.png differ
diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java
index d4945c012..50fb6a565 100644
--- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java
+++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java
@@ -58,11 +58,12 @@ public class ArchivedCommitsCommand implements CommandMarker {
FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf)
.globStatus(new Path(basePath + "/.hoodie/.commits_.archive*"));
List allCommits = new ArrayList<>();
+ int commits = 0;
for (FileStatus fs : fsStatuses) {
//read the archived file
HoodieLogFormat.Reader reader = HoodieLogFormat
.newReader(FSUtils.getFs(basePath, HoodieCLI.conf),
- new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema(), false);
+ new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
List readRecords = new ArrayList<>();
//read the avro blocks
@@ -70,10 +71,17 @@ public class ArchivedCommitsCommand implements CommandMarker {
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
List records = blk.getRecords();
readRecords.addAll(records);
+ if(commits == limit) {
+ break;
+ }
+ commits++;
}
List readCommits = readRecords.stream().map(r -> (GenericRecord) r)
- .map(r -> readCommit(r)).limit(limit).collect(Collectors.toList());
+ .map(r -> readCommit(r)).collect(Collectors.toList());
allCommits.addAll(readCommits);
+ if(commits == limit) {
+ break;
+ }
}
return HoodiePrintHelper.print(
new String[]{"CommitTime", "CommitType", "CommitDetails"},
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java
index 75dd49d6c..6579ccf9f 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java
@@ -105,6 +105,15 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
// Default memory size per compaction, excess spills to disk
public static final String DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES = String.valueOf(1024*1024*1024L); //1GB
+ // used to choose a trade off between IO vs Memory when performing compaction process
+ // Depending on outputfile_size and memory provided, choose true to avoid OOM for large file size + small memory
+ public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = "hoodie.compaction.lazy.block.read";
+ public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "false";
+
+ // used to choose whether to enable reverse log reading (reverse log traversal)
+ public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = "hoodie.compaction.reverse.log.read";
+ public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false";
+
private HoodieCompactionConfig(Properties props) {
super(props);
}
@@ -225,6 +234,18 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
return this;
}
+ public Builder withCompactionLazyBlockReadEnabled(Boolean compactionLazyBlockReadEnabled) {
+ props.setProperty(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
+ String.valueOf(compactionLazyBlockReadEnabled));
+ return this;
+ }
+
+ public Builder withCompactionReverseLogReadEnabled(Boolean compactionReverseLogReadEnabled) {
+ props.setProperty(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP,
+ String.valueOf(compactionReverseLogReadEnabled));
+ return this;
+ }
+
public HoodieCompactionConfig build() {
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP),
@@ -262,6 +283,10 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB);
setDefaultOnCondition(props, !props.containsKey(MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP),
MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP, DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES);
+ setDefaultOnCondition(props, !props.containsKey(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP),
+ COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED);
+ setDefaultOnCondition(props, !props.containsKey(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP),
+ COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED);
HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));
Preconditions.checkArgument(
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java
index 9933f3c9c..dfae2a082 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java
@@ -24,14 +24,14 @@ import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
import com.uber.hoodie.metrics.MetricsReporterType;
+import org.apache.spark.storage.StorageLevel;
+import javax.annotation.concurrent.Immutable;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStream;
import java.util.Map;
import java.util.Properties;
-import javax.annotation.concurrent.Immutable;
-import org.apache.spark.storage.StorageLevel;
/**
* Class storing configs for the {@link com.uber.hoodie.HoodieWriteClient}
@@ -215,6 +215,14 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
.parseLong(props.getProperty(HoodieCompactionConfig.MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP));
}
+ public Boolean getCompactionLazyBlockReadEnabled() {
+ return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP));
+ }
+
+ public Boolean getCompactionReverseLogReadEnabled() {
+ return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP));
+ }
+
/**
* index properties
**/
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java
index 0720b133f..7f00961f2 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java
@@ -159,11 +159,14 @@ public class HoodieAppendHandle extends HoodieIOH
return Optional.empty();
}
+ // TODO (NA) - Perform a schema check of current input record with the last schema on log file
+ // to make sure we don't append records with older (shorter) schema than already appended
public void doAppend() {
int maxBlockSize = config.getLogFileDataBlockMaxSize(); int numberOfRecords = 0;
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, commitTime);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
while (recordItr.hasNext()) {
HoodieRecord record = recordItr.next();
// update the new location of the record, so we know where to find it next
@@ -178,7 +181,7 @@ public class HoodieAppendHandle extends HoodieIOH
// Recompute averageRecordSize before writing a new block and update existing value with avg of new and old
logger.info("AvgRecordSize => " + averageRecordSize);
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record))/2;
- doAppend(metadata);
+ doAppend(header);
numberOfRecords = 0;
}
Optional indexedRecord = getIndexedRecord(record);
@@ -189,18 +192,18 @@ public class HoodieAppendHandle extends HoodieIOH
}
numberOfRecords++;
}
- doAppend(metadata);
+ doAppend(header);
}
- private void doAppend(Map metadata) {
+ private void doAppend(Map header) {
try {
if (recordList.size() > 0) {
- writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, schema, metadata));
+ writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, header));
recordList.clear();
}
if (keysToDelete.size() > 0) {
writer = writer.appendBlock(
- new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), metadata));
+ new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), header));
keysToDelete.clear();
}
} catch (Exception e) {
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java
index 6e1f14ab7..d0454e3d2 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java
@@ -18,6 +18,7 @@ package com.uber.hoodie.io;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry;
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
@@ -30,6 +31,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
+import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.AvroUtils;
@@ -39,6 +41,7 @@ import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileStream;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
@@ -47,6 +50,7 @@ import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
+import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -190,7 +194,9 @@ public class HoodieCommitArchiveLog {
for (HoodieInstant hoodieInstant : instants) {
records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
}
- HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, wrapperSchema);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString());
+ HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header);
this.writer = writer.appendBlock(block);
} catch (Exception e) {
throw new HoodieCommitException("Failed to archive commits", e);
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java
index c72406d12..709c0e9fe 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java
@@ -154,7 +154,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs,
metaClient.getBasePath(),
- operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemorySizePerCompactionInBytes());
+ operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemorySizePerCompactionInBytes(),
+ config.getCompactionLazyBlockReadEnabled(), config.getCompactionReverseLogReadEnabled());
if (!scanner.iterator().hasNext()) {
return Lists.newArrayList();
}
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java
index 88d7b8d8a..9cca2ea90 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java
@@ -265,14 +265,15 @@ public class HoodieMergeOnReadTable extends
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
Long numRollbackBlocks = 0L;
// generate metadata
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME,
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME,
metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
- metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit);
+ header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, commit);
+ header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
+ String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
// if update belongs to an existing log file
writer = writer.appendBlock(new HoodieCommandBlock(
- HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK,
- metadata));
+ header));
numRollbackBlocks++;
filesToNumBlocksRollback
.put(this.getMetaClient().getFs().getFileStatus(writer.getLogFile().getPath()),
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java
index 3add748c9..1e5d015d5 100644
--- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java
+++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java
@@ -122,7 +122,7 @@ public class TestHoodieCommitArchiveLog {
//read the file
HoodieLogFormat.Reader reader = HoodieLogFormat
.newReader(fs, new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")),
- HoodieArchivedMetaEntry.getClassSchema(), false);
+ HoodieArchivedMetaEntry.getClassSchema());
int archivedRecordsCount = 0;
List readRecords = new ArrayList<>();
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java
index c4a527914..0c587c671 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java
@@ -101,6 +101,19 @@ public class HoodieLogFile implements Serializable {
};
}
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ HoodieLogFile that = (HoodieLogFile) o;
+ return path != null ? path.equals(that.path) : that.path == null;
+ }
+
+ @Override
+ public int hashCode() {
+ return path != null ? path.hashCode() : 0;
+ }
+
@Override
public String toString() {
return "HoodieLogFile {" + path + '}';
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java
index 465b53867..98dd29b78 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java
@@ -30,6 +30,7 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.SpillableMapUtils;
import com.uber.hoodie.common.util.collection.ExternalSpillableMap;
import com.uber.hoodie.exception.HoodieIOException;
+import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -48,14 +49,25 @@ import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
+import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME;
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK;
-import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.LogMetadataType.INSTANT_TIME;
/**
* Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of
* records which will be used as a lookup table when merging the base columnar file with the redo
* log file.
+ * NOTE: If readBlockLazily is turned on, does not merge, instead keeps reading log blocks and merges everything at once
+ * This is an optimization to avoid seek() back and forth to read new block (forward seek())
+ * and lazily read content of seen block (reverse and forward seek()) during merge
+ * | | Read Block 1 Metadata | | Read Block 1 Data |
+ * | | Read Block 2 Metadata | | Read Block 2 Data |
+ * | I/O Pass 1 | ..................... | I/O Pass 2 | ................. |
+ * | | Read Block N Metadata | | Read Block N Data |
+ *
+ * This results in two I/O passes over the log file.
+ *
*/
+
public class HoodieCompactedLogRecordScanner implements
Iterable> {
@@ -77,10 +89,11 @@ public class HoodieCompactedLogRecordScanner implements
// Merge strategy to use when combining records from log
private String payloadClassFQN;
// Store the last instant log blocks (needed to implement rollback)
- Deque currentInstantLogBlocks = new ArrayDeque<>();
+ private Deque currentInstantLogBlocks = new ArrayDeque<>();
public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths,
- Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes) {
+ Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes,
+ boolean readBlocksLazily, boolean reverseReader) {
this.readerSchema = readerSchema;
this.latestInstantTime = latestInstantTime;
this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath);
@@ -88,137 +101,141 @@ public class HoodieCompactedLogRecordScanner implements
this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass();
try {
- // Store merged records for all versions for this log file, set the maxInMemoryMapSize to half,
- // assign other half to the temporary map needed to read next block
- records = new ExternalSpillableMap<>(maxMemorySizeInBytes, readerSchema,
+ // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize
+ this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, readerSchema,
payloadClassFQN, Optional.empty());
// iterate over the paths
- Iterator logFilePathsItr = logFilePaths.iterator();
- while (logFilePathsItr.hasNext()) {
- HoodieLogFile logFile = new HoodieLogFile(new Path(logFilePathsItr.next()));
- log.info("Scanning log file " + logFile.getPath());
+ HoodieLogFormatReader logFormatReaderWrapper =
+ new HoodieLogFormatReader(fs,
+ logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile)))
+ .collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader);
+ while (logFormatReaderWrapper.hasNext()) {
+ HoodieLogFile logFile = logFormatReaderWrapper.getLogFile();
+ log.info("Scanning log file " + logFile);
totalLogFiles.incrementAndGet();
- // Use the HoodieLogFormatReader to iterate through the blocks in the log file
- HoodieLogFormatReader reader = new HoodieLogFormatReader(fs, logFile, readerSchema, true);
- while (reader.hasNext()) {
- HoodieLogBlock r = reader.next();
- if (r.getBlockType() != CORRUPT_BLOCK &&
- !HoodieTimeline.compareTimestamps(r.getLogMetadata().get(INSTANT_TIME), this.latestInstantTime,
- HoodieTimeline.LESSER_OR_EQUAL)) {
- //hit a block with instant time greater than should be processed, stop processing further
- break;
- }
- switch (r.getBlockType()) {
- case AVRO_DATA_BLOCK:
- log.info("Reading a data block from file " + logFile.getPath());
- // Consider the following scenario
- // (Time 0, C1, Task T1) -> Running
- // (Time 1, C1, Task T1) -> Failed (Wrote either a corrupt block or a correct DataBlock (B1) with commitTime C1
- // (Time 2, C1, Task T1.2) -> Running (Task T1 was retried and the attempt number is 2)
- // (Time 3, C1, Task T1.2) -> Finished (Wrote a correct DataBlock B2)
- // Now a logFile L1 can have 2 correct Datablocks (B1 and B2) which are the same.
- // Say, commit C1 eventually failed and a rollback is triggered.
- // Rollback will write only 1 rollback block (R1) since it assumes one block is written per ingestion batch for a file,
- // but in reality we need to rollback (B1 & B2)
- // The following code ensures the same rollback block (R1) is used to rollback both B1 & B2
- if(isNewInstantBlock(r)) {
- // If this is a avro data block, then merge the last block records into the main result
- merge(records, currentInstantLogBlocks);
- }
- // store the current block
- currentInstantLogBlocks.push(r);
- break;
- case DELETE_BLOCK:
- log.info("Reading a delete block from file " + logFile.getPath());
- if (isNewInstantBlock(r)) {
- // Block with the keys listed as to be deleted, data and delete blocks written in different batches
- // so it is safe to merge
- // This is a delete block, so lets merge any records from previous data block
- merge(records, currentInstantLogBlocks);
- }
- // store deletes so can be rolled back
- currentInstantLogBlocks.push(r);
- break;
- case COMMAND_BLOCK:
- log.info("Reading a command block from file " + logFile.getPath());
- // This is a command block - take appropriate action based on the command
- HoodieCommandBlock commandBlock = (HoodieCommandBlock) r;
- String targetInstantForCommandBlock = r.getLogMetadata()
- .get(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME);
- switch (commandBlock.getType()) { // there can be different types of command blocks
- case ROLLBACK_PREVIOUS_BLOCK:
- // Rollback the last read log block
- // Get commit time from last record block, compare with targetCommitTime, rollback only if equal,
- // this is required in scenarios of invalid/extra rollback blocks written due to failures during
- // the rollback operation itself and ensures the same rollback block (R1) is used to rollback
- // both B1 & B2 with same instant_time
- int numBlocksRolledBack = 0;
- while(!currentInstantLogBlocks.isEmpty()) {
- HoodieLogBlock lastBlock = currentInstantLogBlocks.peek();
- // handle corrupt blocks separately since they may not have metadata
- if (lastBlock.getBlockType() == CORRUPT_BLOCK) {
- log.info(
- "Rolling back the last corrupted log block read in " + logFile.getPath());
- currentInstantLogBlocks.pop();
- numBlocksRolledBack++;
- }
- // rollback last data block or delete block
- else if (lastBlock.getBlockType() != CORRUPT_BLOCK &&
- targetInstantForCommandBlock
- .contentEquals(lastBlock.getLogMetadata().get(INSTANT_TIME))) {
- log.info("Rolling back the last log block read in " + logFile.getPath());
- currentInstantLogBlocks.pop();
- numBlocksRolledBack++;
- }
- // invalid or extra rollback block
- else if(!targetInstantForCommandBlock
- .contentEquals(currentInstantLogBlocks.peek().getLogMetadata().get(INSTANT_TIME))) {
- log.warn("Invalid or extra rollback command block in " + logFile.getPath());
- break;
- }
- // this should not happen ideally
- else {
- log.warn("Unable to apply rollback command block in " + logFile.getPath());
- }
- }
- log.info("Number of applied rollback blocks " + numBlocksRolledBack);
- break;
-
- }
- break;
- case CORRUPT_BLOCK:
- log.info("Found a corrupt block in " + logFile.getPath());
- // If there is a corrupt block - we will assume that this was the next data block
- currentInstantLogBlocks.push(r);
- break;
- }
+ // Use the HoodieLogFileReader to iterate through the blocks in the log file
+ HoodieLogBlock r = logFormatReaderWrapper.next();
+ if (r.getBlockType() != CORRUPT_BLOCK &&
+ !HoodieTimeline.compareTimestamps(r.getLogBlockHeader().get(INSTANT_TIME),
+ this.latestInstantTime,
+ HoodieTimeline.LESSER_OR_EQUAL)) {
+ //hit a block with instant time greater than should be processed, stop processing further
+ break;
}
- // merge the last read block when all the blocks are done reading
- if (!currentInstantLogBlocks.isEmpty()) {
- log.info("Merging the final blocks in " + logFile.getPath());
- merge(records, currentInstantLogBlocks);
+ switch (r.getBlockType()) {
+ case AVRO_DATA_BLOCK:
+ log.info("Reading a data block from file " + logFile.getPath());
+ if (isNewInstantBlock(r) && !readBlocksLazily) {
+ // If this is an avro data block belonging to a different commit/instant,
+ // then merge the last blocks and records into the main result
+ merge(records, currentInstantLogBlocks);
+ }
+ // store the current block
+ currentInstantLogBlocks.push(r);
+ break;
+ case DELETE_BLOCK:
+ log.info("Reading a delete block from file " + logFile.getPath());
+ if (isNewInstantBlock(r) && !readBlocksLazily) {
+ // If this is a delete data block belonging to a different commit/instant,
+ // then merge the last blocks and records into the main result
+ merge(records, currentInstantLogBlocks);
+ }
+ // store deletes so can be rolled back
+ currentInstantLogBlocks.push(r);
+ break;
+ case COMMAND_BLOCK:
+ // Consider the following scenario
+ // (Time 0, C1, Task T1) -> Running
+ // (Time 1, C1, Task T1) -> Failed (Wrote either a corrupt block or a correct DataBlock (B1) with commitTime C1
+ // (Time 2, C1, Task T1.2) -> Running (Task T1 was retried and the attempt number is 2)
+ // (Time 3, C1, Task T1.2) -> Finished (Wrote a correct DataBlock B2)
+ // Now a logFile L1 can have 2 correct Datablocks (B1 and B2) which are the same.
+ // Say, commit C1 eventually failed and a rollback is triggered.
+ // Rollback will write only 1 rollback block (R1) since it assumes one block is written per ingestion batch for a file,
+ // but in reality we need to rollback (B1 & B2)
+ // The following code ensures the same rollback block (R1) is used to rollback both B1 & B2
+ log.info("Reading a command block from file " + logFile.getPath());
+ // This is a command block - take appropriate action based on the command
+ HoodieCommandBlock commandBlock = (HoodieCommandBlock) r;
+ String targetInstantForCommandBlock = r.getLogBlockHeader()
+ .get(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME);
+ switch (commandBlock.getType()) { // there can be different types of command blocks
+ case ROLLBACK_PREVIOUS_BLOCK:
+ // Rollback the last read log block
+ // Get commit time from last record block, compare with targetCommitTime, rollback only if equal,
+ // this is required in scenarios of invalid/extra rollback blocks written due to failures during
+ // the rollback operation itself and ensures the same rollback block (R1) is used to rollback
+ // both B1 & B2 with same instant_time
+ int numBlocksRolledBack = 0;
+ while (!currentInstantLogBlocks.isEmpty()) {
+ HoodieLogBlock lastBlock = currentInstantLogBlocks.peek();
+ // handle corrupt blocks separately since they may not have metadata
+ if (lastBlock.getBlockType() == CORRUPT_BLOCK) {
+ log.info(
+ "Rolling back the last corrupted log block read in " + logFile.getPath());
+ currentInstantLogBlocks.pop();
+ numBlocksRolledBack++;
+ }
+ // rollback last data block or delete block
+ else if (lastBlock.getBlockType() != CORRUPT_BLOCK &&
+ targetInstantForCommandBlock
+ .contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) {
+ log.info("Rolling back the last log block read in " + logFile.getPath());
+ currentInstantLogBlocks.pop();
+ numBlocksRolledBack++;
+ }
+ // invalid or extra rollback block
+ else if (!targetInstantForCommandBlock
+ .contentEquals(
+ currentInstantLogBlocks.peek().getLogBlockHeader().get(INSTANT_TIME))) {
+ log.warn("TargetInstantTime " + targetInstantForCommandBlock +
+ " invalid or extra rollback command block in " + logFile.getPath());
+ break;
+ }
+ // this should not happen ideally
+ else {
+ log.warn("Unable to apply rollback command block in " + logFile.getPath());
+ }
+ }
+ log.info("Number of applied rollback blocks " + numBlocksRolledBack);
+ break;
+
+ }
+ break;
+ case CORRUPT_BLOCK:
+ log.info("Found a corrupt block in " + logFile.getPath());
+ // If there is a corrupt block - we will assume that this was the next data block
+ currentInstantLogBlocks.push(r);
+ break;
}
}
+ // merge the last read block when all the blocks are done reading
+ if (!currentInstantLogBlocks.isEmpty()) {
+ log.info("Merging the final data blocks");
+ merge(records, currentInstantLogBlocks);
+ }
} catch (IOException e) {
- throw new HoodieIOException("IOException when reading compacting log files");
+ throw new HoodieIOException("IOException when reading log file ");
}
this.totalRecordsToUpdate = records.size();
log.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes);
- log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries());
- log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize());
- log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries());
+ log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records
+ .getInMemoryMapNumEntries());
+ log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records
+ .getCurrentInMemoryMapSize());
+ log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records
+ .getDiskBasedMapNumEntries());
log.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes());
}
/**
* Checks if the current logblock belongs to a later instant
- * @param logBlock
- * @return
*/
private boolean isNewInstantBlock(HoodieLogBlock logBlock) {
- return currentInstantLogBlocks.size() > 0 && currentInstantLogBlocks.peek().getBlockType() != CORRUPT_BLOCK
- && !logBlock.getLogMetadata().get(INSTANT_TIME)
- .contentEquals(currentInstantLogBlocks.peek().getLogMetadata().get(INSTANT_TIME));
+ return currentInstantLogBlocks.size() > 0
+ && currentInstantLogBlocks.peek().getBlockType() != CORRUPT_BLOCK
+ && !logBlock.getLogBlockHeader().get(INSTANT_TIME)
+ .contentEquals(currentInstantLogBlocks.peek().getLogBlockHeader().get(INSTANT_TIME));
}
/**
@@ -228,7 +245,10 @@ public class HoodieCompactedLogRecordScanner implements
*/
private Map> loadRecordsFromBlock(
HoodieAvroDataBlock dataBlock) throws IOException {
- Map> recordsFromLastBlock = Maps.newHashMap();
+ // TODO (NA) - Instead of creating a new HashMap use the spillable map
+ Map> recordsFromLastBlock = Maps
+ .newHashMap();
+ // TODO (NA) - Implemnt getRecordItr() in HoodieAvroDataBlock and use that here
List recs = dataBlock.getRecords();
totalLogRecords.addAndGet(recs.size());
recs.forEach(rec -> {
@@ -255,7 +275,7 @@ public class HoodieCompactedLogRecordScanner implements
* Merge the last seen log blocks with the accumulated records
*/
private void merge(Map> records,
- Deque lastBlocks) throws IOException {
+ Deque lastBlocks) throws IOException {
while (!lastBlocks.isEmpty()) {
// poll the element at the bottom of the stack since that's the order it was inserted
HoodieLogBlock lastBlock = lastBlocks.pollLast();
@@ -265,6 +285,7 @@ public class HoodieCompactedLogRecordScanner implements
break;
case DELETE_BLOCK:
// TODO : If delete is the only block written and/or records are present in parquet file
+ // TODO : Mark as tombstone (optional.empty()) for data instead of deleting the entry
Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(records::remove);
break;
case CORRUPT_BLOCK:
@@ -278,7 +299,7 @@ public class HoodieCompactedLogRecordScanner implements
* Merge the records read from a single data block with the accumulated records
*/
private void merge(Map> records,
- Map> recordsFromLastBlock) {
+ Map> recordsFromLastBlock) {
recordsFromLastBlock.forEach((key, hoodieRecord) -> {
if (records.containsKey(key)) {
// Merge and store the merged record
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java
new file mode 100644
index 000000000..f9a01c8c8
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java
@@ -0,0 +1,410 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.table.log;
+
+import com.google.common.base.Preconditions;
+import com.uber.hoodie.common.model.HoodieLogFile;
+import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
+import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
+import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock;
+import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
+import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
+import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
+import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
+import com.uber.hoodie.exception.CorruptedLogFileException;
+import com.uber.hoodie.exception.HoodieIOException;
+import com.uber.hoodie.exception.HoodieNotSupportedException;
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * 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)
+ */
+class HoodieLogFileReader implements HoodieLogFormat.Reader {
+
+ private static final int DEFAULT_BUFFER_SIZE = 4096;
+ private final static Logger log = LogManager.getLogger(HoodieLogFileReader.class);
+
+ private final FSDataInputStream inputStream;
+ private final HoodieLogFile logFile;
+ private static final byte[] oldMagicBuffer = new byte[4];
+ private static final byte[] magicBuffer = new byte[6];
+ private final Schema readerSchema;
+ private HoodieLogBlock nextBlock = null;
+ private LogFormatVersion nextBlockVersion;
+ private boolean readBlockLazily;
+ private long reverseLogFilePosition;
+ private long lastReverseLogFilePosition;
+ private boolean reverseReader;
+
+ HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
+ boolean readBlockLazily, boolean reverseReader) throws IOException {
+ this.inputStream = fs.open(logFile.getPath(), bufferSize);
+ this.logFile = logFile;
+ this.readerSchema = readerSchema;
+ this.readBlockLazily = readBlockLazily;
+ this.reverseReader = reverseReader;
+ if(this.reverseReader) {
+ this.reverseLogFilePosition = this.lastReverseLogFilePosition = fs.getFileStatus(logFile.getPath()).getLen();
+ }
+ addShutDownHook();
+ }
+
+ HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema,
+ boolean readBlockLazily, boolean reverseReader) throws IOException {
+ this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, readBlockLazily, reverseReader);
+ }
+
+ HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) throws IOException {
+ this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, false, false);
+ }
+
+ @Override
+ public HoodieLogFile getLogFile() {
+ return logFile;
+ }
+
+ /**
+ * Close the inputstream when the JVM exits
+ */
+ private void addShutDownHook() {
+ Runtime.getRuntime().addShutdownHook(new Thread() {
+ public void run() {
+ try {
+ inputStream.close();
+ } catch (Exception e) {
+ log.warn("unable to close input stream for log file " + logFile, e);
+ // fail silently for any sort of exception
+ }
+ }
+ });
+ }
+
+ // TODO : convert content and block length to long by using ByteBuffer, raw byte [] allows for max of Integer size
+ private HoodieLogBlock readBlock() throws IOException {
+
+ int blocksize = -1;
+ int type = -1;
+ HoodieLogBlockType blockType = null;
+ Map header = null;
+
+ try {
+
+ if (isOldMagic()) {
+ // 1 Read the block type for a log block
+ type = inputStream.readInt();
+
+ Preconditions.checkArgument(type < HoodieLogBlockType.values().length,
+ "Invalid block byte type found " + type);
+ blockType = HoodieLogBlockType.values()[type];
+
+ // 2 Read the total size of the block
+ blocksize = inputStream.readInt();
+ } else {
+ // 1 Read the total size of the block
+ blocksize = (int) inputStream.readLong();
+ }
+
+ } catch (Exception e) {
+ // An exception reading any of the above indicates a corrupt block
+ // Create a corrupt block by finding the next OLD_MAGIC marker or EOF
+ return createCorruptBlock();
+ }
+
+ // 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 block) or EOF
+ // If we did not find either of it, then this block is a corrupted block.
+ boolean isCorrupted = isBlockCorrupt(blocksize);
+ if (isCorrupted) {
+ return createCorruptBlock();
+ }
+
+ // 2. Read the version for this log format
+ this.nextBlockVersion = readVersion();
+
+ // 3. Read the block type for a log block
+ if (nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION) {
+ type = inputStream.readInt();
+
+ Preconditions.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
+ if (nextBlockVersion.hasHeader()) {
+ header = HoodieLogBlock.getLogMetadata(inputStream);
+ }
+
+ int contentLength = blocksize;
+ // 5. Read the content length for the content
+ if (nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION) {
+ contentLength = (int) inputStream.readLong();
+ }
+
+ // 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();
+ byte[] content = HoodieLogBlock.readOrSkipContent(inputStream, contentLength, readBlockLazily);
+
+ // 7. Read footer if any
+ Map footer = null;
+ if (nextBlockVersion.hasFooter()) {
+ footer = HoodieLogBlock.getLogMetadata(inputStream);
+ }
+
+ // 8. Read log block length, if present. This acts as a reverse pointer when traversing a log file in reverse
+ long logBlockLength = 0;
+ if (nextBlockVersion.hasLogBlockLength()) {
+ logBlockLength = inputStream.readLong();
+ }
+
+ // 9. Read the log block end position in the log file
+ long blockEndPos = inputStream.getPos();
+
+ switch (blockType) {
+ // based on type read the block
+ case AVRO_DATA_BLOCK:
+ if (nextBlockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) {
+ return HoodieAvroDataBlock.getBlock(content, readerSchema);
+ } else {
+ return HoodieAvroDataBlock.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily,
+ contentPosition, contentLength, blockEndPos, readerSchema, header, footer);
+ }
+ case DELETE_BLOCK:
+ return HoodieDeleteBlock.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily,
+ contentPosition, contentLength, blockEndPos, header, footer);
+ case COMMAND_BLOCK:
+ return HoodieCommandBlock.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily,
+ contentPosition, contentLength, blockEndPos, header, footer);
+ default:
+ throw new HoodieNotSupportedException("Unsupported Block " + blockType);
+ }
+ }
+
+ private HoodieLogBlock createCorruptBlock() throws IOException {
+ log.info("Log " + logFile + " has a corrupted block at " + inputStream.getPos());
+ long currentPos = inputStream.getPos();
+ long nextBlockOffset = scanForNextAvailableBlockOffset();
+ // Rewind to the initial start and read corrupted bytes till the nextBlockOffset
+ inputStream.seek(currentPos);
+ log.info("Next available block in " + logFile + " starts at " + nextBlockOffset);
+ int corruptedBlockSize = (int) (nextBlockOffset - currentPos);
+ long contentPosition = inputStream.getPos();
+ byte[] corruptedBytes = HoodieLogBlock.readOrSkipContent(inputStream, corruptedBlockSize, readBlockLazily);
+ return HoodieCorruptBlock.getBlock(logFile, inputStream, Optional.ofNullable(corruptedBytes), readBlockLazily,
+ contentPosition, corruptedBlockSize, corruptedBlockSize, new HashMap<>(), new HashMap<>());
+ }
+
+ private boolean isBlockCorrupt(int blocksize) throws IOException {
+ long currentPos = inputStream.getPos();
+ try {
+ inputStream.seek(currentPos + blocksize);
+ } catch (EOFException e) {
+ // this is corrupt
+ return true;
+ }
+
+ try {
+ readMagic();
+ // all good - either we found the sync marker or EOF. Reset position and continue
+ return false;
+ } catch (CorruptedLogFileException e) {
+ // This is a corrupted block
+ return true;
+ } finally {
+ inputStream.seek(currentPos);
+ }
+ }
+
+ private long scanForNextAvailableBlockOffset() throws IOException {
+ while (true) {
+ long currentPos = inputStream.getPos();
+ try {
+ boolean isEOF = readMagic();
+ return isEOF ? inputStream.getPos() : currentPos;
+ } catch (CorruptedLogFileException e) {
+ // No luck - advance and try again
+ inputStream.seek(currentPos + 1);
+ }
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ this.inputStream.close();
+ }
+
+ @Override
+ /**
+ * hasNext is not idempotent. TODO - Fix this. It is okay for now - PR
+ */
+ public boolean hasNext() {
+ try {
+ boolean isEOF = readMagic();
+ if (isEOF) {
+ return false;
+ }
+ this.nextBlock = readBlock();
+ return nextBlock != null;
+ } catch (IOException e) {
+ throw new HoodieIOException("IOException when reading logfile " + logFile, e);
+ }
+ }
+
+ /**
+ * Read log format version from log file, if present
+ * For old log files written with Magic header OLD_MAGIC and without version, return DEFAULT_VERSION
+ *
+ * @throws IOException
+ */
+ private LogFormatVersion readVersion() throws IOException {
+ // If not old log file format (written with Magic header OLD_MAGIC), then read log version
+ if (Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC)) {
+ Arrays.fill(oldMagicBuffer, (byte) 0);
+ return new HoodieLogFormatVersion(HoodieLogFormatVersion.DEFAULT_VERSION);
+ }
+ return new HoodieLogFormatVersion(inputStream.readInt());
+ }
+
+ private boolean isOldMagic() {
+ return Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC);
+ }
+
+
+ private boolean readMagic() throws IOException {
+ try {
+ long pos = inputStream.getPos();
+ // 1. Read magic header from the start of the block
+ inputStream.readFully(magicBuffer, 0, 6);
+ if (!Arrays.equals(magicBuffer, HoodieLogFormat.MAGIC)) {
+ inputStream.seek(pos);
+ // 1. Read old magic header from the start of the block
+ // (for backwards compatibility of older log files written without log version)
+ inputStream.readFully(oldMagicBuffer, 0, 4);
+ if (!Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC)) {
+ throw new CorruptedLogFileException(
+ logFile + "could not be read. Did not find the magic bytes at the start of the block");
+ }
+ }
+ return false;
+ } catch (EOFException e) {
+ // We have reached the EOF
+ return true;
+ }
+ }
+
+ @Override
+ public HoodieLogBlock next() {
+ if (nextBlock == null) {
+ // may be hasNext is not called
+ hasNext();
+ }
+ return nextBlock;
+ }
+
+ /**
+ * hasPrev is not idempotent
+ *
+ * @return
+ */
+ public boolean hasPrev() {
+ try {
+ if(!this.reverseReader) {
+ throw new HoodieNotSupportedException("Reverse log reader has not been enabled");
+ }
+ reverseLogFilePosition = lastReverseLogFilePosition;
+ reverseLogFilePosition -= Long.BYTES;
+ lastReverseLogFilePosition = reverseLogFilePosition;
+ inputStream.seek(reverseLogFilePosition);
+ } catch (Exception e) {
+ // Either reached EOF while reading backwards or an exception
+ return false;
+ }
+ return true;
+ }
+
+ /**
+ * This is a reverse iterator
+ * Note: At any point, an instance of HoodieLogFileReader should either iterate reverse (prev)
+ * or forward (next). Doing both in the same instance is not supported
+ * WARNING : Every call to prev() should be preceded with hasPrev()
+ *
+ * @return
+ * @throws IOException
+ */
+ public HoodieLogBlock prev() throws IOException {
+
+ if(!this.reverseReader) {
+ throw new HoodieNotSupportedException("Reverse log reader has not been enabled");
+ }
+ long blockSize = inputStream.readLong();
+ long blockEndPos = inputStream.getPos();
+ // blocksize should read everything about a block including the length as well
+ try {
+ inputStream.seek(reverseLogFilePosition - blockSize);
+ } catch (Exception e) {
+ // this could be a corrupt block
+ inputStream.seek(blockEndPos);
+ throw new CorruptedLogFileException("Found possible corrupted block, cannot read log file in reverse, " +
+ "fallback to forward reading of logfile");
+ }
+ boolean hasNext = hasNext();
+ reverseLogFilePosition -= blockSize;
+ lastReverseLogFilePosition = reverseLogFilePosition;
+ return this.nextBlock;
+ }
+
+ /**
+ * Reverse pointer, does not read the block. Return the current position of the log file (in reverse)
+ * If the pointer (inputstream) is moved in any way, it is the job of the client of this class to
+ * seek/reset it back to the file position returned from the method to expect correct results
+ *
+ * @return
+ * @throws IOException
+ */
+ public long moveToPrev() throws IOException {
+
+ if(!this.reverseReader) {
+ throw new HoodieNotSupportedException("Reverse log reader has not been enabled");
+ }
+ inputStream.seek(lastReverseLogFilePosition);
+ long blockSize = inputStream.readLong();
+ // blocksize should be everything about a block including the length as well
+ inputStream.seek(reverseLogFilePosition - blockSize);
+ reverseLogFilePosition -= blockSize;
+ lastReverseLogFilePosition = reverseLogFilePosition;
+ return reverseLogFilePosition;
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException("Remove not supported for HoodieLogFileReader");
+ }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java
index 58734b165..d5deb9d03 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java
@@ -19,17 +19,18 @@ package com.uber.hoodie.common.table.log;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.FSUtils;
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Iterator;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Iterator;
+
/**
- * File Format for Hoodie Log Files. The File Format consists of blocks each seperated with a MAGIC
+ * File Format for Hoodie Log Files. The File Format consists of blocks each seperated with a OLD_MAGIC
* sync marker. A Block can either be a Data block, Command block or Delete Block. Data Block -
* Contains log records serialized as Avro Binary Format Command Block - Specific commands like
* RoLLBACK_PREVIOUS-BLOCK - Tombstone for the previously written block Delete Block - List of keys
@@ -42,7 +43,21 @@ public interface HoodieLogFormat {
* this file specific (generate a random 4 byte magic and stick it in the file header), but this I
* think is suffice for now - PR
*/
- byte[] MAGIC = new byte[]{'H', 'U', 'D', 'I'};
+ byte[] OLD_MAGIC = new byte[]{'H', 'U', 'D', 'I'};
+
+ /**
+ * Magic 6 bytes we put at the start of every block in the log file.
+ * This is added to maintain backwards compatiblity due to lack of log format/block
+ * version in older log files. All new log block will now write this OLD_MAGIC value
+ */
+ byte[] MAGIC = new byte[]{'#', 'H', 'U', 'D', 'I', '#'};
+
+ /**
+ * The current version of the log format. Anytime the log format changes
+ * this version needs to be bumped and corresponding changes need to be made to
+ * {@link HoodieLogFormatVersion}
+ */
+ int currentVersion = 1;
/**
* Writer interface to allow appending block to this file format
@@ -196,9 +211,8 @@ public interface HoodieLogFormat {
return new WriterBuilder();
}
- static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema,
- boolean readMetadata)
+ static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema)
throws IOException {
- return new HoodieLogFormatReader(fs, logFile, readerSchema, readMetadata);
+ return new HoodieLogFileReader(fs, logFile, readerSchema, false, false);
}
}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java
index 4168e27cf..f62c9f0e5 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java
@@ -16,192 +16,85 @@
package com.uber.hoodie.common.table.log;
-import com.google.common.base.Preconditions;
import com.uber.hoodie.common.model.HoodieLogFile;
-import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
-import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
-import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock;
-import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
-import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
-import com.uber.hoodie.exception.CorruptedLogFileException;
import com.uber.hoodie.exception.HoodieIOException;
-import com.uber.hoodie.exception.HoodieNotSupportedException;
-import java.io.EOFException;
-import java.io.IOException;
-import java.util.Arrays;
import org.apache.avro.Schema;
-import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-/**
- * 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)
- */
+import java.io.IOException;
+import java.util.List;
+
public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
- private static final int DEFAULT_BUFFER_SIZE = 4096;
- private final static Logger log = LogManager.getLogger(HoodieLogFormatReader.class);
-
- private final FSDataInputStream inputStream;
- private final HoodieLogFile logFile;
- private static final byte[] magicBuffer = new byte[4];
+ private final List logFiles;
+ private HoodieLogFileReader currentReader;
+ private final FileSystem fs;
private final Schema readerSchema;
- private HoodieLogBlock nextBlock = null;
- private boolean readMetadata = true;
+ private final boolean readBlocksLazily;
+ private final boolean reverseLogReader;
- HoodieLogFormatReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
- boolean readMetadata) throws IOException {
- this.inputStream = fs.open(logFile.getPath(), bufferSize);
- this.logFile = logFile;
+ HoodieLogFormatReader(FileSystem fs, List logFiles,
+ Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader) throws IOException {
+ this.logFiles = logFiles;
+ this.fs = fs;
this.readerSchema = readerSchema;
- this.readMetadata = readMetadata;
- }
-
- HoodieLogFormatReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema,
- boolean readMetadata) throws IOException {
- this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, readMetadata);
- }
-
- @Override
- public HoodieLogFile getLogFile() {
- return logFile;
- }
-
- private HoodieLogBlock readBlock() throws IOException {
- // 2. Read the block type
- int ordinal = inputStream.readInt();
- Preconditions.checkArgument(ordinal < HoodieLogBlockType.values().length,
- "Invalid block byte ordinal found " + ordinal);
- HoodieLogBlockType blockType = HoodieLogBlockType.values()[ordinal];
-
- // 3. Read the size of the block
- int blocksize = inputStream.readInt();
-
- // 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 block) or EOF
- // If we did not find either of it, then this block is a corrupted block.
- boolean isCorrupted = isBlockCorrupt(blocksize);
- if (isCorrupted) {
- return createCorruptBlock();
- }
-
- // 4. Read the content
- // TODO - have a max block size and reuse this buffer in the ByteBuffer (hard to guess max block size for now)
- byte[] content = new byte[blocksize];
- inputStream.readFully(content, 0, blocksize);
-
- switch (blockType) {
- // based on type read the block
- case AVRO_DATA_BLOCK:
- return HoodieAvroDataBlock.fromBytes(content, readerSchema, readMetadata);
- case DELETE_BLOCK:
- return HoodieDeleteBlock.fromBytes(content, readMetadata);
- case COMMAND_BLOCK:
- return HoodieCommandBlock.fromBytes(content, readMetadata);
- default:
- throw new HoodieNotSupportedException("Unsupported Block " + blockType);
+ this.readBlocksLazily = readBlocksLazily;
+ this.reverseLogReader = reverseLogReader;
+ if(logFiles.size() > 0) {
+ HoodieLogFile nextLogFile = logFiles.remove(0);
+ this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, readBlocksLazily,
+ false);
}
}
- private HoodieLogBlock createCorruptBlock() throws IOException {
- log.info("Log " + logFile + " has a corrupted block at " + inputStream.getPos());
- long currentPos = inputStream.getPos();
- long nextBlockOffset = scanForNextAvailableBlockOffset();
- // Rewind to the initial start and read corrupted bytes till the nextBlockOffset
- inputStream.seek(currentPos);
- log.info("Next available block in " + logFile + " starts at " + nextBlockOffset);
- int corruptedBlockSize = (int) (nextBlockOffset - currentPos);
- byte[] content = new byte[corruptedBlockSize];
- inputStream.readFully(content, 0, corruptedBlockSize);
- return HoodieCorruptBlock.fromBytes(content, corruptedBlockSize, true);
- }
-
- private boolean isBlockCorrupt(int blocksize) throws IOException {
- long currentPos = inputStream.getPos();
- try {
- inputStream.seek(currentPos + blocksize);
- } catch (EOFException e) {
- // this is corrupt
- return true;
- }
-
- try {
- readMagic();
- // all good - either we found the sync marker or EOF. Reset position and continue
- return false;
- } catch (CorruptedLogFileException e) {
- // This is a corrupted block
- return true;
- } finally {
- inputStream.seek(currentPos);
- }
- }
-
- private long scanForNextAvailableBlockOffset() throws IOException {
- while (true) {
- long currentPos = inputStream.getPos();
- try {
- boolean isEOF = readMagic();
- return isEOF ? inputStream.getPos() : currentPos;
- } catch (CorruptedLogFileException e) {
- // No luck - advance and try again
- inputStream.seek(currentPos + 1);
- }
- }
+ HoodieLogFormatReader(FileSystem fs, List logFiles,
+ Schema readerSchema) throws IOException {
+ this(fs, logFiles, readerSchema, false, false);
}
@Override
public void close() throws IOException {
- this.inputStream.close();
+ if (currentReader != null) {
+ currentReader.close();
+ }
}
@Override
- /**
- * hasNext is not idempotent. TODO - Fix this. It is okay for now - PR
- */
public boolean hasNext() {
- try {
- boolean isEOF = readMagic();
- if (isEOF) {
- return false;
- }
- this.nextBlock = readBlock();
- return nextBlock != null;
- } catch (IOException e) {
- throw new HoodieIOException("IOException when reading logfile " + logFile, e);
- }
- }
- private boolean readMagic() throws IOException {
- try {
- // 1. Read magic header from the start of the block
- inputStream.readFully(magicBuffer, 0, 4);
- if (!Arrays.equals(magicBuffer, HoodieLogFormat.MAGIC)) {
- throw new CorruptedLogFileException(
- logFile + "could not be read. Did not find the magic bytes at the start of the block");
- }
+ if(currentReader == null) {
return false;
- } catch (EOFException e) {
- // We have reached the EOF
+ }
+ else if (currentReader.hasNext()) {
return true;
}
+ else if (logFiles.size() > 0) {
+ try {
+ HoodieLogFile nextLogFile = logFiles.remove(0);
+ this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, readBlocksLazily,
+ false);
+ } catch (IOException io) {
+ throw new HoodieIOException("unable to initialize read with log file ", io);
+ }
+ return this.currentReader.hasNext();
+ }
+ return false;
}
@Override
public HoodieLogBlock next() {
- if (nextBlock == null) {
- // may be hasNext is not called
- hasNext();
- }
- return nextBlock;
+ HoodieLogBlock block = currentReader.next();
+ return block;
+ }
+
+ @Override
+ public HoodieLogFile getLogFile() {
+ return currentReader.getLogFile();
}
@Override
public void remove() {
- throw new UnsupportedOperationException("Remove not supported for HoodieLogFormatReader");
}
-}
+
+}
\ No newline at end of file
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java
index f32629571..9ea4600a5 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java
@@ -22,7 +22,6 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat.WriterBuilder;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieException;
-import java.io.IOException;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -32,6 +31,8 @@ import org.apache.hadoop.ipc.RemoteException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
+import java.io.IOException;
+
/**
* HoodieLogFormatWriter can be used to append blocks to a log file Use
* HoodieLogFormat.WriterBuilder to construct
@@ -117,16 +118,39 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
@Override
public Writer appendBlock(HoodieLogBlock block)
throws IOException, InterruptedException {
- byte[] content = block.getBytes();
- // 1. write the magic header for the start of the block
- this.output.write(HoodieLogFormat.MAGIC);
- // 2. Write the block type
- this.output.writeInt(block.getBlockType().ordinal());
- // 3. Write the size of the block
- this.output.writeInt(content.length);
- // 4. Write the contents of the block
- this.output.write(content);
+ // Find current version
+ LogFormatVersion currentLogFormatVersion = new HoodieLogFormatVersion(HoodieLogFormat.currentVersion);
+ long currentSize = this.output.size();
+
+ // 1. Write the magic header for the start of the block
+ this.output.write(HoodieLogFormat.MAGIC);
+
+ // bytes for header
+ byte [] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader());
+ // content bytes
+ byte [] content = block.getContentBytes();
+ // bytes for footer
+ byte [] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter());
+
+ // 2. Write the total size of the block (excluding Magic)
+ this.output.writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length));
+
+ // 3. Write the version of this log block
+ this.output.writeInt(currentLogFormatVersion.getVersion());
+ // 4. Write the block type
+ this.output.writeInt(block.getBlockType().ordinal());
+
+ // 5. Write the headers for the log block
+ this.output.write(headerBytes);
+ // 6. Write the size of the content block
+ this.output.writeLong(content.length);
+ // 7. Write the contents of the data block
+ this.output.write(content);
+ // 8. Write the footers for the log block
+ this.output.write(footerBytes);
+ // 9. Write the total size of the log block (including magic) which is everything written until now (for reverse pointer)
+ this.output.writeLong(this.output.size() - currentSize);
// Flush every block to disk
flush();
@@ -134,6 +158,32 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
return rolloverIfNeeded();
}
+ /**
+ *
+ * This method returns the total LogBlock Length which is the sum of
+ * 1. Number of bytes to write version
+ * 2. Number of bytes to write ordinal
+ * 3. Length of the headers
+ * 4. Number of bytes used to write content length
+ * 5. Length of the content
+ * 6. Length of the footers
+ * 7. Number of bytes to write totalLogBlockLength
+ * @param contentLength
+ * @param headerLength
+ * @param footerLength
+ * @return
+ */
+ private int getLogBlockLength(int contentLength, int headerLength, int footerLength) {
+ return
+ Integer.BYTES + // Number of bytes to write version
+ Integer.BYTES + // Number of bytes to write ordinal
+ headerLength + // Length of the headers
+ Long.BYTES + // Number of bytes used to write content length
+ contentLength + // Length of the content
+ footerLength + // Length of the footers
+ Long.BYTES; // Number of bytes to write totalLogBlockLength at end of block (for reverse pointer)
+ }
+
private Writer rolloverIfNeeded() throws IOException, InterruptedException {
// Roll over if the size is past the threshold
if (getCurrentSize() > sizeThreshold) {
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java
new file mode 100644
index 000000000..8bba078c7
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.table.log;
+
+/**
+ * A set of feature flags associated with a log format.
+ * Versions are changed when the log format changes.
+ * TODO(na) - Implement policies around major/minor versions
+ */
+abstract class LogFormatVersion {
+ private final int version;
+
+ LogFormatVersion(int version) {
+ this.version = version;
+ }
+
+ public int getVersion() {
+ return version;
+ }
+
+ public abstract boolean hasMagicHeader();
+
+ public abstract boolean hasContent();
+
+ public abstract boolean hasContentLength();
+
+ public abstract boolean hasOrdinal();
+
+ public abstract boolean hasHeader();
+
+ public abstract boolean hasFooter();
+
+ public abstract boolean hasLogBlockLength();
+}
+
+/**
+ * Implements logic to determine behavior for feature flags for {@link LogFormatVersion}
+ */
+final class HoodieLogFormatVersion extends LogFormatVersion {
+
+ public final static int DEFAULT_VERSION = 0;
+
+ HoodieLogFormatVersion(int version) {
+ super(version);
+ }
+ @Override
+ public boolean hasMagicHeader() {
+ switch (super.getVersion()) {
+ case DEFAULT_VERSION:
+ return true;
+ default:
+ return true;
+ }
+ }
+
+ @Override
+ public boolean hasContent() {
+ switch (super.getVersion()) {
+ case DEFAULT_VERSION:
+ return true;
+ default:
+ return true;
+ }
+ }
+
+ @Override
+ public boolean hasContentLength() {
+ switch (super.getVersion()) {
+ case DEFAULT_VERSION:
+ return true;
+ default:
+ return true;
+ }
+ }
+
+ @Override
+ public boolean hasOrdinal() {
+ switch (super.getVersion()) {
+ case DEFAULT_VERSION:
+ return true;
+ default:
+ return true;
+ }
+ }
+
+ @Override
+ public boolean hasHeader() {
+ switch (super.getVersion()) {
+ case DEFAULT_VERSION:
+ return false;
+ default:
+ return true;
+ }
+ }
+
+ @Override
+ public boolean hasFooter() {
+ switch (super.getVersion()) {
+ case DEFAULT_VERSION:
+ return false;
+ case 1:
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public boolean hasLogBlockLength() {
+ switch (super.getVersion()) {
+ case DEFAULT_VERSION:
+ return false;
+ case 1:
+ return true;
+ }
+ return false;
+ }
+}
\ No newline at end of file
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java
index ebb72bb4b..94e89e793 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java
@@ -16,6 +16,8 @@
package com.uber.hoodie.common.table.log.block;
+import com.google.common.annotations.VisibleForTesting;
+import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.exception.HoodieIOException;
@@ -27,43 +29,134 @@ import org.apache.avro.io.Decoder;
import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.Encoder;
import org.apache.avro.io.EncoderFactory;
+import org.apache.hadoop.fs.FSDataInputStream;
+import javax.annotation.Nonnull;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Optional;
/**
* DataBlock contains a list of records serialized using Avro.
* The Datablock contains
- * 1. Compressed Writer Schema length
- * 2. Compressed Writer Schema content
- * 3. Total number of records in the block
- * 4. Size of a record
- * 5. Actual avro serialized content of the record
+ * 1. Data Block version
+ * 2. Total number of records in the block
+ * 3. Size of a record
+ * 4. Actual avro serialized content of the record
*/
public class HoodieAvroDataBlock extends HoodieLogBlock {
private List records;
private Schema schema;
- public HoodieAvroDataBlock(List records, Schema schema, Map metadata) {
- super(metadata);
+ public HoodieAvroDataBlock(@Nonnull List records,
+ @Nonnull Map header,
+ @Nonnull Map footer) {
+ super(header, footer, Optional.empty(), Optional.empty(), null, false);
this.records = records;
- this.schema = schema;
+ this.schema = Schema.parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
}
- public HoodieAvroDataBlock(List records, Schema schema) {
- this(records, schema, null);
+ public HoodieAvroDataBlock(@Nonnull List records,
+ @Nonnull Map header) {
+ this(records, header, new HashMap<>());
+ }
+
+ private HoodieAvroDataBlock(Optional content, @Nonnull FSDataInputStream inputStream,
+ boolean readBlockLazily, Optional blockContentLocation,
+ Schema readerSchema, @Nonnull Map headers,
+ @Nonnull Map footer) {
+ super(headers, footer, blockContentLocation, content, inputStream, readBlockLazily);
+ this.schema = readerSchema;
+ }
+
+ public static HoodieLogBlock getBlock(HoodieLogFile logFile,
+ FSDataInputStream inputStream,
+ Optional content,
+ boolean readBlockLazily,
+ long position,
+ long blockSize,
+ long blockEndpos,
+ Schema readerSchema,
+ Map header,
+ Map footer) {
+
+ return new HoodieAvroDataBlock(content, inputStream, readBlockLazily,
+ Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)),
+ readerSchema, header, footer);
+
+ }
+
+ @Override
+ public byte[] getContentBytes() throws IOException {
+
+ // In case this method is called before realizing records from content
+ if (getContent().isPresent()) {
+ return getContent().get();
+ } else if (readBlockLazily && !getContent().isPresent() && records == null) {
+ // read block lazily
+ createRecordsFromContentBytes();
+ }
+
+ Schema schema = Schema.parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
+ GenericDatumWriter writer = new GenericDatumWriter<>(schema);
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ DataOutputStream output = new DataOutputStream(baos);
+
+ // 1. Write out the log block version
+ output.writeInt(HoodieLogBlock.version);
+
+ // 2. Write total number of records
+ output.writeInt(records.size());
+
+ // 3. Write the records
+ Iterator itr = records.iterator();
+ while (itr.hasNext()) {
+ IndexedRecord s = itr.next();
+ ByteArrayOutputStream temp = new ByteArrayOutputStream();
+ Encoder encoder = EncoderFactory.get().binaryEncoder(temp, null);
+ try {
+ // Encode the record into bytes
+ writer.write(s, encoder);
+ encoder.flush();
+
+ // Get the size of the bytes
+ int size = temp.toByteArray().length;
+ // Write the record size
+ output.writeInt(size);
+ // Write the content
+ output.write(temp.toByteArray());
+ itr.remove();
+ } catch (IOException e) {
+ throw new HoodieIOException("IOException converting HoodieAvroDataBlock to bytes", e);
+ }
+ }
+ output.close();
+ return baos.toByteArray();
+ }
+
+ @Override
+ public HoodieLogBlockType getBlockType() {
+ return HoodieLogBlockType.AVRO_DATA_BLOCK;
}
- //TODO : (na) lazily create IndexedRecords only when required
public List getRecords() {
+ if (records == null) {
+ try {
+ // in case records are absent, read content lazily and then convert to IndexedRecords
+ createRecordsFromContentBytes();
+ } catch (IOException io) {
+ throw new HoodieIOException("Unable to convert content bytes to records", io);
+ }
+ }
return records;
}
@@ -71,18 +164,114 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
return schema;
}
- @Override
- public byte[] getBytes() throws IOException {
+ //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
+ private void createRecordsFromContentBytes() throws IOException {
+
+ if (readBlockLazily && !getContent().isPresent()) {
+ // read log block contents from disk
+ inflate();
+ }
+
+ SizeAwareDataInputStream dis =
+ new SizeAwareDataInputStream(
+ new DataInputStream(new ByteArrayInputStream(getContent().get())));
+
+ // 1. Read version for this data block
+ int version = dis.readInt();
+ HoodieAvroDataBlockVersion logBlockVersion = new HoodieAvroDataBlockVersion(version);
+
+ // 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;
+ }
+
+ GenericDatumReader reader = new GenericDatumReader<>(writerSchema, schema);
+ // 2. Get the total records
+ int totalRecords = 0;
+ if (logBlockVersion.hasRecordCount()) {
+ totalRecords = dis.readInt();
+ }
+ List records = new ArrayList<>(totalRecords);
+
+ // 3. Read the content
+ for (int i = 0; i < totalRecords; i++) {
+ int recordLength = dis.readInt();
+ Decoder decoder = DecoderFactory.get()
+ .binaryDecoder(getContent().get(), dis.getNumberOfBytesRead(), recordLength, null);
+ IndexedRecord record = reader.read(null, decoder);
+ records.add(record);
+ dis.skipBytes(recordLength);
+ }
+ dis.close();
+ this.records = records;
+ // Free up content to be GC'd, deflate
+ deflate();
+ }
+
+ /*****************************************************DEPRECATED METHODS**********************************************/
+
+ @Deprecated
+ @VisibleForTesting
+ /**
+ * This constructor is retained to provide backwards compatibility to HoodieArchivedLogs
+ * which were written using HoodieLogFormat V1
+ */
+ public HoodieAvroDataBlock(List records, Schema schema) {
+ super(new HashMap<>(), new HashMap<>(), Optional.empty(), Optional.empty(), null, false);
+ this.records = records;
+ this.schema = schema;
+ }
+
+ @Deprecated
+ /**
+ * This method is retained to provide backwards compatibility to HoodieArchivedLogs which were written using HoodieLogFormat V1
+ */
+ public static HoodieLogBlock getBlock(byte[] content, Schema readerSchema) throws IOException {
+
+ SizeAwareDataInputStream dis = new SizeAwareDataInputStream(
+ new DataInputStream(new ByteArrayInputStream(content)));
+
+ // 1. Read the schema written out
+ int schemaLength = dis.readInt();
+ byte[] compressedSchema = new byte[schemaLength];
+ dis.readFully(compressedSchema, 0, schemaLength);
+ Schema writerSchema = new Schema.Parser().parse(HoodieAvroUtils.decompress(compressedSchema));
+
+ if (readerSchema == null) {
+ readerSchema = writerSchema;
+ }
+
+ GenericDatumReader reader = new GenericDatumReader<>(writerSchema, readerSchema);
+ // 2. Get the total records
+ int totalRecords = dis.readInt();
+ List records = new ArrayList<>(totalRecords);
+
+ // 3. Read the content
+ for (int i = 0; i < totalRecords; i++) {
+ int recordLength = dis.readInt();
+ Decoder decoder = DecoderFactory.get()
+ .binaryDecoder(content, dis.getNumberOfBytesRead(), recordLength, null);
+ IndexedRecord record = reader.read(null, decoder);
+ records.add(record);
+ dis.skipBytes(recordLength);
+ }
+ dis.close();
+ return new HoodieAvroDataBlock(records, readerSchema);
+ }
+
+ @Deprecated
+ @VisibleForTesting
+ public byte[] getBytes(Schema schema) throws IOException {
GenericDatumWriter writer = new GenericDatumWriter<>(schema);
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
- // 1. Write out metadata
- if (super.getLogMetadata() != null) {
- output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata()));
- }
-
// 2. Compress and Write schema out
byte[] schemaContent = HoodieAvroUtils.compress(schema.toString());
output.writeInt(schemaContent.length);
@@ -118,45 +307,4 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
return baos.toByteArray();
}
- @Override
- public HoodieLogBlockType getBlockType() {
- return HoodieLogBlockType.AVRO_DATA_BLOCK;
- }
-
- //TODO (na) - Break down content into smaller chunks of byte [] to be GC as they are used
- public static HoodieLogBlock fromBytes(byte[] content, Schema readerSchema, boolean readMetadata) throws IOException {
-
- SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
- Map metadata = null;
- // 1. Read the metadata written out, if applicable
- if (readMetadata) {
- metadata = HoodieLogBlock.getLogMetadata(dis);
- }
- // 1. Read the schema written out
- int schemaLength = dis.readInt();
- byte[] compressedSchema = new byte[schemaLength];
- dis.readFully(compressedSchema, 0, schemaLength);
- Schema writerSchema = new Schema.Parser().parse(HoodieAvroUtils.decompress(compressedSchema));
-
- if (readerSchema == null) {
- readerSchema = writerSchema;
- }
-
- //TODO : (na) lazily create IndexedRecords only when required
- GenericDatumReader reader = new GenericDatumReader<>(writerSchema, readerSchema);
- // 2. Get the total records
- int totalRecords = dis.readInt();
- List records = new ArrayList<>(totalRecords);
-
- // 3. Read the content
- for (int i=0;i metadata) {
- super(metadata);
- this.type = type;
+ public HoodieCommandBlock(Map header) {
+ this(Optional.empty(), null, false, Optional.empty(), header, new HashMap<>());
}
- public HoodieCommandBlock(HoodieCommandBlockTypeEnum type) {
- this(type, null);
- }
-
- @Override
- public byte[] getBytes() throws IOException {
-
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- DataOutputStream output = new DataOutputStream(baos);
- if (super.getLogMetadata() != null) {
- output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata()));
- }
- output.writeInt(type.ordinal());
- output.close();
- return baos.toByteArray();
+ private HoodieCommandBlock(Optional content, FSDataInputStream inputStream,
+ boolean readBlockLazily, Optional blockContentLocation,
+ Map header, Map footer) {
+ super(header, footer, blockContentLocation, content, inputStream, readBlockLazily);
+ this.type = HoodieCommandBlockTypeEnum.values()[Integer
+ .parseInt(header.get(HeaderMetadataType.COMMAND_BLOCK_TYPE))];
}
public HoodieCommandBlockTypeEnum getType() {
@@ -66,13 +53,23 @@ public class HoodieCommandBlock extends HoodieLogBlock {
return HoodieLogBlockType.COMMAND_BLOCK;
}
- public static HoodieLogBlock fromBytes(byte[] content, boolean readMetadata) throws IOException {
- SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
- Map metadata = null;
- if (readMetadata) {
- metadata = HoodieLogBlock.getLogMetadata(dis);
- }
- int ordinal = dis.readInt();
- return new HoodieCommandBlock(HoodieCommandBlockTypeEnum.values()[ordinal], metadata);
+ @Override
+ public byte[] getContentBytes() {
+ return new byte[0];
+ }
+
+ public static HoodieLogBlock getBlock(HoodieLogFile logFile,
+ FSDataInputStream inputStream,
+ Optional content,
+ boolean readBlockLazily,
+ long position,
+ long blockSize,
+ long blockEndpos,
+ Map header,
+ Map footer) {
+
+ return new HoodieCommandBlock(content, inputStream, readBlockLazily,
+ Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)),
+ header, footer);
}
}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java
index 5819c99e1..c75c8ea62 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java
@@ -16,14 +16,12 @@
package com.uber.hoodie.common.table.log.block;
-import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
+import com.uber.hoodie.common.model.HoodieLogFile;
+import org.apache.hadoop.fs.FSDataInputStream;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
import java.io.IOException;
import java.util.Map;
+import java.util.Optional;
/**
* Corrupt block is emitted whenever the scanner finds the length of the block written at the
@@ -31,26 +29,20 @@ import java.util.Map;
*/
public class HoodieCorruptBlock extends HoodieLogBlock {
- private final byte[] corruptedBytes;
-
- private HoodieCorruptBlock(byte[] corruptedBytes, Map metadata) {
- super(metadata);
- this.corruptedBytes = corruptedBytes;
- }
-
- private HoodieCorruptBlock(byte[] corruptedBytes) {
- this(corruptedBytes, null);
+ private HoodieCorruptBlock(Optional corruptedBytes, FSDataInputStream inputStream,
+ boolean readBlockLazily, Optional blockContentLocation,
+ Map header, Map footer) {
+ super(header, footer, blockContentLocation, corruptedBytes, inputStream, readBlockLazily);
}
@Override
- public byte[] getBytes() throws IOException {
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- DataOutputStream output = new DataOutputStream(baos);
- if (super.getLogMetadata() != null) {
- output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata()));
+ public byte[] getContentBytes() throws IOException {
+
+ if (!getContent().isPresent() && readBlockLazily) {
+ // read content from disk
+ inflate();
}
- output.write(corruptedBytes);
- return baos.toByteArray();
+ return getContent().get();
}
@Override
@@ -58,26 +50,17 @@ public class HoodieCorruptBlock extends HoodieLogBlock {
return HoodieLogBlockType.CORRUPT_BLOCK;
}
- public byte[] getCorruptedBytes() {
- return corruptedBytes;
- }
+ public static HoodieLogBlock getBlock(HoodieLogFile logFile,
+ FSDataInputStream inputStream,
+ Optional corruptedBytes,
+ boolean readBlockLazily,
+ long position,
+ long blockSize,
+ long blockEndPos,
+ Map header,
+ Map footer) throws IOException {
- public static HoodieLogBlock fromBytes(byte[] content, int blockSize, boolean readMetadata)
- throws IOException {
- SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
- Map metadata = null;
- int bytesRemaining = blockSize;
- if (readMetadata) {
- try { //attempt to read metadata
- metadata = HoodieLogBlock.getLogMetadata(dis);
- bytesRemaining = blockSize - HoodieLogBlock.getLogMetadataBytes(metadata).length;
- } catch (IOException e) {
- // unable to read metadata, possibly corrupted
- metadata = null;
- }
- }
- byte[] corruptedBytes = new byte[bytesRemaining];
- dis.readFully(corruptedBytes);
- return new HoodieCorruptBlock(corruptedBytes, metadata);
+ return new HoodieCorruptBlock(corruptedBytes, inputStream, readBlockLazily,
+ Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer);
}
}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java
index 3751124fe..4de25b5da 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java
@@ -16,48 +16,82 @@
package com.uber.hoodie.common.table.log.block;
+import com.uber.hoodie.common.model.HoodieLogFile;
+import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
+import com.uber.hoodie.exception.HoodieIOException;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.FSDataInputStream;
+
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.charset.Charset;
+import java.util.HashMap;
import java.util.Map;
-
-import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
-import org.apache.commons.lang3.StringUtils;
+import java.util.Optional;
/**
* Delete block contains a list of keys to be deleted from scanning the blocks so far
*/
public class HoodieDeleteBlock extends HoodieLogBlock {
- private final String[] keysToDelete;
+ private String[] keysToDelete;
- public HoodieDeleteBlock(String[] keysToDelete, Map metadata) {
- super(metadata);
+ public HoodieDeleteBlock(String[] keysToDelete,
+ Map header) {
+ this(Optional.empty(), null, false, Optional.empty(), header, new HashMap<>());
this.keysToDelete = keysToDelete;
}
- public HoodieDeleteBlock(String[] keysToDelete) {
- this(keysToDelete, null);
+
+ private HoodieDeleteBlock(Optional content, FSDataInputStream inputStream,
+ boolean readBlockLazily, Optional blockContentLocation,
+ Map header, Map footer) {
+ super(header, footer, blockContentLocation, content, inputStream, readBlockLazily);
}
@Override
- public byte[] getBytes() throws IOException {
+ public byte[] getContentBytes() throws IOException {
+
+ // In case this method is called before realizing keys from content
+ if (getContent().isPresent()) {
+ return getContent().get();
+ } else if (readBlockLazily && !getContent().isPresent() && keysToDelete == null) {
+ // read block lazily
+ getKeysToDelete();
+ }
+
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
- if (super.getLogMetadata() != null) {
- output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata()));
- }
- byte[] bytesToWrite = StringUtils.join(keysToDelete, ',').getBytes(Charset.forName("utf-8"));
+ byte[] bytesToWrite = StringUtils.join(getKeysToDelete(), ',').getBytes(Charset.forName("utf-8"));
+ output.writeInt(HoodieLogBlock.version);
output.writeInt(bytesToWrite.length);
output.write(bytesToWrite);
return baos.toByteArray();
}
public String[] getKeysToDelete() {
- return keysToDelete;
+ try {
+ if (keysToDelete == null) {
+ if (!getContent().isPresent() && readBlockLazily) {
+ // read content from disk
+ inflate();
+ }
+ SizeAwareDataInputStream dis =
+ new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(getContent().get())));
+ int version = dis.readInt();
+ int dataLength = dis.readInt();
+ byte[] data = new byte[dataLength];
+ dis.readFully(data);
+ this.keysToDelete = new String(data).split(",");
+ deflate();
+ }
+ return keysToDelete;
+ } catch (IOException io) {
+ throw new HoodieIOException("Unable to generate keys to delete from block content", io);
+ }
}
@Override
@@ -65,15 +99,17 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
return HoodieLogBlockType.DELETE_BLOCK;
}
- public static HoodieLogBlock fromBytes(byte[] content, boolean readMetadata) throws IOException {
- SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
- Map metadata = null;
- if (readMetadata) {
- metadata = HoodieLogBlock.getLogMetadata(dis);
- }
- int dataLength = dis.readInt();
- byte[] data = new byte[dataLength];
- dis.readFully(data);
- return new HoodieDeleteBlock(new String(data).split(","), metadata);
+ public static HoodieLogBlock getBlock(HoodieLogFile logFile,
+ FSDataInputStream inputStream,
+ Optional content,
+ boolean readBlockLazily,
+ long position,
+ long blockSize,
+ long blockEndPos,
+ Map header,
+ Map footer) throws IOException {
+
+ return new HoodieDeleteBlock(content, inputStream, readBlockLazily,
+ Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer);
}
}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java
index d21332f2f..e7735b0db 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java
@@ -17,21 +17,66 @@
package com.uber.hoodie.common.table.log.block;
import com.google.common.collect.Maps;
-import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
+import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.exception.HoodieException;
+import com.uber.hoodie.exception.HoodieIOException;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.IOException;
import java.util.Map;
+import java.util.Optional;
+import javax.annotation.Nonnull;
+import org.apache.hadoop.fs.FSDataInputStream;
/**
* Abstract class defining a block in HoodieLogFile
*/
public abstract class HoodieLogBlock {
- public byte[] getBytes() throws IOException {
+ /**
+ * The current version of the log block. Anytime the logBlock format changes
+ * this version needs to be bumped and corresponding changes need to be made to
+ * {@link HoodieLogBlockVersion}
+ * TODO : Change this to a class, something like HoodieLogBlockVersionV1/V2 and implement/override operations there
+ */
+ public static int version = 1;
+ // Header for each log block
+ private final Map logBlockHeader;
+ // Footer for each log block
+ private final Map logBlockFooter;
+ // Location of a log block on disk
+ private final Optional blockContentLocation;
+ // data for a specific block
+ private Optional content;
+ // TODO : change this to just InputStream so this works for any FileSystem
+ // create handlers to return specific type of inputstream based on FS
+ // input stream corresponding to the log file where this logBlock belongs
+ protected FSDataInputStream inputStream;
+ // Toggle flag, whether to read blocks lazily (I/O intensive) or not (Memory intensive)
+ protected boolean readBlockLazily;
+
+ public HoodieLogBlock(@Nonnull Map logBlockHeader,
+ @Nonnull Map logBlockFooter,
+ @Nonnull Optional blockContentLocation,
+ @Nonnull Optional content,
+ FSDataInputStream inputStream,
+ boolean readBlockLazily) {
+ this.logBlockHeader = logBlockHeader;
+ this.logBlockFooter = logBlockFooter;
+ this.blockContentLocation = blockContentLocation;
+ this.content = content;
+ this.inputStream = inputStream;
+ this.readBlockLazily = readBlockLazily;
+ }
+
+ // Return the bytes representation of the data belonging to a LogBlock
+ public byte[] getContentBytes() throws IOException {
+ throw new HoodieException("No implementation was provided");
+ }
+
+ public byte [] getMagic() {
throw new HoodieException("No implementation was provided");
}
@@ -39,8 +84,25 @@ public abstract class HoodieLogBlock {
throw new HoodieException("No implementation was provided");
}
- //log metadata for each log block
- private Map logMetadata;
+ public long getLogBlockLength() {
+ throw new HoodieException("No implementation was provided");
+ }
+
+ public Optional getBlockContentLocation() {
+ return this.blockContentLocation;
+ }
+
+ public Map getLogBlockHeader() {
+ return logBlockHeader;
+ }
+
+ public Map getLogBlockFooter() {
+ return logBlockFooter;
+ }
+
+ public Optional getContent() {
+ return content;
+ }
/**
* Type of the log block WARNING: This enum is serialized as the ordinal. Only add new enums at
@@ -54,32 +116,71 @@ public abstract class HoodieLogBlock {
}
/**
- * Metadata abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal.
+ * Log Metadata headers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal.
* Only add new enums at the end.
*/
- public enum LogMetadataType {
+ public enum HeaderMetadataType {
INSTANT_TIME,
- TARGET_INSTANT_TIME
+ TARGET_INSTANT_TIME,
+ SCHEMA,
+ COMMAND_BLOCK_TYPE
}
- public HoodieLogBlock(Map logMetadata) {
- this.logMetadata = logMetadata;
+ /**
+ * Log Metadata footers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal.
+ * Only add new enums at the end.
+ */
+ public enum FooterMetadataType {
}
- public Map getLogMetadata() {
- return logMetadata;
+ /**
+ * This class is used to store the Location of the Content of a Log Block. It's used when a client chooses for a
+ * IO intensive CompactedScanner, the location helps to lazily read contents from the log file
+ */
+ public static final class HoodieLogBlockContentLocation {
+ // The logFile that contains this block
+ private final HoodieLogFile logFile;
+ // The filePosition in the logFile for the contents of this block
+ private final long contentPositionInLogFile;
+ // The number of bytes / size of the contents of this block
+ private final long blockSize;
+ // The final position where the complete block ends
+ private final long blockEndPos;
+
+ HoodieLogBlockContentLocation(HoodieLogFile logFile, long contentPositionInLogFile, long blockSize, long blockEndPos) {
+ this.logFile = logFile;
+ this.contentPositionInLogFile = contentPositionInLogFile;
+ this.blockSize = blockSize;
+ this.blockEndPos = blockEndPos;
+ }
+
+ public HoodieLogFile getLogFile() {
+ return logFile;
+ }
+
+ public long getContentPositionInLogFile() {
+ return contentPositionInLogFile;
+ }
+
+ public long getBlockSize() {
+ return blockSize;
+ }
+
+ public long getBlockEndPos() {
+ return blockEndPos;
+ }
}
/**
* Convert log metadata to bytes 1. Write size of metadata 2. Write enum ordinal 3. Write actual
* bytes
*/
- public static byte[] getLogMetadataBytes(Map metadata)
+ public static byte[] getLogMetadataBytes(Map metadata)
throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
output.writeInt(metadata.size());
- for (Map.Entry entry : metadata.entrySet()) {
+ for (Map.Entry entry : metadata.entrySet()) {
output.writeInt(entry.getKey().ordinal());
byte[] bytes = entry.getValue().getBytes();
output.writeInt(bytes.length);
@@ -91,10 +192,10 @@ public abstract class HoodieLogBlock {
/**
* Convert bytes to LogMetadata, follow the same order as {@link HoodieLogBlock#getLogMetadataBytes}
*/
- public static Map getLogMetadata(SizeAwareDataInputStream dis)
+ public static Map getLogMetadata(DataInputStream dis)
throws IOException {
- Map metadata = Maps.newHashMap();
+ Map metadata = Maps.newHashMap();
// 1. Read the metadata written out
int metadataCount = dis.readInt();
try {
@@ -103,7 +204,7 @@ public abstract class HoodieLogBlock {
int metadataEntrySize = dis.readInt();
byte[] metadataEntry = new byte[metadataEntrySize];
dis.readFully(metadataEntry, 0, metadataEntrySize);
- metadata.put(LogMetadataType.values()[metadataEntryIndex], new String(metadataEntry));
+ metadata.put(HeaderMetadataType.values()[metadataEntryIndex], new String(metadataEntry));
metadataCount--;
}
return metadata;
@@ -111,4 +212,60 @@ public abstract class HoodieLogBlock {
throw new IOException("Could not read metadata fields ", eof);
}
}
+
+ /**
+ * Read or Skip block content of a log block in the log file. Depends on lazy reading enabled in
+ * {@link com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner}
+ *
+ * @param inputStream
+ * @param contentLength
+ * @param readBlockLazily
+ * @return
+ * @throws IOException
+ */
+ public static byte [] readOrSkipContent(FSDataInputStream inputStream,
+ Integer contentLength, boolean readBlockLazily) throws IOException {
+ byte [] content = null;
+ 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
+ inputStream.seek(inputStream.getPos() + contentLength);
+ }
+ return content;
+ }
+
+ /**
+ * When lazyReading of blocks is turned on, inflate the content of a log block from disk
+ * @throws IOException
+ */
+ protected void inflate() throws IOException {
+
+ try {
+ content = Optional.of(new byte[(int) this.getBlockContentLocation().get().getBlockSize()]);
+ inputStream.seek(this.getBlockContentLocation().get().getContentPositionInLogFile());
+ inputStream.readFully(content.get(), 0, content.get().length);
+ inputStream.seek(this.getBlockContentLocation().get().getBlockEndPos());
+ } catch(IOException e) {
+ try {
+ // TODO : fs.open() and return inputstream again, need to pass FS configuration
+ // because the inputstream might close/timeout for large number of log blocks to be merged
+ inflate();
+ } catch(IOException io) {
+ throw new HoodieIOException("unable to lazily read log block from disk", io);
+ }
+ }
+ }
+
+ /**
+ * After the content bytes is converted into the required DataStructure by a logBlock, deflate the content
+ * to release byte [] and relieve memory pressure when GC kicks in.
+ * NOTE: This still leaves the heap fragmented
+ */
+ protected void deflate() {
+ content = Optional.empty();
+ }
+
}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java
new file mode 100644
index 000000000..1a9844209
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.table.log.block;
+
+import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.version;
+
+abstract class HoodieLogBlockVersion {
+
+ private final int currentVersion;
+
+ public final static int DEFAULT_VERSION = 0;
+
+ HoodieLogBlockVersion(int version) {
+ this.currentVersion = version;
+ }
+
+ int getVersion() {
+ return currentVersion;
+ }
+}
+
+/**
+ * A set of feature flags associated with a data log block format.
+ * Versions are changed when the log block format changes.
+ * TODO(na) - Implement policies around major/minor versions
+ */
+final class HoodieAvroDataBlockVersion extends HoodieLogBlockVersion {
+
+ HoodieAvroDataBlockVersion(int version) {
+ super(version);
+ }
+
+ public boolean hasRecordCount() {
+ switch (super.getVersion()) {
+ case DEFAULT_VERSION:
+ return true;
+ default:
+ return true;
+ }
+ }
+}
+
+/**
+ * A set of feature flags associated with a command log block format.
+ * Versions are changed when the log block format changes.
+ * TODO(na) - Implement policies around major/minor versions
+ */
+final class HoodieCommandBlockVersion extends HoodieLogBlockVersion {
+
+ HoodieCommandBlockVersion(int version) {
+ super(version);
+ }
+}
+
+/**
+ * A set of feature flags associated with a delete log block format.
+ * Versions are changed when the log block format changes.
+ * TODO(na) - Implement policies around major/minor versions
+ */
+final class HoodieDeleteBlockVersion extends HoodieLogBlockVersion {
+
+ HoodieDeleteBlockVersion(int version) {
+ super(version);
+ }
+}
\ No newline at end of file
diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java
index db0e87d6f..4db0a58b7 100644
--- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java
+++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java
@@ -291,8 +291,9 @@ public class HoodieTestUtils {
.overBaseCommit(location.getCommitTime())
.withFs(fs).build();
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, location.getCommitTime());
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, location.getCommitTime());
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
logWriter.appendBlock(new HoodieAvroDataBlock(s.getValue().stream().map(r -> {
try {
GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get();
@@ -304,7 +305,7 @@ public class HoodieTestUtils {
} catch (IOException e) {
return null;
}
- }).collect(Collectors.toList()), schema, metadata));
+ }).collect(Collectors.toList()), header));
logWriter.close();
} catch (Exception e) {
fail(e.toString());
diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java
index e57c7ebe7..0acd77fb7 100644
--- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java
+++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java
@@ -27,7 +27,6 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat.Reader;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
-import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock;
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
@@ -35,7 +34,7 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
-import com.uber.hoodie.common.util.collection.DiskBasedMap;
+import com.uber.hoodie.exception.CorruptedLogFileException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -49,12 +48,15 @@ import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
import java.util.Collections;
-import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -68,12 +70,26 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@SuppressWarnings("Duplicates")
+@RunWith(Parameterized.class)
public class HoodieLogFormatTest {
private FileSystem fs;
private Path partitionPath;
private static String basePath;
+ private Boolean readBlocksLazily = true;
+
+ public HoodieLogFormatTest(Boolean readBlocksLazily) {
+ this.readBlocksLazily = readBlocksLazily;
+ }
+
+ @Parameterized.Parameters(name = "LogBlockReadMode")
+ public static Collection data() {
+ return Arrays.asList(new Boolean[][]{
+ {true},{false}
+ });
+ }
+
@BeforeClass
public static void setUpClass() throws IOException, InterruptedException {
// Append is not supported in LocalFileSystem. HDFS needs to be setup.
@@ -119,10 +135,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List records = SchemaTestUtil.generateTestRecords(0, 100);
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
- getSimpleSchema(), metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
long size = writer.getCurrentSize();
assertTrue("We just wrote a block - size should be > 0", size > 0);
@@ -138,10 +154,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List records = SchemaTestUtil.generateTestRecords(0, 100);
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
- getSimpleSchema(), metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
// Write out a block
writer = writer.appendBlock(dataBlock);
// Get the size of the block
@@ -153,8 +169,8 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
- dataBlock = new HoodieAvroDataBlock(records,
- getSimpleSchema(), metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ dataBlock = new HoodieAvroDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
assertEquals("This should be a new log file and hence size should be 0", 0,
writer.getCurrentSize());
@@ -168,10 +184,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List records = SchemaTestUtil.generateTestRecords(0, 100);
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
- getSimpleSchema(), metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize();
writer.close();
@@ -180,8 +196,8 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
- dataBlock = new HoodieAvroDataBlock(records,
- getSimpleSchema(), metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ dataBlock = new HoodieAvroDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
long size2 = writer.getCurrentSize();
assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1);
@@ -195,8 +211,8 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
- dataBlock = new HoodieAvroDataBlock(records,
- getSimpleSchema(), metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ dataBlock = new HoodieAvroDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
long size3 = writer.getCurrentSize();
assertTrue("We just wrote a new block - size3 should be > size2", size3 > size2);
@@ -220,10 +236,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List records = SchemaTestUtil.generateTestRecords(0, 100);
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
- getSimpleSchema(), metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ HoodieAvroDataBlock 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 without closing the file
@@ -233,8 +249,8 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
- dataBlock = new HoodieAvroDataBlock(records,
- getSimpleSchema(), metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ dataBlock = new HoodieAvroDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
long size2 = writer.getCurrentSize();
assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1);
@@ -256,10 +272,10 @@ public class HoodieLogFormatTest {
// Some data & append two times.
List records = SchemaTestUtil.generateTestRecords(0, 100);
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
- getSimpleSchema(), metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
for (int i = 0; i < 2; i++) {
HoodieLogFormat.newWriterBuilder().onParentPath(testPath)
@@ -284,15 +300,15 @@ public class HoodieLogFormatTest {
List copyOfRecords = records.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
- getSimpleSchema(), metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat
- .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
+ .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
assertTrue("We wrote a block, we should be able to read it", reader.hasNext());
HoodieLogBlock nextBlock = reader.next();
assertEquals("The next block should be a data block", HoodieLogBlockType.AVRO_DATA_BLOCK,
@@ -316,10 +332,10 @@ public class HoodieLogFormatTest {
List copyOfRecords1 = records1.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
- getSimpleSchema(), metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -330,8 +346,8 @@ public class HoodieLogFormatTest {
List copyOfRecords2 = records2.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- dataBlock = new HoodieAvroDataBlock(records2,
- getSimpleSchema(), metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ dataBlock = new HoodieAvroDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -343,13 +359,13 @@ public class HoodieLogFormatTest {
List copyOfRecords3 = records3.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- dataBlock = new HoodieAvroDataBlock(records3,
- getSimpleSchema(), metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ dataBlock = new HoodieAvroDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat
- .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
+ .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
assertTrue("First block should be available", reader.hasNext());
HoodieLogBlock nextBlock = reader.next();
HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock;
@@ -373,6 +389,51 @@ public class HoodieLogFormatTest {
dataBlockRead.getRecords());
}
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testBasicAppendAndScanMultipleFiles()
+ throws IOException, URISyntaxException, InterruptedException {
+ Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
+ .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withSizeThreshold(1024).withFileId("test-fileid1")
+ .overBaseCommit("100").withFs(fs).build();
+ Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+
+ Set logFiles = new HashSet<>();
+ List> allRecords = new ArrayList<>();
+ // create 4 log files
+ while(writer.getLogFile().getLogVersion() != 4) {
+ logFiles.add(writer.getLogFile());
+ List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
+ List copyOfRecords1 = records1.stream().map(record ->
+ HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
+ .collect(Collectors.toList());
+ allRecords.add(copyOfRecords1);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
+ writer = writer.appendBlock(dataBlock);
+ }
+ writer.close();
+
+ // scan all log blocks (across multiple log files)
+ HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
+ logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()),
+ schema, "100", 10240L, readBlocksLazily, false);
+
+ List scannedRecords = new ArrayList<>();
+ for(HoodieRecord record: scanner) {
+ scannedRecords.add((IndexedRecord) record.getData().getInsertValue(schema).get());
+ }
+
+ assertEquals("Scanner records count should be the same as appended records",
+ scannedRecords.size(), allRecords.stream().flatMap(records -> records.stream())
+ .collect(Collectors.toList()).size());
+
+ }
+
+
@Test
public void testAppendAndReadOnCorruptedLog()
throws IOException, URISyntaxException, InterruptedException {
@@ -380,10 +441,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List records = SchemaTestUtil.generateTestRecords(0, 100);
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
- getSimpleSchema(), metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -392,19 +453,20 @@ public class HoodieLogFormatTest {
FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
outputStream.write(HoodieLogFormat.MAGIC);
- outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
// Write out a length that does not confirm with the content
- outputStream.writeInt(1000);
- // Write out some metadata
- // TODO : test for failure to write metadata - NA ?
- outputStream.write(HoodieLogBlock.getLogMetadataBytes(metadata));
+ outputStream.writeLong(1000);
+ outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
+ outputStream.writeInt(HoodieLogFormat.currentVersion);
+ // Write out a length that does not confirm with the content
+ outputStream.writeLong(500);
+ // Write out some bytes
outputStream.write("something-random".getBytes());
outputStream.flush();
outputStream.close();
// First round of reads - we should be able to read the first block and then EOF
Reader reader = HoodieLogFormat
- .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
+ .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
assertTrue("First block should be available", reader.hasNext());
reader.next();
assertTrue("We should have corrupted block next", reader.hasNext());
@@ -412,19 +474,20 @@ public class HoodieLogFormatTest {
assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK,
block.getBlockType());
HoodieCorruptBlock corruptBlock = (HoodieCorruptBlock) block;
- assertEquals("", "something-random", new String(corruptBlock.getCorruptedBytes()));
+ //assertEquals("", "something-random", new String(corruptBlock.getCorruptedBytes()));
assertFalse("There should be no more block left", reader.hasNext());
// Simulate another failure back to back
outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
outputStream.write(HoodieLogFormat.MAGIC);
- outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
// Write out a length that does not confirm with the content
- outputStream.writeInt(1000);
- // Write out some metadata
- // TODO : test for failure to write metadata - NA ?
- outputStream.write(HoodieLogBlock.getLogMetadataBytes(metadata));
+ outputStream.writeLong(1000);
+ outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
+ outputStream.writeInt(HoodieLogFormat.currentVersion);
+ // Write out a length that does not confirm with the content
+ outputStream.writeLong(500);
+ // Write out some bytes
outputStream.write("something-else-random".getBytes());
outputStream.flush();
outputStream.close();
@@ -434,13 +497,14 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
- dataBlock = new HoodieAvroDataBlock(records, getSimpleSchema(), metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
+ dataBlock = new HoodieAvroDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.close();
// Second round of reads - we should be able to read the first and last block
reader = HoodieLogFormat
- .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
+ .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
assertTrue("First block should be available", reader.hasNext());
reader.next();
assertTrue("We should get the 1st corrupted block next", reader.hasNext());
@@ -450,7 +514,7 @@ public class HoodieLogFormatTest {
assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK,
block.getBlockType());
corruptBlock = (HoodieCorruptBlock) block;
- assertEquals("", "something-else-random", new String(corruptBlock.getCorruptedBytes()));
+ //assertEquals("", "something-else-random", new String(corruptBlock.getCorruptedBytes()));
assertTrue("We should get the last block next", reader.hasNext());
reader.next();
assertFalse("We should have no more blocks left", reader.hasNext());
@@ -471,10 +535,10 @@ public class HoodieLogFormatTest {
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
- schema, metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
// Write 2
@@ -482,7 +546,8 @@ public class HoodieLogFormatTest {
List copyOfRecords2 = records2.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ dataBlock = new HoodieAvroDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -493,7 +558,7 @@ public class HoodieLogFormatTest {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
- schema, "100", 10240L);
+ schema, "100", 10240L, readBlocksLazily, false);
assertEquals("", 200, scanner.getTotalLogRecords());
Set readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -521,32 +586,35 @@ public class HoodieLogFormatTest {
List copyOfRecords1 = records1.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
- schema, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
// Write 2
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101");
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
- dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ dataBlock = new HoodieAvroDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
// Rollback the last write
- metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101");
- HoodieCommandBlock commandBlock = new HoodieCommandBlock(
- HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101");
+ header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
+ String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
+ HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(commandBlock);
// Write 3
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "102");
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102");
List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List copyOfRecords3 = records3.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- dataBlock = new HoodieAvroDataBlock(records3, schema, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ dataBlock = new HoodieAvroDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -557,8 +625,9 @@ public class HoodieLogFormatTest {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
- schema, "102", 10240L);
- assertEquals("We only read 200 records, but only 200 of them are valid", 200, scanner.getTotalLogRecords());
+ schema, "102", 10240L, readBlocksLazily, false);
+ assertEquals("We read 200 records from 2 write batches", 200,
+ scanner.getTotalLogRecords());
Set readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records", 200, readKeys.size());
@@ -585,48 +654,53 @@ public class HoodieLogFormatTest {
List copyOfRecords1 = records1.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
- schema, metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.close();
// Write 2
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101");
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf());
FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
outputStream.write(HoodieLogFormat.MAGIC);
- outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
// Write out a length that does not confirm with the content
- outputStream.writeInt(100);
- // Write out some metadata
- // TODO : test for failure to write metadata - NA ?
- outputStream.write(HoodieLogBlock.getLogMetadataBytes(metadata));
+ outputStream.writeLong(1000);
+
+ outputStream.writeInt(HoodieLogFormat.currentVersion);
+ outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
+
+ // Write out some header
+ outputStream.write(HoodieLogBlock.getLogMetadataBytes(header));
+ outputStream.writeLong("something-random".getBytes().length);
outputStream.write("something-random".getBytes());
outputStream.flush();
outputStream.close();
// Rollback the last write
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "102");
- metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101");
- HoodieCommandBlock commandBlock = new HoodieCommandBlock(
- HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102");
+ header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101");
+ header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
+ String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
+ HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
writer = writer.appendBlock(commandBlock);
// Write 3
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "103");
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103");
List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List copyOfRecords3 = records3.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- dataBlock = new HoodieAvroDataBlock(records3, schema, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ dataBlock = new HoodieAvroDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -637,8 +711,9 @@ public class HoodieLogFormatTest {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
- schema, "103", 10240L);
- assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords());
+ schema, "103", 10240L, true, false);
+ assertEquals("We would read 200 records", 200,
+ scanner.getTotalLogRecords());
Set readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records", 200, readKeys.size());
@@ -665,19 +740,19 @@ public class HoodieLogFormatTest {
List copyOfRecords1 = records1.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
- schema, metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
// Write 2
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101");
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List copyOfRecords2 = records2.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
+ dataBlock = new HoodieAvroDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
copyOfRecords1.addAll(copyOfRecords2);
@@ -689,9 +764,9 @@ public class HoodieLogFormatTest {
// Delete 50 keys
List deletedKeys = originalKeys.subList(0, 50);
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "102");
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102");
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]),
- metadata);
+ header);
writer = writer.appendBlock(deleteBlock);
List allLogFiles = FSUtils
@@ -701,8 +776,9 @@ public class HoodieLogFormatTest {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
- schema, "102", 10240L);
- assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords());
+ schema, "102", 10240L, readBlocksLazily, false);
+ assertEquals("We still would read 200 records", 200,
+ scanner.getTotalLogRecords());
final List readKeys = new ArrayList<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 150 records", 150, readKeys.size());
@@ -713,14 +789,16 @@ public class HoodieLogFormatTest {
readKeys);
// Rollback the last block
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "103");
- metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "102");
- HoodieCommandBlock commandBlock = new HoodieCommandBlock(
- HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103");
+ header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "102");
+ header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
+ String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
+ HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(commandBlock);
readKeys.clear();
- scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L);
+ scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily,
+ false);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records after rollback of delete", 200,
readKeys.size());
@@ -741,17 +819,18 @@ public class HoodieLogFormatTest {
List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List copyOfRecords1 = records1.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
- schema, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
// Write 2
List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
- dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ dataBlock = new HoodieAvroDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
List originalKeys = copyOfRecords1.stream()
@@ -762,12 +841,13 @@ public class HoodieLogFormatTest {
// Delete 50 keys
List deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]),
- metadata);
+ header);
writer = writer.appendBlock(deleteBlock);
// Attempt 1 : Write rollback block for a failed write
- HoodieCommandBlock commandBlock = new HoodieCommandBlock(
- HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
+ String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
+ HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
try {
writer = writer.appendBlock(commandBlock);
// Say job failed, retry writing 2 rollback in the next rollback(..) attempt
@@ -785,7 +865,7 @@ public class HoodieLogFormatTest {
// all data must be rolled back before merge
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
- allLogFiles, schema, "100", 10240L);
+ allLogFiles, schema, "100", 10240L, readBlocksLazily, false);
assertEquals("We would have scanned 0 records because of rollback", 0, scanner.getTotalLogRecords());
final List readKeys = new ArrayList<>();
@@ -809,11 +889,11 @@ public class HoodieLogFormatTest {
List copyOfRecords1 = records1.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
- schema, metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
List originalKeys = copyOfRecords1.stream()
@@ -824,12 +904,13 @@ public class HoodieLogFormatTest {
// Delete 50 keys
List deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]),
- metadata);
+ header);
writer = writer.appendBlock(deleteBlock);
// Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
- HoodieCommandBlock commandBlock = new HoodieCommandBlock(
- HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
+ String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
+ HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(commandBlock);
writer = writer.appendBlock(commandBlock);
@@ -839,7 +920,7 @@ public class HoodieLogFormatTest {
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
- allLogFiles, schema, "100", 10240L);
+ allLogFiles, schema, "100", 10240L, readBlocksLazily, false);
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
}
@@ -855,16 +936,17 @@ public class HoodieLogFormatTest {
// Write 1
List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
- schema, metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
// Write invalid rollback for a failed write (possible for in-flight commits)
- metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101");
- HoodieCommandBlock commandBlock = new HoodieCommandBlock(
- HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101");
+ header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
+ String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
+ HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(commandBlock);
List allLogFiles = FSUtils
@@ -873,7 +955,7 @@ public class HoodieLogFormatTest {
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
- allLogFiles, schema, "100", 10240L);
+ allLogFiles, schema, "100", 10240L, readBlocksLazily, false);
assertEquals("We still would read 100 records", 100,
scanner.getTotalLogRecords());
final List readKeys = new ArrayList<>(100);
@@ -897,11 +979,11 @@ public class HoodieLogFormatTest {
List copyOfRecords1 = records1.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
- schema, metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
@@ -914,13 +996,14 @@ public class HoodieLogFormatTest {
// Delete 50 keys
List deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]),
- metadata);
+ header);
writer = writer.appendBlock(deleteBlock);
// Write 1 rollback block for a failed write
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101");
- HoodieCommandBlock commandBlock = new HoodieCommandBlock(
- HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
+ header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
+ String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
+ HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(commandBlock);
List allLogFiles = FSUtils
@@ -929,7 +1012,7 @@ public class HoodieLogFormatTest {
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
- allLogFiles, schema, "101", 10240L);
+ allLogFiles, schema, "101", 10240L, readBlocksLazily, false);
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
}
@@ -947,11 +1030,11 @@ public class HoodieLogFormatTest {
// Write 1
List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
- metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
- schema, metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
@@ -962,9 +1045,11 @@ public class HoodieLogFormatTest {
FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
outputStream.write(HoodieLogFormat.MAGIC);
+ outputStream.writeLong(1000);
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
+ outputStream.writeInt(HoodieLogFormat.currentVersion);
// Write out a length that does not confirm with the content
- outputStream.writeInt(100);
+ outputStream.writeLong(100);
outputStream.flush();
outputStream.close();
@@ -973,9 +1058,11 @@ public class HoodieLogFormatTest {
outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
outputStream.write(HoodieLogFormat.MAGIC);
+ outputStream.writeLong(1000);
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
+ outputStream.writeInt(HoodieLogFormat.currentVersion);
// Write out a length that does not confirm with the content
- outputStream.writeInt(100);
+ outputStream.writeLong(100);
outputStream.flush();
outputStream.close();
@@ -991,9 +1078,11 @@ public class HoodieLogFormatTest {
outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
outputStream.write(HoodieLogFormat.MAGIC);
+ outputStream.writeLong(1000);
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
+ outputStream.writeInt(HoodieLogFormat.currentVersion);
// Write out a length that does not confirm with the content
- outputStream.writeInt(100);
+ outputStream.writeLong(100);
outputStream.flush();
outputStream.close();
@@ -1001,9 +1090,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1 rollback block for a failed write
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101");
- HoodieCommandBlock commandBlock = new HoodieCommandBlock(
- HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
+ header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
+ String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
+ HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(commandBlock);
List allLogFiles = FSUtils
@@ -1012,9 +1102,282 @@ public class HoodieLogFormatTest {
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
- allLogFiles, schema, "101", 10240L);
+ allLogFiles, schema, "101", 10240L, readBlocksLazily, false);
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
}
+ @Test
+ public void testMagicAndLogVersionsBackwardsCompatibility()
+ throws IOException, InterruptedException, URISyntaxException {
+ // Create the log file
+ Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
+ .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .overBaseCommit("100").withFs(fs).build();
+ Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
+ List records = SchemaTestUtil.generateHoodieTestRecords(0, 100);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ writer.close();
+
+ // Write 1 with OLD_MAGIC and no log format version
+ // Append a log block to end of the log (mimics a log block with old format
+ // fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf());
+ FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
+ // create a block with
+ outputStream.write(HoodieLogFormat.OLD_MAGIC);
+ outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
+ // Write out a length that does not confirm with the content
+ records = SchemaTestUtil.generateHoodieTestRecords(0, 100);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, new String(HoodieAvroUtils.compress(schema.toString())));
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, schema);
+ byte [] content = dataBlock.getBytes(schema);
+ outputStream.writeInt(content.length);
+ // Write out some content
+ outputStream.write(content);
+ outputStream.flush();
+ outputStream.hflush();
+ outputStream.close();
+
+ writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
+ .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .overBaseCommit("100").withFs(fs).build();
+
+ // Write 2 with MAGIC and latest log format version
+ records = SchemaTestUtil.generateHoodieTestRecords(0, 100);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ dataBlock = new HoodieAvroDataBlock(records, header);
+ writer = writer.appendBlock(dataBlock);
+
+ // Write 3 with MAGIC and latest log format version
+ writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
+ .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .overBaseCommit("100").withFs(fs).build();
+ records = SchemaTestUtil.generateHoodieTestRecords(0, 100);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ dataBlock = new HoodieAvroDataBlock(records, header);
+ writer = writer.appendBlock(dataBlock);
+ writer.close();
+
+ Reader reader = HoodieLogFormat
+ .newReader(fs, writer.getLogFile(), schema);
+
+ // Read the first block written with latest version and magic
+ reader.hasNext();
+ HoodieLogBlock block = reader.next();
+ assertEquals(block.getBlockType(), HoodieLogBlockType.AVRO_DATA_BLOCK);
+ HoodieAvroDataBlock dBlock = (HoodieAvroDataBlock) block;
+ assertEquals(dBlock.getRecords().size(), 100);
+
+ // Read second block written with old magic and no version
+ reader.hasNext();
+ block = reader.next();
+ assertEquals(block.getBlockType(), HoodieLogBlockType.AVRO_DATA_BLOCK);
+ dBlock = (HoodieAvroDataBlock) block;
+ assertEquals(dBlock.getRecords().size(), 100);
+
+ //Read third block written with latest version and magic
+ reader.hasNext();
+ block = reader.next();
+ assertEquals(block.getBlockType(), HoodieLogBlockType.AVRO_DATA_BLOCK);
+ dBlock = (HoodieAvroDataBlock) block;
+ assertEquals(dBlock.getRecords().size(), 100);
+
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testBasicAppendAndReadInReverse()
+ throws IOException, URISyntaxException, InterruptedException {
+ Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
+ .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .overBaseCommit("100").withFs(fs).build();
+ Schema schema = getSimpleSchema();
+ List records1 = SchemaTestUtil.generateTestRecords(0, 100);
+ List copyOfRecords1 = records1.stream().map(record ->
+ HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
+ .collect(Collectors.toList());
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
+ writer = writer.appendBlock(dataBlock);
+ writer.close();
+
+ writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
+ .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .overBaseCommit("100").withFs(fs).build();
+ List records2 = SchemaTestUtil.generateTestRecords(0, 100);
+ List copyOfRecords2 = records2.stream().map(record ->
+ HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
+ .collect(Collectors.toList());
+ dataBlock = new HoodieAvroDataBlock(records2, header);
+ writer = writer.appendBlock(dataBlock);
+ writer.close();
+
+ // Close and Open again and append 100 more records
+ writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
+ .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .overBaseCommit("100").withFs(fs).build();
+ List records3 = SchemaTestUtil.generateTestRecords(0, 100);
+ List copyOfRecords3 = records3.stream().map(record ->
+ HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
+ .collect(Collectors.toList());
+ dataBlock = new HoodieAvroDataBlock(records3, header);
+ writer = writer.appendBlock(dataBlock);
+ writer.close();
+
+ HoodieLogFileReader reader =
+ new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), readBlocksLazily,
+ true);
+
+ assertTrue("Last block should be available", reader.hasPrev());
+ HoodieLogBlock prevBlock = reader.prev();
+ HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) prevBlock;
+
+ assertEquals("Third records size should be equal to the written records size",
+ copyOfRecords3.size(), dataBlockRead.getRecords().size());
+ assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords3,
+ dataBlockRead.getRecords());
+
+ assertTrue("Second block should be available", reader.hasPrev());
+ prevBlock = reader.prev();
+ dataBlockRead = (HoodieAvroDataBlock) prevBlock;
+ assertEquals("Read records size should be equal to the written records size",
+ copyOfRecords2.size(), dataBlockRead.getRecords().size());
+ assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords2,
+ dataBlockRead.getRecords());
+
+ assertTrue("First block should be available", reader.hasPrev());
+ prevBlock = reader.prev();
+ dataBlockRead = (HoodieAvroDataBlock) prevBlock;
+ assertEquals("Read records size should be equal to the written records size",
+ copyOfRecords1.size(), dataBlockRead.getRecords().size());
+ assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords1,
+ dataBlockRead.getRecords());
+
+ assertFalse(reader.hasPrev());
+ }
+
+ @Test
+ public void testAppendAndReadOnCorruptedLogInReverse()
+ throws IOException, URISyntaxException, InterruptedException {
+ Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
+ .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .overBaseCommit("100").withFs(fs).build();
+ Schema schema = getSimpleSchema();
+ List records = SchemaTestUtil.generateTestRecords(0, 100);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
+ writer = writer.appendBlock(dataBlock);
+ writer.close();
+
+ // Append some arbit byte[] to thee end of the log (mimics a partially written commit)
+ fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf());
+ FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
+ // create a block with
+ outputStream.write(HoodieLogFormat.OLD_MAGIC);
+ outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
+ // Write out a length that does not confirm with the content
+ outputStream.writeInt(1000);
+ // Write out footer length
+ outputStream.writeInt(1);
+ // Write out some metadata
+ // TODO : test for failure to write metadata - NA ?
+ outputStream.write(HoodieLogBlock.getLogMetadataBytes(header));
+ outputStream.write("something-random".getBytes());
+ outputStream.flush();
+ outputStream.close();
+
+ // Should be able to append a new block
+ writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
+ .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .overBaseCommit("100").withFs(fs).build();
+ records = SchemaTestUtil.generateTestRecords(0, 100);
+ dataBlock = new HoodieAvroDataBlock(records, header);
+ writer = writer.appendBlock(dataBlock);
+ writer.close();
+
+ // First round of reads - we should be able to read the first block and then EOF
+ HoodieLogFileReader reader =
+ new HoodieLogFileReader(fs, writer.getLogFile(), schema, readBlocksLazily, true);
+
+ assertTrue("Last block should be available", reader.hasPrev());
+ HoodieLogBlock block = reader.prev();
+ assertTrue("Last block should be datablock", block instanceof HoodieAvroDataBlock);
+
+ assertTrue("Last block should be available", reader.hasPrev());
+ try {
+ reader.prev();
+ } catch(CorruptedLogFileException e) {
+ e.printStackTrace();
+ // We should have corrupted block
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testBasicAppendAndTraverseInReverse()
+ throws IOException, URISyntaxException, InterruptedException {
+ Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
+ .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .overBaseCommit("100").withFs(fs).build();
+ Schema schema = getSimpleSchema();
+ List records1 = SchemaTestUtil.generateTestRecords(0, 100);
+ List copyOfRecords1 = records1.stream().map(record ->
+ HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
+ .collect(Collectors.toList());
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
+ writer = writer.appendBlock(dataBlock);
+ writer.close();
+
+ writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
+ .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .overBaseCommit("100").withFs(fs).build();
+ List records2 = SchemaTestUtil.generateTestRecords(0, 100);
+ List copyOfRecords2 = records2.stream().map(record ->
+ HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
+ .collect(Collectors.toList());
+ dataBlock = new HoodieAvroDataBlock(records2, header);
+ writer = writer.appendBlock(dataBlock);
+ writer.close();
+
+ // Close and Open again and append 100 more records
+ writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
+ .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .overBaseCommit("100").withFs(fs).build();
+ List records3 = SchemaTestUtil.generateTestRecords(0, 100);
+ List copyOfRecords3 = records3.stream().map(record ->
+ HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
+ .collect(Collectors.toList());
+ dataBlock = new HoodieAvroDataBlock(records3, header);
+ writer = writer.appendBlock(dataBlock);
+ writer.close();
+
+ HoodieLogFileReader reader =
+ new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), readBlocksLazily,
+ true);
+
+ assertTrue("Third block should be available", reader.hasPrev());
+ reader.moveToPrev();
+
+ assertTrue("Second block should be available", reader.hasPrev());
+ reader.moveToPrev();
+
+ // After moving twice, this last reader.prev() should read the First block written
+ assertTrue("First block should be available", reader.hasPrev());
+ HoodieLogBlock prevBlock = reader.prev();
+ HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) prevBlock;
+ assertEquals("Read records size should be equal to the written records size",
+ copyOfRecords1.size(), dataBlockRead.getRecords().size());
+ assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords1,
+ dataBlockRead.getRecords());
+
+ assertFalse(reader.hasPrev());
+ }
}
diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java
index 9f231084a..81cae359b 100644
--- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java
+++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java
@@ -72,6 +72,11 @@ public class HoodieRealtimeRecordReader implements RecordReader deltaRecordMap;
@@ -132,7 +137,8 @@ public class HoodieRealtimeRecordReader implements RecordReader the commit we are trying to read (if using readCommit() API)
for (HoodieRecord extends HoodieRecordPayload> hoodieRecord : compactedLogRecordScanner) {
@@ -140,6 +146,7 @@ public class HoodieRealtimeRecordReader implements RecordReader metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, newCommit);
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, writeSchema, metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
long size = writer.getCurrentSize();
return writer;
diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java
index 95d1d5821..69ae7aff5 100644
--- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java
+++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java
@@ -395,7 +395,7 @@ public class HoodieHiveClient {
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
private MessageType readSchemaFromLogFile(Optional lastCompactionCommitOpt,
Path path) throws IOException {
- Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null, true);
+ Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null);
HoodieAvroDataBlock lastBlock = null;
while (reader.hasNext()) {
HoodieLogBlock block = reader.next();
@@ -404,6 +404,7 @@ public class HoodieHiveClient {
}
}
if (lastBlock != null) {
+ lastBlock.getRecords();
return new parquet.avro.AvroSchemaConverter().convert(lastBlock.getSchema());
}
// Fall back to read the schema from last compaction
diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java
index f2eb5e4f1..10b8c11f8 100644
--- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java
+++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java
@@ -314,9 +314,10 @@ public class TestUtil {
List records = (isLogSchemaSimple ? SchemaTestUtil
.generateTestRecords(0, 100)
: SchemaTestUtil.generateEvolvedTestRecords(100, 100));
- Map metadata = Maps.newHashMap();
- metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, dataFile.getCommitTime());
- HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, schema, metadata);
+ Map header = Maps.newHashMap();
+ header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, dataFile.getCommitTime());
+ header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
+ HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
logWriter.appendBlock(dataBlock);
logWriter.close();
return logWriter.getLogFile();