All Opened hoodie clients in tests needs to be closed
TestMergeOnReadTable must use embedded timeline server
This commit is contained in:
committed by
Balaji Varadarajan
parent
136f8478a3
commit
cd7623e216
@@ -92,7 +92,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
|||||||
public void testRollbackForInflightCompaction() throws Exception {
|
public void testRollbackForInflightCompaction() throws Exception {
|
||||||
// Rollback inflight compaction
|
// Rollback inflight compaction
|
||||||
HoodieWriteConfig cfg = getConfig(false);
|
HoodieWriteConfig cfg = getConfig(false);
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
|
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||||
|
|
||||||
String firstInstantTime = "001";
|
String firstInstantTime = "001";
|
||||||
String secondInstantTime = "004";
|
String secondInstantTime = "004";
|
||||||
|
|||||||
@@ -204,7 +204,7 @@ public class TestClientRollback extends TestHoodieClientBase {
|
|||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
|
||||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||||
|
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, config, false);
|
HoodieWriteClient client = getHoodieWriteClient(config, false);
|
||||||
|
|
||||||
// Rollback commit 1 (this should fail, since commit2 is still around)
|
// Rollback commit 1 (this should fail, since commit2 is still around)
|
||||||
try {
|
try {
|
||||||
@@ -294,7 +294,7 @@ public class TestClientRollback extends TestHoodieClientBase {
|
|||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
|
||||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||||
|
|
||||||
new HoodieWriteClient(jsc, config, false);
|
getHoodieWriteClient(config, false);
|
||||||
|
|
||||||
// Check results, nothing changed
|
// Check results, nothing changed
|
||||||
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||||
@@ -311,7 +311,7 @@ public class TestClientRollback extends TestHoodieClientBase {
|
|||||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
|
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
|
||||||
|
|
||||||
// Turn auto rollback on
|
// Turn auto rollback on
|
||||||
new HoodieWriteClient(jsc, config, true).startCommit();
|
getHoodieWriteClient(config, true).startCommit();
|
||||||
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||||
|
|||||||
@@ -81,27 +81,43 @@ public class TestHoodieClientBase implements Serializable {
|
|||||||
protected transient HoodieTestDataGenerator dataGen = null;
|
protected transient HoodieTestDataGenerator dataGen = null;
|
||||||
|
|
||||||
private HoodieWriteClient writeClient;
|
private HoodieWriteClient writeClient;
|
||||||
|
private HoodieReadClient readClient;
|
||||||
|
|
||||||
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws Exception {
|
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) {
|
||||||
closeClient();
|
return getHoodieWriteClient(cfg, false);
|
||||||
writeClient = new HoodieWriteClient(jsc, cfg);
|
}
|
||||||
|
|
||||||
|
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit) {
|
||||||
|
return getHoodieWriteClient(cfg, rollbackInflightCommit, HoodieIndex.createIndex(cfg, jsc));
|
||||||
|
}
|
||||||
|
|
||||||
|
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit,
|
||||||
|
HoodieIndex index) {
|
||||||
|
closeWriteClient();
|
||||||
|
writeClient = new HoodieWriteClient(jsc, cfg, rollbackInflightCommit, index);
|
||||||
return writeClient;
|
return writeClient;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit)
|
protected HoodieReadClient getHoodieReadClient(String basePath) {
|
||||||
throws Exception {
|
closeReadClient();
|
||||||
closeClient();
|
readClient = new HoodieReadClient(jsc, basePath);
|
||||||
writeClient = new HoodieWriteClient(jsc, cfg, rollbackInflightCommit);
|
return readClient;
|
||||||
return writeClient;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void closeClient() {
|
private void closeWriteClient() {
|
||||||
if (null != writeClient) {
|
if (null != writeClient) {
|
||||||
writeClient.close();
|
writeClient.close();
|
||||||
writeClient = null;
|
writeClient = null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void closeReadClient() {
|
||||||
|
if (null != readClient) {
|
||||||
|
readClient.close();
|
||||||
|
readClient = null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void init() throws IOException {
|
public void init() throws IOException {
|
||||||
// Initialize a local spark env
|
// Initialize a local spark env
|
||||||
@@ -132,7 +148,8 @@ public class TestHoodieClientBase implements Serializable {
|
|||||||
* Properly release resources at end of each test
|
* Properly release resources at end of each test
|
||||||
*/
|
*/
|
||||||
public void tearDown() throws IOException {
|
public void tearDown() throws IOException {
|
||||||
closeClient();
|
closeWriteClient();
|
||||||
|
closeReadClient();
|
||||||
|
|
||||||
if (null != sqlContext) {
|
if (null != sqlContext) {
|
||||||
logger.info("Clearing sql context cache of spark-session used in previous test-case");
|
logger.info("Clearing sql context cache of spark-session used in previous test-case");
|
||||||
|
|||||||
@@ -215,8 +215,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
|||||||
assertNodupesWithinPartition(dedupedRecs);
|
assertNodupesWithinPartition(dedupedRecs);
|
||||||
|
|
||||||
// Perform write-action and check
|
// Perform write-action and check
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc,
|
HoodieWriteClient client = getHoodieWriteClient(
|
||||||
getConfigBuilder().combineInput(true, true).build());
|
getConfigBuilder().combineInput(true, true).build(), false);
|
||||||
client.startCommitWithTime(newCommitTime);
|
client.startCommitWithTime(newCommitTime);
|
||||||
List<WriteStatus> statuses = writeFn.apply(client, records, newCommitTime).collect();
|
List<WriteStatus> statuses = writeFn.apply(client, records, newCommitTime).collect();
|
||||||
assertNoWriteErrors(statuses);
|
assertNoWriteErrors(statuses);
|
||||||
@@ -236,7 +236,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
|||||||
private HoodieWriteClient getWriteClientWithDummyIndex(final boolean isGlobal) throws Exception {
|
private HoodieWriteClient getWriteClientWithDummyIndex(final boolean isGlobal) throws Exception {
|
||||||
HoodieIndex index = mock(HoodieIndex.class);
|
HoodieIndex index = mock(HoodieIndex.class);
|
||||||
when(index.isGlobal()).thenReturn(isGlobal);
|
when(index.isGlobal()).thenReturn(isGlobal);
|
||||||
return new HoodieWriteClient(jsc, getConfigBuilder().build(), false, index);
|
return getHoodieWriteClient(getConfigBuilder().build(), false, index);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -267,7 +267,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
|||||||
private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig,
|
private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig,
|
||||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
|
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
|
||||||
boolean isPrepped) throws Exception {
|
boolean isPrepped) throws Exception {
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, hoodieWriteConfig);
|
HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||||
|
|
||||||
//Write 1 (only inserts)
|
//Write 1 (only inserts)
|
||||||
String newCommitTime = "001";
|
String newCommitTime = "001";
|
||||||
@@ -291,7 +291,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
|||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testDeletes() throws Exception {
|
public void testDeletes() throws Exception {
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, getConfig());
|
HoodieWriteClient client = getHoodieWriteClient(getConfig(), false);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Write 1 (inserts and deletes)
|
* Write 1 (inserts and deletes)
|
||||||
@@ -347,7 +347,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
|||||||
HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
|
HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
|
||||||
dataGen = new HoodieTestDataGenerator(new String[]{testPartitionPath});
|
dataGen = new HoodieTestDataGenerator(new String[]{testPartitionPath});
|
||||||
|
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient client = getHoodieWriteClient(config, false);
|
||||||
|
|
||||||
// Inserts => will write file1
|
// Inserts => will write file1
|
||||||
String commitTime1 = "001";
|
String commitTime1 = "001";
|
||||||
@@ -457,7 +457,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
|||||||
// setup the small file handling params
|
// setup the small file handling params
|
||||||
HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
|
HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
|
||||||
dataGen = new HoodieTestDataGenerator(new String[]{testPartitionPath});
|
dataGen = new HoodieTestDataGenerator(new String[]{testPartitionPath});
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient client = getHoodieWriteClient(config, false);
|
||||||
|
|
||||||
// Inserts => will write file1
|
// Inserts => will write file1
|
||||||
String commitTime1 = "001";
|
String commitTime1 = "001";
|
||||||
|
|||||||
@@ -89,12 +89,12 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
|
|||||||
*/
|
*/
|
||||||
private void testReadFilterExist(HoodieWriteConfig config,
|
private void testReadFilterExist(HoodieWriteConfig config,
|
||||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
|
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getHoodieWriteClient(config);
|
||||||
String newCommitTime = writeClient.startCommit();
|
String newCommitTime = writeClient.startCommit();
|
||||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||||
|
|
||||||
HoodieReadClient readClient = new HoodieReadClient(jsc, config.getBasePath());
|
HoodieReadClient readClient = getHoodieReadClient(config.getBasePath());
|
||||||
JavaRDD<HoodieRecord> filteredRDD = readClient.filterExists(recordsRDD);
|
JavaRDD<HoodieRecord> filteredRDD = readClient.filterExists(recordsRDD);
|
||||||
|
|
||||||
// Should not find any files
|
// Should not find any files
|
||||||
@@ -106,7 +106,7 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
|
|||||||
// Verify there are no errors
|
// Verify there are no errors
|
||||||
assertNoWriteErrors(statuses);
|
assertNoWriteErrors(statuses);
|
||||||
|
|
||||||
readClient = new HoodieReadClient(jsc, config.getBasePath());
|
readClient = getHoodieReadClient(config.getBasePath());
|
||||||
filteredRDD = readClient.filterExists(recordsRDD);
|
filteredRDD = readClient.filterExists(recordsRDD);
|
||||||
List<HoodieRecord> result = filteredRDD.collect();
|
List<HoodieRecord> result = filteredRDD.collect();
|
||||||
// Check results
|
// Check results
|
||||||
@@ -166,7 +166,7 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
|
|||||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> updateFn,
|
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> updateFn,
|
||||||
boolean isPrepped)
|
boolean isPrepped)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, hoodieWriteConfig);
|
HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig);
|
||||||
//Write 1 (only inserts)
|
//Write 1 (only inserts)
|
||||||
String newCommitTime = "001";
|
String newCommitTime = "001";
|
||||||
String initCommitTime = "000";
|
String initCommitTime = "000";
|
||||||
@@ -182,7 +182,7 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
|
|||||||
.map(record -> new HoodieRecord(record.getKey(), null))
|
.map(record -> new HoodieRecord(record.getKey(), null))
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toList()));
|
||||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||||
HoodieReadClient readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath());
|
HoodieReadClient readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
|
||||||
List<HoodieRecord> taggedRecords = readClient.tagLocation(recordRDD).collect();
|
List<HoodieRecord> taggedRecords = readClient.tagLocation(recordRDD).collect();
|
||||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||||
|
|
||||||
@@ -201,7 +201,7 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
|
|||||||
.map(record -> new HoodieRecord(record.getKey(), null))
|
.map(record -> new HoodieRecord(record.getKey(), null))
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toList()));
|
||||||
// Index should be able to locate all updates in correct locations.
|
// Index should be able to locate all updates in correct locations.
|
||||||
readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath());
|
readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
|
||||||
taggedRecords = readClient.tagLocation(recordRDD).collect();
|
taggedRecords = readClient.tagLocation(recordRDD).collect();
|
||||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -48,6 +48,7 @@ import org.apache.spark.api.java.JavaSparkContext;
|
|||||||
import org.apache.spark.sql.Dataset;
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.Row;
|
import org.apache.spark.sql.Row;
|
||||||
import org.apache.spark.sql.SQLContext;
|
import org.apache.spark.sql.SQLContext;
|
||||||
|
import org.junit.After;
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
@@ -63,6 +64,7 @@ public class TestMultiFS implements Serializable {
|
|||||||
private static SQLContext sqlContext;
|
private static SQLContext sqlContext;
|
||||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||||
protected String tableName = "hoodie_rt";
|
protected String tableName = "hoodie_rt";
|
||||||
|
private HoodieWriteClient hdfsWriteClient;
|
||||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
@@ -83,6 +85,22 @@ public class TestMultiFS implements Serializable {
|
|||||||
sqlContext = new SQLContext(jsc);
|
sqlContext = new SQLContext(jsc);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig config) throws Exception {
|
||||||
|
if (null != hdfsWriteClient) {
|
||||||
|
hdfsWriteClient.close();
|
||||||
|
}
|
||||||
|
hdfsWriteClient = new HoodieWriteClient(jsc, config);
|
||||||
|
return hdfsWriteClient;
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void teardown() {
|
||||||
|
if (null != hdfsWriteClient) {
|
||||||
|
hdfsWriteClient.close();
|
||||||
|
hdfsWriteClient = null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@AfterClass
|
@AfterClass
|
||||||
public static void cleanupClass() throws Exception {
|
public static void cleanupClass() throws Exception {
|
||||||
if (jsc != null) {
|
if (jsc != null) {
|
||||||
@@ -98,6 +116,7 @@ public class TestMultiFS implements Serializable {
|
|||||||
FileSystem.closeAll();
|
FileSystem.closeAll();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
protected HoodieWriteConfig getHoodieWriteConfig(String basePath) {
|
protected HoodieWriteConfig getHoodieWriteConfig(String basePath) {
|
||||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withEmbeddedTimelineServerEnabled(true)
|
return HoodieWriteConfig.newBuilder().withPath(basePath).withEmbeddedTimelineServerEnabled(true)
|
||||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
@@ -118,7 +137,7 @@ public class TestMultiFS implements Serializable {
|
|||||||
|
|
||||||
//Create write client to write some records in
|
//Create write client to write some records in
|
||||||
HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath);
|
HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath);
|
||||||
HoodieWriteClient hdfsWriteClient = new HoodieWriteClient(jsc, cfg);
|
HoodieWriteClient hdfsWriteClient = getHoodieWriteClient(cfg);
|
||||||
|
|
||||||
// Write generated data to hdfs (only inserts)
|
// Write generated data to hdfs (only inserts)
|
||||||
String readCommitTime = hdfsWriteClient.startCommit();
|
String readCommitTime = hdfsWriteClient.startCommit();
|
||||||
@@ -139,7 +158,7 @@ public class TestMultiFS implements Serializable {
|
|||||||
.initTableType(jsc.hadoopConfiguration(), tablePath, HoodieTableType.valueOf(tableType), tableName,
|
.initTableType(jsc.hadoopConfiguration(), tablePath, HoodieTableType.valueOf(tableType), tableName,
|
||||||
HoodieAvroPayload.class.getName());
|
HoodieAvroPayload.class.getName());
|
||||||
HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath);
|
HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath);
|
||||||
HoodieWriteClient localWriteClient = new HoodieWriteClient(jsc, localConfig);
|
HoodieWriteClient localWriteClient = getHoodieWriteClient(localConfig);
|
||||||
|
|
||||||
String writeCommitTime = localWriteClient.startCommit();
|
String writeCommitTime = localWriteClient.startCommit();
|
||||||
logger.info("Starting write commit " + writeCommitTime);
|
logger.info("Starting write commit " + writeCommitTime);
|
||||||
|
|||||||
@@ -84,6 +84,7 @@ public class TestHbaseIndex {
|
|||||||
private static String tableName = "test_table";
|
private static String tableName = "test_table";
|
||||||
private String basePath = null;
|
private String basePath = null;
|
||||||
private transient FileSystem fs;
|
private transient FileSystem fs;
|
||||||
|
private HoodieWriteClient writeClient;
|
||||||
|
|
||||||
public TestHbaseIndex() throws Exception {
|
public TestHbaseIndex() throws Exception {
|
||||||
}
|
}
|
||||||
@@ -113,6 +114,11 @@ public class TestHbaseIndex {
|
|||||||
|
|
||||||
@After
|
@After
|
||||||
public void clear() throws Exception {
|
public void clear() throws Exception {
|
||||||
|
if (null != writeClient) {
|
||||||
|
writeClient.close();
|
||||||
|
writeClient = null;
|
||||||
|
}
|
||||||
|
|
||||||
if (basePath != null) {
|
if (basePath != null) {
|
||||||
new File(basePath).delete();
|
new File(basePath).delete();
|
||||||
}
|
}
|
||||||
@@ -128,6 +134,14 @@ public class TestHbaseIndex {
|
|||||||
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
|
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private HoodieWriteClient getWriteClient(HoodieWriteConfig config) throws Exception {
|
||||||
|
if (null != writeClient) {
|
||||||
|
writeClient.close();
|
||||||
|
}
|
||||||
|
writeClient = new HoodieWriteClient(jsc, config);
|
||||||
|
return writeClient;
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSimpleTagLocationAndUpdate() throws Exception {
|
public void testSimpleTagLocationAndUpdate() throws Exception {
|
||||||
|
|
||||||
@@ -139,7 +153,7 @@ public class TestHbaseIndex {
|
|||||||
// Load to memory
|
// Load to memory
|
||||||
HoodieWriteConfig config = getConfig();
|
HoodieWriteConfig config = getConfig();
|
||||||
HBaseIndex index = new HBaseIndex(config);
|
HBaseIndex index = new HBaseIndex(config);
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getWriteClient(config);
|
||||||
writeClient.startCommit();
|
writeClient.startCommit();
|
||||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||||
@@ -178,7 +192,7 @@ public class TestHbaseIndex {
|
|||||||
// Load to memory
|
// Load to memory
|
||||||
HoodieWriteConfig config = getConfig();
|
HoodieWriteConfig config = getConfig();
|
||||||
HBaseIndex index = new HBaseIndex(config);
|
HBaseIndex index = new HBaseIndex(config);
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getWriteClient(config);
|
||||||
|
|
||||||
String newCommitTime = writeClient.startCommit();
|
String newCommitTime = writeClient.startCommit();
|
||||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||||
@@ -231,7 +245,7 @@ public class TestHbaseIndex {
|
|||||||
// only for test, set the hbaseConnection to mocked object
|
// only for test, set the hbaseConnection to mocked object
|
||||||
index.setHbaseConnection(hbaseConnection);
|
index.setHbaseConnection(hbaseConnection);
|
||||||
|
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getWriteClient(config);
|
||||||
|
|
||||||
// start a commit and generate test data
|
// start a commit and generate test data
|
||||||
String newCommitTime = writeClient.startCommit();
|
String newCommitTime = writeClient.startCommit();
|
||||||
@@ -258,7 +272,7 @@ public class TestHbaseIndex {
|
|||||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||||
HoodieWriteConfig config = getConfig();
|
HoodieWriteConfig config = getConfig();
|
||||||
HBaseIndex index = new HBaseIndex(config);
|
HBaseIndex index = new HBaseIndex(config);
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getWriteClient(config);
|
||||||
|
|
||||||
// start a commit and generate test data
|
// start a commit and generate test data
|
||||||
String newCommitTime = writeClient.startCommit();
|
String newCommitTime = writeClient.startCommit();
|
||||||
|
|||||||
@@ -64,6 +64,7 @@ public class TestHoodieCompactor {
|
|||||||
private transient HoodieTestDataGenerator dataGen = null;
|
private transient HoodieTestDataGenerator dataGen = null;
|
||||||
private transient FileSystem fs;
|
private transient FileSystem fs;
|
||||||
private Configuration hadoopConf;
|
private Configuration hadoopConf;
|
||||||
|
private HoodieWriteClient writeClient;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void init() throws IOException {
|
public void init() throws IOException {
|
||||||
@@ -84,6 +85,11 @@ public class TestHoodieCompactor {
|
|||||||
|
|
||||||
@After
|
@After
|
||||||
public void clean() {
|
public void clean() {
|
||||||
|
if (null != writeClient) {
|
||||||
|
writeClient.close();
|
||||||
|
writeClient = null;
|
||||||
|
}
|
||||||
|
|
||||||
if (basePath != null) {
|
if (basePath != null) {
|
||||||
new File(basePath).delete();
|
new File(basePath).delete();
|
||||||
}
|
}
|
||||||
@@ -92,6 +98,14 @@ public class TestHoodieCompactor {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private HoodieWriteClient getWriteClient(HoodieWriteConfig config) throws Exception {
|
||||||
|
if (null != writeClient) {
|
||||||
|
writeClient.close();
|
||||||
|
}
|
||||||
|
writeClient = new HoodieWriteClient(jsc, config);
|
||||||
|
return writeClient;
|
||||||
|
}
|
||||||
|
|
||||||
private HoodieWriteConfig getConfig() {
|
private HoodieWriteConfig getConfig() {
|
||||||
return getConfigBuilder()
|
return getConfigBuilder()
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||||
@@ -123,7 +137,7 @@ public class TestHoodieCompactor {
|
|||||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||||
HoodieWriteConfig config = getConfig();
|
HoodieWriteConfig config = getConfig();
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getWriteClient(config);
|
||||||
|
|
||||||
String newCommitTime = writeClient.startCommit();
|
String newCommitTime = writeClient.startCommit();
|
||||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||||
@@ -140,7 +154,7 @@ public class TestHoodieCompactor {
|
|||||||
public void testWriteStatusContentsAfterCompaction() throws Exception {
|
public void testWriteStatusContentsAfterCompaction() throws Exception {
|
||||||
// insert 100 records
|
// insert 100 records
|
||||||
HoodieWriteConfig config = getConfig();
|
HoodieWriteConfig config = getConfig();
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getWriteClient(config);
|
||||||
String newCommitTime = "100";
|
String newCommitTime = "100";
|
||||||
writeClient.startCommitWithTime(newCommitTime);
|
writeClient.startCommitWithTime(newCommitTime);
|
||||||
|
|
||||||
|
|||||||
@@ -63,6 +63,7 @@ public class TestHoodieMergeHandle {
|
|||||||
protected transient FileSystem fs;
|
protected transient FileSystem fs;
|
||||||
protected String basePath = null;
|
protected String basePath = null;
|
||||||
protected transient HoodieTestDataGenerator dataGen = null;
|
protected transient HoodieTestDataGenerator dataGen = null;
|
||||||
|
private HoodieWriteClient writeClient;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void init() throws IOException {
|
public void init() throws IOException {
|
||||||
@@ -83,6 +84,11 @@ public class TestHoodieMergeHandle {
|
|||||||
|
|
||||||
@After
|
@After
|
||||||
public void clean() {
|
public void clean() {
|
||||||
|
if (null != writeClient) {
|
||||||
|
writeClient.close();
|
||||||
|
writeClient = null;
|
||||||
|
}
|
||||||
|
|
||||||
if (basePath != null) {
|
if (basePath != null) {
|
||||||
new File(basePath).delete();
|
new File(basePath).delete();
|
||||||
}
|
}
|
||||||
@@ -91,6 +97,14 @@ public class TestHoodieMergeHandle {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private HoodieWriteClient getWriteClient(HoodieWriteConfig config) throws Exception {
|
||||||
|
if (null != writeClient) {
|
||||||
|
writeClient.close();
|
||||||
|
}
|
||||||
|
writeClient = new HoodieWriteClient(jsc, config);
|
||||||
|
return writeClient;
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUpsertsForMultipleRecordsInSameFile() throws Exception {
|
public void testUpsertsForMultipleRecordsInSameFile() throws Exception {
|
||||||
// Create records in a single partition
|
// Create records in a single partition
|
||||||
@@ -99,7 +113,7 @@ public class TestHoodieMergeHandle {
|
|||||||
|
|
||||||
// Build a write config with bulkinsertparallelism set
|
// Build a write config with bulkinsertparallelism set
|
||||||
HoodieWriteConfig cfg = getConfigBuilder().build();
|
HoodieWriteConfig cfg = getConfigBuilder().build();
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
HoodieWriteClient client = getWriteClient(cfg);
|
||||||
FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -250,7 +264,7 @@ public class TestHoodieMergeHandle {
|
|||||||
public void testHoodieMergeHandleWriteStatMetrics() throws Exception {
|
public void testHoodieMergeHandleWriteStatMetrics() throws Exception {
|
||||||
// insert 100 records
|
// insert 100 records
|
||||||
HoodieWriteConfig config = getConfigBuilder().build();
|
HoodieWriteConfig config = getConfigBuilder().build();
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getWriteClient(config);
|
||||||
String newCommitTime = "100";
|
String newCommitTime = "100";
|
||||||
writeClient.startCommitWithTime(newCommitTime);
|
writeClient.startCommitWithTime(newCommitTime);
|
||||||
|
|
||||||
|
|||||||
@@ -92,6 +92,7 @@ public class TestMergeOnReadTable {
|
|||||||
private static HdfsTestService hdfsTestService;
|
private static HdfsTestService hdfsTestService;
|
||||||
private transient JavaSparkContext jsc = null;
|
private transient JavaSparkContext jsc = null;
|
||||||
private transient SQLContext sqlContext;
|
private transient SQLContext sqlContext;
|
||||||
|
private HoodieWriteClient writeClient;
|
||||||
|
|
||||||
@AfterClass
|
@AfterClass
|
||||||
public static void cleanUp() throws Exception {
|
public static void cleanUp() throws Exception {
|
||||||
@@ -139,6 +140,11 @@ public class TestMergeOnReadTable {
|
|||||||
|
|
||||||
@After
|
@After
|
||||||
public void clean() {
|
public void clean() {
|
||||||
|
if (null != writeClient) {
|
||||||
|
writeClient.close();
|
||||||
|
writeClient = null;
|
||||||
|
}
|
||||||
|
|
||||||
if (basePath != null) {
|
if (basePath != null) {
|
||||||
new File(basePath).delete();
|
new File(basePath).delete();
|
||||||
}
|
}
|
||||||
@@ -147,10 +153,18 @@ public class TestMergeOnReadTable {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private HoodieWriteClient getWriteClient(HoodieWriteConfig config) throws Exception {
|
||||||
|
if (null != writeClient) {
|
||||||
|
writeClient.close();
|
||||||
|
}
|
||||||
|
writeClient = new HoodieWriteClient(jsc, config);
|
||||||
|
return writeClient;
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSimpleInsertAndUpdate() throws Exception {
|
public void testSimpleInsertAndUpdate() throws Exception {
|
||||||
HoodieWriteConfig cfg = getConfig(true);
|
HoodieWriteConfig cfg = getConfig(true);
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
HoodieWriteClient client = getWriteClient(cfg);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Write 1 (only inserts)
|
* Write 1 (only inserts)
|
||||||
@@ -234,7 +248,7 @@ public class TestMergeOnReadTable {
|
|||||||
@Test
|
@Test
|
||||||
public void testMetadataAggregateFromWriteStatus() throws Exception {
|
public void testMetadataAggregateFromWriteStatus() throws Exception {
|
||||||
HoodieWriteConfig cfg = getConfigBuilder(false).withWriteStatusClass(MetadataMergeWriteStatus.class).build();
|
HoodieWriteConfig cfg = getConfigBuilder(false).withWriteStatusClass(MetadataMergeWriteStatus.class).build();
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
HoodieWriteClient client = getWriteClient(cfg);
|
||||||
|
|
||||||
String newCommitTime = "001";
|
String newCommitTime = "001";
|
||||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||||
@@ -257,7 +271,7 @@ public class TestMergeOnReadTable {
|
|||||||
@Test
|
@Test
|
||||||
public void testSimpleInsertUpdateAndDelete() throws Exception {
|
public void testSimpleInsertUpdateAndDelete() throws Exception {
|
||||||
HoodieWriteConfig cfg = getConfig(true);
|
HoodieWriteConfig cfg = getConfig(true);
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
HoodieWriteClient client = getWriteClient(cfg);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Write 1 (only inserts, written as parquet file)
|
* Write 1 (only inserts, written as parquet file)
|
||||||
@@ -342,7 +356,7 @@ public class TestMergeOnReadTable {
|
|||||||
HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE);
|
HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE);
|
||||||
|
|
||||||
HoodieWriteConfig cfg = getConfig(true);
|
HoodieWriteConfig cfg = getConfig(true);
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
HoodieWriteClient client = getWriteClient(cfg);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Write 1 (only inserts)
|
* Write 1 (only inserts)
|
||||||
@@ -401,7 +415,7 @@ public class TestMergeOnReadTable {
|
|||||||
public void testRollbackWithDeltaAndCompactionCommit() throws Exception {
|
public void testRollbackWithDeltaAndCompactionCommit() throws Exception {
|
||||||
|
|
||||||
HoodieWriteConfig cfg = getConfig(false);
|
HoodieWriteConfig cfg = getConfig(false);
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
HoodieWriteClient client = getWriteClient(cfg);
|
||||||
|
|
||||||
// Test delta commit rollback
|
// Test delta commit rollback
|
||||||
/**
|
/**
|
||||||
@@ -445,7 +459,7 @@ public class TestMergeOnReadTable {
|
|||||||
*/
|
*/
|
||||||
final String commitTime1 = "002";
|
final String commitTime1 = "002";
|
||||||
// WriteClient with custom config (disable small file handling)
|
// WriteClient with custom config (disable small file handling)
|
||||||
client = new HoodieWriteClient(jsc, getHoodieWriteConfigWithSmallFileHandlingOff());
|
client = getWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff());
|
||||||
client.startCommitWithTime(commitTime1);
|
client.startCommitWithTime(commitTime1);
|
||||||
|
|
||||||
List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
|
List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
|
||||||
@@ -475,7 +489,7 @@ public class TestMergeOnReadTable {
|
|||||||
* Write 3 (inserts + updates - testing successful delta commit)
|
* Write 3 (inserts + updates - testing successful delta commit)
|
||||||
*/
|
*/
|
||||||
final String commitTime2 = "002";
|
final String commitTime2 = "002";
|
||||||
client = new HoodieWriteClient(jsc, cfg);
|
client = getWriteClient(cfg);
|
||||||
client.startCommitWithTime(commitTime2);
|
client.startCommitWithTime(commitTime2);
|
||||||
|
|
||||||
copyOfRecords = new ArrayList<>(records);
|
copyOfRecords = new ArrayList<>(records);
|
||||||
@@ -566,7 +580,7 @@ public class TestMergeOnReadTable {
|
|||||||
public void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception {
|
public void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception {
|
||||||
|
|
||||||
HoodieWriteConfig cfg = getConfig(false);
|
HoodieWriteConfig cfg = getConfig(false);
|
||||||
final HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
final HoodieWriteClient client = getWriteClient(cfg);
|
||||||
List<String> allCommits = new ArrayList<>();
|
List<String> allCommits = new ArrayList<>();
|
||||||
/**
|
/**
|
||||||
* Write 1 (only inserts)
|
* Write 1 (only inserts)
|
||||||
@@ -611,7 +625,7 @@ public class TestMergeOnReadTable {
|
|||||||
newCommitTime = "002";
|
newCommitTime = "002";
|
||||||
allCommits.add(newCommitTime);
|
allCommits.add(newCommitTime);
|
||||||
// WriteClient with custom config (disable small file handling)
|
// WriteClient with custom config (disable small file handling)
|
||||||
HoodieWriteClient nClient = new HoodieWriteClient(jsc, getHoodieWriteConfigWithSmallFileHandlingOff());
|
HoodieWriteClient nClient = getWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff());
|
||||||
nClient.startCommitWithTime(newCommitTime);
|
nClient.startCommitWithTime(newCommitTime);
|
||||||
|
|
||||||
List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
|
List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
|
||||||
@@ -741,7 +755,7 @@ public class TestMergeOnReadTable {
|
|||||||
@Test
|
@Test
|
||||||
public void testUpsertPartitioner() throws Exception {
|
public void testUpsertPartitioner() throws Exception {
|
||||||
HoodieWriteConfig cfg = getConfig(true);
|
HoodieWriteConfig cfg = getConfig(true);
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
HoodieWriteClient client = getWriteClient(cfg);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Write 1 (only inserts, written as parquet file)
|
* Write 1 (only inserts, written as parquet file)
|
||||||
@@ -822,7 +836,7 @@ public class TestMergeOnReadTable {
|
|||||||
public void testLogFileCountsAfterCompaction() throws Exception {
|
public void testLogFileCountsAfterCompaction() throws Exception {
|
||||||
// insert 100 records
|
// insert 100 records
|
||||||
HoodieWriteConfig config = getConfig(true);
|
HoodieWriteConfig config = getConfig(true);
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getWriteClient(config);
|
||||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||||
String newCommitTime = "100";
|
String newCommitTime = "100";
|
||||||
writeClient.startCommitWithTime(newCommitTime);
|
writeClient.startCommitWithTime(newCommitTime);
|
||||||
@@ -897,7 +911,7 @@ public class TestMergeOnReadTable {
|
|||||||
public void testMetadataValuesAfterInsertUpsertAndCompaction() throws Exception {
|
public void testMetadataValuesAfterInsertUpsertAndCompaction() throws Exception {
|
||||||
// insert 100 records
|
// insert 100 records
|
||||||
HoodieWriteConfig config = getConfig(false);
|
HoodieWriteConfig config = getConfig(false);
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getWriteClient(config);
|
||||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||||
String newCommitTime = "100";
|
String newCommitTime = "100";
|
||||||
writeClient.startCommitWithTime(newCommitTime);
|
writeClient.startCommitWithTime(newCommitTime);
|
||||||
@@ -940,7 +954,7 @@ public class TestMergeOnReadTable {
|
|||||||
// insert 100 records
|
// insert 100 records
|
||||||
// Setting IndexType to be InMemory to simulate Global Index nature
|
// Setting IndexType to be InMemory to simulate Global Index nature
|
||||||
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
|
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getWriteClient(config);
|
||||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||||
String newCommitTime = "100";
|
String newCommitTime = "100";
|
||||||
writeClient.startCommitWithTime(newCommitTime);
|
writeClient.startCommitWithTime(newCommitTime);
|
||||||
@@ -979,7 +993,7 @@ public class TestMergeOnReadTable {
|
|||||||
// insert 100 records
|
// insert 100 records
|
||||||
// Setting IndexType to be InMemory to simulate Global Index nature
|
// Setting IndexType to be InMemory to simulate Global Index nature
|
||||||
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
|
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getWriteClient(config);
|
||||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||||
String newCommitTime = "100";
|
String newCommitTime = "100";
|
||||||
writeClient.startCommitWithTime(newCommitTime);
|
writeClient.startCommitWithTime(newCommitTime);
|
||||||
@@ -1033,7 +1047,7 @@ public class TestMergeOnReadTable {
|
|||||||
// insert 100 records
|
// insert 100 records
|
||||||
// Setting IndexType to be InMemory to simulate Global Index nature
|
// Setting IndexType to be InMemory to simulate Global Index nature
|
||||||
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
|
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
HoodieWriteClient writeClient = getWriteClient(config);
|
||||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||||
String newCommitTime = "100";
|
String newCommitTime = "100";
|
||||||
writeClient.startCommitWithTime(newCommitTime);
|
writeClient.startCommitWithTime(newCommitTime);
|
||||||
@@ -1088,7 +1102,7 @@ public class TestMergeOnReadTable {
|
|||||||
public void testRollingStatsInMetadata() throws Exception {
|
public void testRollingStatsInMetadata() throws Exception {
|
||||||
|
|
||||||
HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
|
HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
HoodieWriteClient client = getWriteClient(cfg);
|
||||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||||
|
|
||||||
@@ -1178,7 +1192,7 @@ public class TestMergeOnReadTable {
|
|||||||
public void testRollingStatsWithSmallFileHandling() throws Exception {
|
public void testRollingStatsWithSmallFileHandling() throws Exception {
|
||||||
|
|
||||||
HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
|
HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
HoodieWriteClient client = getWriteClient(cfg);
|
||||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||||
Map<String, Long> fileIdToInsertsMap = new HashMap<>();
|
Map<String, Long> fileIdToInsertsMap = new HashMap<>();
|
||||||
Map<String, Long> fileIdToUpsertsMap = new HashMap<>();
|
Map<String, Long> fileIdToUpsertsMap = new HashMap<>();
|
||||||
|
|||||||
Reference in New Issue
Block a user