1
0

FSUtils.getAllPartitionsPaths() works based on .hoodie_partition_metadata

- clean/rollback/write paths covered by existing tests
 - Snapshot copier fixed to copy metadata file also, and test fixed
 - Existing tables need to be repaired by addition of metadata, before this can be rolled out
This commit is contained in:
Vinoth Chandar
2017-03-26 17:15:52 -07:00
committed by vinoth chandar
parent 3129770fd0
commit f9fd16069d
9 changed files with 214 additions and 207 deletions

View File

@@ -22,6 +22,7 @@ import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.table.HoodieTableConfig;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
@@ -50,6 +51,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Hoodie snapshot copy job which copies latest files from all partitions to another place, for snapshot backup.
@@ -71,13 +73,14 @@ public class HoodieSnapshotCopier implements Serializable {
final TableFileSystemView fsView = new HoodieTableFileSystemView(tableMetadata,
tableMetadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
// Get the latest commit
final Optional<HoodieInstant>
latestCommit = tableMetadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
Optional<HoodieInstant> latestCommit = tableMetadata.getActiveTimeline()
.getCommitTimeline().filterCompletedInstants().lastInstant();
if(!latestCommit.isPresent()) {
logger.warn("No commits present. Nothing to snapshot");
} else {
logger.info(String.format("Starting to snapshot latest version files which are also no-late-than %targetBasePath.", latestCommit.get()));
return;
}
final String latestCommitTimestamp = latestCommit.get().getTimestamp();
logger.info(String.format("Starting to snapshot latest version files which are also no-late-than %s.", latestCommitTimestamp));
List<String> partitions = FSUtils.getAllPartitionPaths(fs, baseDir);
if (partitions.size() > 0) {
@@ -90,51 +93,47 @@ public class HoodieSnapshotCopier implements Serializable {
fs.delete(new Path(outputDir), true);
}
jsc.parallelize(partitions, partitions.size()).flatMap(new FlatMapFunction<String, Tuple2<String, String>>() {
@Override
public Iterator<Tuple2<String, String>> call(String partition) throws Exception {
// Only take latest version files <= latestCommit.
FileSystem fs = FSUtils.getFs();
List<Tuple2<String, String>> filePaths = new ArrayList<>();
for (HoodieDataFile hoodieDataFile : fsView
.getLatestVersionInPartition(partition, latestCommit.get().getTimestamp())
.collect(Collectors.toList())) {
filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath()));
}
return filePaths.iterator();
}
}).foreach(new VoidFunction<Tuple2<String, String>>() {
@Override
public void call(Tuple2<String, String> tuple) throws Exception {
String partition = tuple._1();
Path sourceFilePath = new Path(tuple._2());
Path toPartitionPath = new Path(outputDir, partition);
FileSystem fs = FSUtils.getFs();
jsc.parallelize(partitions, partitions.size())
.flatMap(partition -> {
// Only take latest version files <= latestCommit.
FileSystem fs1 = FSUtils.getFs();
List<Tuple2<String, String>> filePaths = new ArrayList<>();
Stream<HoodieDataFile> dataFiles = fsView.getLatestVersionInPartition(partition, latestCommitTimestamp);
dataFiles.forEach(hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath())));
if (!fs.exists(toPartitionPath)) {
fs.mkdirs(toPartitionPath);
}
FileUtil.copy(fs, sourceFilePath, fs, new Path(toPartitionPath, sourceFilePath.getName()),
false, fs.getConf());
}
// also need to copy over partition metadata
Path partitionMetaFile = new Path(new Path(baseDir, partition), HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE);
if (fs1.exists(partitionMetaFile)) {
filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString()));
}
return filePaths.iterator();
}).foreach(tuple -> {
String partition = tuple._1();
Path sourceFilePath = new Path(tuple._2());
Path toPartitionPath = new Path(outputDir, partition);
FileSystem fs1 = FSUtils.getFs();
if (!fs1.exists(toPartitionPath)) {
fs1.mkdirs(toPartitionPath);
}
FileUtil.copy(fs1, sourceFilePath, fs1,
new Path(toPartitionPath, sourceFilePath.getName()), false, fs1.getConf());
});
// Also copy the .commit files
logger.info(String.format("Copying .commit files which are no-late-than %targetBasePath.", latestCommit.get()));
logger.info(String.format("Copying .commit files which are no-late-than %s.", latestCommitTimestamp));
FileStatus[] commitFilesToCopy = fs.listStatus(
new Path(baseDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME), new PathFilter() {
@Override
public boolean accept(Path commitFilePath) {
if (commitFilePath.getName().equals(HoodieTableConfig.HOODIE_PROPERTIES_FILE)) {
return true;
} else {
String commitTime =
FSUtils.getCommitFromCommitFile(commitFilePath.getName());
return tableMetadata.getActiveTimeline().getCommitTimeline()
.compareTimestamps(commitTime, latestCommit.get().getTimestamp(), HoodieTimeline.GREATER);
}
}
});
new Path(baseDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME), (commitFilePath) -> {
if (commitFilePath.getName().equals(HoodieTableConfig.HOODIE_PROPERTIES_FILE)) {
return true;
} else {
String commitTime =
FSUtils.getCommitFromCommitFile(commitFilePath.getName());
return tableMetadata.getActiveTimeline().getCommitTimeline()
.compareTimestamps(commitTime, latestCommitTimestamp, HoodieTimeline.LESSER_OR_EQUAL);
}
});
for (FileStatus commitStatus : commitFilesToCopy) {
Path targetFilePath = new Path(
outputDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitStatus

View File

@@ -16,6 +16,7 @@
package com.uber.hoodie.utilities;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileSystem;
@@ -64,77 +65,79 @@ public class TestHoodieSnapshotCopier {
HoodieSnapshotCopier copier = new HoodieSnapshotCopier();
copier.snapshot(jsc, basePath, outputPath);
// Nothing changed except _SUCCESS
// Nothing changed; we just bail out
assertEquals(fs.listStatus(new Path(basePath)).length, 1);
assertTrue(fs.exists(new Path(outputPath + "/_SUCCESS")));
assertFalse(fs.exists(new Path(outputPath + "/_SUCCESS")));
}
//TODO - uncomment this after fixing test failures
// @Test
// public void testSnapshotCopy() throws Exception {
// // Generate some commits and corresponding parquets
// String commitTime1 = "20160501010101";
// String commitTime2 = "20160502020601";
// String commitTime3 = "20160506030611";
// new File(basePath + "/.hoodie").mkdirs();
// new File(basePath + "/.hoodie/hoodie.properties").createNewFile();
// // Only first two have commit files
// new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
// new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
// new File(basePath + "/.hoodie/" + commitTime3 + ".inflight").createNewFile();
//
// // Some parquet files
// new File(basePath + "/2016/05/01/").mkdirs();
// new File(basePath + "/2016/05/02/").mkdirs();
// new File(basePath + "/2016/05/06/").mkdirs();
//
// // Make commit1
// File file11 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime1, 1, "id11"));
// file11.createNewFile();
// File file12 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime1, 1, "id12"));
// file12.createNewFile();
// File file13 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime1, 1, "id13"));
// file13.createNewFile();
//
// // Make commit2
// File file21 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime2, 1, "id21"));
// file21.createNewFile();
// File file22 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime2, 1, "id22"));
// file22.createNewFile();
// File file23 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime2, 1, "id23"));
// file23.createNewFile();
//
// // Make commit3
// File file31 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime3, 1, "id31"));
// file31.createNewFile();
// File file32 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime3, 1, "id32"));
// file32.createNewFile();
// File file33 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime3, 1, "id33"));
// file33.createNewFile();
//
// // Do a snapshot copy
// HoodieSnapshotCopier copier = new HoodieSnapshotCopier();
// copier.snapshot(jsc, basePath, outputPath);
//
// // Check results
// assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file11.getName())));
// assertTrue(fs.exists(new Path(outputPath + "/2016/05/02/" + file12.getName())));
// assertTrue(fs.exists(new Path(outputPath + "/2016/05/06/" + file13.getName())));
// assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file21.getName())));
// assertTrue(fs.exists(new Path(outputPath + "/2016/05/02/" + file22.getName())));
// assertTrue(fs.exists(new Path(outputPath + "/2016/05/06/" + file23.getName())));
// assertFalse(fs.exists(new Path(outputPath + "/2016/05/01/" + file31.getName())));
// assertFalse(fs.exists(new Path(outputPath + "/2016/05/02/" + file32.getName())));
// assertFalse(fs.exists(new Path(outputPath + "/2016/05/06/" + file33.getName())));
//
// assertTrue(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime1 + ".commit")));
// assertTrue(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime2 + ".commit")));
// assertFalse(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime3 + ".commit")));
// assertFalse(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime3 + ".inflight")));
// assertTrue(fs.exists(new Path(outputPath + "/.hoodie/hoodie.properties")));
//
// assertTrue(fs.exists(new Path(outputPath + "/_SUCCESS")));
// }
@Test
public void testSnapshotCopy() throws Exception {
// Generate some commits and corresponding parquets
String commitTime1 = "20160501010101";
String commitTime2 = "20160502020601";
String commitTime3 = "20160506030611";
new File(basePath + "/.hoodie").mkdirs();
new File(basePath + "/.hoodie/hoodie.properties").createNewFile();
// Only first two have commit files
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".inflight").createNewFile();
// Some parquet files
new File(basePath + "/2016/05/01/").mkdirs();
new File(basePath + "/2016/05/02/").mkdirs();
new File(basePath + "/2016/05/06/").mkdirs();
HoodieTestDataGenerator.writePartitionMetadata(fs,
new String[] {"2016/05/01", "2016/05/02", "2016/05/06"},
basePath);
// Make commit1
File file11 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime1, 1, "id11"));
file11.createNewFile();
File file12 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime1, 1, "id12"));
file12.createNewFile();
File file13 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime1, 1, "id13"));
file13.createNewFile();
// Make commit2
File file21 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime2, 1, "id21"));
file21.createNewFile();
File file22 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime2, 1, "id22"));
file22.createNewFile();
File file23 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime2, 1, "id23"));
file23.createNewFile();
// Make commit3
File file31 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime3, 1, "id31"));
file31.createNewFile();
File file32 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime3, 1, "id32"));
file32.createNewFile();
File file33 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime3, 1, "id33"));
file33.createNewFile();
// Do a snapshot copy
HoodieSnapshotCopier copier = new HoodieSnapshotCopier();
copier.snapshot(jsc, basePath, outputPath);
// Check results
assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file11.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/02/" + file12.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/06/" + file13.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file21.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/02/" + file22.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/06/" + file23.getName())));
assertFalse(fs.exists(new Path(outputPath + "/2016/05/01/" + file31.getName())));
assertFalse(fs.exists(new Path(outputPath + "/2016/05/02/" + file32.getName())));
assertFalse(fs.exists(new Path(outputPath + "/2016/05/06/" + file33.getName())));
assertTrue(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime1 + ".commit")));
assertTrue(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime2 + ".commit")));
assertFalse(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime3 + ".commit")));
assertFalse(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime3 + ".inflight")));
assertTrue(fs.exists(new Path(outputPath + "/.hoodie/hoodie.properties")));
assertTrue(fs.exists(new Path(outputPath + "/_SUCCESS")));
}
@After
public void cleanup() {