Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -242,11 +242,11 @@ private FileStatus[] fetchFilesFromCommitMetadata(HoodieInstant instantToRollbac
return fs.listStatus(Arrays.stream(filePaths).filter(entry -> {
try {
return fs.exists(entry);
} catch (IOException e) {
} catch (Exception e) {
LOG.error("Exists check failed for " + entry.toString(), e);
}
// if IOException is thrown, do not ignore. lets try to add the file of interest to be deleted. we can't miss any files to be rolled back.
return false;
// if any Exception is thrown, do not ignore. let's try to add the file of interest to be deleted. we can't miss any files to be rolled back.
return true;
}).toArray(Path[]::new), pathFilter);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,35 +20,50 @@

import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata;
import org.apache.hudi.avro.model.HoodieRollbackPlan;
import org.apache.hudi.avro.model.HoodieRollbackRequest;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.marker.WriteMarkersFactory;
import org.apache.hudi.testutils.Assertions;

import org.apache.hadoop.fs.FileSystem;
import org.apache.spark.api.java.JavaRDD;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import org.mockito.Mockito;
import org.mockito.MockitoAnnotations;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;

import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import static org.mockito.ArgumentMatchers.any;

public class TestCopyOnWriteRollbackActionExecutor extends HoodieClientRollbackTestBase {
@BeforeEach
Expand Down Expand Up @@ -133,6 +148,65 @@ public void testCopyOnWriteRollbackActionExecutorForFileListingAsGenerateFile()
assertFalse(testTable.baseFileExists(p2, "002", "id22"));
}

@Test
public void testListBasedRollbackStrategy() throws Exception {
//just generate two partitions
dataGen = new HoodieTestDataGenerator(new String[] {DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH, DEFAULT_THIRD_PARTITION_PATH});
HoodieWriteConfig cfg = getConfigBuilder().withRollbackUsingMarkers(false).build();
// 1. prepare data
HoodieTestDataGenerator.writePartitionMetadataDeprecated(fs, new String[] {DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH}, basePath);
SparkRDDWriteClient client = getHoodieWriteClient(cfg);
/**
* Write 1 (only inserts)
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInsertsContainsAllPartitions(newCommitTime, 3);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime);
Assertions.assertNoWriteErrors(statuses.collect());
//client.commit(newCommitTime, statuses);

/**
* Write 2 (updates)
*/
newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, records);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime);
Assertions.assertNoWriteErrors(statuses.collect());
//client.commit(newCommitTime, statuses);

context = new HoodieSparkEngineContext(jsc);
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table = this.getHoodieTable(metaClient, cfg);
HoodieInstant needRollBackInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "002");
String rollbackInstant = "003";

ListingBasedRollbackStrategy rollbackStrategy = new ListingBasedRollbackStrategy(table, context, table.getConfig(), rollbackInstant);
List<HoodieRollbackRequest> rollBackRequests = rollbackStrategy.getRollbackRequests(needRollBackInstant);
rollBackRequests.forEach(entry -> System.out.println(" " + entry.getPartitionPath() + ", " + entry.getFileId() + " "
+ Arrays.toString(entry.getFilesToBeDeleted().toArray())));

HoodieRollbackRequest rollbackRequest = rollBackRequests.stream().filter(entry -> entry.getPartitionPath().equals(DEFAULT_FIRST_PARTITION_PATH)).findFirst().get();

FileSystem fs = Mockito.mock(FileSystem.class);
MockitoAnnotations.initMocks(this);

// mock to throw exception when fs.exists() is invoked
System.out.println("Fs.exists() call for " + rollbackRequest.getFilesToBeDeleted().get(0).toString());
Mockito.when(fs.exists(any()))
.thenThrow(new IOException("Failing exists call for " + rollbackRequest.getFilesToBeDeleted().get(0)));

rollbackStrategy = new ListingBasedRollbackStrategy(table, context, cfg, rollbackInstant);
List<HoodieRollbackRequest> rollBackRequestsUpdated = rollbackStrategy.getRollbackRequests(needRollBackInstant);
rollBackRequestsUpdated.forEach(entry -> System.out.println(" " + entry.getPartitionPath() + ", " + entry.getFileId() + " "
+ Arrays.toString(entry.getFilesToBeDeleted().toArray())));

assertEquals(rollBackRequests, rollBackRequestsUpdated);
}


// Verify that rollback works with replacecommit
@ParameterizedTest
@ValueSource(booleans = {true, false})
Expand Down