Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,177 @@
/*
* 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.table.repair;

import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
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.util.Option;
import org.apache.hudi.exception.HoodieException;

import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;

import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;

import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;

/**
* Utils for table repair tool.
*/
public final class RepairUtils {
/**
* Tags the instant time of each base or log file from the input file paths.
*
* @param basePath Base path of the table.
* @param baseFileExtension Base file extension, e.g., ".parquet".
* @param allPaths A {@link List} of file paths to tag.
* @return A {@link Map} of instant time in {@link String} to a {@link List} of relative file paths.
*/
public static Map<String, List<String>> tagInstantsOfBaseAndLogFiles(
String basePath, String baseFileExtension, List<Path> allPaths) {
// Instant time -> Set of base and log file paths
Map<String, List<String>> instantToFilesMap = new HashMap<>();
allPaths.forEach(path -> {
String instantTime = path.toString().endsWith(baseFileExtension)
? FSUtils.getCommitTime(path.getName()) : FSUtils.getBaseCommitTimeFromLogPath(path);
instantToFilesMap.computeIfAbsent(instantTime, k -> new ArrayList<>());
instantToFilesMap.get(instantTime).add(
FSUtils.getRelativePartitionPath(new Path(basePath), path));
});
return instantToFilesMap;
}

/**
* Gets the base and log file paths written for a given instant from the timeline.
* This reads the details of the instant metadata.
*
* @param timeline {@link HoodieTimeline} instance, can be active or archived timeline.
* @param instant Instant for lookup.
* @return A {@link Option} of {@link Set} of relative file paths to base path
* if the instant action is supported; empty {@link Option} otherwise.
* @throws IOException if reading instant details fail.
*/
public static Option<Set<String>> getBaseAndLogFilePathsFromTimeline(
HoodieTimeline timeline, HoodieInstant instant) throws IOException {
if (!instant.isCompleted()) {
throw new HoodieException("Cannot get base and log file paths from "
+ "instant not completed: " + instant.getTimestamp());
}

switch (instant.getAction()) {
case COMMIT_ACTION:
case DELTA_COMMIT_ACTION:
final HoodieCommitMetadata commitMetadata =
HoodieCommitMetadata.fromBytes(
timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class);
return Option.of(commitMetadata.getPartitionToWriteStats().values().stream().flatMap(List::stream)
.map(HoodieWriteStat::getPath).collect(Collectors.toSet()));
case REPLACE_COMMIT_ACTION:
final HoodieReplaceCommitMetadata replaceCommitMetadata =
HoodieReplaceCommitMetadata.fromBytes(
timeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class);
return Option.of(replaceCommitMetadata.getPartitionToWriteStats().values().stream().flatMap(List::stream)
.map(HoodieWriteStat::getPath).collect(Collectors.toSet()));
default:
return Option.empty();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should we consider other actions like rollback and savepoint etc.

}
}

/**
* Finds the dangling files to remove for a given instant to repair.
*
* @param instantToRepair Instant timestamp to repair.
* @param baseAndLogFilesFromFs A {@link List} of base and log files based on the file system.
* @param activeTimeline {@link HoodieActiveTimeline} instance.
* @param archivedTimeline {@link HoodieArchivedTimeline} instance.
* @return A {@link List} of relative file paths to base path for removing.
*/
public static List<String> findInstantFilesToRemove(
String instantToRepair, List<String> baseAndLogFilesFromFs,
HoodieActiveTimeline activeTimeline, HoodieArchivedTimeline archivedTimeline) {
// Skips the instant if it is requested or inflight in active timeline
if (activeTimeline.filter(instant -> instant.getTimestamp().equals(instantToRepair)
&& !instant.isCompleted()).getInstants().findAny().isPresent()) {
return Collections.emptyList();
}

try {
boolean doesInstantExist = false;
Option<Set<String>> filesFromTimeline = Option.empty();
Option<HoodieInstant> instantOption = activeTimeline.filterCompletedInstants().filter(
instant -> instant.getTimestamp().equals(instantToRepair)).firstInstant();
if (instantOption.isPresent()) {
// Completed instant in active timeline
doesInstantExist = true;
filesFromTimeline = RepairUtils.getBaseAndLogFilePathsFromTimeline(
activeTimeline, instantOption.get());
} else {
instantOption = archivedTimeline.filterCompletedInstants().filter(
instant -> instant.getTimestamp().equals(instantToRepair)).firstInstant();
if (instantOption.isPresent()) {
// Completed instant in archived timeline
doesInstantExist = true;
filesFromTimeline = RepairUtils.getBaseAndLogFilePathsFromTimeline(
archivedTimeline, instantOption.get());
}
}

if (doesInstantExist) {
if (!filesFromTimeline.isPresent() || filesFromTimeline.get().isEmpty()) {
// Skips if no instant details
return Collections.emptyList();
}
// Excludes committed base and log files from timeline
Set<String> filesToRemove = new HashSet<>(baseAndLogFilesFromFs);
filesToRemove.removeAll(filesFromTimeline.get());
return new ArrayList<>(filesToRemove);
} else {
// The instant does not exist in the whole timeline (neither completed nor requested/inflight),
// this means the files from this instant are dangling, which should be removed
return baseAndLogFilesFromFs;
}
} catch (IOException e) {
// In case of failure, does not remove any files for the instant
return Collections.emptyList();
}
}

/**
* Serializable path filter class for Spark job.
*/
public interface SerializablePathFilter extends PathFilter, Serializable {
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,6 @@

package org.apache.hudi.common.table.timeline;

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.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
import org.apache.hudi.common.model.HoodieLogFile;
Expand All @@ -32,10 +28,16 @@
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.Option;
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.apache.log4j.LogManager;
import org.apache.log4j.Logger;

import javax.annotation.Nonnull;

import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
Expand Down Expand Up @@ -65,7 +67,7 @@
*/
public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
private static final Pattern ARCHIVE_FILE_PATTERN =
Pattern.compile("^\\.commits_\\.archive\\.([0-9]*)$");
Pattern.compile("^\\.commits_\\.archive\\.([0-9]*)$");

private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE_PREFIX = "commits";
private static final String ACTION_TYPE_KEY = "actionType";
Expand Down Expand Up @@ -112,6 +114,11 @@ public void loadInstantDetailsInMemory(String startTs, String endTs) {
loadInstants(startTs, endTs);
}

public void loadCompletedInstantDetailsInMemory() {
loadInstants(null, true,
record -> HoodieInstant.State.COMPLETED.toString().equals(record.get(ACTION_STATE).toString()));
}

public void loadCompactionDetailsInMemory(String compactionInstantTime) {
loadCompactionDetailsInMemory(compactionInstantTime, compactionInstantTime);
}
Expand Down Expand Up @@ -143,15 +150,17 @@ public HoodieArchivedTimeline reload() {
}

private HoodieInstant readCommit(GenericRecord record, boolean loadDetails) {
final String instantTime = record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString();
final String instantTime = record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString();
final String action = record.get(ACTION_TYPE_KEY).toString();
if (loadDetails) {
getMetadataKey(action).map(key -> {
Object actionData = record.get(key);
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));
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;
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,11 @@

package org.apache.hudi.common.util;

import org.apache.hudi.exception.HoodieIOException;

import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

Expand Down Expand Up @@ -98,6 +103,31 @@ public static void copy(InputStream inputStream, OutputStream outputStream) thro
}
}

/**
* Copies the file content from source path to destination path.
*
* @param fileSystem {@link FileSystem} instance.
* @param sourceFilePath Source file path.
* @param destFilePath Destination file path.
*/
public static void copy(
FileSystem fileSystem, org.apache.hadoop.fs.Path sourceFilePath,
org.apache.hadoop.fs.Path destFilePath) {
FSDataInputStream fsDataInputStream = null;
FSDataOutputStream fsDataOutputStream = null;
try {
fsDataInputStream = fileSystem.open(sourceFilePath);
fsDataOutputStream = fileSystem.create(destFilePath, false);
copy(fsDataInputStream, fsDataOutputStream);
} catch (IOException e) {
throw new HoodieIOException(String.format("Cannot copy from %s to %s",
sourceFilePath.toString(), destFilePath.toString()), e);
} finally {
closeQuietly(fsDataInputStream);
closeQuietly(fsDataOutputStream);
}
}

public static byte[] readAsByteArray(InputStream input) throws IOException {
return readAsByteArray(input, 128);
}
Expand Down
Loading