[HUDI-711] Refactor exporter main logic (#1436)
* Refactor exporter main logic * break main method into multiple readable methods * fix bug of passing wrong file list * avoid deleting output path when exists * throw exception to early abort on multiple cases * use JavaSparkContext instead of SparkSession * improve unit test for expected exceptions
This commit is contained in:
@@ -31,8 +31,10 @@ import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.utilities.HoodieSnapshotExporter.Config;
|
||||
import org.apache.hudi.utilities.HoodieSnapshotExporter.OutputFormatValidator;
|
||||
import org.apache.hudi.utilities.HoodieSnapshotExporter.Partitioner;
|
||||
import org.apache.hudi.utilities.exception.HoodieSnapshotExporterException;
|
||||
|
||||
import com.beust.jcommander.ParameterException;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
@@ -43,11 +45,12 @@ import org.apache.spark.sql.Column;
|
||||
import org.apache.spark.sql.DataFrameWriter;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.runners.Enclosed;
|
||||
import org.junit.rules.ExpectedException;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.junit.runners.Parameterized.Parameter;
|
||||
@@ -56,9 +59,9 @@ import org.junit.runners.Parameterized.Parameters;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
@@ -85,7 +88,6 @@ public class TestHoodieSnapshotExporter {
|
||||
sourcePath = dfsBasePath + "/source/";
|
||||
targetPath = dfsBasePath + "/target/";
|
||||
dfs.mkdirs(new Path(sourcePath));
|
||||
dfs.mkdirs(new Path(targetPath));
|
||||
HoodieTableMetaClient
|
||||
.initTableType(jsc.hadoopConfiguration(), sourcePath, HoodieTableType.COPY_ON_WRITE, TABLE_NAME,
|
||||
HoodieAvroPayload.class.getName());
|
||||
@@ -140,7 +142,7 @@ public class TestHoodieSnapshotExporter {
|
||||
|
||||
@Test
|
||||
public void testExportAsHudi() throws IOException {
|
||||
new HoodieSnapshotExporter().export(SparkSession.builder().config(jsc.getConf()).getOrCreate(), cfg);
|
||||
new HoodieSnapshotExporter().export(jsc, cfg);
|
||||
|
||||
// Check results
|
||||
assertTrue(dfs.exists(new Path(targetPath + "/.hoodie/" + COMMIT_TIME + ".clean")));
|
||||
@@ -159,18 +161,61 @@ public class TestHoodieSnapshotExporter {
|
||||
assertTrue(dfs.exists(new Path(partition + "/.hoodie_partition_metadata")));
|
||||
assertTrue(dfs.exists(new Path(targetPath + "/_SUCCESS")));
|
||||
}
|
||||
}
|
||||
|
||||
public static class TestHoodieSnapshotExporterForEarlyAbort extends ExporterTestHarness {
|
||||
|
||||
private HoodieSnapshotExporter.Config cfg;
|
||||
|
||||
@Rule
|
||||
public ExpectedException exceptionRule = ExpectedException.none();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
cfg = new Config();
|
||||
cfg.sourceBasePath = sourcePath;
|
||||
cfg.targetOutputPath = targetPath;
|
||||
cfg.outputFormat = OutputFormatValidator.HUDI;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExportEmptyDataset() throws IOException {
|
||||
public void testExportWhenTargetPathExists() throws IOException {
|
||||
// make target output path present
|
||||
dfs.mkdirs(new Path(targetPath));
|
||||
|
||||
// export
|
||||
exceptionRule.expect(HoodieSnapshotExporterException.class);
|
||||
exceptionRule.expectMessage("The target output path already exists.");
|
||||
new HoodieSnapshotExporter().export(jsc, cfg);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExportDatasetWithNoCommit() throws IOException {
|
||||
// delete commit files
|
||||
List<Path> commitFiles = Arrays.stream(dfs.listStatus(new Path(sourcePath + "/.hoodie")))
|
||||
.map(FileStatus::getPath)
|
||||
.filter(filePath -> filePath.getName().endsWith(".commit"))
|
||||
.collect(Collectors.toList());
|
||||
for (Path p : commitFiles) {
|
||||
dfs.delete(p, false);
|
||||
}
|
||||
|
||||
// export
|
||||
exceptionRule.expect(HoodieSnapshotExporterException.class);
|
||||
exceptionRule.expectMessage("No commits present. Nothing to snapshot.");
|
||||
new HoodieSnapshotExporter().export(jsc, cfg);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExportDatasetWithNoPartition() throws IOException {
|
||||
// delete all source data
|
||||
dfs.delete(new Path(sourcePath + "/" + PARTITION_PATH), true);
|
||||
|
||||
// export
|
||||
new HoodieSnapshotExporter().export(SparkSession.builder().config(jsc.getConf()).getOrCreate(), cfg);
|
||||
|
||||
// Check results
|
||||
assertEquals("Target path should be empty.", 0, dfs.listStatus(new Path(targetPath)).length);
|
||||
assertFalse(dfs.exists(new Path(targetPath + "/_SUCCESS")));
|
||||
exceptionRule.expect(HoodieSnapshotExporterException.class);
|
||||
exceptionRule.expectMessage("The source dataset has 0 partition to snapshot.");
|
||||
new HoodieSnapshotExporter().export(jsc, cfg);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -191,7 +236,7 @@ public class TestHoodieSnapshotExporter {
|
||||
cfg.sourceBasePath = sourcePath;
|
||||
cfg.targetOutputPath = targetPath;
|
||||
cfg.outputFormat = format;
|
||||
new HoodieSnapshotExporter().export(SparkSession.builder().config(jsc.getConf()).getOrCreate(), cfg);
|
||||
new HoodieSnapshotExporter().export(jsc, cfg);
|
||||
assertEquals(NUM_RECORDS, sqlContext.read().format(format).load(targetPath).count());
|
||||
assertTrue(dfs.exists(new Path(targetPath + "/_SUCCESS")));
|
||||
}
|
||||
@@ -228,7 +273,7 @@ public class TestHoodieSnapshotExporter {
|
||||
public void testExportWithPartitionField() throws IOException {
|
||||
// `driver` field is set in HoodieTestDataGenerator
|
||||
cfg.outputPartitionField = "driver";
|
||||
new HoodieSnapshotExporter().export(SparkSession.builder().config(jsc.getConf()).getOrCreate(), cfg);
|
||||
new HoodieSnapshotExporter().export(jsc, cfg);
|
||||
|
||||
assertEquals(NUM_RECORDS, sqlContext.read().format("json").load(targetPath).count());
|
||||
assertTrue(dfs.exists(new Path(targetPath + "/_SUCCESS")));
|
||||
@@ -238,7 +283,7 @@ public class TestHoodieSnapshotExporter {
|
||||
@Test
|
||||
public void testExportForUserDefinedPartitioner() throws IOException {
|
||||
cfg.outputPartitioner = UserDefinedPartitioner.class.getName();
|
||||
new HoodieSnapshotExporter().export(SparkSession.builder().config(jsc.getConf()).getOrCreate(), cfg);
|
||||
new HoodieSnapshotExporter().export(jsc, cfg);
|
||||
|
||||
assertEquals(NUM_RECORDS, sqlContext.read().format("json").load(targetPath).count());
|
||||
assertTrue(dfs.exists(new Path(targetPath + "/_SUCCESS")));
|
||||
|
||||
Reference in New Issue
Block a user