Skip to content

Commit

Permalink
[HUDI-6539] New LSM tree style archived timeline
Browse files Browse the repository at this point in the history
  • Loading branch information
danny0405 committed Jul 21, 2023
1 parent f2fdf8a commit a71bc4b
Show file tree
Hide file tree
Showing 22 changed files with 1,397 additions and 704 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -174,8 +174,8 @@ public String compactionShowArchived(
HoodieTimeline.COMPACTION_ACTION, compactionInstantTime);
try {
archivedTimeline.loadCompactionDetailsInMemory(compactionInstantTime);
HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeAvroRecordMetadata(
archivedTimeline.getInstantDetails(instant).get(), HoodieCompactionPlan.getClassSchema());
HoodieCompactionPlan compactionPlan =
TimelineMetadataUtils.deserializeCompactionPlan(archivedTimeline.getInstantDetails(instant).get());
return printCompaction(compactionPlan, sortByField, descending, limit, headerOnly, partition);
} finally {
archivedTimeline.clearInstantDetailsFromMemory(compactionInstantTime);
Expand Down Expand Up @@ -365,17 +365,10 @@ Function<HoodieInstant, HoodieCompactionPlan> compactionPlanReader(

private HoodieCompactionPlan readCompactionPlanForArchivedTimeline(HoodieArchivedTimeline archivedTimeline,
HoodieInstant instant) {
// filter inflight compaction
if (HoodieTimeline.COMPACTION_ACTION.equals(instant.getAction())
&& HoodieInstant.State.INFLIGHT.equals(instant.getState())) {
try {
return TimelineMetadataUtils.deserializeAvroRecordMetadata(archivedTimeline.getInstantDetails(instant).get(),
HoodieCompactionPlan.getClassSchema());
} catch (Exception e) {
throw new HoodieException(e.getMessage(), e);
}
} else {
return null;
try {
return TimelineMetadataUtils.deserializeCompactionPlan(archivedTimeline.getInstantDetails(instant).get());
} catch (Exception e) {
throw new HoodieException(e.getMessage(), e);
}
}

Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
@@ -0,0 +1,258 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hudi.client;

import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
import org.apache.hudi.avro.model.HoodieMergeArchiveFilePlan;
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.collection.ClosableIterator;
import org.apache.hudi.exception.HoodieIOException;

import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nonnull;

import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.Spliterator;
import java.util.Spliterators;
import java.util.function.Function;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.StreamSupport;

/**
* Tools used for migrating to new LSM tree style archived timeline.
*/
public class LegacyArchivedMetaEntryReader {
private static final Logger LOG = LoggerFactory.getLogger(LegacyArchivedMetaEntryReader.class);

private static final Pattern ARCHIVE_FILE_PATTERN =
Pattern.compile("^\\.commits_\\.archive\\.([0-9]+).*");

public static final String MERGE_ARCHIVE_PLAN_NAME = "mergeArchivePlan";

private static final String ACTION_TYPE_KEY = "actionType";
private static final String ACTION_STATE = "actionState";
private static final String STATE_TRANSITION_TIME = "stateTransitionTime";

private final HoodieTableMetaClient metaClient;

private final Map<String, byte[]> readCommits;

public LegacyArchivedMetaEntryReader(HoodieTableMetaClient metaClient) {
this.metaClient = metaClient;
this.readCommits = new HashMap<>();
}

/**
* @deprecated Use {@code HoodieArchivedTimeline#readCommit} instead.
*/
private HoodieInstant readCommit(GenericRecord record, boolean loadDetails) {
final String instantTime = record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString();
final String action = record.get(ACTION_TYPE_KEY).toString();
final String stateTransitionTime = (String) record.get(STATE_TRANSITION_TIME);
if (loadDetails) {
getMetadataKey(action).map(key -> {
Object actionData = record.get(key);
if (actionData != null) {
if (action.equals(HoodieTimeline.COMPACTION_ACTION)) {
this.readCommits.put(instantTime, HoodieAvroUtils.indexedRecordToBytes((IndexedRecord) actionData));
} else {
this.readCommits.put(instantTime, actionData.toString().getBytes(StandardCharsets.UTF_8));
}
}
return null;
});
}
return new HoodieInstant(HoodieInstant.State.valueOf(record.get(ACTION_STATE).toString()), action,
instantTime, stateTransitionTime);
}

@Nonnull
private Option<String> getMetadataKey(String action) {
switch (action) {
case HoodieTimeline.CLEAN_ACTION:
return Option.of("hoodieCleanMetadata");
case HoodieTimeline.COMMIT_ACTION:
case HoodieTimeline.DELTA_COMMIT_ACTION:
return Option.of("hoodieCommitMetadata");
case HoodieTimeline.ROLLBACK_ACTION:
return Option.of("hoodieRollbackMetadata");
case HoodieTimeline.SAVEPOINT_ACTION:
return Option.of("hoodieSavePointMetadata");
case HoodieTimeline.COMPACTION_ACTION:
case HoodieTimeline.LOG_COMPACTION_ACTION:
return Option.of("hoodieCompactionPlan");
case HoodieTimeline.REPLACE_COMMIT_ACTION:
return Option.of("hoodieReplaceCommitMetadata");
case HoodieTimeline.INDEXING_ACTION:
return Option.of("hoodieIndexCommitMetadata");
default:
LOG.error(String.format("Unknown action in metadata (%s)", action));
return Option.empty();
}
}

/**
* This is method to read selected instants. Do NOT use this directly use one of the helper methods above
* If loadInstantDetails is set to true, this would also update 'readCommits' map with commit details
* If filter is specified, only the filtered instants are loaded
* If commitsFilter is specified, only the filtered records are loaded
*
* @deprecated Use {@code HoodieArchivedTimeline.loadInstantsV2} instead.
*/
private List<HoodieInstant> loadInstants(
HoodieArchivedTimeline.TimeRangeFilter filter,
boolean loadInstantDetails,
Function<GenericRecord, Boolean> commitsFilter) {
try {
// List all files
FileStatus[] fsStatuses = metaClient.getFs().globStatus(
new Path(metaClient.getArchivePath() + "/.commits_.archive*"));

// Sort files by version suffix in reverse (implies reverse chronological order)
Arrays.sort(fsStatuses, new ArchiveLogVersionComparator());

Set<HoodieInstant> instantsInRange = new HashSet<>();
for (FileStatus fs : fsStatuses) {
// Read the archived file
try (HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(metaClient.getFs(),
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema())) {
int instantsInPreviousFile = instantsInRange.size();
// Read the avro blocks
while (reader.hasNext()) {
HoodieLogBlock block = reader.next();
if (block instanceof HoodieAvroDataBlock) {
HoodieAvroDataBlock avroBlock = (HoodieAvroDataBlock) block;
// TODO If we can store additional metadata in datablock, we can skip parsing records
// (such as startTime, endTime of records in the block)
try (ClosableIterator<HoodieRecord<IndexedRecord>> itr = avroBlock.getRecordIterator(HoodieRecord.HoodieRecordType.AVRO)) {
StreamSupport.stream(Spliterators.spliteratorUnknownSize(itr, Spliterator.IMMUTABLE), true)
// Filter blocks in desired time window
.map(r -> (GenericRecord) r.getData())
.filter(commitsFilter::apply)
.map(r -> readCommit(r, loadInstantDetails))
.filter(c -> filter == null || filter.isInRange(c.getTimestamp()))
.forEach(instantsInRange::add);
}
}
}

if (filter != null) {
int instantsInCurrentFile = instantsInRange.size() - instantsInPreviousFile;
if (instantsInPreviousFile > 0 && instantsInCurrentFile == 0) {
// Note that this is an optimization to skip reading unnecessary archived files
// This signals we crossed lower bound of desired time window.
break;
}
}
} catch (Exception oriException) {
// merge small archive files may leave uncompleted archive file which will cause exception.
// need to ignore this kind of exception here.
try {
if (isCorruptCompactionFile(metaClient, fs)) {
continue;
}
throw oriException;
} catch (Exception e) {
// If anything wrong during parsing merge archive plan, we need to throw the original exception.
// For example corrupted archive file and corrupted plan are both existed.
throw oriException;
}
}
}

ArrayList<HoodieInstant> result = new ArrayList<>(instantsInRange);
Collections.sort(result);
return result;
} catch (IOException e) {
throw new HoodieIOException(
"Could not load archived commit timeline from path " + metaClient.getArchivePath(), e);
}
}

public static boolean isCorruptCompactionFile(HoodieTableMetaClient metaClient, FileStatus fileStatus) throws IOException {
Path planPath = new Path(metaClient.getArchivePath(), MERGE_ARCHIVE_PLAN_NAME);
HoodieWrapperFileSystem fileSystem = metaClient.getFs();
if (fileSystem.exists(planPath)) {
HoodieMergeArchiveFilePlan plan = TimelineMetadataUtils.deserializeAvroMetadata(FileIOUtils.readDataFromPath(fileSystem, planPath).get(), HoodieMergeArchiveFilePlan.class);
String mergedArchiveFileName = plan.getMergedArchiveFileName();
if (!StringUtils.isNullOrEmpty(mergedArchiveFileName) && fileStatus.getPath().getName().equalsIgnoreCase(mergedArchiveFileName)) {
LOG.warn("Catch exception because of reading uncompleted merging archive file " + mergedArchiveFileName + ". Ignore it here.");
return true;
}
}
return false;
}

/**
* Sort files by reverse order of version suffix in file name.
*/
public static class ArchiveLogVersionComparator implements Comparator<FileStatus>, Serializable {
@Override
public int compare(FileStatus f1, FileStatus f2) {
return Integer.compare(getArchivedFileSuffix(f2), getArchivedFileSuffix(f1));
}
}

private static int getArchivedFileSuffix(FileStatus f) {
try {
Matcher fileMatcher = ARCHIVE_FILE_PATTERN.matcher(f.getPath().getName());
if (fileMatcher.matches()) {
return Integer.parseInt(fileMatcher.group(1));
}
} catch (NumberFormatException e) {
// log and ignore any format warnings
LOG.warn("error getting suffix for archived file: " + f.getPath());
}
// return default value in case of any errors
return 0;
}
}
Loading

0 comments on commit a71bc4b

Please sign in to comment.