Adding another metric to HoodieWriteStat to determine if there were inserts converted to updates, added one test for this
This commit is contained in:
committed by
vinoth chandar
parent
989afddd54
commit
88274b8261
@@ -26,6 +26,7 @@ import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
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.timeline.HoodieActiveTimeline;
|
||||
@@ -35,6 +36,7 @@ import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieStorageConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
@@ -46,12 +48,13 @@ import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public class TestHoodieMergeHandleDuplicateRecords {
|
||||
public class TestHoodieMergeHandle {
|
||||
|
||||
protected transient JavaSparkContext jsc = null;
|
||||
protected transient SQLContext sqlContext;
|
||||
@@ -62,7 +65,7 @@ public class TestHoodieMergeHandleDuplicateRecords {
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeHandleDuplicateRecords"));
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeHandle"));
|
||||
|
||||
//SQLContext stuff
|
||||
sqlContext = new SQLContext(jsc);
|
||||
@@ -241,6 +244,82 @@ public class TestHoodieMergeHandleDuplicateRecords {
|
||||
assertEquals(21, record2Count);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHoodieMergeHandleWriteStatMetrics() throws Exception {
|
||||
// insert 100 records
|
||||
HoodieWriteConfig config = getConfigBuilder().build();
|
||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
||||
String newCommitTime = "100";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
// All records should be inserts into new parquet
|
||||
Assert.assertTrue(statuses.stream()
|
||||
.filter(status -> status.getStat().getPrevCommit() != HoodieWriteStat.NULL_COMMIT).count() > 0);
|
||||
// Num writes should be equal to the number of records inserted
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumWrites()).reduce((a,b) -> a + b).get(), 100);
|
||||
// Num update writes should be equal to the number of records updated
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumUpdateWrites()).reduce((a,b) -> a + b).get(), 0);
|
||||
// Num update writes should be equal to the number of insert records converted to updates as part of small file
|
||||
// handling
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumInserts()).reduce((a,b) -> a + b).get(), 100);
|
||||
|
||||
// Update all the 100 records
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
newCommitTime = "101";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
statuses = writeClient.upsert(updatedRecordsRDD, newCommitTime).collect();
|
||||
|
||||
// All records should be upserts into existing parquet
|
||||
Assert.assertEquals(statuses.stream()
|
||||
.filter(status -> status.getStat().getPrevCommit() == HoodieWriteStat.NULL_COMMIT).count(), 0);
|
||||
// Num writes should be equal to the number of records inserted
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumWrites()).reduce((a,b) -> a + b).get(), 100);
|
||||
// Num update writes should be equal to the number of records updated
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumUpdateWrites()).reduce((a,b) -> a + b).get(), 100);
|
||||
// Num update writes should be equal to the number of insert records converted to updates as part of small file
|
||||
// handling
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumInserts()).reduce((a,b) -> a + b).get(), 0);
|
||||
|
||||
|
||||
newCommitTime = "102";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> allRecords = dataGen.generateInserts(newCommitTime, 100);
|
||||
allRecords.addAll(updatedRecords);
|
||||
JavaRDD<HoodieRecord> allRecordsRDD = jsc.parallelize(allRecords, 1);
|
||||
statuses = writeClient.upsert(allRecordsRDD, newCommitTime).collect();
|
||||
|
||||
// All records should be upserts into existing parquet (with inserts as updates small file handled)
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.filter(status -> status.getStat().getPrevCommit() == HoodieWriteStat.NULL_COMMIT).count(), 0);
|
||||
// Num writes should be equal to the total number of records written
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumWrites()).reduce((a,b) -> a + b).get(), 200);
|
||||
// Num update writes should be equal to the number of records updated (including inserts converted as updates)
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumUpdateWrites()).reduce((a,b) -> a + b).get(), 100);
|
||||
// Num update writes should be equal to the number of insert records converted to updates as part of small file
|
||||
// handling
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumInserts()).reduce((a,b) -> a + b).get(), 100);
|
||||
|
||||
}
|
||||
|
||||
private Dataset<Row> getRecords() {
|
||||
// Check the entire dataset has 8 records still
|
||||
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
|
||||
@@ -827,8 +827,8 @@ public class TestMergeOnReadTable {
|
||||
writeClient.commit(newCommitTime, statuses);
|
||||
|
||||
// rollback a successful commit
|
||||
// Sleep for small interval to force a new rollback start time.
|
||||
Thread.sleep(5);
|
||||
// Sleep for small interval (at least 1 second) to force a new rollback start time.
|
||||
Thread.sleep(1000);
|
||||
writeClient.rollback(newCommitTime);
|
||||
final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
Reference in New Issue
Block a user