1. Small file size handling for inserts into log files. In summary, the total size of the log file is compared with the parquet max file size and if there is scope to add inserts the add it.
This commit is contained in:
committed by
vinoth chandar
parent
324de298bc
commit
459e523d9e
@@ -251,7 +251,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
HashMap<String, TreeSet<String>> fileIdToVersions = new HashMap<>();
|
||||
for (HoodieInstant entry : timeline.getInstants().collect(Collectors.toList())) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(timeline.getInstantDetails(entry).get());
|
||||
.fromBytes(timeline.getInstantDetails(entry).get(), HoodieCommitMetadata.class);
|
||||
|
||||
for (HoodieWriteStat wstat : commitMetadata.getWriteStats(partitionPath)) {
|
||||
if (!fileIdToVersions.containsKey(wstat.getFileId())) {
|
||||
|
||||
@@ -28,6 +28,8 @@ 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.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRollingStat;
|
||||
import com.uber.hoodie.common.model.HoodieRollingStatMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
@@ -46,6 +48,7 @@ import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
@@ -54,6 +57,7 @@ import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import scala.Option;
|
||||
|
||||
@@ -563,7 +567,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime);
|
||||
HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(commitInstant).get());
|
||||
.fromBytes(commitTimeline.getInstantDetails(commitInstant).get(), HoodieCommitMetadata.class);
|
||||
String basePath = table.getMetaClient().getBasePath();
|
||||
Collection<String> commitPathNames = commitMetadata.getFileIdAndFullPaths(basePath).values();
|
||||
|
||||
@@ -571,7 +575,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
FileInputStream inputStream = new FileInputStream(filename);
|
||||
String everything = IOUtils.toString(inputStream);
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString());
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString(),
|
||||
HoodieCommitMetadata.class);
|
||||
HashMap<String, String> paths = metadata.getFileIdAndFullPaths(basePath);
|
||||
inputStream.close();
|
||||
|
||||
@@ -581,6 +586,79 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to ensure commit metadata points to valid files
|
||||
*/
|
||||
@Test
|
||||
public void testRollingStatsInMetadata() throws Exception {
|
||||
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
String commitTime = "000";
|
||||
client.startCommitWithTime(commitTime);
|
||||
|
||||
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));
|
||||
|
||||
// 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(),
|
||||
HoodieCommitMetadata.class);
|
||||
HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata.fromJsonString(metadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY), HoodieRollingStatMetadata.class);
|
||||
int inserts = 0;
|
||||
for (Map.Entry<String, Map<String, HoodieRollingStat>> pstat : rollingStatMetadata.getPartitionToRollingStats()
|
||||
.entrySet()) {
|
||||
for (Map.Entry<String, HoodieRollingStat> stat : pstat.getValue().entrySet()) {
|
||||
inserts += stat.getValue().getInserts();
|
||||
}
|
||||
}
|
||||
Assert.assertEquals(inserts, 200);
|
||||
|
||||
// Update + Inserts such that they just expand file1
|
||||
commitTime = "001";
|
||||
client.startCommitWithTime(commitTime);
|
||||
|
||||
records = dataGen.generateUpdates(commitTime, records);
|
||||
writeRecords = jsc.parallelize(records, 1);
|
||||
result = client.upsert(writeRecords, commitTime);
|
||||
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, result));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
|
||||
// Read from commit file
|
||||
filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
inputStream = new FileInputStream(filename);
|
||||
everything = IOUtils.toString(inputStream);
|
||||
metadata = HoodieCommitMetadata.fromJsonString(everything.toString(), HoodieCommitMetadata.class);
|
||||
rollingStatMetadata = HoodieCommitMetadata.fromJsonString(metadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY), HoodieRollingStatMetadata.class);
|
||||
inserts = 0;
|
||||
int upserts = 0;
|
||||
for (Map.Entry<String, Map<String, HoodieRollingStat>> pstat : rollingStatMetadata.getPartitionToRollingStats()
|
||||
.entrySet()) {
|
||||
for (Map.Entry<String, HoodieRollingStat> stat : pstat.getValue().entrySet()) {
|
||||
inserts += stat.getValue().getInserts();
|
||||
upserts += stat.getValue().getUpserts();
|
||||
}
|
||||
}
|
||||
Assert.assertEquals(inserts, 200);
|
||||
Assert.assertEquals(upserts, 200);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Build Hoodie Write Config for small data file sizes
|
||||
*/
|
||||
|
||||
@@ -109,7 +109,8 @@ public class HoodieClientTestUtils {
|
||||
List<HoodieInstant> commitsToReturn) throws IOException {
|
||||
HashMap<String, String> fileIdToFullPath = new HashMap<>();
|
||||
for (HoodieInstant commit : commitsToReturn) {
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get(),
|
||||
HoodieCommitMetadata.class);
|
||||
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
|
||||
}
|
||||
return fileIdToFullPath;
|
||||
|
||||
@@ -31,9 +31,12 @@ import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.TestRawTripPayload.MetadataMergeWriteStatus;
|
||||
import com.uber.hoodie.common.minicluster.HdfsTestService;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
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.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRollingStat;
|
||||
import com.uber.hoodie.common.model.HoodieRollingStatMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
@@ -896,6 +899,213 @@ public class TestMergeOnReadTable {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to ensure rolling stats are correctly written to metadata file
|
||||
*/
|
||||
@Test
|
||||
public void testRollingStatsInMetadata() throws Exception {
|
||||
|
||||
HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
String commitTime = "000";
|
||||
client.startCommitWithTime(commitTime);
|
||||
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<WriteStatus> statuses = client.insert(writeRecords, commitTime);
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table
|
||||
.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class);
|
||||
HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class);
|
||||
int inserts = 0;
|
||||
for (Map.Entry<String, Map<String, HoodieRollingStat>> pstat :
|
||||
rollingStatMetadata.getPartitionToRollingStats().entrySet()) {
|
||||
for (Map.Entry<String, HoodieRollingStat> stat : pstat.getValue().entrySet()) {
|
||||
inserts += stat.getValue().getInserts();
|
||||
}
|
||||
}
|
||||
Assert.assertEquals(inserts, 200);
|
||||
|
||||
commitTime = "001";
|
||||
client.startCommitWithTime(commitTime);
|
||||
records = dataGen.generateUpdates(commitTime, records);
|
||||
writeRecords = jsc.parallelize(records, 1);
|
||||
statuses = client.upsert(writeRecords, commitTime);
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table
|
||||
.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class);
|
||||
rollingStatMetadata = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class);
|
||||
inserts = 0;
|
||||
int upserts = 0;
|
||||
for (Map.Entry<String, Map<String, HoodieRollingStat>> pstat : rollingStatMetadata.getPartitionToRollingStats()
|
||||
.entrySet()) {
|
||||
for (Map.Entry<String, HoodieRollingStat> stat : pstat.getValue().entrySet()) {
|
||||
inserts += stat.getValue().getInserts();
|
||||
upserts += stat.getValue().getUpserts();
|
||||
}
|
||||
}
|
||||
|
||||
Assert.assertEquals(inserts, 200);
|
||||
Assert.assertEquals(upserts, 200);
|
||||
|
||||
client.rollback(commitTime);
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table
|
||||
.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class);
|
||||
rollingStatMetadata = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class);
|
||||
inserts = 0;
|
||||
upserts = 0;
|
||||
for (Map.Entry<String, Map<String, HoodieRollingStat>> pstat : rollingStatMetadata.getPartitionToRollingStats()
|
||||
.entrySet()) {
|
||||
for (Map.Entry<String, HoodieRollingStat> stat : pstat.getValue().entrySet()) {
|
||||
inserts += stat.getValue().getInserts();
|
||||
upserts += stat.getValue().getUpserts();
|
||||
}
|
||||
}
|
||||
Assert.assertEquals(inserts, 200);
|
||||
Assert.assertEquals(upserts, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to ensure rolling stats are correctly written to the metadata file, identifies small files and corrects them
|
||||
*/
|
||||
@Test
|
||||
public void testRollingStatsWithSmallFileHandling() throws Exception {
|
||||
|
||||
HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
Map<String, Long> fileIdToInsertsMap = new HashMap<>();
|
||||
Map<String, Long> fileIdToUpsertsMap = new HashMap<>();
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
String commitTime = "000";
|
||||
client.startCommitWithTime(commitTime);
|
||||
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<WriteStatus> statuses = client.insert(writeRecords, commitTime);
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table
|
||||
.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class);
|
||||
HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class);
|
||||
int inserts = 0;
|
||||
for (Map.Entry<String, Map<String, HoodieRollingStat>> pstat :
|
||||
rollingStatMetadata.getPartitionToRollingStats().entrySet()) {
|
||||
for (Map.Entry<String, HoodieRollingStat> stat : pstat.getValue().entrySet()) {
|
||||
inserts += stat.getValue().getInserts();
|
||||
fileIdToInsertsMap.put(stat.getKey(), stat.getValue().getInserts());
|
||||
fileIdToUpsertsMap.put(stat.getKey(), stat.getValue().getUpserts());
|
||||
}
|
||||
}
|
||||
Assert.assertEquals(inserts, 200);
|
||||
|
||||
commitTime = "001";
|
||||
client.startCommitWithTime(commitTime);
|
||||
// generate updates + inserts. inserts should be handled into small files
|
||||
records = dataGen.generateUpdates(commitTime, records);
|
||||
records.addAll(dataGen.generateInserts(commitTime, 200));
|
||||
writeRecords = jsc.parallelize(records, 1);
|
||||
statuses = client.upsert(writeRecords, commitTime);
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table
|
||||
.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class);
|
||||
rollingStatMetadata = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class);
|
||||
inserts = 0;
|
||||
int upserts = 0;
|
||||
for (Map.Entry<String, Map<String, HoodieRollingStat>> pstat : rollingStatMetadata.getPartitionToRollingStats()
|
||||
.entrySet()) {
|
||||
for (Map.Entry<String, HoodieRollingStat> stat : pstat.getValue().entrySet()) {
|
||||
// No new file id should be created, all the data should be written to small files already there
|
||||
assertTrue(fileIdToInsertsMap.containsKey(stat.getKey()));
|
||||
assertTrue(fileIdToUpsertsMap.containsKey(stat.getKey()));
|
||||
inserts += stat.getValue().getInserts();
|
||||
upserts += stat.getValue().getUpserts();
|
||||
}
|
||||
}
|
||||
|
||||
Assert.assertEquals(inserts, 400);
|
||||
Assert.assertEquals(upserts, 200);
|
||||
|
||||
// Test small file handling after compaction
|
||||
commitTime = "002";
|
||||
client.scheduleCompactionAtInstant(commitTime, Optional.of(metadata.getExtraMetadata()));
|
||||
statuses = client.compact(commitTime);
|
||||
client.commitCompaction(commitTime, statuses, Optional.empty());
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table
|
||||
.getActiveTimeline().getCommitsTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class);
|
||||
HoodieRollingStatMetadata rollingStatMetadata1 = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class);
|
||||
|
||||
// Ensure that the rolling stats from the extra metadata of delta commits is copied over to the compaction commit
|
||||
for (Map.Entry<String, Map<String, HoodieRollingStat>> entry : rollingStatMetadata.getPartitionToRollingStats()
|
||||
.entrySet()) {
|
||||
Assert.assertTrue(rollingStatMetadata1.getPartitionToRollingStats().containsKey(entry.getKey()));
|
||||
Assert.assertEquals(rollingStatMetadata1.getPartitionToRollingStats().get(entry.getKey()).size(), entry
|
||||
.getValue().size());
|
||||
}
|
||||
|
||||
// Write inserts + updates
|
||||
commitTime = "003";
|
||||
client.startCommitWithTime(commitTime);
|
||||
// generate updates + inserts. inserts should be handled into small files
|
||||
records = dataGen.generateUpdates(commitTime, records);
|
||||
records.addAll(dataGen.generateInserts(commitTime, 200));
|
||||
writeRecords = jsc.parallelize(records, 1);
|
||||
statuses = client.upsert(writeRecords, commitTime);
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table
|
||||
.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class);
|
||||
rollingStatMetadata = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class);
|
||||
inserts = 0;
|
||||
upserts = 0;
|
||||
for (Map.Entry<String, Map<String, HoodieRollingStat>> pstat : rollingStatMetadata.getPartitionToRollingStats()
|
||||
.entrySet()) {
|
||||
for (Map.Entry<String, HoodieRollingStat> stat : pstat.getValue().entrySet()) {
|
||||
// No new file id should be created, all the data should be written to small files already there
|
||||
assertTrue(fileIdToInsertsMap.containsKey(stat.getKey()));
|
||||
inserts += stat.getValue().getInserts();
|
||||
upserts += stat.getValue().getUpserts();
|
||||
}
|
||||
}
|
||||
|
||||
Assert.assertEquals(inserts, 600);
|
||||
Assert.assertEquals(upserts, 600);
|
||||
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getConfig(Boolean autoCommit) {
|
||||
return getConfigBuilder(autoCommit).build();
|
||||
|
||||
Reference in New Issue
Block a user