Incorporating code review feedback for finalizeWrite for COW #4
This commit is contained in:
@@ -58,10 +58,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
|||||||
private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
|
private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
|
||||||
private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class";
|
private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class";
|
||||||
private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName();
|
private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName();
|
||||||
private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER = "hoodie.copyonwrite.use.temp.folder";
|
private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE = "hoodie.copyonwrite.use.temp.folder.for.create";
|
||||||
private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER = "false";
|
private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE = "false";
|
||||||
private static final String HOODIE_MERGEHANDLE_USE_TEMP_FOLDER = "hoodie.mergehandle.use.temp.folder";
|
private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = "hoodie.copyonwrite.use.temp.folder.for.merge";
|
||||||
private static final String DEFAULT_HOODIE_MERGEHANDLE_USE_TEMP_FOLDER = "false";
|
private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = "false";
|
||||||
private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism";
|
private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism";
|
||||||
private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM;
|
private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM;
|
||||||
|
|
||||||
@@ -120,12 +120,17 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
|||||||
return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP);
|
return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP);
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean shouldUseTempFolderForCopyOnWrite() {
|
public boolean shouldUseTempFolderForCopyOnWriteForCreate() {
|
||||||
return Boolean.parseBoolean(props.getProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER));
|
return Boolean.parseBoolean(props.getProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE));
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean shouldUseTempFolderForMergeHandle() {
|
public boolean shouldUseTempFolderForCopyOnWriteForMerge() {
|
||||||
return Boolean.parseBoolean(props.getProperty(HOODIE_MERGEHANDLE_USE_TEMP_FOLDER));
|
return Boolean.parseBoolean(props.getProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE));
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean shouldUseTempFolderForCopyOnWrite() {
|
||||||
|
return shouldUseTempFolderForCopyOnWriteForCreate() ||
|
||||||
|
shouldUseTempFolderForCopyOnWriteForMerge();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getFinalizeWriteParallelism() {
|
public int getFinalizeWriteParallelism() {
|
||||||
@@ -403,13 +408,17 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder withUseTempFolderCopyOnWrite(boolean shouldUseTempFolderCopyOnWrite) {
|
public Builder withUseTempFolderCopyOnWriteForCreate(
|
||||||
props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER, String.valueOf(shouldUseTempFolderCopyOnWrite));
|
boolean shouldUseTempFolderCopyOnWriteForCreate) {
|
||||||
|
props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE, String.valueOf
|
||||||
|
(shouldUseTempFolderCopyOnWriteForCreate));
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder withUseTempFolderMergeHandle(boolean shouldUseTempFolderMergeHandle) {
|
public Builder withUseTempFolderCopyOnWriteForMerge(
|
||||||
props.setProperty(HOODIE_MERGEHANDLE_USE_TEMP_FOLDER, String.valueOf(shouldUseTempFolderMergeHandle));
|
boolean shouldUseTempFolderCopyOnWriteForMerge) {
|
||||||
|
props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE, String.valueOf
|
||||||
|
(shouldUseTempFolderCopyOnWriteForMerge));
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -441,10 +450,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
|||||||
HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING);
|
HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING);
|
||||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP),
|
setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP),
|
||||||
HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS);
|
HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS);
|
||||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_COPYONWRITE_USE_TEMP_FOLDER),
|
setDefaultOnCondition(props, !props.containsKey(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE),
|
||||||
HOODIE_COPYONWRITE_USE_TEMP_FOLDER, DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER);
|
HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE, DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE);
|
||||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_MERGEHANDLE_USE_TEMP_FOLDER),
|
setDefaultOnCondition(props, !props.containsKey(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE),
|
||||||
HOODIE_MERGEHANDLE_USE_TEMP_FOLDER, DEFAULT_HOODIE_MERGEHANDLE_USE_TEMP_FOLDER);
|
HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE, DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE);
|
||||||
setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM),
|
setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM),
|
||||||
FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM);
|
FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM);
|
||||||
|
|
||||||
|
|||||||
@@ -58,8 +58,9 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
|||||||
|
|
||||||
final int sparkPartitionId = TaskContext.getPartitionId();
|
final int sparkPartitionId = TaskContext.getPartitionId();
|
||||||
this.path = makeNewPath(partitionPath, sparkPartitionId, status.getFileId());
|
this.path = makeNewPath(partitionPath, sparkPartitionId, status.getFileId());
|
||||||
if (config.shouldUseTempFolderForCopyOnWrite()) {
|
if (config.shouldUseTempFolderForCopyOnWriteForCreate()) {
|
||||||
this.tempPath = makeTempPath(partitionPath, sparkPartitionId, status.getFileId(), TaskContext.get().stageId(), TaskContext.get().taskAttemptId());
|
this.tempPath = makeTempPath(partitionPath, sparkPartitionId, status.getFileId(),
|
||||||
|
TaskContext.get().stageId(), TaskContext.get().taskAttemptId());
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
@@ -144,9 +145,6 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
|||||||
|
|
||||||
private Path getStorageWriterPath() {
|
private Path getStorageWriterPath() {
|
||||||
// Use tempPath for storage writer if possible
|
// Use tempPath for storage writer if possible
|
||||||
if (this.tempPath != null) {
|
return (this.tempPath == null) ? this.path : this.tempPath;
|
||||||
return this.tempPath;
|
|
||||||
}
|
|
||||||
return this.path;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -101,7 +101,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
|||||||
String relativePath = new Path(record.getPartitionPath() + "/" + FSUtils
|
String relativePath = new Path(record.getPartitionPath() + "/" + FSUtils
|
||||||
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
|
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
|
||||||
newFilePath = new Path(config.getBasePath(), relativePath);
|
newFilePath = new Path(config.getBasePath(), relativePath);
|
||||||
if (config.shouldUseTempFolderForCopyOnWrite() && config.shouldUseTempFolderForMergeHandle()) {
|
if (config.shouldUseTempFolderForCopyOnWriteForMerge()) {
|
||||||
this.tempPath = makeTempPath(record.getPartitionPath(), TaskContext.getPartitionId(), fileId, TaskContext.get().stageId(), TaskContext.get().taskAttemptId());
|
this.tempPath = makeTempPath(record.getPartitionPath(), TaskContext.getPartitionId(), fileId, TaskContext.get().stageId(), TaskContext.get().taskAttemptId());
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -111,7 +111,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
|||||||
}
|
}
|
||||||
|
|
||||||
logger.info(String.format("Merging new data into oldPath %s, as newPath %s",
|
logger.info(String.format("Merging new data into oldPath %s, as newPath %s",
|
||||||
oldFilePath.toString(), getNewFilePath().toString()));
|
oldFilePath.toString(), getStorageWriterPath().toString()));
|
||||||
// file name is same for all records, in this bunch
|
// file name is same for all records, in this bunch
|
||||||
writeStatus.setFileId(fileId);
|
writeStatus.setFileId(fileId);
|
||||||
writeStatus.setPartitionPath(record.getPartitionPath());
|
writeStatus.setPartitionPath(record.getPartitionPath());
|
||||||
@@ -124,7 +124,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
|||||||
}
|
}
|
||||||
// Create the writer for writing the new version file
|
// Create the writer for writing the new version file
|
||||||
storageWriter = HoodieStorageWriterFactory
|
storageWriter = HoodieStorageWriterFactory
|
||||||
.getStorageWriter(commitTime, getNewFilePath(), hoodieTable, config, schema);
|
.getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema);
|
||||||
|
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
logger.error("Error in update task at commit " + commitTime, e);
|
logger.error("Error in update task at commit " + commitTime, e);
|
||||||
@@ -190,18 +190,18 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
|||||||
if (copyOldRecord) {
|
if (copyOldRecord) {
|
||||||
// this should work as it is, since this is an existing record
|
// this should work as it is, since this is an existing record
|
||||||
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
|
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
|
||||||
+ getOldFilePath() + " to new file " + getNewFilePath();
|
+ getOldFilePath() + " to new file " + getStorageWriterPath();
|
||||||
try {
|
try {
|
||||||
storageWriter.writeAvro(key, oldRecord);
|
storageWriter.writeAvro(key, oldRecord);
|
||||||
} catch (ClassCastException e) {
|
} catch (ClassCastException e) {
|
||||||
logger.error(
|
logger.error(
|
||||||
"Schema mismatch when rewriting old record " + oldRecord + " from file "
|
"Schema mismatch when rewriting old record " + oldRecord + " from file "
|
||||||
+ getOldFilePath() + " to file " + getNewFilePath() + " with schema " + schema
|
+ getOldFilePath() + " to file " + getStorageWriterPath() + " with schema " + schema
|
||||||
.toString(true));
|
.toString(true));
|
||||||
throw new HoodieUpsertException(errMsg, e);
|
throw new HoodieUpsertException(errMsg, e);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
logger.error("Failed to merge old record into new file for key " + key + " from old file "
|
logger.error("Failed to merge old record into new file for key " + key + " from old file "
|
||||||
+ getOldFilePath() + " to new file " + getNewFilePath(), e);
|
+ getOldFilePath() + " to new file " + getStorageWriterPath(), e);
|
||||||
throw new HoodieUpsertException(errMsg, e);
|
throw new HoodieUpsertException(errMsg, e);
|
||||||
}
|
}
|
||||||
recordsWritten++;
|
recordsWritten++;
|
||||||
@@ -223,7 +223,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
|||||||
storageWriter.close();
|
storageWriter.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, getNewFilePath()));
|
writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, getStorageWriterPath()));
|
||||||
writeStatus.getStat().setNumWrites(recordsWritten);
|
writeStatus.getStat().setNumWrites(recordsWritten);
|
||||||
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
||||||
writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten);
|
writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten);
|
||||||
@@ -237,12 +237,9 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
|||||||
return oldFilePath;
|
return oldFilePath;
|
||||||
}
|
}
|
||||||
|
|
||||||
private Path getNewFilePath() {
|
private Path getStorageWriterPath() {
|
||||||
// Use tempPath for storage writer if possible
|
// Use tempPath for storage writer if possible
|
||||||
if (this.tempPath != null) {
|
return (this.tempPath == null) ? this.newFilePath : this.tempPath;
|
||||||
return this.tempPath;
|
|
||||||
}
|
|
||||||
return this.newFilePath;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public WriteStatus getWriteStatus() {
|
public WriteStatus getWriteStatus() {
|
||||||
|
|||||||
@@ -579,6 +579,12 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
return stats;
|
return stats;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Finalize the written data files
|
||||||
|
*
|
||||||
|
* @param writeStatuses List of WriteStatus
|
||||||
|
* @return number of files finalized
|
||||||
|
*/
|
||||||
@Override
|
@Override
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public Optional<Integer> finalizeWrite(JavaSparkContext jsc, List writeStatuses) {
|
public Optional<Integer> finalizeWrite(JavaSparkContext jsc, List writeStatuses) {
|
||||||
@@ -618,6 +624,10 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
return Optional.of(results.size());
|
return Optional.of(results.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clean temporary data files that are produced from previous failed commit or retried spark
|
||||||
|
* stages.
|
||||||
|
*/
|
||||||
private void cleanTemporaryDataFiles(JavaSparkContext jsc) {
|
private void cleanTemporaryDataFiles(JavaSparkContext jsc) {
|
||||||
if (!config.shouldUseTempFolderForCopyOnWrite()) {
|
if (!config.shouldUseTempFolderForCopyOnWrite()) {
|
||||||
return;
|
return;
|
||||||
|
|||||||
@@ -280,5 +280,4 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
|||||||
* @return number of files finalized
|
* @return number of files finalized
|
||||||
*/
|
*/
|
||||||
public abstract Optional<Integer> finalizeWrite(JavaSparkContext jsc, List<Tuple2<String, HoodieWriteStat>> writeStatuses);
|
public abstract Optional<Integer> finalizeWrite(JavaSparkContext jsc, List<Tuple2<String, HoodieWriteStat>> writeStatuses);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -48,6 +48,7 @@ import com.uber.hoodie.config.HoodieStorageConfig;
|
|||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||||
import com.uber.hoodie.index.HoodieIndex;
|
import com.uber.hoodie.index.HoodieIndex;
|
||||||
|
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileInputStream;
|
import java.io.FileInputStream;
|
||||||
@@ -55,6 +56,7 @@ import java.io.IOException;
|
|||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.LinkedHashMap;
|
import java.util.LinkedHashMap;
|
||||||
@@ -211,9 +213,12 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUpserts() throws Exception {
|
public void testUpserts() throws Exception {
|
||||||
HoodieWriteConfig cfg = getConfig();
|
testUpsertsInternal(getConfig());
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
}
|
||||||
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
|
|
||||||
|
private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig) throws Exception {
|
||||||
|
HoodieWriteClient client = new HoodieWriteClient(jsc, hoodieWriteConfig);
|
||||||
|
HoodieIndex index = HoodieIndex.createIndex(hoodieWriteConfig, jsc);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Write 1 (only inserts)
|
* Write 1 (only inserts)
|
||||||
@@ -304,95 +309,11 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUpsertsWithFinalizeWrite() throws Exception {
|
public void testUpsertsWithFinalizeWrite() throws Exception {
|
||||||
HoodieWriteConfig cfg = getConfigBuilder()
|
HoodieWriteConfig hoodieWriteConfig = getConfigBuilder()
|
||||||
.withUseTempFolderCopyOnWrite(true)
|
.withUseTempFolderCopyOnWriteForCreate(true)
|
||||||
.withUseTempFolderMergeHandle(true)
|
.withUseTempFolderCopyOnWriteForMerge(true)
|
||||||
.build();
|
.build();
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
testUpsertsInternal(hoodieWriteConfig);
|
||||||
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Write 1 (only inserts)
|
|
||||||
*/
|
|
||||||
String newCommitTime = "001";
|
|
||||||
client.startCommitWithTime(newCommitTime);
|
|
||||||
|
|
||||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
|
||||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
|
||||||
|
|
||||||
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
|
|
||||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
|
||||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
|
||||||
|
|
||||||
assertEquals("Expecting a single commit.", 1,
|
|
||||||
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
|
||||||
assertEquals("Latest commit should be 001", newCommitTime,
|
|
||||||
timeline.lastInstant().get().getTimestamp());
|
|
||||||
assertEquals("Must contain 200 records",
|
|
||||||
records.size(),
|
|
||||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
|
|
||||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
|
||||||
|
|
||||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table)
|
|
||||||
.collect();
|
|
||||||
checkTaggedRecords(taggedRecords, "001");
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Write 2 (updates)
|
|
||||||
*/
|
|
||||||
newCommitTime = "004";
|
|
||||||
client.startCommitWithTime(newCommitTime);
|
|
||||||
|
|
||||||
records = dataGen.generateUpdates(newCommitTime, 100);
|
|
||||||
LinkedHashMap<HoodieKey, HoodieRecord> recordsMap = new LinkedHashMap<>();
|
|
||||||
for (HoodieRecord rec : records) {
|
|
||||||
if (!recordsMap.containsKey(rec.getKey())) {
|
|
||||||
recordsMap.put(rec.getKey(), rec);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
List<HoodieRecord> dedupedRecords = new ArrayList<>(recordsMap.values());
|
|
||||||
|
|
||||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
|
||||||
// Verify there are no errors
|
|
||||||
assertNoWriteErrors(statuses);
|
|
||||||
|
|
||||||
// verify there are now 2 commits
|
|
||||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
|
||||||
assertEquals("Expecting two commits.",
|
|
||||||
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2);
|
|
||||||
assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(),
|
|
||||||
newCommitTime);
|
|
||||||
|
|
||||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
|
||||||
table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
|
||||||
|
|
||||||
// Index should be able to locate all updates in correct locations.
|
|
||||||
taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), table).collect();
|
|
||||||
checkTaggedRecords(taggedRecords, "004");
|
|
||||||
|
|
||||||
// Check the entire dataset has 100 records still
|
|
||||||
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
|
|
||||||
for (int i = 0; i < fullPartitionPaths.length; i++) {
|
|
||||||
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
|
|
||||||
}
|
|
||||||
assertEquals("Must contain 200 records",
|
|
||||||
200,
|
|
||||||
HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count());
|
|
||||||
|
|
||||||
// Check that the incremental consumption from time 000
|
|
||||||
assertEquals("Incremental consumption from time 002, should give all records in commit 004",
|
|
||||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
|
||||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "002").count());
|
|
||||||
assertEquals("Incremental consumption from time 001, should give all records in commit 004",
|
|
||||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
|
||||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "001").count());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@@ -1575,6 +1496,32 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
|||||||
.count() == 3);
|
.count() == 3);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCleanTemporaryDataFiles() throws IOException {
|
||||||
|
HoodieTestUtils.createCommitFiles(basePath, "000");
|
||||||
|
List<String> tempFiles = createTempFiles("000", 10);
|
||||||
|
assertEquals("Some temp files are created.",10, tempFiles.size());
|
||||||
|
assertEquals("Some temp files are created.",tempFiles.size(), getTotalTempFiles());
|
||||||
|
|
||||||
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
|
.withUseTempFolderCopyOnWriteForCreate(false)
|
||||||
|
.withUseTempFolderCopyOnWriteForMerge(false).build();
|
||||||
|
HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||||
|
config.getBasePath(), true),
|
||||||
|
config);
|
||||||
|
table.rollback(jsc, Collections.emptyList());
|
||||||
|
assertEquals("Some temp files are created.",tempFiles.size(), getTotalTempFiles());
|
||||||
|
|
||||||
|
config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
|
.withUseTempFolderCopyOnWriteForCreate(true)
|
||||||
|
.withUseTempFolderCopyOnWriteForMerge(false).build();
|
||||||
|
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||||
|
config.getBasePath(), true),
|
||||||
|
config);
|
||||||
|
table.rollback(jsc, Collections.emptyList());
|
||||||
|
assertEquals("All temp files are deleted.",0, getTotalTempFiles());
|
||||||
|
}
|
||||||
|
|
||||||
public void testCommitWritesRelativePaths() throws Exception {
|
public void testCommitWritesRelativePaths() throws Exception {
|
||||||
|
|
||||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||||
@@ -1642,6 +1589,18 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
|||||||
return files;
|
return files;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private List<String> createTempFiles(String commitTime, int numFiles) throws IOException {
|
||||||
|
List<String> files = new ArrayList<>();
|
||||||
|
for (int i = 0; i < numFiles; i++) {
|
||||||
|
files.add(HoodieTestUtils.createNewDataFile(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, commitTime));
|
||||||
|
}
|
||||||
|
return files;
|
||||||
|
}
|
||||||
|
|
||||||
|
private int getTotalTempFiles() throws IOException {
|
||||||
|
return fs.listStatus(new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME)).length;
|
||||||
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
public void clean() {
|
public void clean() {
|
||||||
if (basePath != null) {
|
if (basePath != null) {
|
||||||
|
|||||||
Reference in New Issue
Block a user