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
@@ -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
|
||||
*/
|
||||
|
||||
Reference in New Issue
Block a user