1
0

[HUDI-1089] Refactor hudi-client to support multi-engine (#1827)

- This change breaks `hudi-client` into `hudi-client-common` and `hudi-spark-client` modules 
- Simple usages of Spark using jsc.parallelize() has been redone using EngineContext#map, EngineContext#flatMap etc
- Code changes in the PR, break classes into `BaseXYZ` parent classes with no spark dependencies living in `hudi-client-common`
- Classes on `hudi-spark-client` are named `SparkXYZ` extending the parent classes with all the Spark dependencies
- To simplify/cleanup, HoodieIndex#fetchRecordLocation has been removed and its usages in tests replaced with alternatives

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Mathieu
2020-10-02 05:25:29 +08:00
committed by GitHub
parent 5aaaf8bff1
commit 1f7add9291
380 changed files with 6071 additions and 4128 deletions

View File

@@ -23,7 +23,8 @@ import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -162,10 +163,10 @@ public class SavepointsCommand implements CommandMarker {
return String.format("Savepoint \"%s\" deleted.", instantTime);
}
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
return new HoodieWriteClient(jsc, config, false);
return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config, false);
}
}

View File

@@ -22,12 +22,13 @@ import org.apache.hudi.cli.DeDupeType;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.cli.DedupeSparkJob;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.client.utils.ClientUtils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTableVersion;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.config.HoodieBootstrapConfig;
@@ -35,8 +36,8 @@ import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieSavepointException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.upgrade.UpgradeDowngrade;
import org.apache.hudi.table.action.compact.strategy.UnBoundedCompactionStrategy;
import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade;
import org.apache.hudi.utilities.HDFSParquetImporter;
import org.apache.hudi.utilities.HDFSParquetImporter.Config;
import org.apache.hudi.utilities.HoodieCleaner;
@@ -343,7 +344,7 @@ public class SparkMain {
}
private static int rollback(JavaSparkContext jsc, String instantTime, String basePath) throws Exception {
HoodieWriteClient client = createHoodieClient(jsc, basePath);
SparkRDDWriteClient client = createHoodieClient(jsc, basePath);
if (client.rollback(instantTime)) {
LOG.info(String.format("The commit \"%s\" rolled back.", instantTime));
return 0;
@@ -355,7 +356,7 @@ public class SparkMain {
private static int createSavepoint(JavaSparkContext jsc, String commitTime, String user,
String comments, String basePath) throws Exception {
HoodieWriteClient client = createHoodieClient(jsc, basePath);
SparkRDDWriteClient client = createHoodieClient(jsc, basePath);
try {
client.savepoint(commitTime, user, comments);
LOG.info(String.format("The commit \"%s\" has been savepointed.", commitTime));
@@ -367,7 +368,7 @@ public class SparkMain {
}
private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime, String basePath) throws Exception {
HoodieWriteClient client = createHoodieClient(jsc, basePath);
SparkRDDWriteClient client = createHoodieClient(jsc, basePath);
try {
client.restoreToSavepoint(savepointTime);
LOG.info(String.format("The commit \"%s\" rolled back.", savepointTime));
@@ -379,7 +380,7 @@ public class SparkMain {
}
private static int deleteSavepoint(JavaSparkContext jsc, String savepointTime, String basePath) throws Exception {
HoodieWriteClient client = createHoodieClient(jsc, basePath);
SparkRDDWriteClient client = createHoodieClient(jsc, basePath);
try {
client.deleteSavepoint(savepointTime);
LOG.info(String.format("Savepoint \"%s\" deleted.", savepointTime));
@@ -401,9 +402,10 @@ public class SparkMain {
*/
protected static int upgradeOrDowngradeTable(JavaSparkContext jsc, String basePath, String toVersion) {
HoodieWriteConfig config = getWriteConfig(basePath);
HoodieTableMetaClient metaClient = ClientUtils.createMetaClient(jsc.hadoopConfiguration(), config, false);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), false,
config.getConsistencyGuardConfig(), Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())));
try {
UpgradeDowngrade.run(metaClient, HoodieTableVersion.valueOf(toVersion), config, jsc, null);
new SparkUpgradeDowngrade(metaClient, config, new HoodieSparkEngineContext(jsc)).run(metaClient, HoodieTableVersion.valueOf(toVersion), config, new HoodieSparkEngineContext(jsc), null);
LOG.info(String.format("Table at \"%s\" upgraded / downgraded to version \"%s\".", basePath, toVersion));
return 0;
} catch (Exception e) {
@@ -412,9 +414,9 @@ public class SparkMain {
}
}
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
HoodieWriteConfig config = getWriteConfig(basePath);
return new HoodieWriteClient(jsc, config);
return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config);
}
private static HoodieWriteConfig getWriteConfig(String basePath) {

View File

@@ -21,7 +21,7 @@ package org.apache.hudi.cli.utils;
import org.apache.hudi.cli.HoodieCliSparkConfig;
import org.apache.hudi.cli.commands.SparkEnvCommand;
import org.apache.hudi.cli.commands.SparkMain;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
@@ -91,7 +91,7 @@ public class SparkUtil {
sparkConf.set(HoodieCliSparkConfig.CLI_MAPRED_OUTPUT_COMPRESSION_CODEC, "org.apache.hadoop.io.compress.GzipCodec");
sparkConf.set(HoodieCliSparkConfig.CLI_MAPRED_OUTPUT_COMPRESSION_TYPE, "BLOCK");
HoodieWriteClient.registerClasses(sparkConf);
SparkRDDWriteClient.registerClasses(sparkConf);
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
jsc.hadoopConfiguration().setBoolean(HoodieCliSparkConfig.CLI_PARQUET_ENABLE_SUMMARY_METADATA, false);
FSUtils.prepareHadoopConf(jsc.hadoopConfiguration());

View File

@@ -30,6 +30,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTimelineArchiveLog;
import org.junit.jupiter.api.AfterEach;
@@ -92,8 +93,9 @@ public class TestArchivedCommitsCommand extends AbstractShellIntegrationTest {
metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
// archive
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf);
archiveLog.archiveIfRequired(jsc);
HoodieSparkTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
archiveLog.archiveIfRequired(context);
}
@AfterEach

View File

@@ -35,6 +35,7 @@ import org.apache.hudi.common.util.NumericUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTimelineArchiveLog;
import org.apache.hadoop.fs.FileSystem;
@@ -179,8 +180,9 @@ public class TestCommitsCommand extends AbstractShellIntegrationTest {
// archive
metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient());
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, jsc.hadoopConfiguration());
archiveLog.archiveIfRequired(jsc);
HoodieSparkTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
archiveLog.archiveIfRequired(context);
CommandResult cr = getShell().executeCommand(String.format("commits showarchived --startTs %s --endTs %s", "100", "104"));
assertTrue(cr.isSuccess());

View File

@@ -24,7 +24,7 @@ import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.cli.testutils.AbstractShellIntegrationTest;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.AbstractHoodieWriteClient;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
@@ -90,7 +90,7 @@ public class TestRollbacksCommand extends AbstractShellIntegrationTest {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(tablePath)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
try (HoodieWriteClient client = getHoodieWriteClient(config)) {
try (AbstractHoodieWriteClient client = getHoodieWriteClient(config)) {
// Rollback inflight commit3 and commit2
client.rollback("102");
client.rollback("101");