-
Notifications
You must be signed in to change notification settings - Fork 2.5k
Implement HoodieLogFormat replacing Avro as the default log format #162
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -1,2 +1,4 @@ | ||
| language: java | ||
| jdk: | ||
| - oraclejdk8 | ||
| sudo: required |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
207 changes: 207 additions & 0 deletions
207
...ommon/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,207 @@ | ||
| /* | ||
| * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) | ||
| * | ||
| * Licensed under the Apache License, Version 2.0 (the "License"); | ||
| * you may not use this file except in compliance with the License. | ||
| * You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package com.uber.hoodie.common.table.log; | ||
|
|
||
| import com.google.common.collect.Maps; | ||
| import com.uber.hoodie.common.model.HoodieAvroPayload; | ||
| import com.uber.hoodie.common.model.HoodieKey; | ||
| import com.uber.hoodie.common.model.HoodieRecord; | ||
| import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; | ||
| import com.uber.hoodie.common.table.log.block.HoodieCommandBlock; | ||
| import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum; | ||
| import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; | ||
| import com.uber.hoodie.exception.HoodieIOException; | ||
| import java.io.IOException; | ||
| import java.util.Arrays; | ||
| import java.util.Collection; | ||
| import java.util.Collections; | ||
| import java.util.Iterator; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Optional; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
| 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. | ||
| * | ||
| * TODO(FIX) - Does not apply application specific merge logic - defaults to HoodieAvroPayload | ||
| */ | ||
| public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<HoodieAvroPayload>> { | ||
| private final static Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class); | ||
|
|
||
| // Final list of compacted/merged records to iterate | ||
| private final Collection<HoodieRecord<HoodieAvroPayload>> logRecords; | ||
| // Reader schema for the records | ||
| private final Schema readerSchema; | ||
| // Total log files read - for metrics | ||
| private AtomicLong totalLogFiles = new AtomicLong(0); | ||
| // Total log records read - for metrics | ||
| private AtomicLong totalLogRecords = new AtomicLong(0); | ||
| // Total final list of compacted/merged records | ||
| private long totalRecordsToUpdate; | ||
|
|
||
| public HoodieCompactedLogRecordScanner(FileSystem fs, List<String> logFilePaths, | ||
| Schema readerSchema) { | ||
| this.readerSchema = readerSchema; | ||
|
|
||
| Map<String, HoodieRecord<HoodieAvroPayload>> records = Maps.newHashMap(); | ||
| // iterate over the paths | ||
| logFilePaths.stream().map(s -> new HoodieLogFile(new Path(s))).forEach(s -> { | ||
| log.info("Scanning log file " + s.getPath()); | ||
| totalLogFiles.incrementAndGet(); | ||
| try { | ||
| // Use the HoodieLogFormatReader to iterate through the blocks in the log file | ||
| HoodieLogFormatReader reader = new HoodieLogFormatReader(fs, s, readerSchema); | ||
| // Store the records loaded from the last data block (needed to implement rollback) | ||
| Map<String, HoodieRecord<HoodieAvroPayload>> recordsFromLastBlock = Maps.newHashMap(); | ||
| reader.forEachRemaining(r -> { | ||
| switch (r.getBlockType()) { | ||
| case AVRO_DATA_BLOCK: | ||
| log.info("Reading a data block from file " + s.getPath()); | ||
| // If this is a avro data block, then merge the last block records into the main result | ||
| merge(records, recordsFromLastBlock); | ||
| // Load the merged records into recordsFromLastBlock | ||
| HoodieAvroDataBlock dataBlock = (HoodieAvroDataBlock) r; | ||
| loadRecordsFromBlock(dataBlock, recordsFromLastBlock); | ||
| break; | ||
| case DELETE_BLOCK: | ||
| log.info("Reading a delete block from file " + s.getPath()); | ||
| // This is a delete block, so lets merge any records from previous data block | ||
| merge(records, recordsFromLastBlock); | ||
| // Delete the keys listed as to be deleted | ||
| HoodieDeleteBlock deleteBlock = (HoodieDeleteBlock) r; | ||
| Arrays.stream(deleteBlock.getKeysToDelete()).forEach(records::remove); | ||
| break; | ||
| case COMMAND_BLOCK: | ||
| log.info("Reading a command block from file " + s.getPath()); | ||
| // This is a command block - take appropriate action based on the command | ||
| HoodieCommandBlock commandBlock = (HoodieCommandBlock) r; | ||
| if (commandBlock.getType() == HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK) { | ||
| log.info("Rolling back the last data block read in " + s.getPath()); | ||
| // rollback the last read data block | ||
| recordsFromLastBlock.clear(); | ||
| } | ||
| break; | ||
| case CORRUPT_BLOCK: | ||
| log.info("Found a corrupt block in " + s.getPath()); | ||
| // If there is a corrupt block - we will assume that this was the next data block | ||
| // so merge the last block records (TODO - handle when the corrupted block was a tombstone written partially?) | ||
| merge(records, recordsFromLastBlock); | ||
| recordsFromLastBlock.clear(); | ||
| break; | ||
| } | ||
| }); | ||
|
|
||
| // merge the last read block when all the blocks are done reading | ||
| if (!recordsFromLastBlock.isEmpty()) { | ||
| log.info("Merging the final data block in " + s.getPath()); | ||
| merge(records, recordsFromLastBlock); | ||
| } | ||
|
|
||
| } catch (IOException e) { | ||
| throw new HoodieIOException("IOException when reading log file " + s); | ||
| } | ||
| }); | ||
| this.logRecords = Collections.unmodifiableCollection(records.values()); | ||
| this.totalRecordsToUpdate = records.size(); | ||
| } | ||
|
|
||
| /** | ||
| * Iterate over the GenericRecord in the block, read the hoodie key and partition path | ||
| * and merge with the HoodieAvroPayload if the same key was found before | ||
| * | ||
| * @param dataBlock | ||
| * @param recordsFromLastBlock | ||
| */ | ||
| private void loadRecordsFromBlock( | ||
| HoodieAvroDataBlock dataBlock, | ||
| Map<String, HoodieRecord<HoodieAvroPayload>> recordsFromLastBlock) { | ||
| recordsFromLastBlock.clear(); | ||
| List<IndexedRecord> recs = dataBlock.getRecords(); | ||
| totalLogRecords.addAndGet(recs.size()); | ||
| recs.forEach(rec -> { | ||
| String key = ((GenericRecord) rec).get(HoodieRecord.RECORD_KEY_METADATA_FIELD) | ||
| .toString(); | ||
| String partitionPath = | ||
| ((GenericRecord) rec).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD) | ||
| .toString(); | ||
| HoodieRecord<HoodieAvroPayload> hoodieRecord = new HoodieRecord<>( | ||
| new HoodieKey(key, partitionPath), | ||
| new HoodieAvroPayload(Optional.of(((GenericRecord) rec)))); | ||
| if (recordsFromLastBlock.containsKey(key)) { | ||
| // Merge and store the merged record | ||
| HoodieAvroPayload combinedValue = recordsFromLastBlock.get(key).getData() | ||
| .preCombine(hoodieRecord.getData()); | ||
| recordsFromLastBlock | ||
| .put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), | ||
| combinedValue)); | ||
| } else { | ||
| // Put the record as is | ||
| recordsFromLastBlock.put(key, hoodieRecord); | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| /** | ||
| * Merge the records read from a single data block with the accumulated records | ||
| * | ||
| * @param records | ||
| * @param recordsFromLastBlock | ||
| */ | ||
| private void merge(Map<String, HoodieRecord<HoodieAvroPayload>> records, | ||
| Map<String, HoodieRecord<HoodieAvroPayload>> recordsFromLastBlock) { | ||
| recordsFromLastBlock.forEach((key, hoodieRecord) -> { | ||
| if (records.containsKey(key)) { | ||
| // Merge and store the merged record | ||
| HoodieAvroPayload combinedValue = records.get(key).getData() | ||
| .preCombine(hoodieRecord.getData()); | ||
| records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), | ||
| combinedValue)); | ||
| } else { | ||
| // Put the record as is | ||
| records.put(key, hoodieRecord); | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| @Override | ||
| public Iterator<HoodieRecord<HoodieAvroPayload>> iterator() { | ||
| return logRecords.iterator(); | ||
| } | ||
|
|
||
| public long getTotalLogFiles() { | ||
| return totalLogFiles.get(); | ||
| } | ||
|
|
||
| public long getTotalLogRecords() { | ||
| return totalLogRecords.get(); | ||
| } | ||
|
|
||
| public long getTotalRecordsToUpdate() { | ||
| return totalRecordsToUpdate; | ||
| } | ||
| } | ||
|
|
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
may be just
LOGinstead ofHOODIE_LOG?