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

@@ -18,31 +18,30 @@ package com.uber.hoodie;
import com.google.common.collect.Iterables;
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -60,7 +59,6 @@ import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
@@ -122,6 +120,15 @@ public class TestHoodieClient implements Serializable {
}
}
private void assertPartitionMetadata(String[] partitionPaths, FileSystem fs) throws IOException {
for (String partitionPath: partitionPaths) {
assertTrue(HoodiePartitionMetadata.hasPartitionMetadata(fs, new Path(basePath, partitionPath)));
HoodiePartitionMetadata pmeta = new HoodiePartitionMetadata(fs, new Path(basePath, partitionPath));
pmeta.readFromFS();
assertEquals(3, pmeta.getPartitionDepth());
}
}
private void checkTaggedRecords(List<HoodieRecord> taggedRecords, String commitTime) {
for (HoodieRecord rec : taggedRecords) {
assertTrue("Record " + rec + " found with no location.", rec.isCurrentLocationKnown());
@@ -203,6 +210,9 @@ public class TestHoodieClient implements Serializable {
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
// check the partition metadata is written out
assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs);
// verify that there is a commit
HoodieReadClient readClient = new HoodieReadClient(jsc, basePath, sqlContext);
assertEquals("Expecting a single commit.", readClient.listCommitsSince("000").size(), 1);
@@ -348,6 +358,7 @@ public class TestHoodieClient implements Serializable {
.build()).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
FileSystem fs = FSUtils.getFs();
HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
/**
* Write 1 (only inserts)
@@ -433,6 +444,7 @@ public class TestHoodieClient implements Serializable {
.build()).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
FileSystem fs = FSUtils.getFs();
HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
/**
* Write 1 (only inserts)
@@ -712,6 +724,10 @@ public class TestHoodieClient implements Serializable {
String commitTime2 = "20160502020601";
String commitTime3 = "20160506030611";
new File(basePath + "/.hoodie").mkdirs();
HoodieTestDataGenerator.writePartitionMetadata(FSUtils.getFs(),
new String[] {"2016/05/01", "2016/05/02", "2016/05/06"},
basePath);
// Only first two have commit files
HoodieTestUtils.createCommitFiles(basePath, commitTime1, commitTime2);
@@ -801,6 +817,9 @@ public class TestHoodieClient implements Serializable {
String commitTime2 = "20160502020601";
String commitTime3 = "20160506030611";
new File(basePath + "/.hoodie").mkdirs();
HoodieTestDataGenerator.writePartitionMetadata(FSUtils.getFs(),
new String[] {"2016/05/01", "2016/05/02", "2016/05/06"},
basePath);
// One good commit
HoodieTestUtils.createCommitFiles(basePath, commitTime1);
@@ -984,7 +1003,6 @@ public class TestHoodieClient implements Serializable {
// setup the small file handling params
HoodieWriteConfig config = getSmallInsertWriteConfig(INSERT_SPLIT_LIMIT); // hold upto 20 records max
dataGen = new HoodieTestDataGenerator(new String[] {TEST_PARTITION_PATH});
HoodieWriteClient client = new HoodieWriteClient(jsc, config);
// Inserts => will write file1
@@ -995,6 +1013,7 @@ public class TestHoodieClient implements Serializable {
List<WriteStatus> statuses= client.insert(insertRecordsRDD1, commitTime1).collect();
assertNoWriteErrors(statuses);
assertPartitionMetadata(new String[]{TEST_PARTITION_PATH}, FSUtils.getFs());
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
String file1 = statuses.get(0).getFileId();