[HUDI-650] Modify handleUpdate path to validate partitionPath (#1368)
This commit is contained in:
@@ -118,7 +118,8 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
||||
updateRecords.add(record1);
|
||||
|
||||
try {
|
||||
HoodieMergeHandle mergeHandle = new HoodieMergeHandle(config2, "101", table2, updateRecords.iterator(), fileId);
|
||||
HoodieMergeHandle mergeHandle = new HoodieMergeHandle(config2, "101", table2,
|
||||
updateRecords.iterator(), record1.getPartitionPath(), fileId);
|
||||
Configuration conf = new Configuration();
|
||||
AvroReadSupport.setAvroReadSchema(conf, mergeHandle.getWriterSchema());
|
||||
List<GenericRecord> oldRecords = ParquetUtils.readAvroRecords(conf,
|
||||
|
||||
@@ -18,8 +18,10 @@
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hudi.common.HoodieClientTestHarness;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieClientTestHarness;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.TestRawTripPayload;
|
||||
@@ -41,9 +43,6 @@ import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.table.HoodieCopyOnWriteTable.UpsertPartitioner;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
@@ -53,6 +52,7 @@ import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
@@ -61,8 +61,6 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
@@ -209,8 +207,8 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
final HoodieCopyOnWriteTable newTable = new HoodieCopyOnWriteTable(config, jsc);
|
||||
List<WriteStatus> statuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return newTable.handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(),
|
||||
updatedRecords.iterator());
|
||||
return newTable.handleUpdate(newCommitTime, updatedRecord1.getPartitionPath(),
|
||||
updatedRecord1.getCurrentLocation().getFileId(), updatedRecords.iterator());
|
||||
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
|
||||
|
||||
// Check the updated file
|
||||
@@ -470,7 +468,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
@Test
|
||||
public void testInsertUpsertWithHoodieAvroPayload() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build();
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
final HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
String commitTime = "000";
|
||||
@@ -484,13 +482,15 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
String fileId = writeStatus.getFileId();
|
||||
metaClient.getFs().create(new Path(basePath + "/.hoodie/000.commit")).close();
|
||||
final HoodieCopyOnWriteTable table2 = new HoodieCopyOnWriteTable(config, jsc);
|
||||
|
||||
final List<HoodieRecord> updates =
|
||||
dataGen.generateUpdatesWithHoodieAvroPayload(commitTime, writeStatus.getWrittenRecords());
|
||||
dataGen.generateUpdatesWithHoodieAvroPayload(commitTime, inserts);
|
||||
|
||||
jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return table2.handleUpdate("001", fileId, updates.iterator());
|
||||
String partitionPath = updates.get(0).getPartitionPath();
|
||||
long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count();
|
||||
final List<List<WriteStatus>> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return table.handleUpdate(commitTime, partitionPath, fileId, updates.iterator());
|
||||
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
assertEquals(updates.size() - numRecordsInPartition, updateStatus.get(0).get(0).getTotalErrorRecords());
|
||||
}
|
||||
|
||||
@After
|
||||
|
||||
@@ -18,17 +18,20 @@
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hudi.common.HoodieClientTestHarness;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.client.HoodieReadClient;
|
||||
import org.apache.hudi.client.HoodieWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieClientTestHarness;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieMergeOnReadTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.TestRawTripPayload.MetadataMergeWriteStatus;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieFileGroup;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -52,10 +55,6 @@ import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
@@ -1208,6 +1207,85 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to validate invoking table.handleUpdate() with input records from multiple partitions will fail.
|
||||
*/
|
||||
@Test
|
||||
public void testHandleUpdateWithMultiplePartitions() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
try (HoodieWriteClient client = getWriteClient(cfg);) {
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts, written as parquet file)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieMergeOnReadTable hoodieTable = (HoodieMergeOnReadTable) HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp());
|
||||
|
||||
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
assertFalse(commit.isPresent());
|
||||
|
||||
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
BaseFileOnlyView roView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
|
||||
Stream<HoodieBaseFile> dataFilesToRead = roView.getLatestBaseFiles();
|
||||
assertFalse(dataFilesToRead.findAny().isPresent());
|
||||
|
||||
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
||||
dataFilesToRead = roView.getLatestBaseFiles();
|
||||
assertTrue("should list the parquet files we wrote in the delta commit",
|
||||
dataFilesToRead.findAny().isPresent());
|
||||
|
||||
/**
|
||||
* Write 2 (only updates, written to .log file)
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
writeRecords = jsc.parallelize(records, 1);
|
||||
statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
/**
|
||||
* Write 3 (only deletes, written to .log file)
|
||||
*/
|
||||
final String newDeleteTime = "004";
|
||||
final String partitionPath = records.get(0).getPartitionPath();
|
||||
final String fileId = statuses.get(0).getFileId();
|
||||
client.startCommitWithTime(newDeleteTime);
|
||||
|
||||
List<HoodieRecord> fewRecordsForDelete = dataGen.generateDeletesFromExistingRecords(records);
|
||||
JavaRDD<HoodieRecord> deleteRDD = jsc.parallelize(fewRecordsForDelete, 1);
|
||||
|
||||
// initialize partitioner
|
||||
hoodieTable.getUpsertPartitioner(new WorkloadProfile(deleteRDD));
|
||||
final List<List<WriteStatus>> deleteStatus = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return hoodieTable.handleUpdate(newDeleteTime, partitionPath, fileId, fewRecordsForDelete.iterator());
|
||||
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
|
||||
// Verify there are errors because records are from multiple partitions (but handleUpdate is invoked for
|
||||
// specific partition)
|
||||
WriteStatus status = deleteStatus.get(0).get(0);
|
||||
assertTrue(status.hasErrors());
|
||||
long numRecordsInPartition = fewRecordsForDelete.stream().filter(u ->
|
||||
u.getPartitionPath().equals(partitionPath)).count();
|
||||
assertEquals(fewRecordsForDelete.size() - numRecordsInPartition, status.getTotalErrorRecords());
|
||||
}
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getConfig(Boolean autoCommit) {
|
||||
return getConfigBuilder(autoCommit).build();
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user