[HUDI-756] Organize Cleaning Action execution into a single package in hudi-client (#1485)
- Introduced a thin abstraction ActionExecutor, that all actions will implement - Pulled cleaning code from table, writeclient into a single package - CleanHelper is now CleanPlanner, HoodieCleanClient is no longer around - Minor refactor of HoodieTable factory method - HoodieTable.create() methods with and without metaclient passed in - HoodieTable constructor now does not do a redundant instantiation - Fixed existing unit tests to work at the HoodieWriteClient level
This commit is contained in:
@@ -99,7 +99,7 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
List<String> partitionPaths =
|
||||
FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning());
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, getConfig(), jsc);
|
||||
final BaseFileOnlyView view1 = table.getBaseFileOnlyView();
|
||||
|
||||
List<HoodieBaseFile> dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
@@ -124,7 +124,7 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
table = HoodieTable.create(metaClient, getConfig(), jsc);
|
||||
final BaseFileOnlyView view2 = table.getBaseFileOnlyView();
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> view2.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("004"))).collect(Collectors.toList());
|
||||
@@ -143,7 +143,7 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
client.rollbackToSavepoint(savepoint.getTimestamp());
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
table = HoodieTable.create(metaClient, getConfig(), jsc);
|
||||
final BaseFileOnlyView view3 = table.getBaseFileOnlyView();
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view3.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("002"));
|
||||
|
||||
@@ -42,7 +42,6 @@ import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.metrics.HoodieMetrics;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -85,10 +84,6 @@ public class TestHoodieClientBase extends HoodieClientTestHarness {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
protected HoodieCleanClient getHoodieCleanClient(HoodieWriteConfig cfg) {
|
||||
return new HoodieCleanClient(jsc, cfg, new HoodieMetrics(cfg, cfg.getTableName()));
|
||||
}
|
||||
|
||||
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) {
|
||||
return getHoodieWriteClient(cfg, false);
|
||||
}
|
||||
@@ -161,7 +156,7 @@ public class TestHoodieClientBase extends HoodieClientTestHarness {
|
||||
}
|
||||
|
||||
protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
((SyncableFileSystemView) (table.getSliceView())).reset();
|
||||
return table;
|
||||
}
|
||||
@@ -255,7 +250,7 @@ public class TestHoodieClientBase extends HoodieClientTestHarness {
|
||||
final HoodieIndex index = HoodieIndex.createIndex(writeConfig, jsc);
|
||||
List<HoodieRecord> records = recordGenFunction.apply(commit, numRecords);
|
||||
final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, writeConfig, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, writeConfig, jsc);
|
||||
JavaRDD<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, table);
|
||||
return taggedRecords.collect();
|
||||
};
|
||||
@@ -276,7 +271,7 @@ public class TestHoodieClientBase extends HoodieClientTestHarness {
|
||||
final HoodieIndex index = HoodieIndex.createIndex(writeConfig, jsc);
|
||||
List<HoodieKey> records = keyGenFunction.apply(numRecords);
|
||||
final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, writeConfig, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, writeConfig, jsc);
|
||||
JavaRDD<HoodieRecord> recordsToDelete = jsc.parallelize(records, 1)
|
||||
.map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload()));
|
||||
JavaRDD<HoodieRecord> taggedRecords = index.tagLocation(recordsToDelete, jsc, table);
|
||||
|
||||
@@ -800,7 +800,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, cfg, jsc);
|
||||
|
||||
String instantTime = "000";
|
||||
client.startCommitWithTime(instantTime);
|
||||
|
||||
@@ -31,11 +31,11 @@ import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.table.HoodieCopyOnWriteTable;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
@@ -67,7 +67,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
||||
public void testSchemaEvolutionOnUpdate() throws Exception {
|
||||
// Create a bunch of records with a old version of schema
|
||||
final HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
|
||||
final HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
final HoodieTable<?> table = HoodieTable.create(config, jsc);
|
||||
|
||||
final List<WriteStatus> statuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
@@ -102,7 +102,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
||||
final WriteStatus insertResult = statuses.get(0);
|
||||
String fileId = insertResult.getFileId();
|
||||
|
||||
final HoodieCopyOnWriteTable table2 = new HoodieCopyOnWriteTable(config2, jsc);
|
||||
final HoodieTable table2 = HoodieTable.create(config2, jsc);
|
||||
Assert.assertEquals(1, jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
// New content with values for the newly added field
|
||||
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
|
||||
@@ -142,7 +142,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
try (HoodieWriteClient writeClient = getWriteClient(config);) {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
|
||||
@@ -162,7 +162,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
|
||||
assertEquals(200, javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size());
|
||||
assertEquals(200, javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
@@ -183,7 +183,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = index.tagLocation(writeRecords, jsc, hoodieTable);
|
||||
@@ -201,7 +201,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
|
||||
assertEquals(10, javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size());
|
||||
assertEquals(10, javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
@@ -227,7 +227,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
|
||||
// commit this upsert
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them
|
||||
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
|
||||
assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 200);
|
||||
@@ -242,7 +242,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
// Rollback the last commit
|
||||
writeClient.rollback(newCommitTime);
|
||||
|
||||
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled
|
||||
// back commit
|
||||
javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
|
||||
@@ -271,7 +271,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Insert 250 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
@@ -296,7 +296,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Insert 200 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
@@ -408,7 +408,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
try (HoodieWriteClient writeClient = getWriteClient(config);) {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
|
||||
@@ -428,7 +428,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
|
||||
assertEquals(200, javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size());
|
||||
assertEquals(200, javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
@@ -448,7 +448,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
try (HoodieWriteClient writeClient = getWriteClient(config);) {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
|
||||
@@ -462,7 +462,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
|
||||
assertEquals(10, javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size());
|
||||
assertEquals(10, javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
@@ -499,7 +499,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
|
||||
assertTrue(index.canIndexLogFiles());
|
||||
try {
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
index.fetchRecordLocation(jsc.parallelize(new ArrayList<HoodieKey>(), 1), jsc, hoodieTable);
|
||||
fail("HbaseIndex supports fetchRecordLocation");
|
||||
} catch (UnsupportedOperationException ex) {
|
||||
|
||||
@@ -160,7 +160,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, jsc, table);
|
||||
// Still 0, as no valid commit
|
||||
assertEquals(filesList.size(), 0);
|
||||
@@ -170,7 +170,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
|
||||
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
table = HoodieTable.create(metaClient, config, jsc);
|
||||
filesList = index.loadInvolvedFiles(partitions, jsc, table);
|
||||
assertEquals(filesList.size(), 4);
|
||||
|
||||
@@ -284,7 +284,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
// Also create the metadata and config
|
||||
HoodieWriteConfig config = makeConfig();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Let's tag
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
|
||||
@@ -324,7 +324,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
// Also create the metadata and config
|
||||
HoodieWriteConfig config = makeConfig();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Let's tag
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
|
||||
@@ -345,7 +345,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
// We do the tag again
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
table = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table);
|
||||
|
||||
@@ -394,7 +394,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
// Also create the metadata and config
|
||||
HoodieWriteConfig config = makeConfig();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Let's tag
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
|
||||
@@ -416,7 +416,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
// We do the tag again
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
table = HoodieTable.create(metaClient, config, jsc);
|
||||
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, table);
|
||||
|
||||
// Check results
|
||||
@@ -465,7 +465,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
|
||||
HoodieWriteConfig config = makeConfig();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table);
|
||||
|
||||
@@ -129,7 +129,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01");
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
// partitions will NOT be respected by this loadInvolvedFiles(...) call
|
||||
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, jsc, table);
|
||||
// Still 0, as no valid commit
|
||||
@@ -140,7 +140,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
|
||||
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
table = HoodieTable.create(metaClient, config, jsc);
|
||||
filesList = index.loadInvolvedFiles(partitions, jsc, table);
|
||||
assertEquals(filesList.size(), 4);
|
||||
|
||||
@@ -259,7 +259,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Add some commits
|
||||
new File(basePath + "/.hoodie").mkdirs();
|
||||
@@ -344,7 +344,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(originalRecord), schema, null, false);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Add some commits
|
||||
new File(basePath + "/.hoodie").mkdirs();
|
||||
|
||||
@@ -44,7 +44,7 @@ public class TestHoodieStorageWriterFactory extends TestHoodieClientBase {
|
||||
final String instantTime = "100";
|
||||
final Path parquetPath = new Path(basePath + "/partition/path/f1_1-0-1_000.parquet");
|
||||
final HoodieWriteConfig cfg = getConfig();
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, cfg, jsc);
|
||||
SparkTaskContextSupplier supplier = new SparkTaskContextSupplier();
|
||||
HoodieStorageWriter<IndexedRecord> parquetWriter = HoodieStorageWriterFactory.getStorageWriter(instantTime,
|
||||
parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
|
||||
|
||||
@@ -21,7 +21,6 @@ package org.apache.hudi.table;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.HoodieCleanClient;
|
||||
import org.apache.hudi.client.HoodieWriteClient;
|
||||
import org.apache.hudi.client.TestHoodieClientBase;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
@@ -104,7 +103,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
* @param insertFn Insertion API for testing
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
private String insertFirstBigBatchForClientCleanerTest(HoodieWriteConfig cfg, HoodieWriteClient client,
|
||||
private void insertFirstBigBatchForClientCleanerTest(HoodieWriteConfig cfg, HoodieWriteClient client,
|
||||
Function2<List<HoodieRecord>, String, Integer> recordGenFunction,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> insertFn) throws Exception {
|
||||
|
||||
@@ -126,7 +125,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, client.getConfig(), jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, client.getConfig(), jsc);
|
||||
|
||||
assertFalse(table.getCompletedCommitsTimeline().empty());
|
||||
String instantTime = table.getCompletedCommitsTimeline().getInstants().findFirst().get().getTimestamp();
|
||||
@@ -137,7 +136,6 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, table).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
return newCommitTime;
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -207,7 +205,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
|
||||
Map<HoodieFileGroupId, FileSlice> compactionFileIdToLatestFileSlice = new HashMap<>();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, getConfig(), jsc);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
TableFileSystemView fsView = table.getFileSystemView();
|
||||
Option<Boolean> added = Option.fromJavaOptional(fsView.getAllFileGroups(partitionPath).findFirst().map(fg -> {
|
||||
@@ -244,7 +242,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
table = HoodieTable.create(metaClient, getConfig(), jsc);
|
||||
HoodieTimeline timeline = table.getMetaClient().getCommitsTimeline();
|
||||
|
||||
TableFileSystemView fsView = table.getFileSystemView();
|
||||
@@ -376,7 +374,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table1 = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable table1 = HoodieTable.create(metaClient, cfg, jsc);
|
||||
HoodieTimeline activeTimeline = table1.getCompletedCommitsTimeline();
|
||||
Option<HoodieInstant> earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1);
|
||||
Set<HoodieInstant> acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet());
|
||||
@@ -412,7 +410,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
*
|
||||
* @param config HoodieWriteConfig
|
||||
*/
|
||||
private List<HoodieCleanStat> runCleaner(HoodieWriteConfig config) {
|
||||
private List<HoodieCleanStat> runCleaner(HoodieWriteConfig config) throws IOException {
|
||||
return runCleaner(config, false);
|
||||
}
|
||||
|
||||
@@ -421,9 +419,8 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
*
|
||||
* @param config HoodieWriteConfig
|
||||
*/
|
||||
private List<HoodieCleanStat> runCleaner(HoodieWriteConfig config, boolean simulateRetryFailure) {
|
||||
HoodieCleanClient writeClient = getHoodieCleanClient(config);
|
||||
|
||||
private List<HoodieCleanStat> runCleaner(HoodieWriteConfig config, boolean simulateRetryFailure) throws IOException {
|
||||
HoodieWriteClient<?> writeClient = getHoodieWriteClient(config);
|
||||
String cleanInstantTs = getNextInstant();
|
||||
HoodieCleanMetadata cleanMetadata1 = writeClient.clean(cleanInstantTs);
|
||||
|
||||
@@ -432,18 +429,16 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
}
|
||||
|
||||
if (simulateRetryFailure) {
|
||||
metaClient.reloadActiveTimeline()
|
||||
.revertToInflight(new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, cleanInstantTs));
|
||||
final HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieCleanMetadata cleanMetadata2 = writeClient.runClean(table,
|
||||
HoodieTimeline.getCleanInflightInstant(cleanInstantTs));
|
||||
HoodieInstant completedCleanInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, cleanInstantTs);
|
||||
metaClient.reloadActiveTimeline().revertToInflight(completedCleanInstant);
|
||||
HoodieCleanMetadata cleanMetadata2 = writeClient.clean(getNextInstant());
|
||||
Assert.assertEquals(cleanMetadata1.getEarliestCommitToRetain(), cleanMetadata2.getEarliestCommitToRetain());
|
||||
Assert.assertEquals(new Integer(0), cleanMetadata2.getTotalFilesDeleted());
|
||||
Assert.assertEquals(cleanMetadata1.getPartitionMetadata().keySet(),
|
||||
cleanMetadata2.getPartitionMetadata().keySet());
|
||||
Assert.assertEquals(cleanMetadata1.getPartitionMetadata().keySet(), cleanMetadata2.getPartitionMetadata().keySet());
|
||||
final HoodieCleanMetadata retriedCleanMetadata = CleanerUtils.getCleanerMetadata(HoodieTableMetaClient.reload(metaClient), completedCleanInstant);
|
||||
cleanMetadata1.getPartitionMetadata().keySet().forEach(k -> {
|
||||
HoodieCleanPartitionMetadata p1 = cleanMetadata1.getPartitionMetadata().get(k);
|
||||
HoodieCleanPartitionMetadata p2 = cleanMetadata2.getPartitionMetadata().get(k);
|
||||
HoodieCleanPartitionMetadata p2 = retriedCleanMetadata.getPartitionMetadata().get(k);
|
||||
Assert.assertEquals(p1.getDeletePathPatterns(), p2.getDeletePathPatterns());
|
||||
Assert.assertEquals(p1.getSuccessDeleteFiles(), p2.getFailedDeleteFiles());
|
||||
Assert.assertEquals(p1.getPartitionPath(), p2.getPartitionPath());
|
||||
@@ -636,8 +631,11 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
Collections.singletonList(fileName2)));
|
||||
newExpected.put(partition2, new Tuple3<>(deletePathPatterns2, successDeleteFiles2, failedDeleteFiles2));
|
||||
|
||||
HoodieCleanMetadata metadata =
|
||||
CleanerUtils.convertCleanMetadata(metaClient, instantTime, Option.of(0L), Arrays.asList(cleanStat1, cleanStat2));
|
||||
HoodieCleanMetadata metadata = CleanerUtils.convertCleanMetadata(
|
||||
instantTime,
|
||||
Option.of(0L),
|
||||
Arrays.asList(cleanStat1, cleanStat2)
|
||||
);
|
||||
metadata.setVersion(CleanerUtils.CLEAN_METADATA_VERSION_1);
|
||||
|
||||
// NOw upgrade and check
|
||||
@@ -896,7 +894,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
table.getActiveTimeline().createNewInstant(new HoodieInstant(State.REQUESTED,
|
||||
HoodieTimeline.COMMIT_ACTION, "000"));
|
||||
table.getActiveTimeline().transitionRequestedToInflight(
|
||||
@@ -988,7 +986,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
* Test clean previous corrupted cleanFiles.
|
||||
*/
|
||||
@Test
|
||||
public void testCleanPreviousCorruptedCleanFiles() {
|
||||
public void testCleanPreviousCorruptedCleanFiles() throws IOException {
|
||||
HoodieWriteConfig config =
|
||||
HoodieWriteConfig.newBuilder()
|
||||
.withPath(basePath).withAssumeDatePartitioning(true)
|
||||
@@ -1042,7 +1040,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
if (j == i && j <= maxNumFileIdsForCompaction) {
|
||||
expFileIdToPendingCompaction.put(fileId, compactionInstants[j]);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
FileSlice slice =
|
||||
table.getSliceView().getLatestFileSlices(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
|
||||
.filter(fs -> fs.getFileId().equals(fileId)).findFirst().get();
|
||||
@@ -1084,7 +1082,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
|
||||
// Test for safety
|
||||
final HoodieTableMetaClient newMetaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
final HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
final HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
expFileIdToPendingCompaction.forEach((fileId, value) -> {
|
||||
String baseInstantForCompaction = fileIdToLatestInstantBeforeCompaction.get(fileId);
|
||||
|
||||
@@ -98,7 +98,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
String instantTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
Pair<Path, String> newPathWithWriteToken = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
HoodieRecord record = mock(HoodieRecord.class);
|
||||
@@ -132,7 +132,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
String partitionPath = "/2016/01/31";
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Get some records belong to the same partition (2016/01/31)
|
||||
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
@@ -207,7 +207,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
Thread.sleep(1000);
|
||||
String newCommitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
final HoodieCopyOnWriteTable newTable = new HoodieCopyOnWriteTable(config, jsc);
|
||||
final HoodieCopyOnWriteTable newTable = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, jsc);
|
||||
List<WriteStatus> statuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return newTable.handleUpdate(newCommitTime, updatedRecord1.getPartitionPath(),
|
||||
updatedRecord1.getCurrentLocation().getFileId(), updatedRecords.iterator());
|
||||
@@ -274,7 +274,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Get some records belong to the same partition (2016/01/31)
|
||||
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
@@ -309,8 +309,8 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
public void testInsertRecords() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String instantTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
// Case 1:
|
||||
// 10 records for partition 1, 1 record for partition 2.
|
||||
@@ -363,7 +363,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
.limitFileSize(64 * 1024).parquetBlockSize(64 * 1024).parquetPageSize(64 * 1024).build()).build();
|
||||
String instantTime = HoodieTestUtils.makeNewCommitTime();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
// Approx 1150 records are written for block size of 64KB
|
||||
@@ -400,7 +400,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
HoodieClientTestUtils.fakeCommitFile(basePath, "001");
|
||||
HoodieClientTestUtils.fakeDataFile(basePath, testPartitionPath, "001", "file1", fileSize);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath});
|
||||
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts);
|
||||
@@ -472,7 +472,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
final HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
final HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, jsc);
|
||||
String instantTime = "000";
|
||||
// Perform inserts of 100 records to test CreateHandle and BufferedExecutor
|
||||
final List<HoodieRecord> inserts = dataGen.generateInsertsWithHoodieAvroPayload(instantTime, 100);
|
||||
@@ -483,9 +483,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
WriteStatus writeStatus = ws.get(0).get(0);
|
||||
String fileId = writeStatus.getFileId();
|
||||
metaClient.getFs().create(new Path(basePath + "/.hoodie/000.commit")).close();
|
||||
final HoodieCopyOnWriteTable table2 = new HoodieCopyOnWriteTable(config, jsc);
|
||||
final List<HoodieRecord> updates =
|
||||
dataGen.generateUpdatesWithHoodieAvroPayload(instantTime, inserts);
|
||||
final List<HoodieRecord> updates = dataGen.generateUpdatesWithHoodieAvroPayload(instantTime, inserts);
|
||||
|
||||
String partitionPath = updates.get(0).getPartitionPath();
|
||||
long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count();
|
||||
|
||||
@@ -120,7 +120,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, jsc);
|
||||
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
@@ -228,7 +228,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, jsc);
|
||||
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
@@ -337,7 +337,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
client.rollback(newCommitTime);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, jsc);
|
||||
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
HoodieTableFileSystemView roView =
|
||||
new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
||||
@@ -369,7 +369,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, jsc);
|
||||
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
@@ -450,7 +450,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
.filter(file -> file.getPath().getName().contains(commitTime2)).count(), 0);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
hoodieTable = HoodieTable.create(metaClient, cfg, jsc);
|
||||
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
||||
dataFiles = roView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList());
|
||||
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
|
||||
@@ -516,7 +516,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, jsc);
|
||||
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
@@ -679,7 +679,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, jsc);
|
||||
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
@@ -769,7 +769,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
|
||||
// Verify that all data file has one log file
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
// In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state
|
||||
((SyncableFileSystemView) (table.getSliceView())).reset();
|
||||
|
||||
@@ -793,7 +793,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
|
||||
// Verify that recently written compacted data file has no log file
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
table = HoodieTable.create(metaClient, config, jsc);
|
||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
|
||||
assertTrue("Compaction commit should be > than last insert", HoodieTimeline
|
||||
@@ -826,7 +826,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
writeClient.commit(newCommitTime, statuses);
|
||||
|
||||
HoodieTable table =
|
||||
HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc);
|
||||
HoodieTable.create(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc);
|
||||
SliceView tableRTFileSystemView = table.getSliceView();
|
||||
|
||||
long numLogFiles = 0;
|
||||
@@ -902,7 +902,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
writeClient.rollback(newCommitTime);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
SliceView tableRTFileSystemView = table.getSliceView();
|
||||
|
||||
long numLogFiles = 0;
|
||||
@@ -939,7 +939,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
statuses.collect();
|
||||
|
||||
HoodieTable table =
|
||||
HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc);
|
||||
HoodieTable.create(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc);
|
||||
SliceView tableRTFileSystemView = table.getSliceView();
|
||||
|
||||
long numLogFiles = 0;
|
||||
@@ -960,7 +960,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
writeClient.commitCompaction(newCommitTime, statuses, Option.empty());
|
||||
// Trigger a rollback of compaction
|
||||
writeClient.rollback(newCommitTime);
|
||||
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc);
|
||||
table = HoodieTable.create(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc);
|
||||
tableRTFileSystemView = table.getSliceView();
|
||||
((SyncableFileSystemView) tableRTFileSystemView).reset();
|
||||
Option<HoodieInstant> lastInstant = ((SyncableFileSystemView) tableRTFileSystemView).getLastInstant();
|
||||
@@ -981,7 +981,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
|
||||
try (HoodieWriteClient client = getWriteClient(cfg);) {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, cfg, jsc);
|
||||
|
||||
// Create a commit without rolling stats in metadata to test backwards compatibility
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
@@ -1002,10 +1002,9 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
table = HoodieTable.create(cfg, jsc);
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(
|
||||
table.getActiveTimeline()
|
||||
.getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
|
||||
table.getActiveTimeline().getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
|
||||
HoodieCommitMetadata.class);
|
||||
HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata.fromBytes(
|
||||
metadata.getExtraMetadata().get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(),
|
||||
@@ -1027,7 +1026,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
table = HoodieTable.create(cfg, jsc);
|
||||
metadata = HoodieCommitMetadata.fromBytes(
|
||||
table.getActiveTimeline()
|
||||
.getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
|
||||
@@ -1051,7 +1050,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
client.rollback(instantTime);
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
table = HoodieTable.create(cfg, jsc);
|
||||
metadata = HoodieCommitMetadata.fromBytes(
|
||||
table.getActiveTimeline()
|
||||
.getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
|
||||
@@ -1078,7 +1077,6 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
*/
|
||||
@Test
|
||||
public void testRollingStatsWithSmallFileHandling() throws Exception {
|
||||
|
||||
HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
|
||||
try (HoodieWriteClient client = getWriteClient(cfg);) {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
@@ -1095,7 +1093,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable table = HoodieTable.create(cfg, jsc);
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(
|
||||
table.getActiveTimeline()
|
||||
.getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
|
||||
@@ -1124,7 +1122,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
table = HoodieTable.create(cfg, jsc);
|
||||
metadata = HoodieCommitMetadata.fromBytes(
|
||||
table.getActiveTimeline()
|
||||
.getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
|
||||
@@ -1155,7 +1153,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
client.commitCompaction(instantTime, statuses, Option.empty());
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
table = HoodieTable.create(cfg, jsc);
|
||||
metadata = HoodieCommitMetadata.fromBytes(
|
||||
table.getActiveTimeline()
|
||||
.getInstantDetails(table.getActiveTimeline().getCommitsTimeline().lastInstant().get()).get(),
|
||||
@@ -1183,7 +1181,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
table = HoodieTable.create(cfg, jsc);
|
||||
metadata = HoodieCommitMetadata.fromBytes(
|
||||
table.getActiveTimeline()
|
||||
.getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
|
||||
@@ -1229,7 +1227,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieMergeOnReadTable hoodieTable = (HoodieMergeOnReadTable) HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieMergeOnReadTable hoodieTable = (HoodieMergeOnReadTable) HoodieTable.create(metaClient, cfg, jsc);
|
||||
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
|
||||
@@ -119,7 +119,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
|
||||
// Reload and rollback inflight compaction
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, jsc);
|
||||
// hoodieTable.rollback(jsc,
|
||||
// new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), false);
|
||||
|
||||
|
||||
@@ -100,7 +100,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
@Test(expected = HoodieNotSupportedException.class)
|
||||
public void testCompactionOnCopyOnWriteFail() throws Exception {
|
||||
metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, getConfig(), jsc);
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
}
|
||||
@@ -109,7 +109,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
public void testCompactionEmpty() throws Exception {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
try (HoodieWriteClient writeClient = getWriteClient(config);) {
|
||||
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
@@ -138,7 +138,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
|
||||
// Update all the 100 records
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
newCommitTime = "101";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
@@ -154,7 +154,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
|
||||
// Verify that all data file has one log file
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
table = HoodieTable.create(metaClient, config, jsc);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> groupedLogFiles =
|
||||
table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
@@ -165,7 +165,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
|
||||
// Do a compaction
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
table = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
JavaRDD<WriteStatus> result =
|
||||
|
||||
Reference in New Issue
Block a user