Adding ability for inserts to be written to log files
This commit is contained in:
committed by
vinoth chandar
parent
34827d50e1
commit
3da063f83b
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.minicluster.HdfsTestService;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
@@ -32,6 +33,7 @@ import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
@@ -44,6 +46,8 @@ import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
@@ -52,8 +56,6 @@ import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class TestHoodieCommitArchiveLog {
|
||||
|
||||
private String basePath;
|
||||
private Configuration hadoopConf;
|
||||
//NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class)
|
||||
//The implementation and gurantees of many API's differ, for example check rename(src,dst)
|
||||
// We need to use DFS here instead of LocalFs since the FsDataInputStream.getWrappedStream() returns a
|
||||
@@ -62,6 +64,9 @@ public class TestHoodieCommitArchiveLog {
|
||||
private static MiniDFSCluster dfsCluster;
|
||||
private static DistributedFileSystem dfs;
|
||||
private static HdfsTestService hdfsTestService;
|
||||
private String basePath;
|
||||
private Configuration hadoopConf;
|
||||
private JavaSparkContext jsc = null;
|
||||
|
||||
@AfterClass
|
||||
public static void cleanUp() throws Exception {
|
||||
@@ -91,11 +96,24 @@ public class TestHoodieCommitArchiveLog {
|
||||
public void init() throws Exception {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieCommitArchiveLog"));
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
hadoopConf = dfs.getConf();
|
||||
jsc.hadoopConfiguration().addResource(dfs.getConf());
|
||||
dfs.mkdirs(new Path(basePath));
|
||||
HoodieTestUtils.init(hadoopConf, basePath);
|
||||
}
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveEmptyDataset() throws IOException {
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
@@ -103,7 +121,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
.forTable("test-trip-table").build();
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg,
|
||||
new HoodieTableMetaClient(dfs.getConf(), cfg.getBasePath(), true));
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
boolean result = archiveLog.archiveIfRequired(jsc);
|
||||
assertTrue(result);
|
||||
}
|
||||
|
||||
@@ -148,7 +166,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg,
|
||||
new HoodieTableMetaClient(dfs.getConf(), basePath, true));
|
||||
|
||||
assertTrue(archiveLog.archiveIfRequired());
|
||||
assertTrue(archiveLog.archiveIfRequired(jsc));
|
||||
|
||||
//reload the timeline and remove the remaining commits
|
||||
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
|
||||
@@ -199,7 +217,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
boolean result = archiveLog.archiveIfRequired(jsc);
|
||||
assertTrue(result);
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants());
|
||||
@@ -222,7 +240,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
boolean result = archiveLog.archiveIfRequired(jsc);
|
||||
assertTrue(result);
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
|
||||
assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("100"));
|
||||
@@ -249,7 +267,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
boolean result = archiveLog.archiveIfRequired(jsc);
|
||||
assertTrue(result);
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals(
|
||||
|
||||
Reference in New Issue
Block a user