1
0

Writes relative paths to .commit files instead of absolute paths

Clean up code

Removed commented out code

Fixed merge conflict with master
This commit is contained in:
gekath
2017-06-02 11:28:47 -04:00
committed by prazanna
parent 0ed3fac5e3
commit db7311f85e
12 changed files with 92 additions and 34 deletions

View File

@@ -210,9 +210,9 @@ public class HoodieReadClient implements Serializable {
HoodieCommitMetadata metadata =
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
// get files from each commit, and replace any previous versions
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths());
String basePath = hoodieTable.getMetaClient().getBasePath();
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
}
return sqlContextOpt.get().read()
.parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]))
.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTimestamp));
@@ -234,11 +234,12 @@ public class HoodieReadClient implements Serializable {
}
try {
HoodieCommitMetadata commitMetdata =
HoodieCommitMetadata commitMetadata =
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commitInstant).get());
Collection<String> paths = commitMetdata.getFileIdAndFullPaths().values();
String basePath = hoodieTable.getMetaClient().getBasePath();
HashMap<String, String> paths = commitMetadata.getFileIdAndFullPaths(basePath);
return sqlContextOpt.get().read()
.parquet(paths.toArray(new String[paths.size()]))
.parquet(paths.values().toArray(new String[paths.size()]))
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime));
} catch (Exception e) {
throw new HoodieException("Error reading commit " + commitTime, e);

View File

@@ -80,6 +80,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
fileSystemView.getLatestDataFilesForFileId(record.getPartitionPath(), fileId)
.findFirst().get().getFileName();
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
Path path = new Path(record.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
writeStatus.getStat().setPrevCommit(baseCommitTime);
writeStatus.setFileId(fileId);
writeStatus.setPartitionPath(record.getPartitionPath());
@@ -103,7 +105,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
+ " on commit " + commitTime + " on HDFS path " + hoodieTable
.getMetaClient().getBasePath() + partitionPath, e);
}
writeStatus.getStat().setFullPath(currentLogFile.getPath().toString());
writeStatus.getStat().setPath(path.toString());
}
// update the new location of the record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));

View File

@@ -123,12 +123,14 @@ public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOH
try {
storageWriter.close();
String relativePath = path.toString().replace(new Path(config.getBasePath()) + "/", "");
HoodieWriteStat stat = new HoodieWriteStat();
stat.setNumWrites(recordsWritten);
stat.setNumDeletes(recordsDeleted);
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
stat.setFileId(status.getFileId());
stat.setFullPath(path.toString());
stat.setPath(relativePath);
stat.setTotalWriteBytes(FSUtils.getFileSize(fs, path));
stat.setTotalWriteErrors(status.getFailedRecords().size());
status.setStat(stat);

View File

@@ -93,9 +93,9 @@ public class HoodieUpdateHandle <T extends HoodieRecordPayload> extends HoodieIO
oldFilePath = new Path(
config.getBasePath() + "/" + record.getPartitionPath() + "/"
+ latestValidFilePath);
newFilePath = new Path(
config.getBasePath() + "/" + record.getPartitionPath() + "/" + FSUtils
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
String relativePath = new Path( record.getPartitionPath() + "/" + FSUtils
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
newFilePath = new Path(config.getBasePath(), relativePath);
// handle cases of partial failures, for update task
if (fs.exists(newFilePath)) {
@@ -108,7 +108,7 @@ public class HoodieUpdateHandle <T extends HoodieRecordPayload> extends HoodieIO
writeStatus.setFileId(fileId);
writeStatus.setPartitionPath(record.getPartitionPath());
writeStatus.getStat().setFileId(fileId);
writeStatus.getStat().setFullPath(newFilePath.toString());
writeStatus.getStat().setPath(relativePath);
}
keyToNewRecords.put(record.getRecordKey(), record);
// update the new location of the record, so we know where to find it next

View File

@@ -114,6 +114,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
for (CompactionWriteStat stat : updateStatusMap) {
metadata.addWriteStat(stat.getPartitionPath(), stat);
}
log.info("Compaction finished with result " + metadata);
//noinspection ConstantConditions

View File

@@ -45,6 +45,7 @@ import com.uber.hoodie.table.HoodieTable;
import java.util.Map;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.SparkConf;
@@ -60,6 +61,7 @@ import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
@@ -101,7 +103,6 @@ public class TestHoodieClient implements Serializable {
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(basePath);
dataGen = new HoodieTestDataGenerator();
}
@@ -616,7 +617,7 @@ public class TestHoodieClient implements Serializable {
if (!fileIdToVersions.containsKey(wstat.getFileId())) {
fileIdToVersions.put(wstat.getFileId(), new TreeSet<>());
}
fileIdToVersions.get(wstat.getFileId()).add(FSUtils.getCommitTime(new Path(wstat.getFullPath()).getName()));
fileIdToVersions.get(wstat.getFileId()).add(FSUtils.getCommitTime(new Path(wstat.getPath()).getName()));
}
}
@@ -1136,7 +1137,6 @@ public class TestHoodieClient implements Serializable {
List<HoodieCleanStat> hoodieCleanStatsFour = table.clean(jsc);
assertEquals("Must not clean any files" , 0, getCleanStat(hoodieCleanStatsFour, partitionPaths[0]).getSuccessDeleteFiles().size());
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file3P0C2));
}
@Test
public void testKeepLatestCommits() throws IOException {
@@ -1298,6 +1298,47 @@ public class TestHoodieClient implements Serializable {
stageOneShuffleReadTaskRecordsCountMap.values().stream().filter(a -> a > 10 && a < 100).count() == 3);
}
public void testCommitWritesRelativePaths() throws Exception {
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
FileSystem fs = FSUtils.getFs();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg);
String commitTime = "000";
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
assertTrue("Commit should succeed", client.commit(commitTime, result));
assertTrue("After explicit commit, commit file should be created",
HoodieTestUtils.doesCommitExist(basePath, commitTime));
// Get parquet file paths from commit metadata
String actionType = table.getCompactedCommitActionType();
HoodieInstant commitInstant =
new HoodieInstant(false, actionType, commitTime);
HoodieTimeline commitTimeline = table.getCompletedCompactionCommitTimeline();
HoodieCommitMetadata commitMetadata =
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commitInstant).get());
String basePath = table.getMetaClient().getBasePath();
Collection<String> commitPathNames = commitMetadata.getFileIdAndFullPaths(basePath).values();
// Read from commit file
String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
FileInputStream inputStream = new FileInputStream(filename);
String everything = IOUtils.toString(inputStream);
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString());
HashMap<String, String> paths = metadata.getFileIdAndFullPaths(basePath);
inputStream.close();
// Compare values in both to make sure they are equal.
for (String pathName: paths.values()) {
assertTrue(commitPathNames.contains(pathName));
}
private HoodieCleanStat getCleanStat(List<HoodieCleanStat> hoodieCleanStatsTwo,
String partitionPath) {
return hoodieCleanStatsTwo.stream()

View File

@@ -129,8 +129,9 @@ public class TestHoodieCompactor {
HoodieCompactionMetadata result =
compactor.compact(jsc, getConfig(), table);
String basePath = table.getMetaClient().getBasePath();
assertTrue("If there is nothing to compact, result will be empty",
result.getFileIdAndFullPaths().isEmpty());
result.getFileIdAndFullPaths(basePath).isEmpty());
}
@Test