|
|
|
|
@@ -19,6 +19,7 @@
|
|
|
|
|
package org.apache.hudi.utilities.functional;
|
|
|
|
|
|
|
|
|
|
import org.apache.hudi.client.SparkRDDWriteClient;
|
|
|
|
|
import org.apache.hudi.common.fs.FSUtils;
|
|
|
|
|
import org.apache.hudi.common.model.HoodieAvroPayload;
|
|
|
|
|
import org.apache.hudi.common.model.HoodieRecord;
|
|
|
|
|
import org.apache.hudi.common.model.HoodieTableType;
|
|
|
|
|
@@ -27,7 +28,7 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
|
|
|
|
import org.apache.hudi.config.HoodieIndexConfig;
|
|
|
|
|
import org.apache.hudi.config.HoodieWriteConfig;
|
|
|
|
|
import org.apache.hudi.index.HoodieIndex.IndexType;
|
|
|
|
|
import org.apache.hudi.testutils.FunctionalTestHarness;
|
|
|
|
|
import org.apache.hudi.testutils.SparkClientFunctionalTestHarness;
|
|
|
|
|
import org.apache.hudi.utilities.HoodieSnapshotExporter;
|
|
|
|
|
import org.apache.hudi.utilities.HoodieSnapshotExporter.Config;
|
|
|
|
|
import org.apache.hudi.utilities.HoodieSnapshotExporter.OutputFormatValidator;
|
|
|
|
|
@@ -35,6 +36,7 @@ import org.apache.hudi.utilities.HoodieSnapshotExporter.Partitioner;
|
|
|
|
|
import org.apache.hudi.utilities.exception.HoodieSnapshotExporterException;
|
|
|
|
|
|
|
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
|
|
import org.apache.hadoop.fs.LocalFileSystem;
|
|
|
|
|
import org.apache.hadoop.fs.LocatedFileStatus;
|
|
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
|
import org.apache.hadoop.fs.RemoteIterator;
|
|
|
|
|
@@ -54,6 +56,7 @@ import org.junit.jupiter.params.ParameterizedTest;
|
|
|
|
|
import org.junit.jupiter.params.provider.ValueSource;
|
|
|
|
|
|
|
|
|
|
import java.io.IOException;
|
|
|
|
|
import java.nio.file.Paths;
|
|
|
|
|
import java.util.Arrays;
|
|
|
|
|
import java.util.List;
|
|
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
@@ -63,7 +66,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
|
|
|
|
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
|
|
|
|
|
|
|
|
|
@Tag("functional")
|
|
|
|
|
public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
|
|
|
|
|
public class TestHoodieSnapshotExporter extends SparkClientFunctionalTestHarness {
|
|
|
|
|
|
|
|
|
|
static final Logger LOG = LogManager.getLogger(TestHoodieSnapshotExporter.class);
|
|
|
|
|
static final int NUM_RECORDS = 100;
|
|
|
|
|
@@ -72,13 +75,14 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
|
|
|
|
|
static final String TABLE_NAME = "testing";
|
|
|
|
|
String sourcePath;
|
|
|
|
|
String targetPath;
|
|
|
|
|
LocalFileSystem lfs;
|
|
|
|
|
|
|
|
|
|
@BeforeEach
|
|
|
|
|
public void init() throws Exception {
|
|
|
|
|
// Initialize test data dirs
|
|
|
|
|
sourcePath = dfsBasePath() + "/source/";
|
|
|
|
|
targetPath = dfsBasePath() + "/target/";
|
|
|
|
|
dfs().mkdirs(new Path(sourcePath));
|
|
|
|
|
sourcePath = Paths.get(basePath(), "source").toString();
|
|
|
|
|
targetPath = Paths.get(basePath(), "target").toString();
|
|
|
|
|
lfs = (LocalFileSystem) FSUtils.getFs(basePath(), jsc().hadoopConfiguration());
|
|
|
|
|
|
|
|
|
|
HoodieTableMetaClient.withPropertyBuilder()
|
|
|
|
|
.setTableType(HoodieTableType.COPY_ON_WRITE)
|
|
|
|
|
@@ -88,14 +92,14 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
|
|
|
|
|
|
|
|
|
|
// Prepare data as source Hudi dataset
|
|
|
|
|
HoodieWriteConfig cfg = getHoodieWriteConfig(sourcePath);
|
|
|
|
|
SparkRDDWriteClient hdfsWriteClient = new SparkRDDWriteClient(context(), cfg);
|
|
|
|
|
hdfsWriteClient.startCommitWithTime(COMMIT_TIME);
|
|
|
|
|
SparkRDDWriteClient writeClient = getHoodieWriteClient(cfg);
|
|
|
|
|
writeClient.startCommitWithTime(COMMIT_TIME);
|
|
|
|
|
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(new String[] {PARTITION_PATH});
|
|
|
|
|
List<HoodieRecord> records = dataGen.generateInserts(COMMIT_TIME, NUM_RECORDS);
|
|
|
|
|
JavaRDD<HoodieRecord> recordsRDD = jsc().parallelize(records, 1);
|
|
|
|
|
hdfsWriteClient.bulkInsert(recordsRDD, COMMIT_TIME);
|
|
|
|
|
hdfsWriteClient.close();
|
|
|
|
|
RemoteIterator<LocatedFileStatus> itr = dfs().listFiles(new Path(sourcePath), true);
|
|
|
|
|
writeClient.bulkInsert(recordsRDD, COMMIT_TIME);
|
|
|
|
|
writeClient.close();
|
|
|
|
|
RemoteIterator<LocatedFileStatus> itr = lfs.listFiles(new Path(sourcePath), true);
|
|
|
|
|
while (itr.hasNext()) {
|
|
|
|
|
LOG.info(">>> Prepared test file: " + itr.next().getPath());
|
|
|
|
|
}
|
|
|
|
|
@@ -103,8 +107,7 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
|
|
|
|
|
|
|
|
|
|
@AfterEach
|
|
|
|
|
public void cleanUp() throws IOException {
|
|
|
|
|
dfs().delete(new Path(sourcePath), true);
|
|
|
|
|
dfs().delete(new Path(targetPath), true);
|
|
|
|
|
lfs.close();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private HoodieWriteConfig getHoodieWriteConfig(String basePath) {
|
|
|
|
|
@@ -138,18 +141,18 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
|
|
|
|
|
new HoodieSnapshotExporter().export(jsc(), cfg);
|
|
|
|
|
|
|
|
|
|
// Check results
|
|
|
|
|
assertTrue(dfs().exists(new Path(targetPath + "/.hoodie/" + COMMIT_TIME + ".commit")));
|
|
|
|
|
assertTrue(dfs().exists(new Path(targetPath + "/.hoodie/" + COMMIT_TIME + ".commit.requested")));
|
|
|
|
|
assertTrue(dfs().exists(new Path(targetPath + "/.hoodie/" + COMMIT_TIME + ".inflight")));
|
|
|
|
|
assertTrue(dfs().exists(new Path(targetPath + "/.hoodie/hoodie.properties")));
|
|
|
|
|
assertTrue(lfs.exists(new Path(targetPath + "/.hoodie/" + COMMIT_TIME + ".commit")));
|
|
|
|
|
assertTrue(lfs.exists(new Path(targetPath + "/.hoodie/" + COMMIT_TIME + ".commit.requested")));
|
|
|
|
|
assertTrue(lfs.exists(new Path(targetPath + "/.hoodie/" + COMMIT_TIME + ".inflight")));
|
|
|
|
|
assertTrue(lfs.exists(new Path(targetPath + "/.hoodie/hoodie.properties")));
|
|
|
|
|
String partition = targetPath + "/" + PARTITION_PATH;
|
|
|
|
|
long numParquetFiles = Arrays.stream(dfs().listStatus(new Path(partition)))
|
|
|
|
|
long numParquetFiles = Arrays.stream(lfs.listStatus(new Path(partition)))
|
|
|
|
|
.filter(fileStatus -> fileStatus.getPath().toString().endsWith(".parquet"))
|
|
|
|
|
.count();
|
|
|
|
|
assertTrue(numParquetFiles >= 1, "There should exist at least 1 parquet file.");
|
|
|
|
|
assertEquals(NUM_RECORDS, sqlContext().read().parquet(partition).count());
|
|
|
|
|
assertTrue(dfs().exists(new Path(partition + "/.hoodie_partition_metadata")));
|
|
|
|
|
assertTrue(dfs().exists(new Path(targetPath + "/_SUCCESS")));
|
|
|
|
|
assertTrue(lfs.exists(new Path(partition + "/.hoodie_partition_metadata")));
|
|
|
|
|
assertTrue(lfs.exists(new Path(targetPath + "/_SUCCESS")));
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@@ -169,7 +172,7 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
|
|
|
|
|
@Test
|
|
|
|
|
public void testExportWhenTargetPathExists() throws IOException {
|
|
|
|
|
// make target output path present
|
|
|
|
|
dfs().mkdirs(new Path(targetPath));
|
|
|
|
|
lfs.mkdirs(new Path(targetPath));
|
|
|
|
|
|
|
|
|
|
// export
|
|
|
|
|
final Throwable thrown = assertThrows(HoodieSnapshotExporterException.class, () -> {
|
|
|
|
|
@@ -181,12 +184,12 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
|
|
|
|
|
@Test
|
|
|
|
|
public void testExportDatasetWithNoCommit() throws IOException {
|
|
|
|
|
// delete commit files
|
|
|
|
|
List<Path> commitFiles = Arrays.stream(dfs().listStatus(new Path(sourcePath + "/.hoodie")))
|
|
|
|
|
List<Path> commitFiles = Arrays.stream(lfs.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);
|
|
|
|
|
lfs.delete(p, false);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// export
|
|
|
|
|
@@ -199,7 +202,7 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
|
|
|
|
|
@Test
|
|
|
|
|
public void testExportDatasetWithNoPartition() throws IOException {
|
|
|
|
|
// delete all source data
|
|
|
|
|
dfs().delete(new Path(sourcePath + "/" + PARTITION_PATH), true);
|
|
|
|
|
lfs.delete(new Path(sourcePath + "/" + PARTITION_PATH), true);
|
|
|
|
|
|
|
|
|
|
// export
|
|
|
|
|
final Throwable thrown = assertThrows(HoodieSnapshotExporterException.class, () -> {
|
|
|
|
|
@@ -221,7 +224,7 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
|
|
|
|
|
cfg.outputFormat = format;
|
|
|
|
|
new HoodieSnapshotExporter().export(jsc(), cfg);
|
|
|
|
|
assertEquals(NUM_RECORDS, sqlContext().read().format(format).load(targetPath).count());
|
|
|
|
|
assertTrue(dfs().exists(new Path(targetPath + "/_SUCCESS")));
|
|
|
|
|
assertTrue(lfs.exists(new Path(targetPath + "/_SUCCESS")));
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@@ -259,8 +262,8 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
|
|
|
|
|
new HoodieSnapshotExporter().export(jsc(), cfg);
|
|
|
|
|
|
|
|
|
|
assertEquals(NUM_RECORDS, sqlContext().read().format("json").load(targetPath).count());
|
|
|
|
|
assertTrue(dfs().exists(new Path(targetPath + "/_SUCCESS")));
|
|
|
|
|
assertTrue(dfs().listStatus(new Path(targetPath)).length > 1);
|
|
|
|
|
assertTrue(lfs.exists(new Path(targetPath + "/_SUCCESS")));
|
|
|
|
|
assertTrue(lfs.listStatus(new Path(targetPath)).length > 1);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
@@ -269,8 +272,8 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
|
|
|
|
|
new HoodieSnapshotExporter().export(jsc(), cfg);
|
|
|
|
|
|
|
|
|
|
assertEquals(NUM_RECORDS, sqlContext().read().format("json").load(targetPath).count());
|
|
|
|
|
assertTrue(dfs().exists(new Path(targetPath + "/_SUCCESS")));
|
|
|
|
|
assertTrue(dfs().exists(new Path(String.format("%s/%s=%s", targetPath, UserDefinedPartitioner.PARTITION_NAME, PARTITION_PATH))));
|
|
|
|
|
assertTrue(lfs.exists(new Path(targetPath + "/_SUCCESS")));
|
|
|
|
|
assertTrue(lfs.exists(new Path(String.format("%s/%s=%s", targetPath, UserDefinedPartitioner.PARTITION_NAME, PARTITION_PATH))));
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|