[HUDI-217] Provide a unified resource management class to standardize the resource allocation and release for hudi client test cases
This commit is contained in:
@@ -35,7 +35,7 @@ import org.apache.spark.api.java.JavaSparkContext;
|
||||
* Abstract class taking care of holding common member variables (FileSystem, SparkContext, HoodieConfigs)
|
||||
* Also, manages embedded timeline-server if enabled.
|
||||
*/
|
||||
public abstract class AbstractHoodieClient implements Serializable {
|
||||
public abstract class AbstractHoodieClient implements Serializable, AutoCloseable {
|
||||
|
||||
private static final Logger logger = LogManager.getLogger(AbstractHoodieClient.class);
|
||||
|
||||
|
||||
@@ -0,0 +1,206 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.minicluster.HdfsTestService;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* The test harness for resource initialization and cleanup.
|
||||
*/
|
||||
public abstract class HoodieClientTestHarness implements Serializable {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(HoodieClientTestHarness.class);
|
||||
|
||||
protected transient JavaSparkContext jsc = null;
|
||||
protected transient SQLContext sqlContext;
|
||||
protected transient FileSystem fs;
|
||||
protected String basePath = null;
|
||||
protected TemporaryFolder folder = null;
|
||||
protected transient HoodieTestDataGenerator dataGen = null;
|
||||
protected transient ExecutorService executorService;
|
||||
|
||||
//dfs
|
||||
protected String dfsBasePath;
|
||||
protected transient HdfsTestService hdfsTestService;
|
||||
protected transient MiniDFSCluster dfsCluster;
|
||||
protected transient DistributedFileSystem dfs;
|
||||
|
||||
protected void initSparkContexts(String appName) {
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest(appName));
|
||||
jsc.setLogLevel("ERROR");
|
||||
|
||||
//SQLContext stuff
|
||||
sqlContext = new SQLContext(jsc);
|
||||
}
|
||||
|
||||
protected void initSparkContexts() {
|
||||
initSparkContexts("TestHoodieClient");
|
||||
}
|
||||
|
||||
protected void cleanupSparkContexts() {
|
||||
if (sqlContext != null) {
|
||||
logger.info("Clearing sql context cache of spark-session used in previous test-case");
|
||||
sqlContext.clearCache();
|
||||
sqlContext = null;
|
||||
}
|
||||
|
||||
if (jsc != null) {
|
||||
logger.info("Closing spark context used in previous test-case");
|
||||
jsc.close();
|
||||
jsc.stop();
|
||||
jsc = null;
|
||||
}
|
||||
}
|
||||
|
||||
protected void initTempFolderAndPath() throws IOException {
|
||||
folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
}
|
||||
|
||||
protected void cleanupTempFolderAndPath() throws IOException {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
|
||||
if (folder != null) {
|
||||
logger.info("Explicitly removing workspace used in previously run test-case");
|
||||
folder.delete();
|
||||
}
|
||||
}
|
||||
|
||||
protected void initFileSystem() {
|
||||
if (basePath == null) {
|
||||
throw new IllegalStateException("The base path has not been initialized.");
|
||||
}
|
||||
|
||||
if (jsc == null) {
|
||||
throw new IllegalStateException("The Spark context has not been initialized.");
|
||||
}
|
||||
|
||||
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
if (fs instanceof LocalFileSystem) {
|
||||
LocalFileSystem lfs = (LocalFileSystem) fs;
|
||||
// With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream
|
||||
// This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open
|
||||
// So, for the tests, we enforce checksum verification to circumvent the problem
|
||||
lfs.setVerifyChecksum(true);
|
||||
}
|
||||
}
|
||||
|
||||
protected void initFileSystemWithDefaultConfiguration() {
|
||||
fs = FSUtils.getFs(basePath, new Configuration());
|
||||
if (fs instanceof LocalFileSystem) {
|
||||
LocalFileSystem lfs = (LocalFileSystem) fs;
|
||||
// With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream
|
||||
// This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open
|
||||
// So, for the tests, we enforce checksum verification to circumvent the problem
|
||||
lfs.setVerifyChecksum(true);
|
||||
}
|
||||
}
|
||||
|
||||
protected void cleanupFileSystem() throws IOException {
|
||||
if (fs != null) {
|
||||
logger.warn("Closing file-system instance used in previous test-run");
|
||||
fs.close();
|
||||
}
|
||||
}
|
||||
|
||||
protected void initTableType() throws IOException {
|
||||
if (basePath == null) {
|
||||
throw new IllegalStateException("The base path has not been initialized.");
|
||||
}
|
||||
|
||||
if (jsc == null) {
|
||||
throw new IllegalStateException("The Spark context has not been initialized.");
|
||||
}
|
||||
|
||||
HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, getTableType());
|
||||
}
|
||||
|
||||
protected void cleanupTableType() {
|
||||
|
||||
}
|
||||
|
||||
protected void initTestDataGenerator() throws IOException {
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
}
|
||||
|
||||
protected void cleanupTestDataGenerator() throws IOException {
|
||||
dataGen = null;
|
||||
}
|
||||
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.COPY_ON_WRITE;
|
||||
}
|
||||
|
||||
protected void initDFS() throws IOException {
|
||||
FileSystem.closeAll();
|
||||
hdfsTestService = new HdfsTestService();
|
||||
dfsCluster = hdfsTestService.start(true);
|
||||
|
||||
// Create a temp folder as the base path
|
||||
dfs = dfsCluster.getFileSystem();
|
||||
dfsBasePath = dfs.getWorkingDirectory().toString();
|
||||
dfs.mkdirs(new Path(dfsBasePath));
|
||||
}
|
||||
|
||||
protected void cleanupDFS() throws IOException {
|
||||
if (hdfsTestService != null) {
|
||||
hdfsTestService.stop();
|
||||
dfsCluster.shutdown();
|
||||
}
|
||||
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
|
||||
// same JVM
|
||||
FileSystem.closeAll();
|
||||
}
|
||||
|
||||
protected void initExecutorServiceWithFixedThreadPool(int threadNum) {
|
||||
executorService = Executors.newFixedThreadPool(threadNum);
|
||||
}
|
||||
|
||||
protected void cleanupExecutorService() {
|
||||
if (this.executorService != null) {
|
||||
this.executorService.shutdownNow();
|
||||
this.executorService = null;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -59,7 +59,9 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
@@ -71,7 +73,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
return getConfigBuilder(autoCommit).build();
|
||||
}
|
||||
|
||||
protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
|
||||
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
|
||||
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)
|
||||
@@ -84,62 +86,74 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
.build());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
initTempFolderAndPath();
|
||||
initTestDataGenerator();
|
||||
initSparkContexts();
|
||||
initTableType();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
cleanupTableType();
|
||||
cleanupSparkContexts();
|
||||
cleanupTestDataGenerator();
|
||||
cleanupTempFolderAndPath();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollbackForInflightCompaction() throws Exception {
|
||||
// Rollback inflight compaction
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
|
||||
int numRecs = 2000;
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
|
||||
HoodieInstant pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertTrue("Pending Compaction instant has expected instant time",
|
||||
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
|
||||
assertTrue("Pending Compaction instant has expected state",
|
||||
pendingCompactionInstant.getState().equals(State.REQUESTED));
|
||||
HoodieInstant pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertTrue("Pending Compaction instant has expected instant time",
|
||||
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
|
||||
assertTrue("Pending Compaction instant has expected state",
|
||||
pendingCompactionInstant.getState().equals(State.REQUESTED));
|
||||
|
||||
moveCompactionFromRequestedToInflight(compactionInstantTime, client, cfg);
|
||||
moveCompactionFromRequestedToInflight(compactionInstantTime, client, cfg);
|
||||
|
||||
// Reload and rollback inflight compaction
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
hoodieTable.rollback(jsc, compactionInstantTime, false);
|
||||
// Reload and rollback inflight compaction
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
hoodieTable.rollback(jsc, compactionInstantTime, false);
|
||||
|
||||
client.rollbackInflightCompaction(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), hoodieTable);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
pendingCompactionInstant = metaClient.getCommitsAndCompactionTimeline().filterPendingCompactionTimeline()
|
||||
.getInstants().findFirst().get();
|
||||
assertEquals("compaction", pendingCompactionInstant.getAction());
|
||||
assertEquals(State.REQUESTED, pendingCompactionInstant.getState());
|
||||
assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp());
|
||||
client.rollbackInflightCompaction(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), hoodieTable);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
pendingCompactionInstant = metaClient.getCommitsAndCompactionTimeline().filterPendingCompactionTimeline()
|
||||
.getInstants().findFirst().get();
|
||||
assertEquals("compaction", pendingCompactionInstant.getAction());
|
||||
assertEquals(State.REQUESTED, pendingCompactionInstant.getState());
|
||||
assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp());
|
||||
|
||||
// We indirectly test for the race condition where a inflight instant was first deleted then created new. Every
|
||||
// time this happens, the pending compaction instant file in Hoodie Meta path becomes an empty file (Note: Hoodie
|
||||
// reads compaction plan from aux path which is untouched). TO test for regression, we simply get file status
|
||||
// and look at the file size
|
||||
FileStatus fstatus =
|
||||
metaClient.getFs().getFileStatus(new Path(metaClient.getMetaPath(), pendingCompactionInstant.getFileName()));
|
||||
assertTrue(fstatus.getLen() > 0);
|
||||
// We indirectly test for the race condition where a inflight instant was first deleted then created new. Every
|
||||
// time this happens, the pending compaction instant file in Hoodie Meta path becomes an empty file (Note: Hoodie
|
||||
// reads compaction plan from aux path which is untouched). TO test for regression, we simply get file status
|
||||
// and look at the file size
|
||||
FileStatus fstatus =
|
||||
metaClient.getFs().getFileStatus(new Path(metaClient.getMetaPath(), pendingCompactionInstant.getFileName()));
|
||||
assertTrue(fstatus.getLen() > 0);
|
||||
}
|
||||
}
|
||||
|
||||
private Path getInstantPath(HoodieTableMetaClient metaClient, String timestamp, String action, State state) {
|
||||
@@ -151,8 +165,6 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
public void testRollbackInflightIngestionWithPendingCompaction() throws Exception {
|
||||
// Rollback inflight ingestion when there is pending compaction
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
@@ -161,74 +173,78 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
createNextDeltaCommit(inflightInstantTime, records, client, metaClient, cfg, true);
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieInstant pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertTrue("Pending Compaction instant has expected instant time",
|
||||
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
|
||||
HoodieInstant inflightInstant =
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
assertTrue("inflight instant has expected instant time",
|
||||
inflightInstant.getTimestamp().equals(inflightInstantTime));
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
createNextDeltaCommit(inflightInstantTime, records, client, metaClient, cfg, true);
|
||||
|
||||
//This should rollback
|
||||
client.startCommitWithTime(nextInflightInstantTime);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieInstant pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertTrue("Pending Compaction instant has expected instant time",
|
||||
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
|
||||
HoodieInstant inflightInstant =
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
assertTrue("inflight instant has expected instant time",
|
||||
inflightInstant.getTimestamp().equals(inflightInstantTime));
|
||||
|
||||
//Validate
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
inflightInstant =
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
assertTrue("inflight instant has expected instant time",
|
||||
inflightInstant.getTimestamp().equals(nextInflightInstantTime));
|
||||
assertTrue("Expect only one inflight instant",
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().getInstants().count() == 1);
|
||||
//Expect pending Compaction to be present
|
||||
pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertTrue("Pending Compaction instant has expected instant time",
|
||||
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
|
||||
//This should rollback
|
||||
client.startCommitWithTime(nextInflightInstantTime);
|
||||
|
||||
//Validate
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
inflightInstant =
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
assertTrue("inflight instant has expected instant time",
|
||||
inflightInstant.getTimestamp().equals(nextInflightInstantTime));
|
||||
assertTrue("Expect only one inflight instant",
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().getInstants().count() == 1);
|
||||
//Expect pending Compaction to be present
|
||||
pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertTrue("Pending Compaction instant has expected instant time",
|
||||
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInflightCompaction() throws Exception {
|
||||
// There is inflight compaction. Subsequent compaction run must work correctly
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
String thirdInstantTime = "006";
|
||||
String fourthInstantTime = "007";
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
String thirdInstantTime = "006";
|
||||
String fourthInstantTime = "007";
|
||||
|
||||
int numRecs = 2000;
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
// Schedule and mark compaction instant as inflight
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
moveCompactionFromRequestedToInflight(compactionInstantTime, client, cfg);
|
||||
// Schedule and mark compaction instant as inflight
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
moveCompactionFromRequestedToInflight(compactionInstantTime, client, cfg);
|
||||
|
||||
// Complete ingestions
|
||||
runNextDeltaCommits(client, Arrays.asList(thirdInstantTime, fourthInstantTime),
|
||||
records, cfg, false, Arrays.asList(compactionInstantTime));
|
||||
// Complete ingestions
|
||||
runNextDeltaCommits(client, Arrays.asList(thirdInstantTime, fourthInstantTime),
|
||||
records, cfg, false, Arrays.asList(compactionInstantTime));
|
||||
|
||||
// execute inflight compaction
|
||||
executeCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, true);
|
||||
// execute inflight compaction
|
||||
executeCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, true);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -359,47 +375,49 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
public void testCompactionAfterTwoDeltaCommits() throws Exception {
|
||||
// No Delta Commits after compaction request
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
int numRecs = 2000;
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleAndExecuteCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, false);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleAndExecuteCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, false);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInterleavedCompaction() throws Exception {
|
||||
//Case: Two delta commits before and after compaction schedule
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
String thirdInstantTime = "006";
|
||||
String fourthInstantTime = "007";
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
String thirdInstantTime = "006";
|
||||
String fourthInstantTime = "007";
|
||||
|
||||
int numRecs = 2000;
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
|
||||
runNextDeltaCommits(client, Arrays.asList(thirdInstantTime, fourthInstantTime),
|
||||
records, cfg, false, Arrays.asList(compactionInstantTime));
|
||||
executeCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, true);
|
||||
runNextDeltaCommits(client, Arrays.asList(thirdInstantTime, fourthInstantTime),
|
||||
records, cfg, false, Arrays.asList(compactionInstantTime));
|
||||
executeCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, true);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -72,7 +72,9 @@ import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.scheduler.SparkListener;
|
||||
import org.apache.spark.scheduler.SparkListenerTaskEnd;
|
||||
import org.apache.spark.util.AccumulatorV2;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import scala.collection.Iterator;
|
||||
|
||||
@@ -84,9 +86,22 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
private static final int BIG_BATCH_INSERT_SIZE = 500;
|
||||
private static Logger logger = LogManager.getLogger(TestHoodieClientBase.class);
|
||||
|
||||
@Override
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
initTempFolderAndPath();
|
||||
initSparkContexts();
|
||||
initTestDataGenerator();
|
||||
initFileSystem();
|
||||
initTableType();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
cleanupTableType();
|
||||
cleanupSparkContexts();
|
||||
cleanupTestDataGenerator();
|
||||
cleanupFileSystem();
|
||||
cleanupTempFolderAndPath();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -194,108 +209,110 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
.withFinalizeWriteParallelism(1)
|
||||
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
|
||||
.build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =
|
||||
generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts);
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =
|
||||
generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts);
|
||||
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordUpsertGenWrappedFunction =
|
||||
generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateUniqueUpdates);
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordUpsertGenWrappedFunction =
|
||||
generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateUniqueUpdates);
|
||||
|
||||
insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn);
|
||||
|
||||
Map<HoodieFileGroupId, FileSlice> compactionFileIdToLatestFileSlice = new HashMap<>();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
TableFileSystemView fsView = table.getFileSystemView();
|
||||
Option<Boolean> added = Option.fromJavaOptional(fsView.getAllFileGroups(partitionPath).findFirst()
|
||||
.map(fg -> {
|
||||
fg.getLatestFileSlice().map(fs -> compactionFileIdToLatestFileSlice.put(fg.getFileGroupId(), fs));
|
||||
return true;
|
||||
}));
|
||||
if (added.isPresent()) {
|
||||
// Select only one file-group for compaction
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// Create workload with selected file-slices
|
||||
List<Pair<String, FileSlice>> partitionFileSlicePairs = compactionFileIdToLatestFileSlice.entrySet().stream()
|
||||
.map(e -> Pair.of(e.getKey().getPartitionPath(), e.getValue())).collect(Collectors.toList());
|
||||
HoodieCompactionPlan compactionPlan =
|
||||
CompactionUtils.buildFromFileSlices(partitionFileSlicePairs, Option.empty(), Option.empty());
|
||||
List<String> instantTimes = HoodieTestUtils.monotonicIncreasingCommitTimestamps(9, 1);
|
||||
String compactionTime = instantTimes.get(0);
|
||||
table.getActiveTimeline().saveToCompactionRequested(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionTime),
|
||||
AvroUtils.serializeCompactionPlan(compactionPlan));
|
||||
|
||||
instantTimes = instantTimes.subList(1, instantTimes.size());
|
||||
// Keep doing some writes and clean inline. Make sure we have expected number of files
|
||||
// remaining.
|
||||
for (String newInstantTime : instantTimes) {
|
||||
try {
|
||||
client.startCommitWithTime(newInstantTime);
|
||||
List<HoodieRecord> records = recordUpsertGenWrappedFunction.apply(newInstantTime, 100);
|
||||
|
||||
List<WriteStatus> statuses =
|
||||
upsertFn.apply(client, jsc.parallelize(records, 1), newInstantTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc);
|
||||
HoodieTimeline timeline = table.getMetaClient().getCommitsTimeline();
|
||||
insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn);
|
||||
|
||||
Map<HoodieFileGroupId, FileSlice> compactionFileIdToLatestFileSlice = new HashMap<>();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
TableFileSystemView fsView = table.getFileSystemView();
|
||||
// Need to ensure the following
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
// compute all the versions of all files, from time 0
|
||||
HashMap<String, TreeSet<String>> fileIdToVersions = new HashMap<>();
|
||||
for (HoodieInstant entry : timeline.getInstants().collect(Collectors.toList())) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(timeline.getInstantDetails(entry).get(), HoodieCommitMetadata.class);
|
||||
|
||||
for (HoodieWriteStat wstat : commitMetadata.getWriteStats(partitionPath)) {
|
||||
if (!fileIdToVersions.containsKey(wstat.getFileId())) {
|
||||
fileIdToVersions.put(wstat.getFileId(), new TreeSet<>());
|
||||
}
|
||||
fileIdToVersions.get(wstat.getFileId()).add(FSUtils.getCommitTime(new Path(wstat.getPath()).getName()));
|
||||
}
|
||||
}
|
||||
|
||||
List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList());
|
||||
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
if (compactionFileIdToLatestFileSlice.containsKey(fileGroup.getFileGroupId())) {
|
||||
// Ensure latest file-slice selected for compaction is retained
|
||||
Option<HoodieDataFile> dataFileForCompactionPresent =
|
||||
Option.fromJavaOptional(fileGroup.getAllDataFiles().filter(df -> {
|
||||
return compactionFileIdToLatestFileSlice.get(fileGroup.getFileGroupId())
|
||||
.getBaseInstantTime().equals(df.getCommitTime());
|
||||
}).findAny());
|
||||
Assert.assertTrue("Data File selected for compaction is retained",
|
||||
dataFileForCompactionPresent.isPresent());
|
||||
} else {
|
||||
// file has no more than max versions
|
||||
String fileId = fileGroup.getFileGroupId().getFileId();
|
||||
List<HoodieDataFile> dataFiles = fileGroup.getAllDataFiles().collect(Collectors.toList());
|
||||
|
||||
assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions",
|
||||
dataFiles.size() <= maxVersions);
|
||||
|
||||
// Each file, has the latest N versions (i.e cleaning gets rid of older versions)
|
||||
List<String> commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId));
|
||||
for (int i = 0; i < dataFiles.size(); i++) {
|
||||
assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions,
|
||||
Iterables.get(dataFiles, i).getCommitTime(), commitedVersions.get(commitedVersions.size() - 1 - i));
|
||||
}
|
||||
}
|
||||
}
|
||||
Option<Boolean> added = Option.fromJavaOptional(fsView.getAllFileGroups(partitionPath).findFirst()
|
||||
.map(fg -> {
|
||||
fg.getLatestFileSlice().map(fs -> compactionFileIdToLatestFileSlice.put(fg.getFileGroupId(), fs));
|
||||
return true;
|
||||
}));
|
||||
if (added.isPresent()) {
|
||||
// Select only one file-group for compaction
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// Create workload with selected file-slices
|
||||
List<Pair<String, FileSlice>> partitionFileSlicePairs = compactionFileIdToLatestFileSlice.entrySet().stream()
|
||||
.map(e -> Pair.of(e.getKey().getPartitionPath(), e.getValue())).collect(Collectors.toList());
|
||||
HoodieCompactionPlan compactionPlan =
|
||||
CompactionUtils.buildFromFileSlices(partitionFileSlicePairs, Option.empty(), Option.empty());
|
||||
List<String> instantTimes = HoodieTestUtils.monotonicIncreasingCommitTimestamps(9, 1);
|
||||
String compactionTime = instantTimes.get(0);
|
||||
table.getActiveTimeline().saveToCompactionRequested(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionTime),
|
||||
AvroUtils.serializeCompactionPlan(compactionPlan));
|
||||
|
||||
instantTimes = instantTimes.subList(1, instantTimes.size());
|
||||
// Keep doing some writes and clean inline. Make sure we have expected number of files
|
||||
// remaining.
|
||||
for (String newInstantTime : instantTimes) {
|
||||
try {
|
||||
client.startCommitWithTime(newInstantTime);
|
||||
List<HoodieRecord> records = recordUpsertGenWrappedFunction.apply(newInstantTime, 100);
|
||||
|
||||
List<WriteStatus> statuses =
|
||||
upsertFn.apply(client, jsc.parallelize(records, 1), newInstantTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc);
|
||||
HoodieTimeline timeline = table.getMetaClient().getCommitsTimeline();
|
||||
|
||||
TableFileSystemView fsView = table.getFileSystemView();
|
||||
// Need to ensure the following
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
// compute all the versions of all files, from time 0
|
||||
HashMap<String, TreeSet<String>> fileIdToVersions = new HashMap<>();
|
||||
for (HoodieInstant entry : timeline.getInstants().collect(Collectors.toList())) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(timeline.getInstantDetails(entry).get(), HoodieCommitMetadata.class);
|
||||
|
||||
for (HoodieWriteStat wstat : commitMetadata.getWriteStats(partitionPath)) {
|
||||
if (!fileIdToVersions.containsKey(wstat.getFileId())) {
|
||||
fileIdToVersions.put(wstat.getFileId(), new TreeSet<>());
|
||||
}
|
||||
fileIdToVersions.get(wstat.getFileId()).add(FSUtils.getCommitTime(new Path(wstat.getPath()).getName()));
|
||||
}
|
||||
}
|
||||
|
||||
List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList());
|
||||
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
if (compactionFileIdToLatestFileSlice.containsKey(fileGroup.getFileGroupId())) {
|
||||
// Ensure latest file-slice selected for compaction is retained
|
||||
Option<HoodieDataFile> dataFileForCompactionPresent =
|
||||
Option.fromJavaOptional(fileGroup.getAllDataFiles().filter(df -> {
|
||||
return compactionFileIdToLatestFileSlice.get(fileGroup.getFileGroupId())
|
||||
.getBaseInstantTime().equals(df.getCommitTime());
|
||||
}).findAny());
|
||||
Assert.assertTrue("Data File selected for compaction is retained",
|
||||
dataFileForCompactionPresent.isPresent());
|
||||
} else {
|
||||
// file has no more than max versions
|
||||
String fileId = fileGroup.getFileGroupId().getFileId();
|
||||
List<HoodieDataFile> dataFiles = fileGroup.getAllDataFiles().collect(Collectors.toList());
|
||||
|
||||
assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions",
|
||||
dataFiles.size() <= maxVersions);
|
||||
|
||||
// Each file, has the latest N versions (i.e cleaning gets rid of older versions)
|
||||
List<String> commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId));
|
||||
for (int i = 0; i < dataFiles.size(); i++) {
|
||||
assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions,
|
||||
Iterables.get(dataFiles, i).getCommitTime(),
|
||||
commitedVersions.get(commitedVersions.size() - 1 - i));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new RuntimeException(ioe);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new RuntimeException(ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
@@ -43,6 +42,8 @@ import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
@@ -50,9 +51,22 @@ import org.junit.Test;
|
||||
*/
|
||||
public class TestClientRollback extends TestHoodieClientBase {
|
||||
|
||||
@Override
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
initTempFolderAndPath();
|
||||
initTestDataGenerator();
|
||||
initSparkContexts();
|
||||
initFileSystem();
|
||||
initTableType();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
cleanupTableType();
|
||||
cleanupSparkContexts();
|
||||
cleanupTestDataGenerator();
|
||||
cleanupFileSystem();
|
||||
cleanupTempFolderAndPath();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -63,109 +77,110 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
|
||||
.build()).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
/**
|
||||
* Write 2 (updates)
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
/**
|
||||
* Write 2 (updates)
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
client.savepoint("hoodie-unit-test", "test");
|
||||
client.savepoint("hoodie-unit-test", "test");
|
||||
|
||||
/**
|
||||
* Write 3 (updates)
|
||||
*/
|
||||
newCommitTime = "003";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
/**
|
||||
* Write 3 (updates)
|
||||
*/
|
||||
newCommitTime = "003";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(),
|
||||
getConfig().shouldAssumeDatePartitioning());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
final ReadOptimizedView view1 = table.getROFileSystemView();
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(),
|
||||
getConfig().shouldAssumeDatePartitioning());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
final ReadOptimizedView view1 = table.getROFileSystemView();
|
||||
|
||||
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 003 should be present", 3, dataFiles.size());
|
||||
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 003 should be present", 3, dataFiles.size());
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 002 should be present", 3, dataFiles.size());
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 002 should be present", 3, dataFiles.size());
|
||||
|
||||
/**
|
||||
* Write 4 (updates)
|
||||
*/
|
||||
newCommitTime = "004";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
/**
|
||||
* Write 4 (updates)
|
||||
*/
|
||||
newCommitTime = "004";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
final ReadOptimizedView view2 = table.getROFileSystemView();
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
final ReadOptimizedView view2 = table.getROFileSystemView();
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view2.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 004 should be present", 3, dataFiles.size());
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view2.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 004 should be present", 3, dataFiles.size());
|
||||
|
||||
// rolling back to a non existent savepoint must not succeed
|
||||
try {
|
||||
client.rollbackToSavepoint("001");
|
||||
fail("Rolling back to non-existent savepoint should not be allowed");
|
||||
} catch (HoodieRollbackException e) {
|
||||
// this is good
|
||||
// rolling back to a non existent savepoint must not succeed
|
||||
try {
|
||||
client.rollbackToSavepoint("001");
|
||||
fail("Rolling back to non-existent savepoint should not be allowed");
|
||||
} catch (HoodieRollbackException e) {
|
||||
// this is good
|
||||
}
|
||||
|
||||
// rollback to savepoint 002
|
||||
HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstants().findFirst().get();
|
||||
client.rollbackToSavepoint(savepoint.getTimestamp());
|
||||
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
final ReadOptimizedView view3 = table.getROFileSystemView();
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 002 be available", 3, dataFiles.size());
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 003 should be rolled back", 0, dataFiles.size());
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 004 should be rolled back", 0, dataFiles.size());
|
||||
}
|
||||
|
||||
// rollback to savepoint 002
|
||||
HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstants().findFirst().get();
|
||||
client.rollbackToSavepoint(savepoint.getTimestamp());
|
||||
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
final ReadOptimizedView view3 = table.getROFileSystemView();
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 002 be available", 3, dataFiles.size());
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 003 should be rolled back", 0, dataFiles.size());
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 004 should be rolled back", 0, dataFiles.size());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -204,57 +219,58 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
|
||||
HoodieWriteClient client = getHoodieWriteClient(config, false);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(config, false);) {
|
||||
|
||||
// Rollback commit 1 (this should fail, since commit2 is still around)
|
||||
try {
|
||||
// Rollback commit 1 (this should fail, since commit2 is still around)
|
||||
try {
|
||||
client.rollback(commitTime1);
|
||||
assertTrue("Should have thrown an exception ", false);
|
||||
} catch (HoodieRollbackException hrbe) {
|
||||
// should get here
|
||||
}
|
||||
|
||||
// Rollback commit3
|
||||
client.rollback(commitTime3);
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
|
||||
|
||||
// simulate partial failure, where .inflight was not deleted, but data files were.
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3);
|
||||
client.rollback(commitTime3);
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
|
||||
// Rollback commit2
|
||||
client.rollback(commitTime2);
|
||||
assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
|
||||
|
||||
// simulate partial failure, where only .commit => .inflight renaming succeeded, leaving a
|
||||
// .inflight commit and a bunch of data files around.
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, commitTime2);
|
||||
file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21");
|
||||
file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22");
|
||||
file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23");
|
||||
|
||||
client.rollback(commitTime2);
|
||||
assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
|
||||
|
||||
// Let's rollback commit1, Check results
|
||||
client.rollback(commitTime1);
|
||||
assertTrue("Should have thrown an exception ", false);
|
||||
} catch (HoodieRollbackException hrbe) {
|
||||
// should get here
|
||||
assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime1));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
|
||||
}
|
||||
|
||||
// Rollback commit3
|
||||
client.rollback(commitTime3);
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
|
||||
|
||||
// simulate partial failure, where .inflight was not deleted, but data files were.
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3);
|
||||
client.rollback(commitTime3);
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
|
||||
// Rollback commit2
|
||||
client.rollback(commitTime2);
|
||||
assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
|
||||
|
||||
// simulate partial failure, where only .commit => .inflight renaming succeeded, leaving a
|
||||
// .inflight commit and a bunch of data files around.
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, commitTime2);
|
||||
file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21");
|
||||
file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22");
|
||||
file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23");
|
||||
|
||||
client.rollback(commitTime2);
|
||||
assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
|
||||
|
||||
// Let's rollback commit1, Check results
|
||||
client.rollback(commitTime1);
|
||||
assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime1));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -294,35 +310,38 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
|
||||
getHoodieWriteClient(config, false);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(config, false);) {
|
||||
|
||||
// Check results, nothing changed
|
||||
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
|
||||
// Check results, nothing changed
|
||||
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
|
||||
}
|
||||
|
||||
// Turn auto rollback on
|
||||
getHoodieWriteClient(config, true).startCommit();
|
||||
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(config, true)) {
|
||||
client.startCommit();
|
||||
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -39,6 +39,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
@@ -49,15 +50,19 @@ public class TestCompactionAdminClient extends TestHoodieClientBase {
|
||||
private CompactionAdminClient client;
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
super.init();
|
||||
public void setUp() throws Exception {
|
||||
initTempFolderAndPath();
|
||||
initSparkContexts();
|
||||
metaClient = HoodieTestUtils.initTableType(HoodieTestUtils.getDefaultHadoopConf(), basePath, MERGE_ON_READ);
|
||||
client = new CompactionAdminClient(jsc, basePath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
client.close();
|
||||
metaClient = null;
|
||||
cleanupSparkContexts();
|
||||
cleanupTempFolderAndPath();
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -114,8 +119,8 @@ public class TestCompactionAdminClient extends TestHoodieClientBase {
|
||||
public void testRepairCompactionPlan() throws Exception {
|
||||
int numEntriesPerInstant = 10;
|
||||
CompactionTestUtils
|
||||
.setupAndValidateCompactionOperations(metaClient,false, numEntriesPerInstant, numEntriesPerInstant,
|
||||
numEntriesPerInstant, numEntriesPerInstant);
|
||||
.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant, numEntriesPerInstant,
|
||||
numEntriesPerInstant, numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateRepair("000", "001", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
@@ -172,8 +177,8 @@ public class TestCompactionAdminClient extends TestHoodieClientBase {
|
||||
|
||||
/**
|
||||
* Enssure compaction plan is valid
|
||||
*
|
||||
* @param compactionInstant Compaction Instant
|
||||
* @throws Exception
|
||||
*/
|
||||
private void ensureValidCompactionPlan(String compactionInstant) throws Exception {
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
@@ -282,8 +287,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase {
|
||||
newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant, true)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)).forEach(fs -> {
|
||||
Assert.assertFalse("No Data file must be present", fs.getDataFile().isPresent());
|
||||
Assert.assertTrue("No Log Files", fs.getLogFiles().count() == 0);
|
||||
});
|
||||
Assert.assertTrue("No Log Files", fs.getLogFiles().count() == 0); });
|
||||
|
||||
// Ensure same number of log-files before and after renaming per fileId
|
||||
Map<String, Long> fileIdToCountsAfterRenaming =
|
||||
|
||||
@@ -21,36 +21,27 @@ package org.apache.hudi;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.util.ConsistencyGuard;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.FailSafeConsistencyGuard;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class TestConsistencyGuard {
|
||||
private String basePath;
|
||||
protected transient FileSystem fs;
|
||||
public class TestConsistencyGuard extends HoodieClientTestHarness {
|
||||
|
||||
@Before
|
||||
public void setup() throws IOException {
|
||||
TemporaryFolder testFolder = new TemporaryFolder();
|
||||
testFolder.create();
|
||||
basePath = testFolder.getRoot().getAbsolutePath();
|
||||
fs = FSUtils.getFs(basePath, new Configuration());
|
||||
if (fs instanceof LocalFileSystem) {
|
||||
LocalFileSystem lfs = (LocalFileSystem) fs;
|
||||
// With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream
|
||||
// This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open
|
||||
// So, for the tests, we enforce checksum verification to circumvent the problem
|
||||
lfs.setVerifyChecksum(true);
|
||||
}
|
||||
initTempFolderAndPath();
|
||||
initFileSystemWithDefaultConfiguration();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
cleanupFileSystem();
|
||||
cleanupTempFolderAndPath();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@@ -22,9 +22,7 @@ import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
@@ -32,7 +30,6 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
@@ -40,7 +37,6 @@ import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.TestRawTripPayload.MetadataMergeWriteStatus;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
@@ -49,7 +45,6 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
@@ -61,27 +56,15 @@ import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
/**
|
||||
* Base Class providing setup/cleanup and utility methods for testing Hoodie Client facing tests
|
||||
*/
|
||||
public class TestHoodieClientBase implements Serializable {
|
||||
public class TestHoodieClientBase extends HoodieClientTestHarness {
|
||||
|
||||
protected static Logger logger = LogManager.getLogger(TestHoodieClientBase.class);
|
||||
|
||||
protected transient JavaSparkContext jsc = null;
|
||||
protected transient SQLContext sqlContext;
|
||||
protected transient FileSystem fs;
|
||||
protected String basePath = null;
|
||||
protected TemporaryFolder folder = null;
|
||||
protected transient HoodieTestDataGenerator dataGen = null;
|
||||
|
||||
private HoodieWriteClient writeClient;
|
||||
private HoodieReadClient readClient;
|
||||
|
||||
@@ -120,61 +103,6 @@ public class TestHoodieClientBase implements Serializable {
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieClient"));
|
||||
jsc.setLogLevel("ERROR");
|
||||
|
||||
//SQLContext stuff
|
||||
sqlContext = new SQLContext(jsc);
|
||||
|
||||
folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
|
||||
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
if (fs instanceof LocalFileSystem) {
|
||||
LocalFileSystem lfs = (LocalFileSystem) fs;
|
||||
// With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream
|
||||
// This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open
|
||||
// So, for the tests, we enforce checksum verification to circumvent the problem
|
||||
lfs.setVerifyChecksum(true);
|
||||
}
|
||||
HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, getTableType());
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
}
|
||||
|
||||
@After
|
||||
/**
|
||||
* Properly release resources at end of each test
|
||||
*/
|
||||
public void tearDown() throws IOException {
|
||||
closeWriteClient();
|
||||
closeReadClient();
|
||||
|
||||
if (null != sqlContext) {
|
||||
logger.info("Clearing sql context cache of spark-session used in previous test-case");
|
||||
sqlContext.clearCache();
|
||||
}
|
||||
|
||||
if (null != jsc) {
|
||||
logger.info("Closing spark context used in previous test-case");
|
||||
jsc.close();
|
||||
}
|
||||
|
||||
// Create a temp folder as the base path
|
||||
if (null != folder) {
|
||||
logger.info("Explicitly removing workspace used in previously run test-case");
|
||||
folder.delete();
|
||||
}
|
||||
|
||||
if (null != fs) {
|
||||
logger.warn("Closing file-system instance used in previous test-run");
|
||||
fs.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Default HoodieWriteConfig for tests
|
||||
*
|
||||
@@ -200,8 +128,8 @@ public class TestHoodieClientBase implements Serializable {
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
|
||||
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
|
||||
.build());
|
||||
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
|
||||
.build());
|
||||
}
|
||||
|
||||
protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
@@ -469,16 +397,6 @@ public class TestHoodieClientBase implements Serializable {
|
||||
return result;
|
||||
}
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Cleaner state corresponding to a partition path
|
||||
*
|
||||
@@ -537,7 +455,4 @@ public class TestHoodieClientBase implements Serializable {
|
||||
R apply(T1 v1, T2 v2, T3 v3) throws IOException;
|
||||
}
|
||||
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.COPY_ON_WRITE;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -27,7 +27,6 @@ import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
@@ -66,15 +65,30 @@ import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
|
||||
@Override
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
initTempFolderAndPath();
|
||||
initSparkContexts();
|
||||
initTestDataGenerator();
|
||||
initFileSystem();
|
||||
initTableType();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
cleanupTableType();
|
||||
cleanupTestDataGenerator();
|
||||
cleanupSparkContexts();
|
||||
cleanupFileSystem();
|
||||
cleanupTempFolderAndPath();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -137,19 +151,21 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
boolean isPrepped) throws Exception {
|
||||
// Set autoCommit false
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
|
||||
String prevCommitTime = "000";
|
||||
String newCommitTime = "001";
|
||||
int numRecords = 200;
|
||||
JavaRDD<WriteStatus> result =
|
||||
insertFirstBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, writeFn, isPrepped, false, numRecords);
|
||||
String prevCommitTime = "000";
|
||||
String newCommitTime = "001";
|
||||
int numRecords = 200;
|
||||
JavaRDD<WriteStatus> result =
|
||||
insertFirstBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, writeFn, isPrepped, false,
|
||||
numRecords);
|
||||
|
||||
assertFalse("If Autocommit is false, then commit should not be made automatically",
|
||||
HoodieTestUtils.doesCommitExist(basePath, newCommitTime));
|
||||
assertTrue("Commit should succeed", client.commit(newCommitTime, result));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, newCommitTime));
|
||||
assertFalse("If Autocommit is false, then commit should not be made automatically",
|
||||
HoodieTestUtils.doesCommitExist(basePath, newCommitTime));
|
||||
assertTrue("Commit should succeed", client.commit(newCommitTime, result));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, newCommitTime));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -215,15 +231,16 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
assertNodupesWithinPartition(dedupedRecs);
|
||||
|
||||
// Perform write-action and check
|
||||
HoodieWriteClient client = getHoodieWriteClient(
|
||||
getConfigBuilder().combineInput(true, true).build(), false);
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<WriteStatus> statuses = writeFn.apply(client, records, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
assertEquals(2, statuses.size());
|
||||
assertNodupesWithinPartition(
|
||||
statuses.stream().map(WriteStatus::getWrittenRecords)
|
||||
.flatMap(Collection::stream).collect(Collectors.toList()));
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(
|
||||
getConfigBuilder().combineInput(true, true).build(), false);) {
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<WriteStatus> statuses = writeFn.apply(client, records, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
assertEquals(2, statuses.size());
|
||||
assertNodupesWithinPartition(
|
||||
statuses.stream().map(WriteStatus::getWrittenRecords)
|
||||
.flatMap(Collection::stream).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -534,42 +551,43 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
public void testCommitWritesRelativePaths() throws Exception {
|
||||
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
String commitTime = "000";
|
||||
client.startCommitWithTime(commitTime);
|
||||
String commitTime = "000";
|
||||
client.startCommitWithTime(commitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
|
||||
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
|
||||
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, result));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, result));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
|
||||
// Get parquet file paths from commit metadata
|
||||
String actionType = metaClient.getCommitActionType();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime);
|
||||
HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(commitInstant).get(), HoodieCommitMetadata.class);
|
||||
String basePath = table.getMetaClient().getBasePath();
|
||||
Collection<String> commitPathNames = commitMetadata.getFileIdAndFullPaths(basePath).values();
|
||||
// Get parquet file paths from commit metadata
|
||||
String actionType = metaClient.getCommitActionType();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime);
|
||||
HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(commitInstant).get(), HoodieCommitMetadata.class);
|
||||
String basePath = table.getMetaClient().getBasePath();
|
||||
Collection<String> commitPathNames = commitMetadata.getFileIdAndFullPaths(basePath).values();
|
||||
|
||||
// Read from commit file
|
||||
String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
FileInputStream inputStream = new FileInputStream(filename);
|
||||
String everything = FileIOUtils.readAsUTFString(inputStream);
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything, HoodieCommitMetadata.class);
|
||||
HashMap<String, String> paths = metadata.getFileIdAndFullPaths(basePath);
|
||||
inputStream.close();
|
||||
// Read from commit file
|
||||
String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
FileInputStream inputStream = new FileInputStream(filename);
|
||||
String everything = FileIOUtils.readAsUTFString(inputStream);
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything, HoodieCommitMetadata.class);
|
||||
HashMap<String, String> paths = metadata.getFileIdAndFullPaths(basePath);
|
||||
inputStream.close();
|
||||
|
||||
// Compare values in both to make sure they are equal.
|
||||
for (String pathName : paths.values()) {
|
||||
assertTrue(commitPathNames.contains(pathName));
|
||||
// Compare values in both to make sure they are equal.
|
||||
for (String pathName : paths.values()) {
|
||||
assertTrue(commitPathNames.contains(pathName));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -20,7 +20,6 @@ package org.apache.hudi;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
@@ -29,7 +28,9 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@@ -38,9 +39,22 @@ import org.junit.Test;
|
||||
*/
|
||||
public class TestHoodieReadClient extends TestHoodieClientBase {
|
||||
|
||||
@Override
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
initTempFolderAndPath();
|
||||
initTestDataGenerator();
|
||||
initSparkContexts();
|
||||
initFileSystem();
|
||||
initTableType();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
cleanupTableType();
|
||||
cleanupTestDataGenerator();
|
||||
cleanupSparkContexts();
|
||||
cleanupFileSystem();
|
||||
cleanupTempFolderAndPath();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -88,28 +102,30 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
|
||||
*/
|
||||
private void testReadFilterExist(HoodieWriteConfig config,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
|
||||
HoodieWriteClient writeClient = getHoodieWriteClient(config);
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
try (HoodieWriteClient writeClient = getHoodieWriteClient(config);
|
||||
HoodieReadClient readClient = getHoodieReadClient(config.getBasePath());) {
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
|
||||
HoodieReadClient readClient = getHoodieReadClient(config.getBasePath());
|
||||
JavaRDD<HoodieRecord> filteredRDD = readClient.filterExists(recordsRDD);
|
||||
JavaRDD<HoodieRecord> filteredRDD = readClient.filterExists(recordsRDD);
|
||||
|
||||
// Should not find any files
|
||||
assertTrue(filteredRDD.collect().size() == 100);
|
||||
// Should not find any files
|
||||
assertTrue(filteredRDD.collect().size() == 100);
|
||||
|
||||
JavaRDD<HoodieRecord> smallRecordsRDD = jsc.parallelize(records.subList(0, 75), 1);
|
||||
// We create three parquet file, each having one record. (3 different partitions)
|
||||
List<WriteStatus> statuses = writeFn.apply(writeClient, smallRecordsRDD, newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
JavaRDD<HoodieRecord> smallRecordsRDD = jsc.parallelize(records.subList(0, 75), 1);
|
||||
// We create three parquet file, each having one record. (3 different partitions)
|
||||
List<WriteStatus> statuses = writeFn.apply(writeClient, smallRecordsRDD, newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
readClient = getHoodieReadClient(config.getBasePath());
|
||||
filteredRDD = readClient.filterExists(recordsRDD);
|
||||
List<HoodieRecord> result = filteredRDD.collect();
|
||||
// Check results
|
||||
Assert.assertEquals(25, result.size());
|
||||
try (HoodieReadClient anotherReadClient = getHoodieReadClient(config.getBasePath());) {
|
||||
filteredRDD = anotherReadClient.filterExists(recordsRDD);
|
||||
List<HoodieRecord> result = filteredRDD.collect();
|
||||
// Check results
|
||||
Assert.assertEquals(25, result.size());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -165,43 +181,44 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> updateFn,
|
||||
boolean isPrepped)
|
||||
throws Exception {
|
||||
HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig);
|
||||
//Write 1 (only inserts)
|
||||
String newCommitTime = "001";
|
||||
String initCommitTime = "000";
|
||||
int numRecords = 200;
|
||||
JavaRDD<WriteStatus> result =
|
||||
insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, insertFn, isPrepped,
|
||||
true, numRecords);
|
||||
// Construct HoodieRecord from the WriteStatus but set HoodieKey, Data and HoodieRecordLocation accordingly
|
||||
// since they have been modified in the DAG
|
||||
JavaRDD<HoodieRecord> recordRDD =
|
||||
jsc.parallelize(
|
||||
result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
|
||||
.map(record -> new HoodieRecord(record.getKey(), null))
|
||||
.collect(Collectors.toList()));
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieReadClient readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
|
||||
List<HoodieRecord> taggedRecords = readClient.tagLocation(recordRDD).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig);) {
|
||||
//Write 1 (only inserts)
|
||||
String newCommitTime = "001";
|
||||
String initCommitTime = "000";
|
||||
int numRecords = 200;
|
||||
JavaRDD<WriteStatus> result =
|
||||
insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, insertFn, isPrepped,
|
||||
true, numRecords);
|
||||
// Construct HoodieRecord from the WriteStatus but set HoodieKey, Data and HoodieRecordLocation accordingly
|
||||
// since they have been modified in the DAG
|
||||
JavaRDD<HoodieRecord> recordRDD =
|
||||
jsc.parallelize(
|
||||
result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
|
||||
.map(record -> new HoodieRecord(record.getKey(), null))
|
||||
.collect(Collectors.toList()));
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieReadClient readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
|
||||
List<HoodieRecord> taggedRecords = readClient.tagLocation(recordRDD).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
|
||||
// Write 2 (updates)
|
||||
String prevCommitTime = newCommitTime;
|
||||
newCommitTime = "004";
|
||||
numRecords = 100;
|
||||
String commitTimeBetweenPrevAndNew = "002";
|
||||
result = updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
|
||||
Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)),
|
||||
initCommitTime, numRecords, updateFn, isPrepped,
|
||||
true, numRecords, 200, 2);
|
||||
recordRDD =
|
||||
jsc.parallelize(
|
||||
result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
|
||||
.map(record -> new HoodieRecord(record.getKey(), null))
|
||||
.collect(Collectors.toList()));
|
||||
// Index should be able to locate all updates in correct locations.
|
||||
readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
|
||||
taggedRecords = readClient.tagLocation(recordRDD).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
// Write 2 (updates)
|
||||
String prevCommitTime = newCommitTime;
|
||||
newCommitTime = "004";
|
||||
numRecords = 100;
|
||||
String commitTimeBetweenPrevAndNew = "002";
|
||||
result = updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
|
||||
Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)),
|
||||
initCommitTime, numRecords, updateFn, isPrepped,
|
||||
true, numRecords, 200, 2);
|
||||
recordRDD =
|
||||
jsc.parallelize(
|
||||
result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
|
||||
.map(record -> new HoodieRecord(record.getKey(), null))
|
||||
.collect(Collectors.toList()));
|
||||
// Index should be able to locate all updates in correct locations.
|
||||
readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
|
||||
taggedRecords = readClient.tagLocation(recordRDD).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,15 +20,10 @@ package org.apache.hudi;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
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.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.minicluster.HdfsTestService;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
@@ -42,47 +37,37 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
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;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestMultiFS implements Serializable {
|
||||
public class TestMultiFS extends HoodieClientTestHarness {
|
||||
|
||||
private static String dfsBasePath;
|
||||
private static HdfsTestService hdfsTestService;
|
||||
private static MiniDFSCluster dfsCluster;
|
||||
private static DistributedFileSystem dfs;
|
||||
private static Logger logger = LogManager.getLogger(TestMultiFS.class);
|
||||
private static JavaSparkContext jsc;
|
||||
private static SQLContext sqlContext;
|
||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||
protected String tableName = "hoodie_rt";
|
||||
private HoodieWriteClient hdfsWriteClient;
|
||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
|
||||
@BeforeClass
|
||||
public static void initClass() throws Exception {
|
||||
hdfsTestService = new HdfsTestService();
|
||||
dfsCluster = hdfsTestService.start(true);
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts();
|
||||
jsc.getConf().setAppName("hoodie-client-example");
|
||||
jsc.getConf().setMaster("local[1]");
|
||||
jsc.getConf().set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
jsc.getConf().set("spark.kryoserializer.buffer.max", "512m");
|
||||
initDFS();
|
||||
initTestDataGenerator();
|
||||
}
|
||||
|
||||
// Create a temp folder as the base path
|
||||
dfs = dfsCluster.getFileSystem();
|
||||
dfsBasePath = dfs.getWorkingDirectory().toString();
|
||||
dfs.mkdirs(new Path(dfsBasePath));
|
||||
|
||||
SparkConf sparkConf = new SparkConf().setAppName("hoodie-client-example");
|
||||
sparkConf.setMaster("local[1]");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
sparkConf.set("spark.kryoserializer.buffer.max", "512m");
|
||||
jsc = new JavaSparkContext(sparkConf);
|
||||
sqlContext = new SQLContext(jsc);
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
cleanupSparkContexts();
|
||||
cleanupDFS();
|
||||
cleanupTestDataGenerator();
|
||||
}
|
||||
|
||||
private HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig config) throws Exception {
|
||||
@@ -93,30 +78,6 @@ public class TestMultiFS implements Serializable {
|
||||
return hdfsWriteClient;
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() {
|
||||
if (null != hdfsWriteClient) {
|
||||
hdfsWriteClient.close();
|
||||
hdfsWriteClient = null;
|
||||
}
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanupClass() throws Exception {
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
|
||||
if (hdfsTestService != null) {
|
||||
hdfsTestService.stop();
|
||||
dfsCluster.shutdown();
|
||||
}
|
||||
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
|
||||
// same JVM
|
||||
FileSystem.closeAll();
|
||||
}
|
||||
|
||||
|
||||
protected HoodieWriteConfig getHoodieWriteConfig(String basePath) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withEmbeddedTimelineServerEnabled(true)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
@@ -126,10 +87,6 @@ public class TestMultiFS implements Serializable {
|
||||
|
||||
@Test
|
||||
public void readLocalWriteHDFS() throws Exception {
|
||||
|
||||
// Generator of some records to be loaded in.
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
|
||||
// Initialize table and filesystem
|
||||
HoodieTableMetaClient
|
||||
.initTableType(jsc.hadoopConfiguration(), dfsBasePath, HoodieTableType.valueOf(tableType), tableName,
|
||||
@@ -137,44 +94,44 @@ public class TestMultiFS implements Serializable {
|
||||
|
||||
//Create write client to write some records in
|
||||
HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath);
|
||||
HoodieWriteClient hdfsWriteClient = getHoodieWriteClient(cfg);
|
||||
|
||||
// Write generated data to hdfs (only inserts)
|
||||
String readCommitTime = hdfsWriteClient.startCommit();
|
||||
logger.info("Starting commit " + readCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(readCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
hdfsWriteClient.upsert(writeRecords, readCommitTime);
|
||||
|
||||
// Read from hdfs
|
||||
FileSystem fs = FSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), dfsBasePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
Dataset<Row> readRecords = HoodieClientTestUtils.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime);
|
||||
assertEquals("Should contain 100 records", readRecords.count(), records.size());
|
||||
|
||||
// Write to local
|
||||
HoodieTableMetaClient
|
||||
.initTableType(jsc.hadoopConfiguration(), tablePath, HoodieTableType.valueOf(tableType), tableName,
|
||||
HoodieAvroPayload.class.getName());
|
||||
HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath);
|
||||
HoodieWriteClient localWriteClient = getHoodieWriteClient(localConfig);
|
||||
|
||||
String writeCommitTime = localWriteClient.startCommit();
|
||||
logger.info("Starting write commit " + writeCommitTime);
|
||||
List<HoodieRecord> localRecords = dataGen.generateInserts(writeCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> localWriteRecords = jsc.parallelize(localRecords, 1);
|
||||
logger.info("Writing to path: " + tablePath);
|
||||
localWriteClient.upsert(localWriteRecords, writeCommitTime);
|
||||
try (HoodieWriteClient hdfsWriteClient = getHoodieWriteClient(cfg);
|
||||
HoodieWriteClient localWriteClient = getHoodieWriteClient(localConfig);) {
|
||||
|
||||
logger.info("Reading from path: " + tablePath);
|
||||
fs = FSUtils.getFs(tablePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
metaClient = new HoodieTableMetaClient(fs.getConf(), tablePath);
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
Dataset<Row> localReadRecords = HoodieClientTestUtils.readCommit(tablePath, sqlContext, timeline, writeCommitTime);
|
||||
assertEquals("Should contain 100 records", localReadRecords.count(), localRecords.size());
|
||||
// Write generated data to hdfs (only inserts)
|
||||
String readCommitTime = hdfsWriteClient.startCommit();
|
||||
logger.info("Starting commit " + readCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(readCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
hdfsWriteClient.upsert(writeRecords, readCommitTime);
|
||||
|
||||
hdfsWriteClient.close();
|
||||
localWriteClient.close();
|
||||
// Read from hdfs
|
||||
FileSystem fs = FSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), dfsBasePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
Dataset<Row> readRecords = HoodieClientTestUtils.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime);
|
||||
assertEquals("Should contain 100 records", readRecords.count(), records.size());
|
||||
|
||||
// Write to local
|
||||
HoodieTableMetaClient
|
||||
.initTableType(jsc.hadoopConfiguration(), tablePath, HoodieTableType.valueOf(tableType), tableName,
|
||||
HoodieAvroPayload.class.getName());
|
||||
|
||||
String writeCommitTime = localWriteClient.startCommit();
|
||||
logger.info("Starting write commit " + writeCommitTime);
|
||||
List<HoodieRecord> localRecords = dataGen.generateInserts(writeCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> localWriteRecords = jsc.parallelize(localRecords, 1);
|
||||
logger.info("Writing to path: " + tablePath);
|
||||
localWriteClient.upsert(localWriteRecords, writeCommitTime);
|
||||
|
||||
logger.info("Reading from path: " + tablePath);
|
||||
fs = FSUtils.getFs(tablePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
metaClient = new HoodieTableMetaClient(fs.getConf(), tablePath);
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
Dataset<Row> localReadRecords = HoodieClientTestUtils
|
||||
.readCommit(tablePath, sqlContext, timeline, writeCommitTime);
|
||||
assertEquals("Should contain 100 records", localReadRecords.count(), localRecords.size());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -24,6 +24,7 @@ import static org.mockito.Mockito.when;
|
||||
|
||||
import java.util.List;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
@@ -33,28 +34,28 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.func.CopyOnWriteLazyInsertIterable.HoodieInsertValueGenResult;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class TestBoundedInMemoryExecutor {
|
||||
public class TestBoundedInMemoryExecutor extends HoodieClientTestHarness {
|
||||
|
||||
private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator();
|
||||
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
private SparkBoundedInMemoryExecutor<HoodieRecord,
|
||||
Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> executor = null;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
initTestDataGenerator();
|
||||
}
|
||||
|
||||
@After
|
||||
public void afterTest() {
|
||||
if (this.executor != null) {
|
||||
this.executor.shutdownNow();
|
||||
this.executor = null;
|
||||
}
|
||||
public void tearDown() throws Exception {
|
||||
cleanupTestDataGenerator();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExecutor() throws Exception {
|
||||
|
||||
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, 100);
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(commitTime, 100);
|
||||
|
||||
HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
|
||||
when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024);
|
||||
@@ -78,12 +79,20 @@ public class TestBoundedInMemoryExecutor {
|
||||
}
|
||||
};
|
||||
|
||||
executor = new SparkBoundedInMemoryExecutor(hoodieWriteConfig,
|
||||
hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.avroSchema));
|
||||
int result = executor.execute();
|
||||
// It should buffer and write 100 records
|
||||
Assert.assertEquals(result, 100);
|
||||
// There should be no remaining records in the buffer
|
||||
Assert.assertFalse(executor.isRemaining());
|
||||
SparkBoundedInMemoryExecutor<HoodieRecord,
|
||||
Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> executor = null;
|
||||
try {
|
||||
executor = new SparkBoundedInMemoryExecutor(hoodieWriteConfig,
|
||||
hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.avroSchema));
|
||||
int result = executor.execute();
|
||||
// It should buffer and write 100 records
|
||||
Assert.assertEquals(result, 100);
|
||||
// There should be no remaining records in the buffer
|
||||
Assert.assertFalse(executor.isRemaining());
|
||||
} finally {
|
||||
if (executor != null) {
|
||||
executor.shutdownNow();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -28,14 +28,13 @@ import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.Semaphore;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
@@ -55,23 +54,20 @@ import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class TestBoundedInMemoryQueue {
|
||||
public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
|
||||
private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator();
|
||||
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
private ExecutorService executorService = null;
|
||||
|
||||
@Before
|
||||
public void beforeTest() {
|
||||
this.executorService = Executors.newFixedThreadPool(2);
|
||||
public void setUp() throws Exception {
|
||||
initTestDataGenerator();
|
||||
initExecutorServiceWithFixedThreadPool(2);
|
||||
}
|
||||
|
||||
@After
|
||||
public void afterTest() {
|
||||
if (this.executorService != null) {
|
||||
this.executorService.shutdownNow();
|
||||
this.executorService = null;
|
||||
}
|
||||
public void tearDown() throws Exception {
|
||||
cleanupTestDataGenerator();
|
||||
cleanupExecutorService();
|
||||
}
|
||||
|
||||
// Test to ensure that we are reading all records from queue iterator in the same order
|
||||
@@ -80,7 +76,7 @@ public class TestBoundedInMemoryQueue {
|
||||
@Test(timeout = 60000)
|
||||
public void testRecordReading() throws Exception {
|
||||
final int numRecords = 128;
|
||||
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(commitTime, numRecords);
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
|
||||
// Produce
|
||||
@@ -128,7 +124,7 @@ public class TestBoundedInMemoryQueue {
|
||||
Map<String, Tuple2<Integer, Integer>> keyToProducerAndIndexMap = new HashMap<>();
|
||||
|
||||
for (int i = 0; i < numProducers; i++) {
|
||||
List<HoodieRecord> pRecs = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
|
||||
List<HoodieRecord> pRecs = dataGen.generateInserts(commitTime, numRecords);
|
||||
int j = 0;
|
||||
for (HoodieRecord r : pRecs) {
|
||||
Assert.assertTrue(!keyToProducerAndIndexMap.containsKey(r.getRecordKey()));
|
||||
@@ -211,7 +207,7 @@ public class TestBoundedInMemoryQueue {
|
||||
@Test(timeout = 60000)
|
||||
public void testMemoryLimitForBuffering() throws Exception {
|
||||
final int numRecords = 128;
|
||||
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(commitTime, numRecords);
|
||||
// maximum number of records to keep in memory.
|
||||
final int recordLimit = 5;
|
||||
final SizeEstimator<HoodieInsertValueGenResult<HoodieRecord>> sizeEstimator =
|
||||
@@ -262,7 +258,7 @@ public class TestBoundedInMemoryQueue {
|
||||
@Test(timeout = 60000)
|
||||
public void testException() throws Exception {
|
||||
final int numRecords = 256;
|
||||
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(commitTime, numRecords);
|
||||
final SizeEstimator<Tuple2<HoodieRecord, Option<IndexedRecord>>> sizeEstimator =
|
||||
new DefaultSizeEstimator<>();
|
||||
// queue memory limit
|
||||
|
||||
@@ -20,8 +20,6 @@ package org.apache.hudi.func;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
@@ -29,8 +27,8 @@ import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.TestRawTripPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -45,37 +43,24 @@ import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.table.HoodieCopyOnWriteTable;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class TestUpdateMapFunction implements Serializable {
|
||||
|
||||
private String basePath = null;
|
||||
private transient JavaSparkContext jsc = null;
|
||||
public class TestUpdateMapFunction extends HoodieClientTestHarness {
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
this.basePath = folder.getRoot().getAbsolutePath();
|
||||
public void setUp() throws Exception {
|
||||
initTempFolderAndPath();
|
||||
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath);
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestUpdateMapFunction"));
|
||||
initSparkContexts("TestUpdateMapFunction");
|
||||
}
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
public void tearDown() throws Exception {
|
||||
cleanupTempFolderAndPath();
|
||||
cleanupSparkContexts();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@@ -18,12 +18,10 @@
|
||||
|
||||
package org.apache.hudi.index;
|
||||
|
||||
import java.io.File;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieHBaseIndexConfig;
|
||||
@@ -33,55 +31,41 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.hbase.DefaultHBaseQPSResourceAllocator;
|
||||
import org.apache.hudi.index.hbase.HBaseIndex;
|
||||
import org.apache.hudi.index.hbase.HBaseIndexQPSResourceAllocator;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class TestHBaseQPSResourceAllocator {
|
||||
private static JavaSparkContext jsc = null;
|
||||
public class TestHBaseQPSResourceAllocator extends HoodieClientTestHarness {
|
||||
|
||||
private static String tableName = "test_table";
|
||||
private String basePath = null;
|
||||
private static HBaseTestingUtility utility;
|
||||
private static Configuration hbaseConfig;
|
||||
private HBaseTestingUtility utility;
|
||||
private Configuration hbaseConfig;
|
||||
private static String QPS_TEST_SUFFIX_PATH = "qps_test_suffix";
|
||||
|
||||
@AfterClass
|
||||
public static void clean() {
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void init() throws Exception {
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
utility = new HBaseTestingUtility();
|
||||
utility.startMiniCluster();
|
||||
hbaseConfig = utility.getConnection().getConfiguration();
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestQPSResourceAllocator"));
|
||||
initSparkContexts("TestQPSResourceAllocator");
|
||||
|
||||
initTempFolderAndPath();
|
||||
basePath = folder.getRoot().getAbsolutePath() + QPS_TEST_SUFFIX_PATH;
|
||||
// Initialize table
|
||||
initTableType();
|
||||
}
|
||||
|
||||
@After
|
||||
public void clear() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
public void tearDown() throws Exception {
|
||||
cleanupSparkContexts();
|
||||
cleanupTempFolderAndPath();
|
||||
cleanupTableType();
|
||||
if (utility != null) {
|
||||
utility.shutdownMiniCluster();
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() throws Exception {
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath() + QPS_TEST_SUFFIX_PATH;
|
||||
// Initialize table
|
||||
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testsDefaultQPSResourceAllocator() {
|
||||
HoodieWriteConfig config = getConfig(Option.empty());
|
||||
@@ -122,14 +106,14 @@ public class TestHBaseQPSResourceAllocator {
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder(HoodieHBaseIndexConfig hoodieHBaseIndexConfig) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(1, 1).withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).withInlineCompaction(false)
|
||||
.build()).withAutoCommit(false)
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table").withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.HBASE)
|
||||
.withHBaseIndexConfig(hoodieHBaseIndexConfig)
|
||||
.build());
|
||||
.withParallelism(1, 1).withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).withInlineCompaction(false)
|
||||
.build()).withAutoCommit(false)
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table").withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.HBASE)
|
||||
.withHBaseIndexConfig(hoodieHBaseIndexConfig)
|
||||
.build());
|
||||
}
|
||||
|
||||
private HoodieHBaseIndexConfig getConfigWithResourceAllocator(Option<String> resourceAllocatorClass) {
|
||||
|
||||
@@ -24,11 +24,9 @@ import static org.mockito.Matchers.anyObject;
|
||||
import static org.mockito.Mockito.atMost;
|
||||
import static org.mockito.Mockito.times;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
@@ -37,9 +35,9 @@ import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
@@ -56,7 +54,6 @@ import org.apache.hudi.index.hbase.HBaseIndex.HbasePutBatchSizeCalculator;
|
||||
import org.apache.hudi.index.hbase.HBaseIndexQPSResourceAllocator;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
@@ -64,7 +61,6 @@ import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.FixMethodOrder;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.junit.runners.MethodSorters;
|
||||
import org.mockito.Mockito;
|
||||
import scala.Tuple2;
|
||||
@@ -75,14 +71,11 @@ import scala.Tuple2;
|
||||
* MethodSorters.NAME_ASCENDING to make sure the tests run in order. Please alter the order of tests running carefully.
|
||||
*/
|
||||
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
|
||||
public class TestHbaseIndex {
|
||||
public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
|
||||
private static JavaSparkContext jsc = null;
|
||||
private static HBaseTestingUtility utility;
|
||||
private static Configuration hbaseConfig;
|
||||
private static String tableName = "test_table";
|
||||
private String basePath = null;
|
||||
private transient FileSystem fs;
|
||||
private HoodieWriteClient writeClient;
|
||||
|
||||
public TestHbaseIndex() throws Exception {
|
||||
@@ -90,9 +83,6 @@ public class TestHbaseIndex {
|
||||
|
||||
@AfterClass
|
||||
public static void clean() throws Exception {
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
if (utility != null) {
|
||||
utility.shutdownMiniCluster();
|
||||
}
|
||||
@@ -100,37 +90,36 @@ public class TestHbaseIndex {
|
||||
|
||||
@BeforeClass
|
||||
public static void init() throws Exception {
|
||||
|
||||
// Initialize HbaseMiniCluster
|
||||
utility = new HBaseTestingUtility();
|
||||
utility.startMiniCluster();
|
||||
hbaseConfig = utility.getConnection().getConfiguration();
|
||||
utility.createTable(TableName.valueOf(tableName), Bytes.toBytes("_s"));
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHbaseIndex"));
|
||||
initSparkContexts("TestHbaseIndex");
|
||||
jsc.hadoopConfiguration().addResource(utility.getConfiguration());
|
||||
|
||||
// Create a temp folder as the base path
|
||||
initTempFolderAndPath();
|
||||
// Initialize table
|
||||
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
|
||||
initTestDataGenerator();
|
||||
}
|
||||
|
||||
@After
|
||||
public void clear() throws Exception {
|
||||
public void tearDown() throws Exception {
|
||||
if (null != writeClient) {
|
||||
writeClient.close();
|
||||
writeClient = null;
|
||||
}
|
||||
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() throws Exception {
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
// Initialize table
|
||||
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
|
||||
cleanupSparkContexts();
|
||||
cleanupTempFolderAndPath();
|
||||
cleanupTestDataGenerator();
|
||||
}
|
||||
|
||||
private HoodieWriteClient getWriteClient(HoodieWriteConfig config) throws Exception {
|
||||
@@ -145,49 +134,47 @@ public class TestHbaseIndex {
|
||||
public void testSimpleTagLocationAndUpdate() throws Exception {
|
||||
|
||||
String newCommitTime = "001";
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
HoodieWriteClient writeClient = getWriteClient(config);
|
||||
writeClient.startCommit();
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
try (HoodieWriteClient writeClient = getWriteClient(config);) {
|
||||
writeClient.startCommit();
|
||||
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, jsc, hoodieTable);
|
||||
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
|
||||
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
|
||||
|
||||
// Insert 200 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
// Insert 200 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since it was a failed
|
||||
// commit
|
||||
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
|
||||
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(
|
||||
record -> (record.getCurrentLocation() != null && record.getCurrentLocation().getInstantTime()
|
||||
.equals(newCommitTime))).distinct().count() == 200);
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since it was a failed
|
||||
// commit
|
||||
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
|
||||
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(
|
||||
record -> (record.getCurrentLocation() != null && record.getCurrentLocation().getInstantTime()
|
||||
.equals(newCommitTime))).distinct().count() == 200);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTagLocationAndDuplicateUpdate() throws Exception {
|
||||
String newCommitTime = "001";
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 10);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
@@ -215,13 +202,11 @@ public class TestHbaseIndex {
|
||||
assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 10);
|
||||
assertTrue(javaRDD.filter(
|
||||
record -> (record.getCurrentLocation() != null && record.getCurrentLocation().getInstantTime()
|
||||
.equals(newCommitTime))).distinct().count() == 10);
|
||||
.equals(newCommitTime))).distinct().count() == 10);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleTagLocationAndUpdateWithRollback() throws Exception {
|
||||
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
@@ -264,8 +249,6 @@ public class TestHbaseIndex {
|
||||
|
||||
@Test
|
||||
public void testTotalGetsBatching() throws Exception {
|
||||
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
|
||||
@@ -301,8 +284,6 @@ public class TestHbaseIndex {
|
||||
|
||||
@Test
|
||||
public void testTotalPutsBatching() throws Exception {
|
||||
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
HoodieWriteClient writeClient = getWriteClient(config);
|
||||
|
||||
@@ -20,44 +20,30 @@ package org.apache.hudi.index;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.config.HoodieHBaseIndexConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.bloom.HoodieBloomIndex;
|
||||
import org.apache.hudi.index.hbase.HBaseIndex;
|
||||
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;
|
||||
public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
|
||||
@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);
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts("TestHoodieIndex");
|
||||
initTempFolderAndPath();
|
||||
initTableType();
|
||||
}
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
public void tearDown() throws Exception {
|
||||
cleanupSparkContexts();
|
||||
cleanupTempFolderAndPath();
|
||||
cleanupTableType();
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -67,7 +53,7 @@ public class TestHoodieIndex {
|
||||
// Different types
|
||||
HoodieWriteConfig config = clientConfigBuilder.withPath(basePath).withIndexConfig(
|
||||
indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE)
|
||||
.withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder().build()).build()).build();
|
||||
.withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder().build()).build()).build();
|
||||
assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HBaseIndex);
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
|
||||
@@ -37,8 +37,8 @@ import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.TestRawTripPayload;
|
||||
@@ -57,21 +57,16 @@ import org.apache.hudi.io.HoodieKeyLookupHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import scala.Tuple2;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class TestHoodieBloomIndex {
|
||||
public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
private JavaSparkContext jsc = null;
|
||||
private String basePath = null;
|
||||
private transient FileSystem fs;
|
||||
private String schemaStr;
|
||||
private Schema schema;
|
||||
|
||||
@@ -93,14 +88,9 @@ public class TestHoodieBloomIndex {
|
||||
}
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieBloomIndex"));
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts("TestHoodieBloomIndex");
|
||||
initTempFolderAndPath();
|
||||
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
|
||||
// We have some records to be tagged (two different partitions)
|
||||
schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
|
||||
@@ -108,13 +98,9 @@ public class TestHoodieBloomIndex {
|
||||
}
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
public void tearDown() throws Exception {
|
||||
cleanupSparkContexts();
|
||||
cleanupTempFolderAndPath();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig makeConfig() {
|
||||
|
||||
@@ -34,7 +34,7 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.TestRawTripPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
@@ -49,18 +49,13 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class TestHoodieGlobalBloomIndex {
|
||||
public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
private JavaSparkContext jsc = null;
|
||||
private String basePath = null;
|
||||
private transient FileSystem fs;
|
||||
private String schemaStr;
|
||||
private Schema schema;
|
||||
|
||||
@@ -68,14 +63,9 @@ public class TestHoodieGlobalBloomIndex {
|
||||
}
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieGlobalBloomIndex"));
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts("TestHoodieGlobalBloomIndex");
|
||||
initTempFolderAndPath();
|
||||
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
|
||||
// We have some records to be tagged (two different partitions)
|
||||
schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
|
||||
@@ -83,13 +73,9 @@ public class TestHoodieGlobalBloomIndex {
|
||||
}
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
public void tearDown() throws Exception {
|
||||
cleanupSparkContexts();
|
||||
cleanupTempFolderAndPath();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
@@ -32,14 +31,10 @@ import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.minicluster.HdfsTestService;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
@@ -52,59 +47,20 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class TestHoodieCommitArchiveLog {
|
||||
public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
|
||||
//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
|
||||
// FsDataInputStream instead of a InputStream and thus throws java.lang.ClassCastException:
|
||||
// org.apache.hadoop.fs.FSDataInputStream cannot be cast to org.apache.hadoop.fs.FSInputStream
|
||||
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 {
|
||||
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
|
||||
// same JVM
|
||||
FileSystem.closeAll();
|
||||
|
||||
if (hdfsTestService != null) {
|
||||
hdfsTestService.stop();
|
||||
dfsCluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpDFS() throws IOException {
|
||||
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
|
||||
// same JVM
|
||||
FileSystem.closeAll();
|
||||
if (hdfsTestService == null) {
|
||||
hdfsTestService = new HdfsTestService();
|
||||
dfsCluster = hdfsTestService.start(true);
|
||||
// Create a temp folder as the base path
|
||||
dfs = dfsCluster.getFileSystem();
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieCommitArchiveLog"));
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
initDFS();
|
||||
|
||||
initTempFolderAndPath();
|
||||
initSparkContexts("TestHoodieCommitArchiveLog");
|
||||
hadoopConf = dfs.getConf();
|
||||
jsc.hadoopConfiguration().addResource(dfs.getConf());
|
||||
dfs.mkdirs(new Path(basePath));
|
||||
@@ -112,13 +68,11 @@ public class TestHoodieCommitArchiveLog {
|
||||
}
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
public void clean() throws IOException {
|
||||
cleanupDFS();
|
||||
|
||||
cleanupTempFolderAndPath();
|
||||
cleanupSparkContexts();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@@ -21,15 +21,12 @@ package org.apache.hudi.io;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -46,56 +43,36 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.bloom.HoodieBloomIndex;
|
||||
import org.apache.hudi.io.compact.HoodieCompactor;
|
||||
import org.apache.hudi.io.compact.HoodieRealtimeTableCompactor;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
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 TestHoodieCompactor {
|
||||
public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
|
||||
private transient JavaSparkContext jsc = null;
|
||||
private String basePath = null;
|
||||
private HoodieCompactor compactor;
|
||||
private transient HoodieTestDataGenerator dataGen = null;
|
||||
private transient FileSystem fs;
|
||||
private Configuration hadoopConf;
|
||||
private HoodieWriteClient writeClient;
|
||||
private Configuration hadoopConf;
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
public void setUp() throws Exception {
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieCompactor"));
|
||||
initSparkContexts("TestHoodieCompactor");
|
||||
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
initTempFolderAndPath();
|
||||
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
|
||||
fs = FSUtils.getFs(basePath, hadoopConf);
|
||||
HoodieTestUtils.initTableType(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
|
||||
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
compactor = new HoodieRealtimeTableCompactor();
|
||||
initTestDataGenerator();
|
||||
}
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (null != writeClient) {
|
||||
writeClient.close();
|
||||
writeClient = null;
|
||||
}
|
||||
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
public void tearDown() throws Exception {
|
||||
cleanupFileSystem();
|
||||
cleanupTestDataGenerator();
|
||||
cleanupTempFolderAndPath();
|
||||
cleanupSparkContexts();
|
||||
}
|
||||
|
||||
private HoodieWriteClient getWriteClient(HoodieWriteConfig config) throws Exception {
|
||||
@@ -137,74 +114,81 @@ public class TestHoodieCompactor {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieWriteClient writeClient = getWriteClient(config);
|
||||
try (HoodieWriteClient writeClient = getWriteClient(config);) {
|
||||
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
JavaRDD<WriteStatus> result =
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
assertTrue("If there is nothing to compact, result will be empty", result.isEmpty());
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
JavaRDD<WriteStatus> result =
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
assertTrue("If there is nothing to compact, result will be empty", result.isEmpty());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteStatusContentsAfterCompaction() throws Exception {
|
||||
// insert 100 records
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HoodieWriteClient writeClient = getWriteClient(config);
|
||||
String newCommitTime = "100";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
try (HoodieWriteClient writeClient = getWriteClient(config);) {
|
||||
String newCommitTime = "100";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
// Update all the 100 records
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
// Update all the 100 records
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
newCommitTime = "101";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
newCommitTime = "101";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
HoodieIndex index = new HoodieBloomIndex<>(config);
|
||||
updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect();
|
||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
HoodieIndex index = new HoodieBloomIndex<>(config);
|
||||
updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect();
|
||||
|
||||
// Write them to corresponding avro logfiles
|
||||
HoodieTestUtils
|
||||
.writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchemaWithMetadataFields,
|
||||
updatedRecords);
|
||||
// Write them to corresponding avro logfiles
|
||||
HoodieTestUtils
|
||||
.writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchemaWithMetadataFields,
|
||||
updatedRecords);
|
||||
|
||||
// Verify that all data file has one log file
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
for (FileSlice fileSlice : groupedLogFiles) {
|
||||
assertEquals("There should be 1 log file written for every data file", 1, fileSlice.getLogFiles().count());
|
||||
// Verify that all data file has one log file
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
for (FileSlice fileSlice : groupedLogFiles) {
|
||||
assertEquals("There should be 1 log file written for every data file", 1, fileSlice.getLogFiles().count());
|
||||
}
|
||||
}
|
||||
|
||||
// Do a compaction
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
JavaRDD<WriteStatus> result =
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
|
||||
// Verify that all partition paths are present in the WriteStatus result
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<WriteStatus> writeStatuses = result.collect();
|
||||
assertTrue(writeStatuses.stream()
|
||||
.filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath))
|
||||
.count() > 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Do a compaction
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
JavaRDD<WriteStatus> result =
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
|
||||
// Verify that all partition paths are present in the WriteStatus result
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<WriteStatus> writeStatuses = result.collect();
|
||||
assertTrue(writeStatuses.stream()
|
||||
.filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath))
|
||||
.count() > 0);
|
||||
}
|
||||
@Override
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
}
|
||||
|
||||
// TODO - after modifying HoodieReadClient to support realtime tables - add more tests to make
|
||||
|
||||
@@ -22,17 +22,15 @@ import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotEquals;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
@@ -45,56 +43,34 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
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;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public class TestHoodieMergeHandle {
|
||||
public class TestHoodieMergeHandle extends HoodieClientTestHarness {
|
||||
|
||||
protected transient JavaSparkContext jsc = null;
|
||||
protected transient SQLContext sqlContext;
|
||||
protected transient FileSystem fs;
|
||||
protected String basePath = null;
|
||||
protected transient HoodieTestDataGenerator dataGen = null;
|
||||
private HoodieWriteClient writeClient;
|
||||
private HoodieWriteClient writeClient;
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeHandle"));
|
||||
|
||||
//SQLContext stuff
|
||||
sqlContext = new SQLContext(jsc);
|
||||
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts("TestHoodieMergeHandle");
|
||||
initTempFolderAndPath();
|
||||
initFileSystem();
|
||||
initTableType();
|
||||
initTestDataGenerator();
|
||||
}
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (null != writeClient) {
|
||||
writeClient.close();
|
||||
writeClient = null;
|
||||
}
|
||||
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
public void tearDown() throws Exception {
|
||||
cleanupTableType();
|
||||
cleanupFileSystem();
|
||||
cleanupTestDataGenerator();
|
||||
cleanupTempFolderAndPath();
|
||||
cleanupSparkContexts();
|
||||
}
|
||||
|
||||
private HoodieWriteClient getWriteClient(HoodieWriteConfig config) throws Exception {
|
||||
@@ -113,233 +89,235 @@ public class TestHoodieMergeHandle {
|
||||
|
||||
// Build a write config with bulkinsertparallelism set
|
||||
HoodieWriteConfig cfg = getConfigBuilder().build();
|
||||
HoodieWriteClient client = getWriteClient(cfg);
|
||||
FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
try (HoodieWriteClient client = getWriteClient(cfg);) {
|
||||
FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
* This will do a bulk insert of 44 records of which there are 2 records repeated 21 times each.
|
||||
* id1 (21 records), id2 (21 records), id3, id4
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 4);
|
||||
HoodieRecord record1 = records.get(0);
|
||||
HoodieRecord record2 = records.get(1);
|
||||
for (int i = 0; i < 20; i++) {
|
||||
HoodieRecord dup = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime);
|
||||
records.add(dup);
|
||||
}
|
||||
for (int i = 0; i < 20; i++) {
|
||||
HoodieRecord dup = dataGen.generateUpdateRecord(record2.getKey(), newCommitTime);
|
||||
records.add(dup);
|
||||
}
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
List<WriteStatus> statuses = client.bulkInsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there is a commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
Assert.assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp());
|
||||
assertEquals("Must contain 44 records",
|
||||
records.size(),
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
|
||||
|
||||
/**
|
||||
* Write 2 (insert)
|
||||
* This will do a bulk insert of 1 record with the same row_key as record1 in the previous insert - id1.
|
||||
* At this point, we will have 2 files with the row_keys as shown here -
|
||||
* File 1 - id1 (21 records), id2 (21 records), id3, id4
|
||||
* File 2 - id1
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
// Do 1 more bulk insert with the same dup record1
|
||||
List<HoodieRecord> newRecords = new ArrayList<>();
|
||||
HoodieRecord sameAsRecord1 = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime);
|
||||
newRecords.add(sameAsRecord1);
|
||||
writeRecords = jsc.parallelize(newRecords, 1);
|
||||
statuses = client.bulkInsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there are 2 commits
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals("Expecting two commits.", 2, timeline.findInstantsAfter("000", Integer.MAX_VALUE)
|
||||
.countInstants());
|
||||
Assert.assertEquals("Latest commit should be 002", newCommitTime, timeline.lastInstant().get().getTimestamp());
|
||||
Dataset<Row> dataSet = getRecords();
|
||||
assertEquals("Must contain 45 records", 45, dataSet.count());
|
||||
|
||||
/**
|
||||
* Write 3 (insert)
|
||||
* This will bulk insert 2 new completely new records.
|
||||
* At this point, we will have 2 files with the row_keys as shown here -
|
||||
* File 1 - id1 (21 records), id2 (21 records), id3, id4
|
||||
* File 2 - id1
|
||||
* File 3 - id5, id6
|
||||
*/
|
||||
newCommitTime = "003";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
newRecords = dataGen.generateInserts(newCommitTime, 2);
|
||||
writeRecords = jsc.parallelize(newRecords, 1);
|
||||
statuses = client.bulkInsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there are now 3 commits
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals("Expecting three commits.", 3, timeline.findInstantsAfter("000", Integer.MAX_VALUE)
|
||||
.countInstants());
|
||||
Assert.assertEquals("Latest commit should be 003", newCommitTime, timeline.lastInstant().get().getTimestamp());
|
||||
dataSet = getRecords();
|
||||
assertEquals("Must contain 47 records", 47, dataSet.count());
|
||||
|
||||
/**
|
||||
* Write 4 (updates)
|
||||
* This will generate 2 upsert records with id1 and id2. The rider and driver names in the update records
|
||||
* will be rider-004 and driver-004.
|
||||
* After the upsert is complete, all the records with id1 in File 1 and File 2 must be updated, all the records
|
||||
* with id2 in File 2 must also be updated.
|
||||
* Also, none of the other records in File 1, File 2 and File 3 must be updated.
|
||||
*/
|
||||
newCommitTime = "004";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<HoodieRecord> updateRecords = new ArrayList<>();
|
||||
|
||||
// This exists in 001 and 002 and should be updated in both
|
||||
sameAsRecord1 = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime);
|
||||
updateRecords.add(sameAsRecord1);
|
||||
|
||||
// This exists in 001 and should be updated
|
||||
HoodieRecord sameAsRecord2 = dataGen.generateUpdateRecord(record2.getKey(), newCommitTime);
|
||||
updateRecords.add(sameAsRecord2);
|
||||
JavaRDD<HoodieRecord> updateRecordsRDD = jsc.parallelize(updateRecords, 1);
|
||||
statuses = client.upsert(updateRecordsRDD, newCommitTime).collect();
|
||||
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify there are now 4 commits
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals("Expecting four commits.", 4, timeline.findInstantsAfter("000", Integer.MAX_VALUE)
|
||||
.countInstants());
|
||||
Assert.assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime);
|
||||
|
||||
// Check the entire dataset has 47 records still
|
||||
dataSet = getRecords();
|
||||
assertEquals("Must contain 47 records", 47, dataSet.count());
|
||||
Row[] rows = (Row[]) dataSet.collect();
|
||||
int record1Count = 0;
|
||||
int record2Count = 0;
|
||||
for (Row row : rows) {
|
||||
if (row.getAs("_hoodie_record_key").equals(record1.getKey().getRecordKey())) {
|
||||
record1Count++;
|
||||
// assert each duplicate record is updated
|
||||
assertEquals(row.getAs("rider"), "rider-004");
|
||||
assertEquals(row.getAs("driver"), "driver-004");
|
||||
} else if (row.getAs("_hoodie_record_key").equals(record2.getKey().getRecordKey())) {
|
||||
record2Count++;
|
||||
// assert each duplicate record is updated
|
||||
assertEquals(row.getAs("rider"), "rider-004");
|
||||
assertEquals(row.getAs("driver"), "driver-004");
|
||||
} else {
|
||||
assertNotEquals(row.getAs("rider"), "rider-004");
|
||||
assertNotEquals(row.getAs("driver"), "rider-004");
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
* This will do a bulk insert of 44 records of which there are 2 records repeated 21 times each.
|
||||
* id1 (21 records), id2 (21 records), id3, id4
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 4);
|
||||
HoodieRecord record1 = records.get(0);
|
||||
HoodieRecord record2 = records.get(1);
|
||||
for (int i = 0; i < 20; i++) {
|
||||
HoodieRecord dup = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime);
|
||||
records.add(dup);
|
||||
}
|
||||
}
|
||||
// Assert that id1 record count which has been updated to rider-004 and driver-004 is 22, which is the total
|
||||
// number of records with row_key id1
|
||||
assertEquals(22, record1Count);
|
||||
for (int i = 0; i < 20; i++) {
|
||||
HoodieRecord dup = dataGen.generateUpdateRecord(record2.getKey(), newCommitTime);
|
||||
records.add(dup);
|
||||
}
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
List<WriteStatus> statuses = client.bulkInsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// Assert that id2 record count which has been updated to rider-004 and driver-004 is 21, which is the total
|
||||
// number of records with row_key id2
|
||||
assertEquals(21, record2Count);
|
||||
// verify that there is a commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals("Expecting a single commit.", 1,
|
||||
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
Assert.assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp());
|
||||
assertEquals("Must contain 44 records",
|
||||
records.size(),
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
|
||||
|
||||
/**
|
||||
* Write 2 (insert)
|
||||
* This will do a bulk insert of 1 record with the same row_key as record1 in the previous insert - id1.
|
||||
* At this point, we will have 2 files with the row_keys as shown here -
|
||||
* File 1 - id1 (21 records), id2 (21 records), id3, id4
|
||||
* File 2 - id1
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
// Do 1 more bulk insert with the same dup record1
|
||||
List<HoodieRecord> newRecords = new ArrayList<>();
|
||||
HoodieRecord sameAsRecord1 = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime);
|
||||
newRecords.add(sameAsRecord1);
|
||||
writeRecords = jsc.parallelize(newRecords, 1);
|
||||
statuses = client.bulkInsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there are 2 commits
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals("Expecting two commits.", 2, timeline.findInstantsAfter("000", Integer.MAX_VALUE)
|
||||
.countInstants());
|
||||
Assert.assertEquals("Latest commit should be 002", newCommitTime, timeline.lastInstant().get().getTimestamp());
|
||||
Dataset<Row> dataSet = getRecords();
|
||||
assertEquals("Must contain 45 records", 45, dataSet.count());
|
||||
|
||||
/**
|
||||
* Write 3 (insert)
|
||||
* This will bulk insert 2 new completely new records.
|
||||
* At this point, we will have 2 files with the row_keys as shown here -
|
||||
* File 1 - id1 (21 records), id2 (21 records), id3, id4
|
||||
* File 2 - id1
|
||||
* File 3 - id5, id6
|
||||
*/
|
||||
newCommitTime = "003";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
newRecords = dataGen.generateInserts(newCommitTime, 2);
|
||||
writeRecords = jsc.parallelize(newRecords, 1);
|
||||
statuses = client.bulkInsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there are now 3 commits
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals("Expecting three commits.", 3, timeline.findInstantsAfter("000", Integer.MAX_VALUE)
|
||||
.countInstants());
|
||||
Assert.assertEquals("Latest commit should be 003", newCommitTime, timeline.lastInstant().get().getTimestamp());
|
||||
dataSet = getRecords();
|
||||
assertEquals("Must contain 47 records", 47, dataSet.count());
|
||||
|
||||
/**
|
||||
* Write 4 (updates)
|
||||
* This will generate 2 upsert records with id1 and id2. The rider and driver names in the update records
|
||||
* will be rider-004 and driver-004.
|
||||
* After the upsert is complete, all the records with id1 in File 1 and File 2 must be updated, all the records
|
||||
* with id2 in File 2 must also be updated.
|
||||
* Also, none of the other records in File 1, File 2 and File 3 must be updated.
|
||||
*/
|
||||
newCommitTime = "004";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<HoodieRecord> updateRecords = new ArrayList<>();
|
||||
|
||||
// This exists in 001 and 002 and should be updated in both
|
||||
sameAsRecord1 = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime);
|
||||
updateRecords.add(sameAsRecord1);
|
||||
|
||||
// This exists in 001 and should be updated
|
||||
HoodieRecord sameAsRecord2 = dataGen.generateUpdateRecord(record2.getKey(), newCommitTime);
|
||||
updateRecords.add(sameAsRecord2);
|
||||
JavaRDD<HoodieRecord> updateRecordsRDD = jsc.parallelize(updateRecords, 1);
|
||||
statuses = client.upsert(updateRecordsRDD, newCommitTime).collect();
|
||||
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify there are now 4 commits
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals("Expecting four commits.", 4, timeline.findInstantsAfter("000", Integer.MAX_VALUE)
|
||||
.countInstants());
|
||||
Assert.assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime);
|
||||
|
||||
// Check the entire dataset has 47 records still
|
||||
dataSet = getRecords();
|
||||
assertEquals("Must contain 47 records", 47, dataSet.count());
|
||||
Row[] rows = (Row[]) dataSet.collect();
|
||||
int record1Count = 0;
|
||||
int record2Count = 0;
|
||||
for (Row row : rows) {
|
||||
if (row.getAs("_hoodie_record_key").equals(record1.getKey().getRecordKey())) {
|
||||
record1Count++;
|
||||
// assert each duplicate record is updated
|
||||
assertEquals(row.getAs("rider"), "rider-004");
|
||||
assertEquals(row.getAs("driver"), "driver-004");
|
||||
} else if (row.getAs("_hoodie_record_key").equals(record2.getKey().getRecordKey())) {
|
||||
record2Count++;
|
||||
// assert each duplicate record is updated
|
||||
assertEquals(row.getAs("rider"), "rider-004");
|
||||
assertEquals(row.getAs("driver"), "driver-004");
|
||||
} else {
|
||||
assertNotEquals(row.getAs("rider"), "rider-004");
|
||||
assertNotEquals(row.getAs("driver"), "rider-004");
|
||||
}
|
||||
}
|
||||
// Assert that id1 record count which has been updated to rider-004 and driver-004 is 22, which is the total
|
||||
// number of records with row_key id1
|
||||
assertEquals(22, record1Count);
|
||||
|
||||
// Assert that id2 record count which has been updated to rider-004 and driver-004 is 21, which is the total
|
||||
// number of records with row_key id2
|
||||
assertEquals(21, record2Count);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHoodieMergeHandleWriteStatMetrics() throws Exception {
|
||||
// insert 100 records
|
||||
HoodieWriteConfig config = getConfigBuilder().build();
|
||||
HoodieWriteClient writeClient = getWriteClient(config);
|
||||
String newCommitTime = "100";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
try (HoodieWriteClient writeClient = getWriteClient(config);) {
|
||||
String newCommitTime = "100";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
// All records should be inserts into new parquet
|
||||
Assert.assertTrue(statuses.stream()
|
||||
.filter(status -> status.getStat().getPrevCommit() != HoodieWriteStat.NULL_COMMIT).count() > 0);
|
||||
// Num writes should be equal to the number of records inserted
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumWrites()).reduce((a,b) -> a + b).get(), 100);
|
||||
// Num update writes should be equal to the number of records updated
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumUpdateWrites()).reduce((a,b) -> a + b).get(), 0);
|
||||
// Num update writes should be equal to the number of insert records converted to updates as part of small file
|
||||
// handling
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumInserts()).reduce((a,b) -> a + b).get(), 100);
|
||||
// All records should be inserts into new parquet
|
||||
Assert.assertTrue(statuses.stream()
|
||||
.filter(status -> status.getStat().getPrevCommit() != HoodieWriteStat.NULL_COMMIT).count() > 0);
|
||||
// Num writes should be equal to the number of records inserted
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumWrites()).reduce((a, b) -> a + b).get(), 100);
|
||||
// Num update writes should be equal to the number of records updated
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumUpdateWrites()).reduce((a, b) -> a + b).get(), 0);
|
||||
// Num update writes should be equal to the number of insert records converted to updates as part of small file
|
||||
// handling
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumInserts()).reduce((a, b) -> a + b).get(), 100);
|
||||
|
||||
// Update all the 100 records
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
// Update all the 100 records
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
newCommitTime = "101";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
newCommitTime = "101";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
statuses = writeClient.upsert(updatedRecordsRDD, newCommitTime).collect();
|
||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
statuses = writeClient.upsert(updatedRecordsRDD, newCommitTime).collect();
|
||||
|
||||
// All records should be upserts into existing parquet
|
||||
Assert.assertEquals(statuses.stream()
|
||||
.filter(status -> status.getStat().getPrevCommit() == HoodieWriteStat.NULL_COMMIT).count(), 0);
|
||||
// Num writes should be equal to the number of records inserted
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumWrites()).reduce((a,b) -> a + b).get(), 100);
|
||||
// Num update writes should be equal to the number of records updated
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumUpdateWrites()).reduce((a,b) -> a + b).get(), 100);
|
||||
// Num update writes should be equal to the number of insert records converted to updates as part of small file
|
||||
// handling
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumInserts()).reduce((a,b) -> a + b).get(), 0);
|
||||
// All records should be upserts into existing parquet
|
||||
Assert.assertEquals(statuses.stream()
|
||||
.filter(status -> status.getStat().getPrevCommit() == HoodieWriteStat.NULL_COMMIT).count(), 0);
|
||||
// Num writes should be equal to the number of records inserted
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumWrites()).reduce((a, b) -> a + b).get(), 100);
|
||||
// Num update writes should be equal to the number of records updated
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumUpdateWrites()).reduce((a, b) -> a + b).get(), 100);
|
||||
// Num update writes should be equal to the number of insert records converted to updates as part of small file
|
||||
// handling
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumInserts()).reduce((a, b) -> a + b).get(), 0);
|
||||
|
||||
newCommitTime = "102";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
newCommitTime = "102";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
List<HoodieRecord> allRecords = dataGen.generateInserts(newCommitTime, 100);
|
||||
allRecords.addAll(updatedRecords);
|
||||
JavaRDD<HoodieRecord> allRecordsRDD = jsc.parallelize(allRecords, 1);
|
||||
statuses = writeClient.upsert(allRecordsRDD, newCommitTime).collect();
|
||||
|
||||
List<HoodieRecord> allRecords = dataGen.generateInserts(newCommitTime, 100);
|
||||
allRecords.addAll(updatedRecords);
|
||||
JavaRDD<HoodieRecord> allRecordsRDD = jsc.parallelize(allRecords, 1);
|
||||
statuses = writeClient.upsert(allRecordsRDD, newCommitTime).collect();
|
||||
|
||||
// All records should be upserts into existing parquet (with inserts as updates small file handled)
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.filter(status -> status.getStat().getPrevCommit() == HoodieWriteStat.NULL_COMMIT).count(), 0);
|
||||
// Num writes should be equal to the total number of records written
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumWrites()).reduce((a,b) -> a + b).get(), 200);
|
||||
// Num update writes should be equal to the number of records updated (including inserts converted as updates)
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumUpdateWrites()).reduce((a,b) -> a + b).get(), 100);
|
||||
// Num update writes should be equal to the number of insert records converted to updates as part of small file
|
||||
// handling
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumInserts()).reduce((a,b) -> a + b).get(), 100);
|
||||
// Verify all records have location set
|
||||
statuses.forEach(writeStatus -> {
|
||||
writeStatus.getWrittenRecords().forEach(r -> {
|
||||
// Ensure New Location is set
|
||||
Assert.assertTrue(r.getNewLocation().isPresent());
|
||||
// All records should be upserts into existing parquet (with inserts as updates small file handled)
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.filter(status -> status.getStat().getPrevCommit() == HoodieWriteStat.NULL_COMMIT).count(), 0);
|
||||
// Num writes should be equal to the total number of records written
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumWrites()).reduce((a, b) -> a + b).get(), 200);
|
||||
// Num update writes should be equal to the number of records updated (including inserts converted as updates)
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumUpdateWrites()).reduce((a, b) -> a + b).get(), 100);
|
||||
// Num update writes should be equal to the number of insert records converted to updates as part of small file
|
||||
// handling
|
||||
Assert.assertEquals((long) statuses.stream()
|
||||
.map(status -> status.getStat().getNumInserts()).reduce((a, b) -> a + b).get(), 100);
|
||||
// Verify all records have location set
|
||||
statuses.forEach(writeStatus -> {
|
||||
writeStatus.getWrittenRecords().forEach(r -> {
|
||||
// Ensure New Location is set
|
||||
Assert.assertTrue(r.getNewLocation().isPresent());
|
||||
});
|
||||
});
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
private Dataset<Row> getRecords() {
|
||||
|
||||
@@ -31,6 +31,7 @@ import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
@@ -58,33 +59,30 @@ import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class TestCopyOnWriteTable {
|
||||
public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
|
||||
protected static Logger log = LogManager.getLogger(TestCopyOnWriteTable.class);
|
||||
|
||||
private String basePath = null;
|
||||
private transient JavaSparkContext jsc = null;
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestCopyOnWriteTable"));
|
||||
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
this.basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts("TestCopyOnWriteTable");
|
||||
initTempFolderAndPath();
|
||||
initTableType();
|
||||
initTestDataGenerator();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
cleanupSparkContexts();
|
||||
cleanupTempFolderAndPath();
|
||||
cleanupTableType();
|
||||
cleanupTestDataGenerator();
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -208,7 +206,7 @@ public class TestCopyOnWriteTable {
|
||||
List<WriteStatus> statuses =
|
||||
jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return newTable.handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(),
|
||||
updatedRecords.iterator());
|
||||
updatedRecords.iterator());
|
||||
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
|
||||
|
||||
// Check the updated file
|
||||
@@ -439,7 +437,7 @@ public class TestCopyOnWriteTable {
|
||||
final String testPartitionPath = "2016/09/26";
|
||||
// Inserts + Updates .. Check updates go together & inserts subsplit, after expanding
|
||||
// smallest file
|
||||
UpsertPartitioner partitioner = getUpsertPartitioner(1000 * 1024, 400, 100, 800 * 1024,testPartitionPath,
|
||||
UpsertPartitioner partitioner = getUpsertPartitioner(1000 * 1024, 400, 100, 800 * 1024, testPartitionPath,
|
||||
false);
|
||||
List<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
|
||||
|
||||
@@ -479,12 +477,11 @@ public class TestCopyOnWriteTable {
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
final HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
String commitTime = "000";
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
|
||||
// Perform inserts of 100 records to test CreateHandle and BufferedExecutor
|
||||
final List<HoodieRecord> inserts = dataGenerator.generateInsertsWithHoodieAvroPayload(commitTime, 100);
|
||||
final List<HoodieRecord> inserts = dataGen.generateInsertsWithHoodieAvroPayload(commitTime, 100);
|
||||
final List<List<WriteStatus>> ws = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return table.handleInsert(commitTime, UUID.randomUUID().toString(), inserts.iterator());
|
||||
}).map(x -> (List<WriteStatus>)HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
|
||||
WriteStatus writeStatus = ws.get(0).get(0);
|
||||
String fileId = writeStatus.getFileId();
|
||||
@@ -492,11 +489,11 @@ public class TestCopyOnWriteTable {
|
||||
final HoodieCopyOnWriteTable table2 = new HoodieCopyOnWriteTable(config, jsc);
|
||||
|
||||
final List<HoodieRecord> updates =
|
||||
dataGenerator.generateUpdatesWithHoodieAvroPayload(commitTime, writeStatus.getWrittenRecords());
|
||||
dataGen.generateUpdatesWithHoodieAvroPayload(commitTime, writeStatus.getWrittenRecords());
|
||||
|
||||
jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return table2.handleUpdate("001", fileId, updates.iterator());
|
||||
}).map(x -> (List<WriteStatus>)HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
}
|
||||
|
||||
@After
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
Reference in New Issue
Block a user