Skip to content
Closed
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
Expand Up @@ -18,11 +18,19 @@

package org.apache.hudi.table;

import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieReplaceMetadata;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.ActionType;
import org.apache.hudi.common.model.HoodieArchivedLogFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
Expand All @@ -46,12 +54,6 @@
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;

import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

Expand All @@ -62,8 +64,10 @@
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.stream.Collectors;
import java.util.stream.Stream;

Expand Down Expand Up @@ -263,6 +267,7 @@ public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
LOG.info("Wrapper schema " + wrapperSchema.toString());
List<IndexedRecord> records = new ArrayList<>();
for (HoodieInstant hoodieInstant : instants) {
deleteReplacedFiles(hoodieInstant);
try {
records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
if (records.size() >= this.config.getCommitArchivalBatchSize()) {
Expand All @@ -285,6 +290,43 @@ public Path getArchiveFilePath() {
return archiveFilePath;
}

private void deleteReplacedFiles(HoodieInstant instant) {
if (!instant.isCompleted() || !HoodieTimeline.COMMIT_ACTION.equals(instant.getAction())) {
// only delete files for completed instants
return;
}
Option<HoodieInstant> replaceInstantOption = metaClient.getActiveTimeline().getCompletedAndReplaceTimeline()
.filter(replaceInstant -> replaceInstant.getTimestamp().equals(instant.getTimestamp())).firstInstant();

replaceInstantOption.ifPresent(replaceInstant -> {
try {
HoodieReplaceMetadata metadata = TimelineMetadataUtils.deserializeHoodieReplaceMetadata(
metaClient.getActiveTimeline().getInstantDetails(replaceInstant).get());

metadata.getPartitionMetadata().entrySet().stream().forEach(entry ->
deleteFileGroups(entry.getKey(), new HashSet<>(entry.getValue()), instant)
);
} catch (IOException e) {
throw new HoodieCommitException("Failed to archive because cannot delete replace files", e);
}
});
}

private void deleteFileGroups(String partitionPath, Set<String> fileIdsToDelete, HoodieInstant instant) {
try {
FileStatus[] statuses = metaClient.getFs().listStatus(FSUtils.getPartitionPath(metaClient.getBasePath(), partitionPath));
for (FileStatus status : statuses) {
String fileId = FSUtils.getFileIdFromFilePath(status.getPath());
if (fileIdsToDelete.contains(fileId)) {
LOG.info("Delete " + status.getPath() + " to archive " + instant);
metaClient.getFs().delete(status.getPath());
}
}
} catch (IOException e) {
LOG.error("unable to delete file groups that are replaced", e);
}
}

private void writeToFile(Schema wrapperSchema, List<IndexedRecord> records) throws Exception {
if (records.size() > 0) {
Map<HeaderMetadataType, String> header = new HashMap<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@

package org.apache.hudi.io;

import org.apache.hadoop.fs.FileStatus;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
Expand All @@ -40,10 +42,13 @@
import org.junit.jupiter.api.Test;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
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;

Expand Down Expand Up @@ -208,6 +213,40 @@ public void testArchiveTableWithArchival() throws IOException {
verifyInflightInstants(metaClient, 2);
}

@Test
public void testArchiveTableWithReplacedFiles() throws IOException {
HoodieTestUtils.init(hadoopConf, basePath);
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table")
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
.build();

int numCommits = 4;
int commitInstant = 100;
for (int i = 0; i < numCommits; i++) {
createCommitAndReplaceMetadata(commitInstant);
commitInstant += 100;
}

metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
assertEquals(4, timeline.countInstants(), "Loaded 4 commits and the count should match");
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, metaClient);
boolean result = archiveLog.archiveIfRequired(hadoopConf);
assertTrue(result);

FileStatus[] allFiles = metaClient.getFs().listStatus(new Path(basePath + "/" + HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH));
Set<String> allFileIds = Arrays.stream(allFiles).map(fs -> FSUtils.getFileIdFromFilePath(fs.getPath())).collect(Collectors.toSet());

// verify 100-1,200-1 are deleted by archival
assertFalse(allFileIds.contains("file-100-1"));
assertFalse(allFileIds.contains("file-200-1"));
assertTrue(allFileIds.contains("file-100-2"));
assertTrue(allFileIds.contains("file-200-2"));
assertTrue(allFileIds.contains("file-300-1"));
assertTrue(allFileIds.contains("file-400-1"));
}

@Test
public void testArchiveTableWithNoArchival() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
Expand Down Expand Up @@ -430,4 +469,19 @@ public void testConvertCommitMetadata() {
org.apache.hudi.avro.model.HoodieCommitMetadata expectedCommitMetadata = archiveLog.convertCommitMetadata(hoodieCommitMetadata);
assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString());
}

private void createCommitAndReplaceMetadata(int commitInstant) throws IOException {
String commitTime = "" + commitInstant;
String fileId1 = "file-" + commitInstant + "-1";
String fileId2 = "file-" + commitInstant + "-2";

// create replace instant to mark fileId1 as deleted
Map<String, List<String>> partitionToReplaceFiles = new HashMap<>();
partitionToReplaceFiles.putIfAbsent(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, new ArrayList<>());
partitionToReplaceFiles.get(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).add(fileId1);
HoodieTestUtils.createReplaceInstant(partitionToReplaceFiles, commitTime, metaClient);
HoodieTestDataGenerator.createCommitFile(basePath, commitTime, dfs.getConf());
HoodieTestDataGenerator.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, commitTime, fileId1, dfs.getConf());
HoodieTestDataGenerator.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, commitTime, fileId2, dfs.getConf());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -365,6 +365,14 @@ public static void createCompactionRequestedFile(String basePath, String instant
createEmptyFile(basePath, commitFile, configuration);
}

public static void createDataFile(String basePath, String partitionPath, String instantTime, String fileID, Configuration configuration)
throws IOException {

Path dataFilePath = new Path(basePath + "/" + partitionPath + "/"
+ FSUtils.makeDataFileName(instantTime, "1_0_1", fileID));
createEmptyFile(basePath, dataFilePath, configuration);
}

private static void createEmptyFile(String basePath, Path filePath, Configuration configuration) throws IOException {
FileSystem fs = FSUtils.getFs(basePath, configuration);
FSDataOutputStream os = fs.create(filePath, true);
Expand Down
44 changes: 44 additions & 0 deletions hudi-common/src/main/avro/HoodieReplaceMetadata.avsc
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* 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.
*/

/*
* Note that all 'replace' instants are read for every query
* So it is important to keep this small. Please be careful
* before tracking additional information in this file.
* This will be used for 'insert_overwrite' (RFC-18) and also 'clustering' (RFC-19)
*/
{"namespace": "org.apache.hudi.avro.model",
"type": "record",
"name": "HoodieReplaceMetadata",
"fields": [
{"name": "totalFileGroupsReplaced", "type": "int"},
{"name": "command", "type": "string"},
{"name": "partitionMetadata", "type": {
"type" : "map", "values" : {
"type": "array",
"items": "string"
Copy link
Member

Choose a reason for hiding this comment

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

I was expecting this to contain the actual file slices being replaced? seems like we just want to have the partitions here?

}
}
},
{
"name":"version",
"type":["int", "null"],
"default": 1
}
]
}
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION,
INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION,
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION));
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION,
INFLIGHT_REPLACE_EXTENSION, REPLACE_EXTENSION));

private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class);
protected HoodieTableMetaClient metaClient;
Expand Down Expand Up @@ -304,6 +305,22 @@ public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedI
return inflight;
}

/**
* Transition Clean State from inflight to Committed.
*
* @param inflightInstant Inflight instant
* @param data Extra Metadata
* @return commit instant
*/
public HoodieInstant transitionReplaceInflightToComplete(HoodieInstant inflightInstant, Option<byte[]> data) {
ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.REPLACE_ACTION));
ValidationUtils.checkArgument(inflightInstant.isInflight());
HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, REPLACE_ACTION, inflightInstant.getTimestamp());
// Then write to timeline
transitionState(inflightInstant, commitInstant, data);
return commitInstant;
}

private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option<byte[]> data) {
transitionState(fromInstant, toInstant, data, false);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,6 +113,24 @@ public HoodieDefaultTimeline getCommitsAndCompactionTimeline() {
return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details);
}

@Override
public HoodieDefaultTimeline getCommitsReplaceAndCompactionTimeline() {
Set<String> validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_ACTION);
return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details);
}

@Override
public HoodieTimeline getCompletedAndReplaceTimeline() {
Set<String> commitActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION);
Set<String> validCommitTimes = instants.stream().filter(s -> s.isCompleted())
.filter(s -> commitActions.contains(s.getAction()))
.map(s -> s.getTimestamp()).collect(Collectors.toSet());

return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(REPLACE_ACTION))
.filter(s -> s.isCompleted())
.filter(instant -> validCommitTimes.contains(instant.getTimestamp())), details);
}

@Override
public HoodieTimeline filterPendingCompactionTimeline() {
return new HoodieDefaultTimeline(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,6 +162,9 @@ public String getFileName() {
} else if (HoodieTimeline.RESTORE_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightRestoreFileName(timestamp)
: HoodieTimeline.makeRestoreFileName(timestamp);
} else if (HoodieTimeline.REPLACE_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightReplaceFileName(timestamp)
: HoodieTimeline.makeReplaceFileName(timestamp);
}
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ public interface HoodieTimeline extends Serializable {
String CLEAN_ACTION = "clean";
String ROLLBACK_ACTION = "rollback";
String SAVEPOINT_ACTION = "savepoint";
String REPLACE_ACTION = "replace";
String INFLIGHT_EXTENSION = ".inflight";
// With Async Compaction, compaction instant can be in 3 states :
// (compaction-requested), (compaction-inflight), (completed)
Expand All @@ -57,7 +58,7 @@ public interface HoodieTimeline extends Serializable {

String[] VALID_ACTIONS_IN_TIMELINE = {COMMIT_ACTION, DELTA_COMMIT_ACTION,
CLEAN_ACTION, SAVEPOINT_ACTION, RESTORE_ACTION, ROLLBACK_ACTION,
COMPACTION_ACTION};
COMPACTION_ACTION, REPLACE_ACTION};

String COMMIT_EXTENSION = "." + COMMIT_ACTION;
String DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION;
Expand All @@ -78,6 +79,8 @@ public interface HoodieTimeline extends Serializable {
String INFLIGHT_COMPACTION_EXTENSION = StringUtils.join(".", COMPACTION_ACTION, INFLIGHT_EXTENSION);
String INFLIGHT_RESTORE_EXTENSION = "." + RESTORE_ACTION + INFLIGHT_EXTENSION;
String RESTORE_EXTENSION = "." + RESTORE_ACTION;
String INFLIGHT_REPLACE_EXTENSION = "." + REPLACE_ACTION + INFLIGHT_EXTENSION;
String REPLACE_EXTENSION = "." + REPLACE_ACTION;

String INVALID_INSTANT_TS = "0";

Expand Down Expand Up @@ -126,6 +129,21 @@ public interface HoodieTimeline extends Serializable {
*/
HoodieTimeline getCommitsAndCompactionTimeline();

/**
* Timeline to just include commits (commit/deltacommit), replace and compaction actions.
*
* @return
*/
HoodieDefaultTimeline getCommitsReplaceAndCompactionTimeline();


/**
* Timeline to just include replace instants that have valid (commit/deltacommit) actions.
*
* @return
*/
HoodieTimeline getCompletedAndReplaceTimeline();

/**
* Filter this timeline to just include requested and inflight compaction instants.
*
Expand Down Expand Up @@ -348,6 +366,14 @@ static String makeInflightRestoreFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.INFLIGHT_RESTORE_EXTENSION);
}

static String makeReplaceFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.REPLACE_EXTENSION);
}

static String makeInflightReplaceFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.INFLIGHT_REPLACE_EXTENSION);
}

static String makeDeltaFileName(String instantTime) {
return instantTime + HoodieTimeline.DELTA_COMMIT_EXTENSION;
}
Expand Down
Loading