1
0

CodeStyle formatting to conform to basic Checkstyle rules.

The code-style rules follow google style with some changes:

1. Increase line length from 100 to 120
2. Disable JavaDoc related checkstyles as this needs more manual work.

Both source and test code are checked for code-style
This commit is contained in:
Balaji Varadarajan
2018-03-20 16:29:20 -07:00
committed by vinoth chandar
parent 987f5d6b96
commit 788e4f2d2e
200 changed files with 6209 additions and 5975 deletions

View File

@@ -33,10 +33,10 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport {
private String maxRecordKey;
public final static String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY =
public static final String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY =
"com.uber.hoodie.bloomfilter";
public final static String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key";
public final static String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key";
public static final String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key";
public static final String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key";
public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, BloomFilter bloomFilter) {

View File

@@ -16,6 +16,7 @@
package com.uber.hoodie.avro;
import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
@@ -24,11 +25,11 @@ import java.util.Map;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import com.fasterxml.jackson.databind.ObjectMapper;
/**
* Marjority of this is copied from https://github.com/jwills/avro-json/blob/master/src/main/java/com/cloudera/science/avro/common/JsonConverter.java
* Adjusted for expected behavior of our use cases
* Marjority of this is copied from
* https://github.com/jwills/avro-json/blob/master/src/main/java/com/cloudera/science/avro/
* common/JsonConverter.java Adjusted for expected behavior of our use cases
*/
public class MercifulJsonConverter {
@@ -132,10 +133,10 @@ public class MercifulJsonConverter {
}
private boolean isOptional(Schema schema) {
return schema.getType().equals(Schema.Type.UNION) &&
schema.getTypes().size() == 2 &&
(schema.getTypes().get(0).getType().equals(Schema.Type.NULL) ||
schema.getTypes().get(1).getType().equals(Schema.Type.NULL));
return schema.getType().equals(Schema.Type.UNION)
&& schema.getTypes().size() == 2
&& (schema.getTypes().get(0).getType().equals(Schema.Type.NULL)
|| schema.getTypes().get(1).getType().equals(Schema.Type.NULL));
}
private Schema getNonNull(Schema schema) {

View File

@@ -113,9 +113,8 @@ public class HoodieCleanStat implements Serializable {
}
public Builder withEarliestCommitRetained(Optional<HoodieInstant> earliestCommitToRetain) {
this.earliestCommitToRetain = (earliestCommitToRetain.isPresent()) ?
earliestCommitToRetain.get().getTimestamp() :
"-1";
this.earliestCommitToRetain = (earliestCommitToRetain.isPresent())
? earliestCommitToRetain.get().getTimestamp() : "-1";
return this;
}

View File

@@ -210,12 +210,18 @@ public class HoodieCommitMetadata implements Serializable {
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
HoodieCommitMetadata that = (HoodieCommitMetadata) o;
if (!partitionToWriteStats.equals(that.partitionToWriteStats)) return false;
if (!partitionToWriteStats.equals(that.partitionToWriteStats)) {
return false;
}
return compacted.equals(that.compacted);
}

View File

@@ -39,7 +39,6 @@ public class HoodieFileGroup implements Serializable {
};
}
/**
* Partition containing the file group.
*/
@@ -107,10 +106,10 @@ public class HoodieFileGroup implements Serializable {
*/
private boolean isFileSliceCommitted(FileSlice slice) {
String maxCommitTime = lastInstant.get().getTimestamp();
return timeline.containsOrBeforeTimelineStarts(slice.getBaseCommitTime()) &&
HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL);
return timeline.containsOrBeforeTimelineStarts(slice.getBaseCommitTime())
&& HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL);
}
@@ -128,7 +127,7 @@ public class HoodieFileGroup implements Serializable {
/**
* Gets the latest slice - this can contain either
*
* <p>
* - just the log files without data file - (or) data file with 0 or more log files
*/
public Optional<FileSlice> getLatestFileSlice() {

View File

@@ -21,7 +21,7 @@ import java.io.Serializable;
/**
* HoodieKey consists of
*
* <p>
* - recordKey : a recordKey that acts as primary key for a record - partitionPath : path to the
* partition that contains the record
*/
@@ -54,8 +54,8 @@ public class HoodieKey implements Serializable {
return false;
}
HoodieKey otherKey = (HoodieKey) o;
return Objects.equal(recordKey, otherKey.recordKey) &&
Objects.equal(partitionPath, otherKey.partitionPath);
return Objects.equal(recordKey, otherKey.recordKey)
&& Objects.equal(partitionPath, otherKey.partitionPath);
}
@Override

View File

@@ -30,7 +30,7 @@ import org.apache.hadoop.fs.Path;
/**
* Abstracts a single log file. Contains methods to extract metadata like the fileId, version and
* extension from the log file path.
*
* <p>
* Also contains logic to roll-over the log file
*/
public class HoodieLogFile implements Serializable {
@@ -103,8 +103,12 @@ public class HoodieLogFile implements Serializable {
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
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;
}

View File

@@ -101,9 +101,8 @@ public class HoodiePartitionMetadata {
}
} catch (IOException ioe) {
log.warn(
"Error trying to save partition metadata (this is okay, as long as atleast 1 of these succced), "
+
partitionPath, ioe);
"Error trying to save partition metadata (this is okay, as long as "
+ "atleast 1 of these succced), " + partitionPath, ioe);
} finally {
if (!metafileExists) {
try {

View File

@@ -118,10 +118,10 @@ public class HoodieRecord<T extends HoodieRecordPayload> implements Serializable
return false;
}
HoodieRecord that = (HoodieRecord) o;
return Objects.equal(key, that.key) &&
Objects.equal(data, that.data) &&
Objects.equal(currentLocation, that.currentLocation) &&
Objects.equal(newLocation, that.newLocation);
return Objects.equal(key, that.key)
&& Objects.equal(data, that.data)
&& Objects.equal(currentLocation, that.currentLocation)
&& Objects.equal(newLocation, that.newLocation);
}
@Override

View File

@@ -42,8 +42,8 @@ public class HoodieRecordLocation implements Serializable {
return false;
}
HoodieRecordLocation otherLoc = (HoodieRecordLocation) o;
return Objects.equal(commitTime, otherLoc.commitTime) &&
Objects.equal(fileId, otherLoc.fileId);
return Objects.equal(commitTime, otherLoc.commitTime)
&& Objects.equal(fileId, otherLoc.fileId);
}
@Override

View File

@@ -38,13 +38,13 @@ public interface HoodieRecordPayload<T extends HoodieRecordPayload> extends Seri
/**
* This methods lets you write custom merging/combining logic to produce new values as a function
* of current value on storage and whats contained in this object.
*
* <p>
* eg: 1) You are updating counters, you may want to add counts to currentValue and write back
* updated counts 2) You may be reading DB redo logs, and merge them with current image for a
* database row on storage
*
* @param currentValue Current value in storage, to merge/combine this payload with
* @param schema Schema used for record
* @param schema Schema used for record
* @return new combined/merged value to be written back to storage. EMPTY to skip writing this
* record.
*/

View File

@@ -18,16 +18,16 @@ package com.uber.hoodie.common.model;
/**
* Type of the Hoodie Table.
*
* <p>
* Currently, 1 type is supported
*
* <p>
* COPY_ON_WRITE - Performs upserts by versioning entire files, with later versions containing newer
* value of a record.
*
* <p>
* In the future, following might be added.
*
* <p>
* MERGE_ON_READ - Speeds up upserts, by delaying merge until enough work piles up.
*
* <p>
* SIMPLE_LSM - A simple 2 level LSM tree.
*/
public enum HoodieTableType {

View File

@@ -17,9 +17,8 @@
package com.uber.hoodie.common.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import javax.annotation.Nullable;
import java.io.Serializable;
import javax.annotation.Nullable;
import org.apache.hadoop.fs.Path;
/**

View File

@@ -25,39 +25,39 @@ import java.util.concurrent.atomic.AtomicInteger;
*/
public class SizeAwareDataInputStream {
private final DataInputStream dis;
private final AtomicInteger numberOfBytesRead;
private final DataInputStream dis;
private final AtomicInteger numberOfBytesRead;
public SizeAwareDataInputStream(DataInputStream dis) {
this.dis = dis;
this.numberOfBytesRead = new AtomicInteger(0);
}
public SizeAwareDataInputStream(DataInputStream dis) {
this.dis = dis;
this.numberOfBytesRead = new AtomicInteger(0);
}
public int readInt() throws IOException {
numberOfBytesRead.addAndGet(Integer.BYTES);
return dis.readInt();
}
public int readInt() throws IOException {
numberOfBytesRead.addAndGet(Integer.BYTES);
return dis.readInt();
}
public void readFully(byte b[], int off, int len) throws IOException {
numberOfBytesRead.addAndGet(len);
dis.readFully(b, off, len);
}
public void readFully(byte[] b, int off, int len) throws IOException {
numberOfBytesRead.addAndGet(len);
dis.readFully(b, off, len);
}
public void readFully(byte b[]) throws IOException {
numberOfBytesRead.addAndGet(b.length);
dis.readFully(b);
}
public void readFully(byte[] b) throws IOException {
numberOfBytesRead.addAndGet(b.length);
dis.readFully(b);
}
public int skipBytes(int n) throws IOException {
numberOfBytesRead.addAndGet(n);
return dis.skipBytes(n);
}
public int skipBytes(int n) throws IOException {
numberOfBytesRead.addAndGet(n);
return dis.skipBytes(n);
}
public void close() throws IOException {
dis.close();
}
public void close() throws IOException {
dis.close();
}
public Integer getNumberOfBytesRead() {
return numberOfBytesRead.get();
}
public Integer getNumberOfBytesRead() {
return numberOfBytesRead.get();
}
}

View File

@@ -42,7 +42,7 @@ import org.apache.log4j.Logger;
*/
public class HoodieTableConfig implements Serializable {
private final transient static Logger log = LogManager.getLogger(HoodieTableConfig.class);
private static final transient Logger log = LogManager.getLogger(HoodieTableConfig.class);
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";

View File

@@ -48,7 +48,7 @@ import org.apache.log4j.Logger;
*/
public class HoodieTableMetaClient implements Serializable {
private final transient static Logger log = LogManager.getLogger(HoodieTableMetaClient.class);
private static final transient Logger log = LogManager.getLogger(HoodieTableMetaClient.class);
public static String METAFOLDER_NAME = ".hoodie";
public static String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp";

View File

@@ -16,6 +16,9 @@
package com.uber.hoodie.common.table.log;
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 com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -28,17 +31,9 @@ import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
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.common.util.collection.converter.StringConverter;
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
import com.uber.hoodie.common.util.collection.converter.StringConverter;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
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.IOException;
import java.util.ArrayDeque;
import java.util.Arrays;
@@ -49,30 +44,28 @@ import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
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 org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* 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.
*
* 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 | <p> This results in two I/O passes over the log file.
*/
public class HoodieCompactedLogRecordScanner implements
Iterable<HoodieRecord<? extends HoodieRecordPayload>> {
private final static Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class);
private static final Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class);
// Final map of compacted/merged records
private final ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records;
@@ -116,10 +109,10 @@ public class HoodieCompactedLogRecordScanner implements
totalLogFiles.incrementAndGet();
// 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)) {
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;
}
@@ -147,14 +140,16 @@ public class HoodieCompactedLogRecordScanner implements
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 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
// 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;
@@ -163,10 +158,11 @@ public class HoodieCompactedLogRecordScanner implements
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
// 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();
@@ -176,30 +172,29 @@ public class HoodieCompactedLogRecordScanner implements
"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))) {
} else if (lastBlock.getBlockType() != CORRUPT_BLOCK
&& targetInstantForCommandBlock
.contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) {
// rollback last data block or delete block
log.info("Rolling back the last log block read in " + logFile.getPath());
currentInstantLogBlocks.pop();
numBlocksRolledBack++;
}
// invalid or extra rollback block
else if (!targetInstantForCommandBlock
} else if (!targetInstantForCommandBlock
.contentEquals(
currentInstantLogBlocks.peek().getLogBlockHeader().get(INSTANT_TIME))) {
log.warn("TargetInstantTime " + targetInstantForCommandBlock +
" invalid or extra rollback command block in " + logFile.getPath());
// invalid or extra rollback block
log.warn("TargetInstantTime " + targetInstantForCommandBlock
+ " invalid or extra rollback command block in " + logFile.getPath());
break;
}
// this should not happen ideally
else {
} else {
// this should not happen ideally
log.warn("Unable to apply rollback command block in " + logFile.getPath());
}
}
log.info("Number of applied rollback blocks " + numBlocksRolledBack);
break;
default:
throw new UnsupportedOperationException("Command type not yet supported.");
}
break;
@@ -208,6 +203,8 @@ public class HoodieCompactedLogRecordScanner implements
// If there is a corrupt block - we will assume that this was the next data block
currentInstantLogBlocks.push(r);
break;
default:
throw new UnsupportedOperationException("Block type not supported yet");
}
}
// merge the last read block when all the blocks are done reading
@@ -240,10 +237,9 @@ public class HoodieCompactedLogRecordScanner implements
}
/**
* Iterate over the GenericRecord in the block, read the hoodie key and partition path and merge
* with the application specific payload if the same key was found before. Sufficient to just merge
* the log records since the base data is merged on previous compaction.
* Finally, merge this log block with the accumulated records
* Iterate over the GenericRecord in the block, read the hoodie key and partition path and merge with the application
* specific payload if the same key was found before. Sufficient to just merge the log records since the base data is
* merged on previous compaction. Finally, merge this log block with the accumulated records
*/
private Map<String, HoodieRecord<? extends HoodieRecordPayload>> merge(
HoodieAvroDataBlock dataBlock) throws IOException {
@@ -291,6 +287,9 @@ public class HoodieCompactedLogRecordScanner implements
case CORRUPT_BLOCK:
log.warn("Found a corrupt block which was not rolled back");
break;
default:
//TODO <vb> : Need to understand if COMMAND_BLOCK has to be handled?
break;
}
}
}

View File

@@ -28,18 +28,17 @@ 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;
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
@@ -49,28 +48,29 @@ import java.util.Optional;
class HoodieLogFileReader implements HoodieLogFormat.Reader {
private static final int DEFAULT_BUFFER_SIZE = 4096;
private final static Logger log = LogManager.getLogger(HoodieLogFileReader.class);
private static final 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 LogFormatVersion nextBlockVersion;
private HoodieLogFormat.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 {
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();
if (this.reverseReader) {
this.reverseLogFilePosition = this.lastReverseLogFilePosition = fs
.getFileStatus(logFile.getPath()).getLen();
}
addShutDownHook();
}
@@ -80,7 +80,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, readBlockLazily, reverseReader);
}
HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) throws IOException {
HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema)
throws IOException {
this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, false, false);
}
@@ -105,7 +106,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
});
}
// TODO : convert content and block length to long by using ByteBuffer, raw byte [] allows for max of Integer size
// 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;
@@ -137,8 +139,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
}
// 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.
// 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();
@@ -168,7 +170,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
}
// 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)
// 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);
@@ -178,7 +181,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
footer = HoodieLogBlock.getLogMetadata(inputStream);
}
// 8. Read log block length, if present. This acts as a reverse pointer when traversing a log file in reverse
// 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();
@@ -193,15 +197,18 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
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);
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);
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);
return HoodieCommandBlock
.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, header, footer);
default:
throw new HoodieNotSupportedException("Unsupported Block " + blockType);
}
@@ -216,9 +223,12 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
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<>());
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 {
@@ -278,12 +288,10 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
}
/**
* 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
* 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
*/
private LogFormatVersion readVersion() throws IOException {
private HoodieLogFormat.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);
@@ -309,7 +317,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
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");
logFile
+ "could not be read. Did not find the magic bytes at the start of the block");
}
}
return false;
@@ -324,19 +333,17 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
try {
// hasNext() must be called before next()
return readBlock();
} catch(IOException io) {
} catch (IOException io) {
throw new HoodieIOException("IOException when reading logblock from log file " + logFile, io);
}
}
/**
* hasPrev is not idempotent
*
* @return
*/
public boolean hasPrev() {
try {
if(!this.reverseReader) {
if (!this.reverseReader) {
throw new HoodieNotSupportedException("Reverse log reader has not been enabled");
}
reverseLogFilePosition = lastReverseLogFilePosition;
@@ -351,17 +358,13 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
}
/**
* 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
* 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) {
if (!this.reverseReader) {
throw new HoodieNotSupportedException("Reverse log reader has not been enabled");
}
long blockSize = inputStream.readLong();
@@ -372,8 +375,9 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
} 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");
throw new CorruptedLogFileException(
"Found possible corrupted block, cannot read log file in reverse, "
+ "fallback to forward reading of logfile");
}
boolean hasNext = hasNext();
reverseLogFilePosition -= blockSize;
@@ -382,16 +386,14 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
}
/**
* 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
* 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
*/
public long moveToPrev() throws IOException {
if(!this.reverseReader) {
if (!this.reverseReader) {
throw new HoodieNotSupportedException("Reverse log reader has not been enabled");
}
inputStream.seek(lastReverseLogFilePosition);

View File

@@ -19,22 +19,21 @@ 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 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
* to delete - tombstone for keys
* 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 to delete - tombstone for keys
*/
public interface HoodieLogFormat {
@@ -43,19 +42,18 @@ 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[] OLD_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
* 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', '#'};
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}
* 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;
@@ -94,7 +92,7 @@ public interface HoodieLogFormat {
*/
class WriterBuilder {
private final static Logger log = LogManager.getLogger(WriterBuilder.class);
private static final Logger log = LogManager.getLogger(WriterBuilder.class);
// Default max log file size 512 MB
public static final long DEFAULT_SIZE_THRESHOLD = 512 * 1024 * 1024L;
@@ -112,7 +110,8 @@ public interface HoodieLogFormat {
private String logFileId;
// File Commit Time stamp
private String commitTime;
// version number for this log file. If not specified, then the current version will be computed by inspecting the file system
// version number for this log file. If not specified, then the current version will be
// computed by inspecting the file system
private Integer logVersion;
// Location of the directory containing the log
private Path parentPath;
@@ -215,4 +214,35 @@ public interface HoodieLogFormat {
throws IOException {
return new HoodieLogFileReader(fs, logFile, readerSchema, false, false);
}
/**
* 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();
}
}

View File

@@ -19,11 +19,10 @@ 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.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import java.io.IOException;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -36,16 +35,16 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
private final boolean readBlocksLazily;
private final boolean reverseLogReader;
private final static Logger log = LogManager.getLogger(HoodieLogFormatReader.class);
private static final Logger log = LogManager.getLogger(HoodieLogFormatReader.class);
HoodieLogFormatReader(FileSystem fs, List<HoodieLogFile> logFiles,
Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader) throws IOException {
Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader) throws IOException {
this.logFiles = logFiles;
this.fs = fs;
this.readerSchema = readerSchema;
this.readBlocksLazily = readBlocksLazily;
this.reverseLogReader = reverseLogReader;
if(logFiles.size() > 0) {
if (logFiles.size() > 0) {
HoodieLogFile nextLogFile = logFiles.remove(0);
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, readBlocksLazily,
false);
@@ -53,7 +52,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
}
HoodieLogFormatReader(FileSystem fs, List<HoodieLogFile> logFiles,
Schema readerSchema) throws IOException {
Schema readerSchema) throws IOException {
this(fs, logFiles, readerSchema, false, false);
}
@@ -67,16 +66,15 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
@Override
public boolean hasNext() {
if(currentReader == null) {
if (currentReader == null) {
return false;
}
else if (currentReader.hasNext()) {
} else if (currentReader.hasNext()) {
return true;
}
else if (logFiles.size() > 0) {
} else if (logFiles.size() > 0) {
try {
HoodieLogFile nextLogFile = logFiles.remove(0);
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, readBlocksLazily,
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema,
readBlocksLazily,
false);
} catch (IOException io) {
throw new HoodieIOException("unable to initialize read with log file ", io);

View File

@@ -17,46 +17,17 @@
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
* Implements logic to determine behavior for feature flags for
* {@link HoodieLogFormat.LogFormatVersion}.
*/
abstract class LogFormatVersion {
private final int version;
final class HoodieLogFormatVersion extends HoodieLogFormat.LogFormatVersion {
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;
public static final int DEFAULT_VERSION = 0;
HoodieLogFormatVersion(int version) {
super(version);
}
@Override
public boolean hasMagicHeader() {
switch (super.getVersion()) {
@@ -114,8 +85,9 @@ final class HoodieLogFormatVersion extends LogFormatVersion {
return false;
case 1:
return true;
default:
return false;
}
return false;
}
@Override
@@ -125,7 +97,8 @@ final class HoodieLogFormatVersion extends LogFormatVersion {
return false;
case 1:
return true;
default:
return false;
}
return false;
}
}

View File

@@ -22,6 +22,7 @@ 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;
@@ -31,15 +32,13 @@ 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
*/
public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
private final static Logger log = LogManager.getLogger(HoodieLogFormatWriter.class);
private static final Logger log = LogManager.getLogger(HoodieLogFormatWriter.class);
private HoodieLogFile logFile;
private final FileSystem fs;
@@ -49,7 +48,6 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
private FSDataOutputStream output;
/**
*
* @param fs
* @param logFile
* @param bufferSize
@@ -71,7 +69,8 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
try {
this.output = fs.append(path, bufferSize);
} catch (RemoteException e) {
// this happens when either another task executor writing to this file died or data node is going down
// this happens when either another task executor writing to this file died or
// data node is going down
if (e.getClassName().equals(AlreadyBeingCreatedException.class.getName())
&& fs instanceof DistributedFileSystem) {
log.warn("Trying to recover log on path " + path);
@@ -120,21 +119,23 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
throws IOException, InterruptedException {
// Find current version
LogFormatVersion currentLogFormatVersion = new HoodieLogFormatVersion(HoodieLogFormat.currentVersion);
HoodieLogFormat.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());
byte[] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader());
// content bytes
byte [] content = block.getContentBytes();
byte[] content = block.getContentBytes();
// bytes for footer
byte [] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter());
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));
this.output
.writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length));
// 3. Write the version of this log block
this.output.writeInt(currentLogFormatVersion.getVersion());
@@ -149,7 +150,8 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
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)
// 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();
@@ -159,35 +161,27 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
}
/**
*
* 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
* 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
*/
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)
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; // bytes to write totalLogBlockLength at end of block (for reverse ptr)
}
private Writer rolloverIfNeeded() throws IOException, InterruptedException {
// Roll over if the size is past the threshold
if (getCurrentSize() > sizeThreshold) {
//TODO - make an end marker which seals the old log file (no more appends possible to that file).
//TODO - make an end marker which seals the old log file (no more appends possible to that
// file).
log.info("CurrentSize " + getCurrentSize() + " has reached threshold " + sizeThreshold
+ ". Rolling over to the next version");
HoodieLogFile newLogFile = logFile.rollOver(fs);

View File

@@ -44,12 +44,9 @@ import org.apache.avro.io.EncoderFactory;
import org.apache.hadoop.fs.FSDataInputStream;
/**
* DataBlock contains a list of records serialized using Avro.
* The Datablock contains
* 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
* DataBlock contains a list of records serialized using Avro. The Datablock contains 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 {
@@ -216,7 +213,7 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
deflate();
}
/*****************************************************DEPRECATED METHODS**********************************************/
/*********************************DEPRECATED METHODS***********************************/
@Deprecated
@VisibleForTesting
@@ -232,7 +229,8 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
@Deprecated
/**
* This method is retained to provide backwards compatibility to HoodieArchivedLogs which were written using HoodieLogFormat V1
* 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 {

View File

@@ -0,0 +1,37 @@
/*
* 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;
/**
* 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;
}
}
}

View File

@@ -17,11 +17,10 @@
package com.uber.hoodie.common.table.log.block;
import com.uber.hoodie.common.model.HoodieLogFile;
import org.apache.hadoop.fs.FSDataInputStream;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import org.apache.hadoop.fs.FSDataInputStream;
/**
* Command block issues a specific command to the scanner
@@ -30,7 +29,9 @@ public class HoodieCommandBlock extends HoodieLogBlock {
private final HoodieCommandBlockTypeEnum type;
public enum HoodieCommandBlockTypeEnum {ROLLBACK_PREVIOUS_BLOCK}
public enum HoodieCommandBlockTypeEnum {
ROLLBACK_PREVIOUS_BLOCK
}
public HoodieCommandBlock(Map<HeaderMetadataType, String> header) {
this(Optional.empty(), null, false, Optional.empty(), header, new HashMap<>());

View File

@@ -0,0 +1,28 @@
/*
* 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;
/**
* 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);
}
}

View File

@@ -17,11 +17,10 @@
package com.uber.hoodie.common.table.log.block;
import com.uber.hoodie.common.model.HoodieLogFile;
import org.apache.hadoop.fs.FSDataInputStream;
import java.io.IOException;
import java.util.Map;
import java.util.Optional;
import org.apache.hadoop.fs.FSDataInputStream;
/**
* Corrupt block is emitted whenever the scanner finds the length of the block written at the
@@ -61,6 +60,7 @@ public class HoodieCorruptBlock extends HoodieLogBlock {
Map<HeaderMetadataType, String> footer) throws IOException {
return new HoodieCorruptBlock(corruptedBytes, inputStream, readBlockLazily,
Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer);
Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)),
header, footer);
}
}

View File

@@ -19,9 +19,6 @@ 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;
@@ -31,6 +28,8 @@ import java.nio.charset.Charset;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.FSDataInputStream;
/**
* Delete block contains a list of keys to be deleted from scanning the blocks so far
@@ -65,7 +64,8 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
byte[] bytesToWrite = StringUtils.join(getKeysToDelete(), ',').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);
@@ -80,7 +80,8 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
inflate();
}
SizeAwareDataInputStream dis =
new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(getContent().get())));
new SizeAwareDataInputStream(
new DataInputStream(new ByteArrayInputStream(getContent().get())));
int version = dis.readInt();
int dataLength = dis.readInt();
byte[] data = new byte[dataLength];
@@ -110,6 +111,7 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
Map<HeaderMetadataType, String> footer) throws IOException {
return new HoodieDeleteBlock(content, inputStream, readBlockLazily,
Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer);
Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)),
header, footer);
}
}

View File

@@ -0,0 +1,28 @@
/*
* 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;
/**
* 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);
}
}

View File

@@ -36,10 +36,10 @@ import org.apache.hadoop.fs.FSDataInputStream;
public abstract class HoodieLogBlock {
/**
* 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
* 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
@@ -49,7 +49,7 @@ public abstract class HoodieLogBlock {
// Location of a log block on disk
private final Optional<HoodieLogBlockContentLocation> blockContentLocation;
// data for a specific block
private Optional<byte []> content;
private Optional<byte[]> 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
@@ -57,10 +57,10 @@ public abstract class HoodieLogBlock {
// Toggle flag, whether to read blocks lazily (I/O intensive) or not (Memory intensive)
protected boolean readBlockLazily;
public HoodieLogBlock(@Nonnull Map<HeaderMetadataType, String> logBlockHeader,
public HoodieLogBlock(@Nonnull Map<HeaderMetadataType, String> logBlockHeader,
@Nonnull Map<HeaderMetadataType, String> logBlockFooter,
@Nonnull Optional<HoodieLogBlockContentLocation> blockContentLocation,
@Nonnull Optional<byte []> content,
@Nonnull Optional<byte[]> content,
FSDataInputStream inputStream,
boolean readBlockLazily) {
this.logBlockHeader = logBlockHeader;
@@ -76,7 +76,7 @@ public abstract class HoodieLogBlock {
throw new HoodieException("No implementation was provided");
}
public byte [] getMagic() {
public byte[] getMagic() {
throw new HoodieException("No implementation was provided");
}
@@ -116,8 +116,8 @@ public abstract class HoodieLogBlock {
}
/**
* Log Metadata headers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal.
* Only add new enums at the end.
* Log Metadata headers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the
* ordinal. Only add new enums at the end.
*/
public enum HeaderMetadataType {
INSTANT_TIME,
@@ -127,17 +127,19 @@ public abstract class HoodieLogBlock {
}
/**
* Log Metadata footers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal.
* Only add new enums at the end.
* 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 {
}
/**
* 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
* 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
@@ -147,7 +149,8 @@ public abstract class HoodieLogBlock {
// The final position where the complete block ends
private final long blockEndPos;
HoodieLogBlockContentLocation(HoodieLogFile logFile, long contentPositionInLogFile, long blockSize, long blockEndPos) {
HoodieLogBlockContentLocation(HoodieLogFile logFile, long contentPositionInLogFile,
long blockSize, long blockEndPos) {
this.logFile = logFile;
this.contentPositionInLogFile = contentPositionInLogFile;
this.blockSize = blockSize;
@@ -190,7 +193,8 @@ public abstract class HoodieLogBlock {
}
/**
* Convert bytes to LogMetadata, follow the same order as {@link HoodieLogBlock#getLogMetadataBytes}
* Convert bytes to LogMetadata, follow the same order as
* {@link HoodieLogBlock#getLogMetadataBytes}
*/
public static Map<HeaderMetadataType, String> getLogMetadata(DataInputStream dis)
throws IOException {
@@ -216,16 +220,10 @@ public abstract class HoodieLogBlock {
/**
* 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,
public static byte[] readOrSkipContent(FSDataInputStream inputStream,
Integer contentLength, boolean readBlockLazily) throws IOException {
byte [] content = null;
byte[] content = null;
if (!readBlockLazily) {
// Read the contents in memory
content = new byte[contentLength];
@@ -239,7 +237,6 @@ public abstract class HoodieLogBlock {
/**
* When lazyReading of blocks is turned on, inflate the content of a log block from disk
* @throws IOException
*/
protected void inflate() throws IOException {
@@ -248,21 +245,21 @@ public abstract class HoodieLogBlock {
inputStream.seek(this.getBlockContentLocation().get().getContentPositionInLogFile());
inputStream.readFully(content.get(), 0, content.get().length);
inputStream.seek(this.getBlockContentLocation().get().getBlockEndPos());
} catch(IOException e) {
} 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) {
} 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
* 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();

View File

@@ -0,0 +1,33 @@
/*
* 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;
abstract class HoodieLogBlockVersion {
private final int currentVersion;
public static final int DEFAULT_VERSION = 0;
HoodieLogBlockVersion(int version) {
this.currentVersion = version;
}
int getVersion() {
return currentVersion;
}
}

View File

@@ -1,79 +0,0 @@
/*
* 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);
}
}

View File

@@ -51,9 +51,10 @@ import org.apache.log4j.Logger;
*/
public class HoodieActiveTimeline extends HoodieDefaultTimeline {
public static final FastDateFormat COMMIT_FORMATTER = FastDateFormat.getInstance("yyyyMMddHHmmss");
public static final FastDateFormat COMMIT_FORMATTER = FastDateFormat
.getInstance("yyyyMMddHHmmss");
private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
private static final transient Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
private HoodieTableMetaClient metaClient;
/**
@@ -71,12 +72,12 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
Arrays.stream(
HoodieTableMetaClient
.scanFiles(metaClient.getFs(), new Path(metaClient.getMetaPath()), path -> {
// Include only the meta files with extensions that needs to be included
String extension = FSUtils.getFileExtension(path.getName());
return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension));
})).sorted(Comparator.comparing(
// Sort the meta-data by the instant time (first part of the file name)
fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName())))
// Include only the meta files with extensions that needs to be included
String extension = FSUtils.getFileExtension(path.getName());
return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension));
})).sorted(Comparator.comparing(
// Sort the meta-data by the instant time (first part of the file name)
fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName())))
// create HoodieInstantMarkers from FileStatus, which extracts properties
.map(HoodieInstant::new).collect(Collectors.toList());
log.info("Loaded instants " + instants);
@@ -84,13 +85,15 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
throw new HoodieIOException("Failed to scan metadata", e);
}
this.metaClient = metaClient;
// multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
// multiple casts will make this lambda serializable -
// http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details =
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
}
public HoodieActiveTimeline(HoodieTableMetaClient metaClient) {
this(metaClient,
new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION,
CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION});
}
@@ -114,8 +117,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
/**
* Get all instants (commits, delta commits) that produce new data, in the active
* timeline *
* Get all instants (commits, delta commits) that produce new data, in the active timeline *
*/
public HoodieTimeline getCommitsTimeline() {
return getTimelineOfActions(
@@ -123,8 +125,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
/**
* Get all instants (commits, delta commits, clean, savepoint, rollback) that result
* in actions, in the active timeline *
* Get all instants (commits, delta commits, clean, savepoint, rollback) that result in actions,
* in the active timeline *
*/
public HoodieTimeline getAllCommitsTimeline() {
return getTimelineOfActions(

View File

@@ -46,7 +46,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
private HoodieTableMetaClient metaClient;
private Map<String, byte[]> readCommits = new HashMap<>();
private final transient static Logger log = LogManager.getLogger(HoodieArchivedTimeline.class);
private static final transient Logger log = LogManager.getLogger(HoodieArchivedTimeline.class);
public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) {
// Read back the commits to make sure
@@ -68,8 +68,10 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
throw new HoodieIOException(
"Could not load archived commit timeline from path " + archiveLogPath, e);
}
// multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
// multiple casts will make this lambda serializable -
// http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details =
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
this.metaClient = metaClient;
}

View File

@@ -34,7 +34,7 @@ import org.apache.log4j.Logger;
*/
public class HoodieDefaultTimeline implements HoodieTimeline {
private final transient static Logger log = LogManager.getLogger(HoodieDefaultTimeline.class);
private static final transient Logger log = LogManager.getLogger(HoodieDefaultTimeline.class);
protected Function<HoodieInstant, Optional<byte[]>> details;
protected List<HoodieInstant> instants;
@@ -65,9 +65,9 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
@Override
public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) {
return new HoodieDefaultTimeline(instants.stream().filter(
s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), startTs, GREATER) &&
HoodieTimeline.compareTimestamps(
s.getTimestamp(), endTs, LESSER_OR_EQUAL)), details);
s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), startTs, GREATER)
&& HoodieTimeline.compareTimestamps(
s.getTimestamp(), endTs, LESSER_OR_EQUAL)), details);
}
@Override
@@ -133,8 +133,8 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
@Override
public boolean isBeforeTimelineStarts(String instant) {
Optional<HoodieInstant> firstCommit = firstInstant();
return firstCommit.isPresent() &&
HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER);
return firstCommit.isPresent()
&& HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER);
}

View File

@@ -79,25 +79,20 @@ public class HoodieInstant implements Serializable {
*/
public String getFileName() {
if (HoodieTimeline.COMMIT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightCommitFileName(timestamp) :
HoodieTimeline.makeCommitFileName(timestamp);
return isInflight ? HoodieTimeline.makeInflightCommitFileName(timestamp)
: HoodieTimeline.makeCommitFileName(timestamp);
} else if (HoodieTimeline.CLEAN_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightCleanerFileName(timestamp) :
HoodieTimeline.makeCleanerFileName(timestamp);
return isInflight ? HoodieTimeline.makeInflightCleanerFileName(timestamp)
: HoodieTimeline.makeCleanerFileName(timestamp);
} else if (HoodieTimeline.ROLLBACK_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightRollbackFileName(timestamp) :
HoodieTimeline.makeRollbackFileName(timestamp);
return isInflight ? HoodieTimeline.makeInflightRollbackFileName(timestamp)
: HoodieTimeline.makeRollbackFileName(timestamp);
} else if (HoodieTimeline.SAVEPOINT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightSavePointFileName(timestamp) :
HoodieTimeline.makeSavePointFileName(timestamp);
return isInflight ? HoodieTimeline.makeInflightSavePointFileName(timestamp)
: HoodieTimeline.makeSavePointFileName(timestamp);
} else if (HoodieTimeline.DELTA_COMMIT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightDeltaFileName(timestamp) :
HoodieTimeline.makeDeltaFileName(timestamp);
return isInflight ? HoodieTimeline.makeInflightDeltaFileName(timestamp)
: HoodieTimeline.makeDeltaFileName(timestamp);
}
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
}
@@ -111,9 +106,9 @@ public class HoodieInstant implements Serializable {
return false;
}
HoodieInstant that = (HoodieInstant) o;
return isInflight == that.isInflight &&
Objects.equals(action, that.action) &&
Objects.equals(timestamp, that.timestamp);
return isInflight == that.isInflight
&& Objects.equals(action, that.action)
&& Objects.equals(timestamp, that.timestamp);
}
@Override

View File

@@ -55,209 +55,209 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
TableFileSystemView.ReadOptimizedView,
TableFileSystemView.RealtimeView, Serializable {
protected HoodieTableMetaClient metaClient;
// This is the commits that will be visible for all views extending this view
protected HoodieTimeline visibleActiveTimeline;
protected HoodieTableMetaClient metaClient;
// This is the commits that will be visible for all views extending this view
protected HoodieTimeline visibleActiveTimeline;
// mapping from partition paths to file groups contained within them
protected HashMap<String, List<HoodieFileGroup>> partitionToFileGroupsMap;
// mapping from file id to the file group.
protected HashMap<String, HoodieFileGroup> fileGroupMap;
// mapping from partition paths to file groups contained within them
protected HashMap<String, List<HoodieFileGroup>> partitionToFileGroupsMap;
// mapping from file id to the file group.
protected HashMap<String, HoodieFileGroup> fileGroupMap;
/**
* Create a file system view, as of the given timeline
*/
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline) {
this.metaClient = metaClient;
this.visibleActiveTimeline = visibleActiveTimeline;
this.fileGroupMap = new HashMap<>();
this.partitionToFileGroupsMap = new HashMap<>();
/**
* Create a file system view, as of the given timeline
*/
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline) {
this.metaClient = metaClient;
this.visibleActiveTimeline = visibleActiveTimeline;
this.fileGroupMap = new HashMap<>();
this.partitionToFileGroupsMap = new HashMap<>();
}
/**
* Create a file system view, as of the given timeline, with the provided file statuses.
*/
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline,
FileStatus[] fileStatuses) {
this(metaClient, visibleActiveTimeline);
addFilesToView(fileStatuses);
}
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
}
private void writeObject(java.io.ObjectOutputStream out)
throws IOException {
out.defaultWriteObject();
}
/**
* Adds the provided statuses into the file system view, and also caches it inside this object.
*/
private List<HoodieFileGroup> addFilesToView(FileStatus[] statuses) {
Map<Pair<String, String>, List<HoodieDataFile>> dataFiles = convertFileStatusesToDataFiles(
statuses)
.collect(Collectors.groupingBy((dataFile) -> {
String partitionPathStr = FSUtils.getRelativePartitionPath(
new Path(metaClient.getBasePath()),
dataFile.getFileStatus().getPath().getParent());
return Pair.of(partitionPathStr, dataFile.getFileId());
}));
Map<Pair<String, String>, List<HoodieLogFile>> logFiles = convertFileStatusesToLogFiles(
statuses)
.collect(Collectors.groupingBy((logFile) -> {
String partitionPathStr = FSUtils.getRelativePartitionPath(
new Path(metaClient.getBasePath()),
logFile.getPath().getParent());
return Pair.of(partitionPathStr, logFile.getFileId());
}));
Set<Pair<String, String>> fileIdSet = new HashSet<>(dataFiles.keySet());
fileIdSet.addAll(logFiles.keySet());
List<HoodieFileGroup> fileGroups = new ArrayList<>();
fileIdSet.forEach(pair -> {
HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), pair.getValue(),
visibleActiveTimeline);
if (dataFiles.containsKey(pair)) {
dataFiles.get(pair).forEach(dataFile -> group.addDataFile(dataFile));
}
if (logFiles.containsKey(pair)) {
logFiles.get(pair).forEach(logFile -> group.addLogFile(logFile));
}
fileGroups.add(group);
});
// add to the cache.
fileGroups.forEach(group -> {
fileGroupMap.put(group.getId(), group);
if (!partitionToFileGroupsMap.containsKey(group.getPartitionPath())) {
partitionToFileGroupsMap.put(group.getPartitionPath(), new ArrayList<>());
}
partitionToFileGroupsMap.get(group.getPartitionPath()).add(group);
});
return fileGroups;
}
private Stream<HoodieDataFile> convertFileStatusesToDataFiles(FileStatus[] statuses) {
Predicate<FileStatus> roFilePredicate = fileStatus ->
fileStatus.getPath().getName()
.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension());
return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new);
}
private Stream<HoodieLogFile> convertFileStatusesToLogFiles(FileStatus[] statuses) {
Predicate<FileStatus> rtFilePredicate = fileStatus ->
fileStatus.getPath().getName()
.contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension());
return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new);
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles(final String partitionPath) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestDataFile())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles() {
return fileGroupMap.values().stream()
.map(fileGroup -> fileGroup.getLatestDataFile())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
String maxCommitTime) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestDataFileBeforeOrOn(maxCommitTime))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
return fileGroupMap.values().stream()
.map(fileGroup -> fileGroup.getLatestDataFileInRange(commitsToReturn))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getAllDataFiles())
.flatMap(dataFileList -> dataFileList);
}
@Override
public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestFileSlice())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath,
String maxCommitTime) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
return fileGroupMap.values().stream()
.map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<FileSlice> getAllFileSlices(String partitionPath) {
return getAllFileGroups(partitionPath)
.map(group -> group.getAllFileSlices())
.flatMap(sliceList -> sliceList);
}
/**
* Given a partition path, obtain all filegroups within that. All methods, that work at the
* partition level go through this.
*/
@Override
public Stream<HoodieFileGroup> getAllFileGroups(String partitionPathStr) {
// return any previously fetched groups.
if (partitionToFileGroupsMap.containsKey(partitionPathStr)) {
return partitionToFileGroupsMap.get(partitionPathStr).stream();
}
/**
* Create a file system view, as of the given timeline, with the provided file statuses.
*/
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline,
FileStatus[] fileStatuses) {
this(metaClient, visibleActiveTimeline);
addFilesToView(fileStatuses);
}
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
}
private void writeObject(java.io.ObjectOutputStream out)
throws IOException {
out.defaultWriteObject();
}
/**
* Adds the provided statuses into the file system view, and also caches it inside this object.
*/
private List<HoodieFileGroup> addFilesToView(FileStatus[] statuses) {
Map<Pair<String, String>, List<HoodieDataFile>> dataFiles = convertFileStatusesToDataFiles(
statuses)
.collect(Collectors.groupingBy((dataFile) -> {
String partitionPathStr = FSUtils.getRelativePartitionPath(
new Path(metaClient.getBasePath()),
dataFile.getFileStatus().getPath().getParent());
return Pair.of(partitionPathStr, dataFile.getFileId());
}));
Map<Pair<String, String>, List<HoodieLogFile>> logFiles = convertFileStatusesToLogFiles(
statuses)
.collect(Collectors.groupingBy((logFile) -> {
String partitionPathStr = FSUtils.getRelativePartitionPath(
new Path(metaClient.getBasePath()),
logFile.getPath().getParent());
return Pair.of(partitionPathStr, logFile.getFileId());
}));
Set<Pair<String, String>> fileIdSet = new HashSet<>(dataFiles.keySet());
fileIdSet.addAll(logFiles.keySet());
List<HoodieFileGroup> fileGroups = new ArrayList<>();
fileIdSet.forEach(pair -> {
HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), pair.getValue(),
visibleActiveTimeline);
if (dataFiles.containsKey(pair)) {
dataFiles.get(pair).forEach(dataFile -> group.addDataFile(dataFile));
}
if (logFiles.containsKey(pair)) {
logFiles.get(pair).forEach(logFile -> group.addLogFile(logFile));
}
fileGroups.add(group);
});
// add to the cache.
fileGroups.forEach(group -> {
fileGroupMap.put(group.getId(), group);
if (!partitionToFileGroupsMap.containsKey(group.getPartitionPath())) {
partitionToFileGroupsMap.put(group.getPartitionPath(), new ArrayList<>());
}
partitionToFileGroupsMap.get(group.getPartitionPath()).add(group);
});
return fileGroups;
}
private Stream<HoodieDataFile> convertFileStatusesToDataFiles(FileStatus[] statuses) {
Predicate<FileStatus> roFilePredicate = fileStatus ->
fileStatus.getPath().getName()
.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension());
return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new);
}
private Stream<HoodieLogFile> convertFileStatusesToLogFiles(FileStatus[] statuses) {
Predicate<FileStatus> rtFilePredicate = fileStatus ->
fileStatus.getPath().getName()
.contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension());
return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new);
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles(final String partitionPath) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestDataFile())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles() {
return fileGroupMap.values().stream()
.map(fileGroup -> fileGroup.getLatestDataFile())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
String maxCommitTime) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestDataFileBeforeOrOn(maxCommitTime))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
return fileGroupMap.values().stream()
.map(fileGroup -> fileGroup.getLatestDataFileInRange(commitsToReturn))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getAllDataFiles())
.flatMap(dataFileList -> dataFileList);
}
@Override
public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestFileSlice())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath,
String maxCommitTime) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
return fileGroupMap.values().stream()
.map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<FileSlice> getAllFileSlices(String partitionPath) {
return getAllFileGroups(partitionPath)
.map(group -> group.getAllFileSlices())
.flatMap(sliceList -> sliceList);
}
/**
* Given a partition path, obtain all filegroups within that. All methods, that work at the
* partition level go through this.
*/
@Override
public Stream<HoodieFileGroup> getAllFileGroups(String partitionPathStr) {
// return any previously fetched groups.
if (partitionToFileGroupsMap.containsKey(partitionPathStr)) {
return partitionToFileGroupsMap.get(partitionPathStr).stream();
}
try {
// Create the path if it does not exist already
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath);
FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath);
List<HoodieFileGroup> fileGroups = addFilesToView(statuses);
return fileGroups.stream();
} catch (IOException e) {
throw new HoodieIOException(
"Failed to list data files in partition " + partitionPathStr, e);
}
try {
// Create the path if it does not exist already
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath);
FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath);
List<HoodieFileGroup> fileGroups = addFilesToView(statuses);
return fileGroups.stream();
} catch (IOException e) {
throw new HoodieIOException(
"Failed to list data files in partition " + partitionPathStr, e);
}
}
}

View File

@@ -94,8 +94,10 @@ public class FSUtils {
return String.format("%s_%d_%s.parquet", fileId, taskPartitionId, commitTime);
}
public static String makeTempDataFileName(String partitionPath, String commitTime, int taskPartitionId, String fileId, int stageId, long taskAttemptId) {
return String.format("%s_%s_%d_%s_%d_%d.parquet", partitionPath.replace("/", "-"), fileId, taskPartitionId, commitTime, stageId, taskAttemptId);
public static String makeTempDataFileName(String partitionPath, String commitTime,
int taskPartitionId, String fileId, int stageId, long taskAttemptId) {
return String.format("%s_%s_%d_%s_%d_%d.parquet", partitionPath.replace("/", "-"), fileId,
taskPartitionId, commitTime, stageId, taskAttemptId);
}
public static String maskWithoutFileId(String commitTime, int taskPartitionId) {
@@ -299,7 +301,8 @@ public class FSUtils {
Optional<Integer> currentVersion =
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
// handle potential overflow
return (currentVersion.isPresent()) ? currentVersion.get() + 1 : HoodieLogFile.LOGFILE_BASE_VERSION;
return (currentVersion.isPresent()) ? currentVersion.get() + 1
: HoodieLogFile.LOGFILE_BASE_VERSION;
}
public static int getDefaultBufferSize(final FileSystem fs) {
@@ -331,7 +334,8 @@ public class FSUtils {
if (recovered) {
break;
}
// Sleep for 1 second before trying again. Typically it takes about 2-3 seconds to recover under default settings
// Sleep for 1 second before trying again. Typically it takes about 2-3 seconds to recover
// under default settings
Thread.sleep(1000);
}
return recovered;

View File

@@ -45,11 +45,11 @@ import org.apache.avro.io.EncoderFactory;
public class HoodieAvroUtils {
// All metadata fields are optional strings.
private final static Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList(
private static final Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList(
Schema.create(Schema.Type.NULL),
Schema.create(Schema.Type.STRING)));
private final static Schema RECORD_KEY_SCHEMA = initRecordKeySchema();
private static final Schema RECORD_KEY_SCHEMA = initRecordKeySchema();
/**
* Convert a given avro record to bytes

View File

@@ -50,7 +50,7 @@ public class ParquetUtils {
/**
* Read the rowKey list from the given parquet file.
*
* @param filePath The parquet file path.
* @param filePath The parquet file path.
* @param configuration configuration to build fs object
*/
public static Set<String> readRowKeysFromParquet(Configuration configuration, Path filePath) {
@@ -116,8 +116,8 @@ public class ParquetUtils {
if (metadata.containsKey(footerName)) {
footerVals.add(metadata.get(footerName));
} else {
throw new MetadataNotFoundException("Could not find index in Parquet footer. " +
"Looked for key " + footerName + " in " + parquetFilePath);
throw new MetadataNotFoundException("Could not find index in Parquet footer. "
+ "Looked for key " + footerName + " in " + parquetFilePath);
}
}
return footerVals;
@@ -146,7 +146,7 @@ public class ParquetUtils {
"Could not read min/max record key out of footer correctly from %s. read) : %s",
parquetFilePath, minMaxKeys));
}
return new String[]{minMaxKeys.get(0), minMaxKeys.get(1)};
return new String[] {minMaxKeys.get(0), minMaxKeys.get(1)};
}
/**

View File

@@ -55,7 +55,8 @@ public class ReflectionUtils {
}
return (T) clazzCache.get(recordPayloadClass).getConstructor(constructorArgTypes)
.newInstance(payloadArgs);
} catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) {
} catch (InstantiationException | IllegalAccessException
| InvocationTargetException | NoSuchMethodException e) {
throw new HoodieException("Unable to instantiate payload class ", e);
} catch (ClassNotFoundException e) {
throw new HoodieException("Unable to instantiate payload class ", e);

View File

@@ -13,6 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.util;
import com.uber.hoodie.common.model.HoodieKey;
@@ -22,39 +23,28 @@ import com.uber.hoodie.common.util.collection.DiskBasedMap;
import com.uber.hoodie.common.util.collection.converter.Converter;
import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream;
import com.uber.hoodie.exception.HoodieCorruptedDataException;
import org.apache.avro.generic.GenericRecord;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.util.Optional;
import java.util.zip.CRC32;
import org.apache.avro.generic.GenericRecord;
public class SpillableMapUtils {
/**
* Using the schema and payload class, read and convert the bytes on disk to a HoodieRecord
*
* @param file
* @param valuePosition
* @param valueLength
* @return
* @throws IOException
*/
public static byte[] readBytesFromDisk(RandomAccessFile file, long valuePosition, int valueLength) throws IOException {
public static byte[] readBytesFromDisk(RandomAccessFile file, long valuePosition, int valueLength)
throws IOException {
DiskBasedMap.FileEntry fileEntry = readInternal(file, valuePosition, valueLength);
return fileEntry.getValue();
}
/**
* |crc|timestamp|sizeOfKey|SizeOfValue|key|value|
*
* @param file
* @param valuePosition
* @param valueLength
* @return
* @throws IOException
*/
private static DiskBasedMap.FileEntry readInternal(RandomAccessFile file, long valuePosition, int valueLength) throws IOException {
private static DiskBasedMap.FileEntry readInternal(RandomAccessFile file, long valuePosition,
int valueLength) throws IOException {
file.seek(valuePosition);
long crc = file.readLong();
long timestamp = file.readLong();
@@ -69,27 +59,23 @@ public class SpillableMapUtils {
file.read(value, 0, valueSize);
long crcOfReadValue = generateChecksum(value);
if (!(crc == crcOfReadValue)) {
throw new HoodieCorruptedDataException("checksum of payload written to external disk does not match, " +
"data may be corrupted");
throw new HoodieCorruptedDataException("checksum of payload written to external disk does not match, "
+ "data may be corrupted");
}
return new DiskBasedMap.FileEntry(crc, keySize, valueSize, key, value, timestamp);
}
/**
* Write Value and other metadata necessary to disk. Each entry has the following sequence of data
* <p>
* Write Value and other metadata necessary to disk. Each entry has the following sequence of data <p>
* |crc|timestamp|sizeOfKey|SizeOfValue|key|value|
*
* @param outputStream
* @param fileEntry
* @return
* @throws IOException
*/
public static long spillToDisk(SizeAwareDataOutputStream outputStream, DiskBasedMap.FileEntry fileEntry) throws IOException {
public static long spillToDisk(SizeAwareDataOutputStream outputStream,
DiskBasedMap.FileEntry fileEntry) throws IOException {
return spill(outputStream, fileEntry);
}
private static long spill(SizeAwareDataOutputStream outputStream, DiskBasedMap.FileEntry fileEntry)
private static long spill(SizeAwareDataOutputStream outputStream,
DiskBasedMap.FileEntry fileEntry)
throws IOException {
outputStream.writeLong(fileEntry.getCrc());
outputStream.writeLong(fileEntry.getTimestamp());
@@ -102,9 +88,6 @@ public class SpillableMapUtils {
/**
* Generate a checksum for a given set of bytes
*
* @param data
* @return
*/
public static long generateChecksum(byte[] data) {
CRC32 crc = new CRC32();
@@ -113,13 +96,8 @@ public class SpillableMapUtils {
}
/**
* Compute a bytes representation of the payload by serializing the contents
* This is used to estimate the size of the payload (either in memory or when written to disk)
*
* @param <R>
* @param value
* @return
* @throws IOException
* Compute a bytes representation of the payload by serializing the contents This is used to estimate the size of the
* payload (either in memory or when written to disk)
*/
public static <R> long computePayloadSize(R value, Converter<R> valueConverter) throws IOException {
return valueConverter.sizeEstimate(value);
@@ -127,12 +105,6 @@ public class SpillableMapUtils {
/**
* Utility method to convert bytes to HoodieRecord using schema and payload class
*
* @param rec
* @param payloadClazz
* @param <R>
* @return
* @throws IOException
*/
public static <R> R convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz) {
String recKey = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD)

View File

@@ -22,9 +22,6 @@ import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStre
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
@@ -40,18 +37,25 @@ import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* This class provides a disk spillable only map implementation. All of the data is
* currenly written to one file, without any rollover support. It uses the following :
* 1) An in-memory map that tracks the key-> latest ValueMetadata.
* 2) Current position in the file
* NOTE : Only String.class type supported for Key
* This class provides a disk spillable only map implementation. All of the data is currenly written to one file,
* without any rollover support. It uses the following : 1) An in-memory map that tracks the key-> latest ValueMetadata.
* 2) Current position in the file NOTE : Only String.class type supported for Key
*/
final public class DiskBasedMap<T, R> implements Map<T, R> {
public final class DiskBasedMap<T, R> implements Map<T, R> {
private static final Logger log = LogManager.getLogger(DiskBasedMap.class);
// Default file path prefix to put the spillable file
private static String DEFAULT_BASE_FILE_PATH = "/tmp/";
// Stores the key and corresponding value's latest metadata spilled to disk
final private Map<T, ValueMetadata> valueMetadataMap;
private final Map<T, ValueMetadata> valueMetadataMap;
// Key converter to convert key type to bytes
private final Converter<T> keyConverter;
// Value converter to convert value type to bytes
private final Converter<R> valueConverter;
// Read only file access to be able to seek to random positions to readFromDisk values
private RandomAccessFile readOnlyFileHandle;
// Write only OutputStream to be able to ONLY append to the file
@@ -63,100 +67,7 @@ final public class DiskBasedMap<T, R> implements Map<T, R> {
private AtomicLong filePosition;
// FilePath to store the spilled data
private String filePath;
// Default file path prefix to put the spillable file
private static String DEFAULT_BASE_FILE_PATH = "/tmp/";
// Key converter to convert key type to bytes
final private Converter<T> keyConverter;
// Value converter to convert value type to bytes
final private Converter<R> valueConverter;
private static Logger log = LogManager.getLogger(DiskBasedMap.class);
public final class ValueMetadata {
// FilePath to store the spilled data
private String filePath;
// Size (numberOfBytes) of the value written to disk
private Integer sizeOfValue;
// FilePosition of the value written to disk
private Long offsetOfValue;
// Current timestamp when the value was written to disk
private Long timestamp;
protected ValueMetadata(String filePath, int sizeOfValue, long offsetOfValue, long timestamp) {
this.filePath = filePath;
this.sizeOfValue = sizeOfValue;
this.offsetOfValue = offsetOfValue;
this.timestamp = timestamp;
}
public String getFilePath() {
return filePath;
}
public int getSizeOfValue() {
return sizeOfValue;
}
public Long getOffsetOfValue() {
return offsetOfValue;
}
public long getTimestamp() {
return timestamp;
}
}
public static final class FileEntry {
// Checksum of the value written to disk, compared during every readFromDisk to make sure no corruption
private Long crc;
// Size (numberOfBytes) of the key written to disk
private Integer sizeOfKey;
// Size (numberOfBytes) of the value written to disk
private Integer sizeOfValue;
// Actual key
private byte[] key;
// Actual value
private byte[] value;
// Current timestamp when the value was written to disk
private Long timestamp;
public FileEntry(long crc, int sizeOfKey, int sizeOfValue, byte[] key, byte[] value,
long timestamp) {
this.crc = crc;
this.sizeOfKey = sizeOfKey;
this.sizeOfValue = sizeOfValue;
this.key = key;
this.value = value;
this.timestamp = timestamp;
}
public long getCrc() {
return crc;
}
public int getSizeOfKey() {
return sizeOfKey;
}
public int getSizeOfValue() {
return sizeOfValue;
}
public byte[] getKey() {
return key;
}
public byte[] getValue() {
return value;
}
public long getTimestamp() {
return timestamp;
}
}
protected DiskBasedMap(Optional<String> baseFilePath,
Converter<T> keyConverter, Converter<R> valueConverter) throws IOException {
@@ -193,8 +104,8 @@ final public class DiskBasedMap<T, R> implements Map<T, R> {
}
/**
* Register shutdown hook to force flush contents of the data written to FileOutputStream
* from OS page cache (typically 4 KB) to disk
* Register shutdown hook to force flush contents of the data written to FileOutputStream from OS page cache
* (typically 4 KB) to disk
*/
private void addShutDownHook() {
Runtime.getRuntime().addShutdownHook(new Thread() {
@@ -324,4 +235,89 @@ final public class DiskBasedMap<T, R> implements Map<T, R> {
}
return entrySet;
}
public static final class FileEntry {
// Checksum of the value written to disk, compared during every readFromDisk to make sure no corruption
private Long crc;
// Size (numberOfBytes) of the key written to disk
private Integer sizeOfKey;
// Size (numberOfBytes) of the value written to disk
private Integer sizeOfValue;
// Actual key
private byte[] key;
// Actual value
private byte[] value;
// Current timestamp when the value was written to disk
private Long timestamp;
public FileEntry(long crc, int sizeOfKey, int sizeOfValue, byte[] key, byte[] value,
long timestamp) {
this.crc = crc;
this.sizeOfKey = sizeOfKey;
this.sizeOfValue = sizeOfValue;
this.key = key;
this.value = value;
this.timestamp = timestamp;
}
public long getCrc() {
return crc;
}
public int getSizeOfKey() {
return sizeOfKey;
}
public int getSizeOfValue() {
return sizeOfValue;
}
public byte[] getKey() {
return key;
}
public byte[] getValue() {
return value;
}
public long getTimestamp() {
return timestamp;
}
}
public final class ValueMetadata {
// FilePath to store the spilled data
private String filePath;
// Size (numberOfBytes) of the value written to disk
private Integer sizeOfValue;
// FilePosition of the value written to disk
private Long offsetOfValue;
// Current timestamp when the value was written to disk
private Long timestamp;
protected ValueMetadata(String filePath, int sizeOfValue, long offsetOfValue, long timestamp) {
this.filePath = filePath;
this.sizeOfValue = sizeOfValue;
this.offsetOfValue = offsetOfValue;
this.timestamp = timestamp;
}
public String getFilePath() {
return filePath;
}
public int getSizeOfValue() {
return sizeOfValue;
}
public Long getOffsetOfValue() {
return offsetOfValue;
}
public long getTimestamp() {
return timestamp;
}
}
}

View File

@@ -18,11 +18,7 @@ package com.uber.hoodie.common.util.collection;
import com.twitter.common.objectsize.ObjectSizeCalculator;
import com.uber.hoodie.common.util.collection.converter.Converter;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
@@ -31,43 +27,43 @@ import java.util.Iterator;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* An external map that spills content to disk when there is insufficient space for it to grow. <p>
* This map holds 2 types of data structures : <p> (1) Key-Value pairs in a in-memory map (2)
* Key-ValueMetadata pairs in an in-memory map which keeps a marker to the values spilled to disk
* <p> NOTE : Values are only appended to disk. If a remove() is called, the entry is marked removed
* from the in-memory key-valueMetadata map but it's values will be lying around in the temp file on
* disk until the file is cleaned. <p> The setting of the spill threshold faces the following
* trade-off: If the spill threshold is too high, the in-memory map may occupy more memory than is
* available, resulting in OOM. However, if the spill threshold is too low, we spill frequently and
* incur unnecessary disk writes.
* An external map that spills content to disk when there is insufficient space for it to grow. <p> This map holds 2
* types of data structures : <p> (1) Key-Value pairs in a in-memory map (2) Key-ValueMetadata pairs in an in-memory map
* which keeps a marker to the values spilled to disk <p> NOTE : Values are only appended to disk. If a remove() is
* called, the entry is marked removed from the in-memory key-valueMetadata map but it's values will be lying around in
* the temp file on disk until the file is cleaned. <p> The setting of the spill threshold faces the following
* trade-off: If the spill threshold is too high, the in-memory map may occupy more memory than is available, resulting
* in OOM. However, if the spill threshold is too low, we spill frequently and incur unnecessary disk writes.
*/
public class ExternalSpillableMap<T, R> implements Map<T, R> {
// Find the actual estimated payload size after inserting N records
final private static int NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE = 100;
private static final int NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE = 100;
private static final Logger log = LogManager.getLogger(ExternalSpillableMap.class);
// maximum space allowed in-memory for this map
final private long maxInMemorySizeInBytes;
private final long maxInMemorySizeInBytes;
// Map to store key-values in memory until it hits maxInMemorySizeInBytes
private final Map<T, R> inMemoryMap;
// Map to store key-valuemetadata important to find the values spilled to disk
private final DiskBasedMap<T, R> diskBasedMap;
// TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and
// incorrect payload estimation
private final Double sizingFactorForInMemoryMap = 0.8;
// Key converter to convert key type to bytes
private final Converter<T> keyConverter;
// Value converter to convert value type to bytes
private final Converter<R> valueConverter;
// current space occupied by this map in-memory
private Long currentInMemoryMapSize;
// Map to store key-values in memory until it hits maxInMemorySizeInBytes
final private Map<T, R> inMemoryMap;
// Map to store key-valuemetadata important to find the values spilled to disk
final private DiskBasedMap<T, R> diskBasedMap;
// An estimate of the size of each payload written to this map
private volatile long estimatedPayloadSize = 0;
// TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and incorrect payload estimation
final private Double sizingFactorForInMemoryMap = 0.8;
// Key converter to convert key type to bytes
final private Converter<T> keyConverter;
// Value converter to convert value type to bytes
final private Converter<R> valueConverter;
// Flag to determine whether to stop re-estimating payload size
private boolean shouldEstimatePayloadSize = true;
private static Logger log = LogManager.getLogger(ExternalSpillableMap.class);
public ExternalSpillableMap(Long maxInMemorySizeInBytes, Optional<String> baseFilePath,
Converter<T> keyConverter, Converter<R> valueConverter) throws IOException {
this.inMemoryMap = new HashMap<>();
@@ -153,15 +149,14 @@ public class ExternalSpillableMap<T, R> implements Map<T, R> {
this.estimatedPayloadSize =
keyConverter.sizeEstimate(key) + valueConverter.sizeEstimate(value);
log.info("Estimated Payload size => " + estimatedPayloadSize);
}
else if(shouldEstimatePayloadSize &&
inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) {
} else if (shouldEstimatePayloadSize
&& inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) {
// Re-estimate the size of a record by calculating the size of the entire map containing
// N entries and then dividing by the number of entries present (N). This helps to get a
// correct estimation of the size of each record in the JVM.
long totalMapSize = ObjectSizeCalculator.getObjectSize(inMemoryMap);
this.currentInMemoryMapSize = totalMapSize;
this.estimatedPayloadSize = totalMapSize/inMemoryMap.size();
this.estimatedPayloadSize = totalMapSize / inMemoryMap.size();
shouldEstimatePayloadSize = false;
log.info("New Estimated Payload size => " + this.estimatedPayloadSize);
}
@@ -227,9 +222,8 @@ public class ExternalSpillableMap<T, R> implements Map<T, R> {
}
/**
* Iterator that wraps iterating over all the values for this map
* 1) inMemoryIterator - Iterates over all the data in-memory map
* 2) diskLazyFileIterator - Iterates over all the data spilled to disk
* Iterator that wraps iterating over all the values for this map 1) inMemoryIterator - Iterates over all the data
* in-memory map 2) diskLazyFileIterator - Iterates over all the data spilled to disk
*/
private class IteratorWrapper<R> implements Iterator<R> {

View File

@@ -20,7 +20,6 @@ import com.uber.hoodie.common.util.SpillableMapUtils;
import com.uber.hoodie.common.util.collection.converter.Converter;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.util.Iterator;
@@ -29,8 +28,7 @@ import java.util.function.Consumer;
import java.util.stream.Collectors;
/**
* Iterable to lazily fetch values spilled to disk.
* This class uses RandomAccessFile to randomly access the position of
* Iterable to lazily fetch values spilled to disk. This class uses RandomAccessFile to randomly access the position of
* the latest value for a key spilled to disk and returns the result.
*/
public class LazyFileIterable<T, R> implements Iterable<R> {
@@ -63,9 +61,9 @@ public class LazyFileIterable<T, R> implements Iterable<R> {
*/
public class LazyFileIterator<T, R> implements Iterator<R> {
private final Converter<R> valueConverter;
private RandomAccessFile readOnlyFileHandle;
private Iterator<Map.Entry<T, DiskBasedMap.ValueMetadata>> metadataIterator;
private final Converter<R> valueConverter;
public LazyFileIterator(RandomAccessFile file, Map<T, DiskBasedMap.ValueMetadata> map,
Converter<R> valueConverter) throws IOException {

View File

@@ -22,14 +22,9 @@ import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieNotSerializableException;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.Field;
import java.util.Arrays;
import java.util.Optional;
import java.util.stream.Stream;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.lang3.SerializationUtils;

View File

@@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicLong;
* Wrapper for DataOutpuStream to keep track of number of bytes written
*/
public class SizeAwareDataOutputStream {
// Actual outpuStream
private DataOutputStream outputStream;
// Counter to keep track of number of bytes written
@@ -45,12 +46,12 @@ public class SizeAwareDataOutputStream {
outputStream.writeInt(v);
}
public void write(byte [] v) throws IOException {
public void write(byte[] v) throws IOException {
size.addAndGet(v.length);
outputStream.write(v);
}
public void write(byte [] v, int offset, int len) throws IOException {
public void write(byte[] v, int offset, int len) throws IOException {
size.addAndGet(len + offset);
outputStream.write(v, offset, len);
}

View File

@@ -17,7 +17,8 @@
package com.uber.hoodie.exception;
/**
* <p> Exception thrown when any data corruption happens when reading/writing from temporary disk </p>
* <p> Exception thrown when any data corruption happens when reading/writing from temporary disk
* </p>
*/
public class HoodieCorruptedDataException extends HoodieException {