1
0

Adding ability for inserts to be written to log files

This commit is contained in:
Nishith Agarwal
2018-05-13 16:25:11 -07:00
committed by vinoth chandar
parent 34827d50e1
commit 3da063f83b
52 changed files with 1061 additions and 519 deletions

View File

@@ -76,10 +76,10 @@ public class TestCleaner extends TestHoodieClientBase {
/**
* Helper method to do first batch of insert for clean by versions/commits tests
*
* @param cfg Hoodie Write Config
* @param client Hoodie Client
* @param cfg Hoodie Write Config
* @param client Hoodie Client
* @param recordGenFunction Function to generate records for insertion
* @param insertFn Insertion API for testing
* @param insertFn Insertion API for testing
* @throws Exception in case of error
*/
private void insertFirstBigBatchForClientCleanerTest(
@@ -93,7 +93,6 @@ public class TestCleaner extends TestHoodieClientBase {
* (this is basically same as insert part of upsert, just adding it here so we can
* catch breakages in insert(), if the implementation diverges.)
*/
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
String newCommitTime = client.startCommit();
List<HoodieRecord> records = recordGenFunction.apply(newCommitTime, BIG_BATCH_INSERT_SIZE);
@@ -108,7 +107,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, getConfig());
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
assertFalse(table.getCompletedCommitTimeline().empty());
String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp();
@@ -116,7 +115,8 @@ public class TestCleaner extends TestHoodieClientBase {
assertEquals("The clean instant should be the same as the commit instant", commitTime,
table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp());
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect();
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, table).collect();
checkTaggedRecords(taggedRecords, newCommitTime);
}
@@ -158,10 +158,10 @@ public class TestCleaner extends TestHoodieClientBase {
/**
* Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective
*
* @param insertFn Insert API to be tested
* @param upsertFn Upsert API to be tested
* @param insertFn Insert API to be tested
* @param upsertFn Upsert API to be tested
* @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during
* record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs)
* record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs)
* @throws Exception in case of errors
*/
private void testInsertAndCleanByVersions(
@@ -198,8 +198,8 @@ public class TestCleaner extends TestHoodieClientBase {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig());
HoodieTimeline timeline = table.getCommitsTimeline();
HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc);
HoodieTimeline timeline = metadata.getCommitsTimeline();
TableFileSystemView fsView = table.getFileSystemView();
// Need to ensure the following
@@ -280,10 +280,10 @@ public class TestCleaner extends TestHoodieClientBase {
/**
* Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective
*
* @param insertFn Insert API to be tested
* @param upsertFn Upsert API to be tested
* @param insertFn Insert API to be tested
* @param upsertFn Upsert API to be tested
* @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during
* record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs)
* record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs)
* @throws Exception in case of errors
*/
private void testInsertAndCleanByCommits(
@@ -318,7 +318,7 @@ public class TestCleaner extends TestHoodieClientBase {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg);
HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg, jsc);
HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline();
Optional<HoodieInstant> earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1);
Set<HoodieInstant> acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet());
@@ -365,7 +365,8 @@ public class TestCleaner extends TestHoodieClientBase {
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "000");
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "000");
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
assertEquals("Must not clean any files", 0,
@@ -377,7 +378,8 @@ public class TestCleaner extends TestHoodieClientBase {
// make next commit, with 1 insert & 1 update per partition
HoodieTestUtils.createCommitFiles(basePath, "001");
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), config);
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), config,
jsc);
String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001"); // insert
String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001"); // insert
@@ -397,7 +399,7 @@ public class TestCleaner extends TestHoodieClientBase {
// make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "002");
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
config, jsc);
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file2P0C1); // update
@@ -452,7 +454,7 @@ public class TestCleaner extends TestHoodieClientBase {
// make 1 compaction commit
HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001");
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
assertEquals("Must clean three files, one parquet and 2 log files", 3,
getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size());
@@ -479,7 +481,8 @@ public class TestCleaner extends TestHoodieClientBase {
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "000");
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
assertEquals("Must not clean any files", 0,
@@ -492,7 +495,7 @@ public class TestCleaner extends TestHoodieClientBase {
// make next commit, with 1 insert & 1 update per partition
HoodieTestUtils.createCommitFiles(basePath, "001");
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
config, jsc);
String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001"); // insert
String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001"); // insert
@@ -512,7 +515,7 @@ public class TestCleaner extends TestHoodieClientBase {
// make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "002");
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
config, jsc);
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file2P0C1); // update
@@ -527,7 +530,7 @@ public class TestCleaner extends TestHoodieClientBase {
// make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "003");
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
config, jsc);
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "003", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "003", file2P0C1); // update
@@ -568,14 +571,15 @@ public class TestCleaner extends TestHoodieClientBase {
.withUseTempFolderCopyOnWriteForCreate(false)
.withUseTempFolderCopyOnWriteForMerge(false).build();
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
table.rollback(jsc, Collections.emptyList());
assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles());
config = HoodieWriteConfig.newBuilder().withPath(basePath).withUseTempFolderCopyOnWriteForCreate(true)
.withUseTempFolderCopyOnWriteForMerge(false).build();
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
config, jsc);
table.rollback(jsc, Collections.emptyList());
assertEquals("All temp files are deleted.", 0, getTotalTempFiles());
}
@@ -595,7 +599,8 @@ public class TestCleaner extends TestHoodieClientBase {
HoodieTestUtils.createCommitFiles(basePath, "000");
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", hoodieCleanStatsOne.isEmpty());
@@ -655,7 +660,8 @@ public class TestCleaner extends TestHoodieClientBase {
updateAllFilesInPartition(filesP2C0, DEFAULT_THIRD_PARTITION_PATH, "003");
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
assertEquals(100, getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size());
@@ -677,7 +683,7 @@ public class TestCleaner extends TestHoodieClientBase {
* Utility method to create temporary data files
*
* @param commitTime Commit Timestamp
* @param numFiles Number for files to be generated
* @param numFiles Number for files to be generated
* @return generated files
* @throws IOException in case of error
*/

View File

@@ -96,7 +96,7 @@ public class TestClientRollback extends TestHoodieClientBase {
List<String> partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(),
getConfig().shouldAssumeDatePartitioning());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView();
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
@@ -121,7 +121,7 @@ public class TestClientRollback extends TestHoodieClientBase {
assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig());
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> {
@@ -142,7 +142,7 @@ public class TestClientRollback extends TestHoodieClientBase {
client.rollbackToSavepoint(savepoint.getTimestamp());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig());
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> {
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));

View File

@@ -184,9 +184,9 @@ public class TestHoodieClientBase implements Serializable {
final HoodieIndex index = HoodieIndex.createIndex(writeConfig, jsc);
List<HoodieRecord> records = recordGenFunction.apply(commit, numRecords);
final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
HoodieTable.getHoodieTable(metaClient, writeConfig);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, writeConfig, jsc);
JavaRDD<HoodieRecord> taggedRecords =
index.tagLocation(jsc.parallelize(records, 1), HoodieTable.getHoodieTable(metaClient, writeConfig));
index.tagLocation(jsc.parallelize(records, 1), jsc, table);
return taggedRecords.collect();
};
}
@@ -348,7 +348,7 @@ public class TestHoodieClientBase implements Serializable {
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
}
assertEquals("Must contain " + expTotalRecords + " records", expTotalRecords,
HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count());
HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count());
// Check that the incremental consumption from prevCommitTime
assertEquals("Incremental consumption from " + prevCommitTime

View File

@@ -415,7 +415,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
assertEquals("2 files needs to be committed.", 2, statuses.size());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView();
List<HoodieDataFile> files = fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3)
.collect(Collectors.toList());
@@ -519,7 +519,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
assertEquals("2 files needs to be committed.", 2, statuses.size());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
List<HoodieDataFile> files = table.getROFileSystemView()
.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3)
.collect(Collectors.toList());
@@ -544,7 +544,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
String commitTime = "000";
client.startCommitWithTime(commitTime);
@@ -559,9 +559,9 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
HoodieTestUtils.doesCommitExist(basePath, commitTime));
// Get parquet file paths from commit metadata
String actionType = table.getCommitActionType();
String actionType = metaClient.getCommitActionType();
HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime);
HoodieTimeline commitTimeline = table.getCommitTimeline().filterCompletedInstants();
HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(commitInstant).get());
String basePath = table.getMetaClient().getBasePath();

View File

@@ -28,7 +28,6 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.table.HoodieTable;
import java.io.File;
import java.io.IOException;
import java.io.RandomAccessFile;
@@ -43,6 +42,7 @@ import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
@@ -152,14 +152,17 @@ public class HoodieClientTestUtils {
/**
* Reads the paths under the a hoodie dataset out as a DataFrame
*/
public static Dataset<Row> read(String basePath, SQLContext sqlContext, FileSystem fs, String... paths) {
public static Dataset<Row> read(JavaSparkContext jsc, String basePath, SQLContext
sqlContext,
FileSystem
fs, String...
paths) {
List<String> filteredPaths = new ArrayList<>();
try {
HoodieTable hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true);
for (String path : paths) {
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(
hoodieTable.getMetaClient(), hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path)));
metaClient, metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path)));
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(Collectors.toList());
for (HoodieDataFile file : latestFiles) {
filteredPaths.add(file.getPath());

View File

@@ -213,10 +213,10 @@ public class HoodieTestDataGenerator {
/**
* Generates new updates, randomly distributed across the keys above. There can be duplicates within the returned list
*
* @param commitTime Commit Timestamp
* @param n Number of updates (including dups)
* @return list of hoodie record updates
* @throws IOException
*/
public List<HoodieRecord> generateUpdates(String commitTime, Integer n) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
@@ -230,10 +230,10 @@ public class HoodieTestDataGenerator {
/**
* Generates deduped updates of keys previously inserted, randomly distributed across the keys above.
*
* @param commitTime Commit Timestamp
* @param n Number of unique records
* @return list of hoodie record updates
* @throws IOException
*/
public List<HoodieRecord> generateUniqueUpdates(String commitTime, Integer n) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();

View File

@@ -16,7 +16,7 @@
package com.uber.hoodie.func;
import static com.uber.hoodie.func.LazyInsertIterable.getTransformFunction;
import static com.uber.hoodie.func.CopyOnWriteLazyInsertIterable.getTransformFunction;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -38,7 +38,7 @@ public class TestBoundedInMemoryExecutor {
private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator();
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
private SparkBoundedInMemoryExecutor<HoodieRecord,
Tuple2<HoodieRecord, Optional<IndexedRecord>>, Integer> executor = null;
Tuple2<HoodieRecord, Optional<IndexedRecord>>, Integer> executor = null;
@After
public void afterTest() {
@@ -83,4 +83,4 @@ public class TestBoundedInMemoryExecutor {
// There should be no remaining records in the buffer
Assert.assertFalse(executor.isRemaining());
}
}
}

View File

@@ -16,7 +16,7 @@
package com.uber.hoodie.func;
import static com.uber.hoodie.func.LazyInsertIterable.getTransformFunction;
import static com.uber.hoodie.func.CopyOnWriteLazyInsertIterable.getTransformFunction;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.func;
import static org.junit.Assert.fail;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.TestRawTripPayload;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -29,11 +30,14 @@ import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
import java.io.File;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.Path;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
@@ -41,6 +45,7 @@ import org.junit.rules.TemporaryFolder;
public class TestUpdateMapFunction {
private String basePath = null;
private transient JavaSparkContext jsc = null;
@Before
public void init() throws Exception {
@@ -49,6 +54,18 @@ public class TestUpdateMapFunction {
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath);
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestUpdateMapFunction"));
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
@Test
@@ -56,7 +73,7 @@ public class TestUpdateMapFunction {
// Create a bunch of records with a old version of schema
HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(HoodieTestUtils.getDefaultHadoopConf(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
@@ -82,7 +99,7 @@ public class TestUpdateMapFunction {
String fileId = insertResult.next().get(0).getFileId();
System.out.println(fileId);
table = new HoodieCopyOnWriteTable(config, metaClient);
table = new HoodieCopyOnWriteTable(config, jsc);
// New content with values for the newly added field
recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";

View File

@@ -26,8 +26,7 @@ import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.HoodieTableConfig;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieIndexConfig;
@@ -74,7 +73,6 @@ public class TestHbaseIndex {
private static String tableName = "test_table";
private String basePath = null;
private transient FileSystem fs;
private HoodieTableMetaClient metaClient;
public TestHbaseIndex() throws Exception {
}
@@ -117,9 +115,7 @@ public class TestHbaseIndex {
folder.create();
basePath = folder.getRoot().getAbsolutePath();
// Initialize table
metaClient = HoodieTableMetaClient
.initTableType(utility.getConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE, tableName,
HoodieTableConfig.DEFAULT_PAYLOAD_CLASS);
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
}
@Test
@@ -132,13 +128,14 @@ public class TestHbaseIndex {
// Load to memory
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config, jsc);
HBaseIndex index = new HBaseIndex(config);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
writeClient.startCommit();
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Test tagLocation without any entries in index
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, hoodieTable);
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
// Insert 200 records
@@ -147,14 +144,15 @@ public class TestHbaseIndex {
// Now tagLocation for these records, hbaseIndex should not tag them since it was a failed
// commit
javaRDD = index.tagLocation(writeRecords, hoodieTable);
javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
// Now commit this & update location of records inserted and validate no errors
writeClient.commit(newCommitTime, writeStatues);
// Now tagLocation for these records, hbaseIndex should tag them correctly
javaRDD = index.tagLocation(writeRecords, hoodieTable);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200);
assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 200);
assertTrue(javaRDD.filter(
@@ -169,14 +167,14 @@ public class TestHbaseIndex {
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
// Load to memory
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config, jsc);
HBaseIndex index = new HBaseIndex(config);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Insert 200 records
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
@@ -184,9 +182,9 @@ public class TestHbaseIndex {
// commit this upsert
writeClient.commit(newCommitTime, writeStatues);
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Now tagLocation for these records, hbaseIndex should tag them
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, hoodieTable);
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200);
// check tagged records are tagged with correct fileIds
@@ -199,9 +197,10 @@ public class TestHbaseIndex {
// Rollback the last commit
writeClient.rollback(newCommitTime);
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled
// back commit
javaRDD = index.tagLocation(writeRecords, hoodieTable);
javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0);
}
@@ -211,7 +210,7 @@ public class TestHbaseIndex {
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config, jsc);
HBaseIndex index = new HBaseIndex(config);
// Mock hbaseConnection and related entities
Connection hbaseConnection = Mockito.mock(Connection.class);
@@ -228,15 +227,15 @@ public class TestHbaseIndex {
String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Insert 250 records
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
assertNoWriteErrors(writeStatues.collect());
// Now tagLocation for these records, hbaseIndex should tag them
index.tagLocation(writeRecords, hoodieTable);
index.tagLocation(writeRecords, jsc, hoodieTable);
// 3 batches should be executed given batchSize = 100 and parallelism = 1
Mockito.verify(table, times(3)).get((List<Get>) anyObject());
@@ -248,15 +247,15 @@ public class TestHbaseIndex {
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config, jsc);
HBaseIndex index = new HBaseIndex(config);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
// start a commit and generate test data
String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Insert 200 records
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
@@ -276,7 +275,7 @@ public class TestHbaseIndex {
// Get all the files generated
int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count();
index.updateLocation(writeStatues, hoodieTable);
index.updateLocation(writeStatues, jsc, hoodieTable);
// 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated,
// so each fileId ideally gets updates
Mockito.verify(table, atMost(numberOfDataFileIds)).put((List<Put>) anyObject());

View File

@@ -18,27 +18,58 @@ package com.uber.hoodie.index;
import static org.junit.Assert.assertTrue;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.index.hbase.HBaseIndex;
import java.io.File;
import java.io.IOException;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
public class TestHoodieIndex {
private transient JavaSparkContext jsc = null;
private String basePath = null;
@Before
public void init() throws IOException {
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieIndex"));
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
@Test
public void testCreateIndex() throws Exception {
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
// Different types
HoodieWriteConfig config = clientConfigBuilder.withPath("").withIndexConfig(
HoodieWriteConfig config = clientConfigBuilder.withPath(basePath).withIndexConfig(
indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build()).build();
assertTrue(HoodieIndex.createIndex(config, null) instanceof HBaseIndex);
config = clientConfigBuilder.withPath("")
assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HBaseIndex);
config = clientConfigBuilder.withPath(basePath)
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
assertTrue(HoodieIndex.createIndex(config, null) instanceof InMemoryHashIndex);
config = clientConfigBuilder.withPath("")
assertTrue(HoodieIndex.createIndex(config, jsc) instanceof InMemoryHashIndex);
config = clientConfigBuilder.withPath(basePath)
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
assertTrue(HoodieIndex.createIndex(config, null) instanceof HoodieBloomIndex);
assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HoodieBloomIndex);
}
}

View File

@@ -95,6 +95,16 @@ public class TestHoodieBloomIndex {
schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
@Test
public void testLoadUUIDsInMemory() throws IOException {
// Create one RDD of hoodie record
@@ -135,7 +145,7 @@ public class TestHoodieBloomIndex {
@Test
public void testLoadInvolvedFiles() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
HoodieBloomIndex index = new HoodieBloomIndex(config);
// Create some partitions, and put some files
// "2016/01/21": 0 file
@@ -171,8 +181,8 @@ public class TestHoodieBloomIndex {
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, table);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, jsc, table);
// Still 0, as no valid commit
assertEquals(filesList.size(), 0);
@@ -181,7 +191,8 @@ public class TestHoodieBloomIndex {
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
filesList = index.loadInvolvedFiles(partitions, table);
table = HoodieTable.getHoodieTable(metadata, config, jsc);
filesList = index.loadInvolvedFiles(partitions, jsc, table);
assertEquals(filesList.size(), 4);
// these files will not have the key ranges
assertNull(filesList.get(0)._2().getMaxRecordKey());
@@ -205,7 +216,7 @@ public class TestHoodieBloomIndex {
public void testRangePruning() {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
HoodieBloomIndex index = new HoodieBloomIndex(config);
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"),
@@ -290,13 +301,13 @@ public class TestHoodieBloomIndex {
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
try {
bloomIndex.tagLocation(recordRDD, table);
bloomIndex.tagLocation(recordRDD, jsc, table);
} catch (IllegalArgumentException e) {
fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices " + "required");
}
@@ -332,11 +343,11 @@ public class TestHoodieBloomIndex {
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table);
// Should not find any files
for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -350,9 +361,9 @@ public class TestHoodieBloomIndex {
// We do the tag again
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metadata, config);
table = HoodieTable.getHoodieTable(metadata, config, jsc);
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table);
// Check results
for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -397,11 +408,11 @@ public class TestHoodieBloomIndex {
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaPairRDD<HoodieKey, Optional<String>> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table);
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
JavaPairRDD<HoodieKey, Optional<String>> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, table);
// Should not find any files
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
@@ -415,8 +426,8 @@ public class TestHoodieBloomIndex {
// We do the tag again
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metadata, config);
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table);
table = HoodieTable.getHoodieTable(metadata, config, jsc);
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, table);
// Check results
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
@@ -465,10 +476,10 @@ public class TestHoodieBloomIndex {
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table);
// Check results
for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -525,14 +536,4 @@ public class TestHoodieBloomIndex {
}
return filename;
}
@After
public void clean() {
if (jsc != null) {
jsc.stop();
}
if (basePath != null) {
new File(basePath).delete();
}
}
}

View File

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

View File

@@ -107,7 +107,7 @@ public class TestHoodieCompactor {
public void testCompactionOnCopyOnWriteFail() throws Exception {
HoodieTestUtils.initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime());
}
@@ -115,7 +115,7 @@ public class TestHoodieCompactor {
public void testCompactionEmpty() throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = getConfig();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
String newCommitTime = writeClient.startCommit();
@@ -142,15 +142,15 @@ public class TestHoodieCompactor {
// Update all the 100 records
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
newCommitTime = "101";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
HoodieIndex index = new HoodieBloomIndex<>(config, jsc);
updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect();
HoodieIndex index = new HoodieBloomIndex<>(config);
updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect();
// Write them to corresponding avro logfiles
HoodieTestUtils
@@ -158,7 +158,7 @@ public class TestHoodieCompactor {
// Verify that all data file has one log file
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath)
.collect(Collectors.toList());
@@ -169,7 +169,7 @@ public class TestHoodieCompactor {
// Do a compaction
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
JavaRDD<WriteStatus> result = compactor
.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime());

View File

@@ -247,7 +247,8 @@ public class TestHoodieMergeHandleDuplicateRecords {
for (int i = 0; i < fullPartitionPaths.length; i++) {
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
}
Dataset<Row> dataSet = HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths);
Dataset<Row> dataSet = HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs,
fullPartitionPaths);
return dataSet;
}

View File

@@ -30,6 +30,7 @@ import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Random;
import java.util.stream.Collectors;
import org.junit.Test;
@@ -127,7 +128,7 @@ public class TestHoodieCompactionStrategy {
Map<Long, List<Long>> sizesMap) {
List<CompactionOperation> operations = Lists.newArrayList(sizesMap.size());
sizesMap.forEach((k, v) -> {
operations.add(new CompactionOperation(TestHoodieDataFile.newDataFile(k),
operations.add(new CompactionOperation(Optional.of(TestHoodieDataFile.newDataFile(k)),
partitionPaths[new Random().nextInt(partitionPaths.length - 1)],
v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()), config));
});

View File

@@ -77,6 +77,7 @@ public class TestCopyOnWriteTable {
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
}
@Test
@@ -90,9 +91,10 @@ public class TestCopyOnWriteTable {
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieWriteConfig config = makeHoodieClientConfig();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath);
HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath,
UUID.randomUUID().toString());
Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName);
assertTrue(newPath.toString().equals(
this.basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, unitNumber, fileName)));
@@ -117,7 +119,7 @@ public class TestCopyOnWriteTable {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
String partitionPath = "/2016/01/31";
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
// Get some records belong to the same partition (2016/01/31)
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
@@ -187,7 +189,7 @@ public class TestCopyOnWriteTable {
Thread.sleep(1000);
String newCommitTime = HoodieTestUtils.makeNewCommitTime();
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = new HoodieCopyOnWriteTable(config, metadata);
table = new HoodieCopyOnWriteTable(config, jsc);
Iterator<List<WriteStatus>> iter = table
.handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(),
updatedRecords.iterator());
@@ -255,7 +257,7 @@ public class TestCopyOnWriteTable {
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
// Get some records belong to the same partition (2016/01/31)
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
@@ -290,7 +292,7 @@ public class TestCopyOnWriteTable {
String commitTime = HoodieTestUtils.makeNewCommitTime();
FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
// Write a few records, and get atleast one file
// 10 records for partition 1, 1 record for partition 2.
@@ -324,7 +326,7 @@ public class TestCopyOnWriteTable {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
// Case 1:
// 10 records for partition 1, 1 record for partition 2.
@@ -372,7 +374,7 @@ public class TestCopyOnWriteTable {
.build()).build();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
List<HoodieRecord> records = new ArrayList<>();
// Approx 1150 records are written for block size of 64KB
@@ -411,9 +413,9 @@ public class TestCopyOnWriteTable {
HoodieClientTestUtils.fakeDataFile(basePath, testPartitionPath, "001", "file1", fileSize);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath});
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[]{testPartitionPath});
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts);
List<HoodieRecord> updateRecords = dataGenerator.generateUpdates("001", numUpdates);
for (HoodieRecord updateRec : updateRecords) {

View File

@@ -47,6 +47,7 @@ import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.index.HoodieIndex.IndexType;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import java.io.File;
import java.io.IOException;
@@ -70,7 +71,6 @@ import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
@@ -155,7 +155,7 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
@@ -166,7 +166,7 @@ public class TestMergeOnReadTable {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles);
metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent());
@@ -209,9 +209,8 @@ public class TestMergeOnReadTable {
assertTrue(dataFilesToRead.findAny().isPresent());
// verify that there is a commit
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), getConfig(false));
HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants();
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true);
HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants();
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
assertTrue(HoodieTimeline.compareTimestamps("000", latestCompactionCommitTime, HoodieTimeline.LESSER));
@@ -263,7 +262,7 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
@@ -274,7 +273,7 @@ public class TestMergeOnReadTable {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles);
metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent());
@@ -372,7 +371,7 @@ public class TestMergeOnReadTable {
client.rollback(newCommitTime);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCompletedCommitTimeline(), allFiles);
@@ -408,7 +407,7 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
@@ -419,7 +418,7 @@ public class TestMergeOnReadTable {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles);
metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent());
@@ -456,7 +455,7 @@ public class TestMergeOnReadTable {
client.rollback(newCommitTime);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList());
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
@@ -482,8 +481,8 @@ public class TestMergeOnReadTable {
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles);
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles);
final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get()
.getTimestamp();
@@ -500,8 +499,8 @@ public class TestMergeOnReadTable {
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles);
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles);
assertFalse(roView.getLatestDataFiles().filter(file -> {
if (compactedCommitTime.equals(file.getCommitTime())) {
@@ -531,7 +530,7 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
@@ -542,7 +541,7 @@ public class TestMergeOnReadTable {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCommitsTimeline().filterCompletedInstants(), allFiles);
metaClient.getCommitsTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
Map<String, Long> parquetFileIdToSize = dataFilesToRead.collect(
Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize));
@@ -591,7 +590,6 @@ public class TestMergeOnReadTable {
}
@Test
@Ignore
public void testLogFileCountsAfterCompaction() throws Exception {
// insert 100 records
HoodieWriteConfig config = getConfig(true);
@@ -606,15 +604,15 @@ public class TestMergeOnReadTable {
// Update all the 100 records
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
newCommitTime = "101";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
HoodieIndex index = new HoodieBloomIndex<>(config, jsc);
updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect();
HoodieIndex index = new HoodieBloomIndex<>(config);
updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect();
// Write them to corresponding avro logfiles
HoodieTestUtils
@@ -623,7 +621,7 @@ public class TestMergeOnReadTable {
// Verify that all data file has one log file
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath)
.collect(Collectors.toList());
@@ -634,14 +632,14 @@ public class TestMergeOnReadTable {
// Do a compaction
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
String commitTime = writeClient.startCompaction();
JavaRDD<WriteStatus> result = writeClient.compact(commitTime);
// Verify that recently written compacted data file has no log file
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
assertTrue("Compaction commit should be > than last insert", HoodieTimeline.compareTimestamps(
@@ -677,7 +675,7 @@ public class TestMergeOnReadTable {
// total time taken for creating files should be greater than 0
long totalCreateTime = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalCreateTime())
.reduce((a,b) -> a + b).intValue();
.reduce((a, b) -> a + b).intValue();
Assert.assertTrue(totalCreateTime > 0);
// Update all the 100 records
@@ -690,7 +688,7 @@ public class TestMergeOnReadTable {
writeClient.commit(newCommitTime, statuses);
// total time taken for upsert all records should be greater than 0
long totalUpsertTime = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalUpsertTime())
.reduce((a,b) -> a + b).intValue();
.reduce((a, b) -> a + b).intValue();
Assert.assertTrue(totalUpsertTime > 0);
// Do a compaction
@@ -699,22 +697,168 @@ public class TestMergeOnReadTable {
writeClient.commitCompaction(commitTime, statuses);
// total time taken for scanning log files should be greater than 0
long timeTakenForScanner = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalScanTime())
.reduce((a,b) -> a + b).longValue();
.reduce((a, b) -> a + b).longValue();
Assert.assertTrue(timeTakenForScanner > 0);
}
@Test
public void testSimpleInsertsGeneratedIntoLogFiles() throws Exception {
// insert 100 records
// Setting IndexType to be InMemory to simulate Global Index nature
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime);
writeClient.commit(newCommitTime, statuses);
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config,
jsc);
TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView();
long numLogFiles = 0;
for (String partitionPath : dataGen.getPartitionPaths()) {
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getDataFile().isPresent()).count() == 0);
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count() > 0);
numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count();
}
Assert.assertTrue(numLogFiles > 0);
// Do a compaction
String commitTime = writeClient.startCompaction();
statuses = writeClient.compact(commitTime);
Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles);
Assert.assertEquals(statuses.count(), numLogFiles);
writeClient.commitCompaction(commitTime, statuses);
}
@Test
public void testInsertsGeneratedIntoLogFilesRollback() throws Exception {
// insert 100 records
// Setting IndexType to be InMemory to simulate Global Index nature
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime);
// trigger an action
List<WriteStatus> writeStatuses = statuses.collect();
// Ensure that inserts are written to only log files
Assert.assertEquals(writeStatuses.stream().filter(writeStatus -> !writeStatus.getStat().getPath().contains("log")
).count(), 0);
Assert.assertTrue(writeStatuses.stream().filter(writeStatus -> writeStatus.getStat().getPath().contains("log")
).count() > 0);
// rollback a failed commit
boolean rollback = writeClient.rollback(newCommitTime);
Assert.assertTrue(rollback);
newCommitTime = "101";
writeClient.startCommitWithTime(newCommitTime);
// insert 100 records
records = dataGen.generateInserts(newCommitTime, 100);
recordsRDD = jsc.parallelize(records, 1);
statuses = writeClient.insert(recordsRDD, newCommitTime);
writeClient.commit(newCommitTime, statuses);
// rollback a successful commit
writeClient.rollback(newCommitTime);
final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView();
long numLogFiles = 0;
for (String partitionPath : dataGen.getPartitionPaths()) {
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getDataFile().isPresent()).count() == 0);
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count() == 0);
numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count();
}
Assert.assertTrue(numLogFiles == 0);
}
@Test
public void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction() throws Exception {
// insert 100 records
// Setting IndexType to be InMemory to simulate Global Index nature
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime);
writeClient.commit(newCommitTime, statuses);
// trigger an action
statuses.collect();
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config,
jsc);
TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView();
long numLogFiles = 0;
for (String partitionPath : dataGen.getPartitionPaths()) {
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getDataFile().isPresent()).count() == 0);
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count() > 0);
numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count();
}
Assert.assertTrue(numLogFiles > 0);
// Do a compaction
newCommitTime = writeClient.startCompaction();
statuses = writeClient.compact(newCommitTime);
// Ensure all log files have been compacted into parquet files
Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles);
Assert.assertEquals(statuses.count(), numLogFiles);
writeClient.commitCompaction(newCommitTime, statuses);
// Trigger a rollback of compaction
writeClient.rollback(newCommitTime);
for (String partitionPath : dataGen.getPartitionPaths()) {
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getDataFile().isPresent()).count() == 0);
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count() > 0);
}
}
private HoodieWriteConfig getConfig(Boolean autoCommit) {
return getConfigBuilder(autoCommit).build();
}
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
return getConfigBuilder(autoCommit, IndexType.BLOOM);
}
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieIndex.IndexType indexType) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false)
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
.forTable("test-trip-table")
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build());
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build());
}
private void assertNoWriteErrors(List<WriteStatus> statuses) {