diff --git a/.travis.yml b/.travis.yml index 67fa0071f..d36c0cb70 100644 --- a/.travis.yml +++ b/.travis.yml @@ -20,10 +20,10 @@ jdk: - openjdk8 jobs: include: - - name: "Unit tests except hudi-client" - env: MODE=unit MODULES='!hudi-client' HUDI_QUIETER_LOGGING=1 - - name: "Unit tests for hudi-client" - env: MODE=unit MODULES=hudi-client HUDI_QUIETER_LOGGING=1 + - name: "Unit tests except hudi-spark-client" + env: MODE=unit MODULES='!hudi-client/hudi-spark-client' HUDI_QUIETER_LOGGING=1 + - name: "Unit tests for hudi-spark-client" + env: MODE=unit MODULES=hudi-client/hudi-spark-client HUDI_QUIETER_LOGGING=1 - name: "Functional tests" env: MODE=functional HUDI_QUIETER_LOGGING=1 - name: "Integration tests" diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml index a1b398ecc..dda07f9a1 100644 --- a/hudi-cli/pom.xml +++ b/hudi-cli/pom.xml @@ -148,7 +148,14 @@ org.apache.hudi - hudi-client + hudi-client-common + ${project.version} + test + test-jar + + + org.apache.hudi + hudi-spark-client ${project.version} test test-jar diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java index 6d8155295..e4d7cc69e 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java @@ -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); } } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index ce567b98f..f715b16e0 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -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) { diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java index ba9d1e534..430aab969 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java @@ -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()); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java index f69e3d3ec..5c27636da 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java @@ -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 diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java index c2509b12d..84b357622 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java @@ -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()); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java index 10197a325..8cf2be958 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java @@ -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"); diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml new file mode 100644 index 000000000..902de5836 --- /dev/null +++ b/hudi-client/hudi-client-common/pom.xml @@ -0,0 +1,264 @@ + + + + + hudi-client + org.apache.hudi + 0.6.1-SNAPSHOT + + 4.0.0 + + hudi-client-common + ${parent.version} + + hudi-client-common + jar + + + + + org.scala-lang + scala-library + ${scala.version} + + + + + org.apache.hudi + hudi-common + ${project.version} + + + org.apache.hudi + hudi-timeline-service + ${project.version} + + + + + log4j + log4j + + + + + org.apache.parquet + parquet-avro + + + + + io.dropwizard.metrics + metrics-graphite + + + com.rabbitmq + * + + + + + io.dropwizard.metrics + metrics-core + + + io.dropwizard.metrics + metrics-jmx + + + io.prometheus + simpleclient + + + io.prometheus + simpleclient_httpserver + + + io.prometheus + simpleclient_dropwizard + + + io.prometheus + simpleclient_pushgateway + + + + + org.apache.hudi + hudi-common + ${project.version} + tests + test-jar + test + + + + + org.apache.hadoop + hadoop-hdfs + tests + + + + org.mortbay.jetty + * + + + javax.servlet.jsp + * + + + javax.servlet + * + + + + + org.apache.hadoop + hadoop-common + tests + + + org.mortbay.jetty + * + + + javax.servlet.jsp + * + + + javax.servlet + * + + + + + + + org.junit.jupiter + junit-jupiter-api + test + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.vintage + junit-vintage-engine + test + + + org.junit.jupiter + junit-jupiter-params + test + + + org.mockito + mockito-junit-jupiter + test + + + org.junit.platform + junit-platform-runner + test + + + org.junit.platform + junit-platform-suite-api + test + + + org.junit.platform + junit-platform-commons + test + + + + + + + org.jacoco + jacoco-maven-plugin + + + net.alchim31.maven + scala-maven-plugin + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + compile + + compile + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + test-compile + + + + false + + + + org.apache.rat + apache-rat-plugin + + + + + + src/main/resources + + + src/test/resources + + + + diff --git a/hudi-client/src/main/java/org/apache/hudi/async/AsyncCompactService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java similarity index 83% rename from hudi-client/src/main/java/org/apache/hudi/async/AsyncCompactService.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java index 391344f16..d7faf5411 100644 --- a/hudi-client/src/main/java/org/apache/hudi/async/AsyncCompactService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java @@ -17,14 +17,15 @@ package org.apache.hudi.async; -import org.apache.hudi.client.Compactor; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.AbstractCompactor; +import org.apache.hudi.client.AbstractHoodieWriteClient; +import org.apache.hudi.client.common.EngineProperty; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; import java.util.concurrent.BlockingQueue; @@ -40,7 +41,7 @@ import java.util.stream.IntStream; /** * Async Compactor Service that runs in separate thread. Currently, only one compactor is allowed to run at any time. */ -public class AsyncCompactService extends AbstractAsyncService { +public abstract class AsyncCompactService extends HoodieAsyncService { private static final long serialVersionUID = 1L; private static final Logger LOG = LogManager.getLogger(AsyncCompactService.class); @@ -51,23 +52,25 @@ public class AsyncCompactService extends AbstractAsyncService { public static final String COMPACT_POOL_NAME = "hoodiecompact"; private final int maxConcurrentCompaction; - private transient Compactor compactor; - private transient JavaSparkContext jssc; + private transient AbstractCompactor compactor; + private transient HoodieEngineContext context; private transient BlockingQueue pendingCompactions = new LinkedBlockingQueue<>(); private transient ReentrantLock queueLock = new ReentrantLock(); private transient Condition consumed = queueLock.newCondition(); - public AsyncCompactService(JavaSparkContext jssc, HoodieWriteClient client) { - this(jssc, client, false); + public AsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client) { + this(context, client, false); } - public AsyncCompactService(JavaSparkContext jssc, HoodieWriteClient client, boolean runInDaemonMode) { + public AsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client, boolean runInDaemonMode) { super(runInDaemonMode); - this.jssc = jssc; - this.compactor = new Compactor(client); + this.context = context; + this.compactor = createCompactor(client); this.maxConcurrentCompaction = 1; } + protected abstract AbstractCompactor createCompactor(AbstractHoodieWriteClient client); + /** * Enqueues new Pending compaction. */ @@ -127,8 +130,8 @@ public class AsyncCompactService extends AbstractAsyncService { return Pair.of(CompletableFuture.allOf(IntStream.range(0, maxConcurrentCompaction).mapToObj(i -> CompletableFuture.supplyAsync(() -> { try { // Set Compactor Pool Name for allowing users to prioritize compaction - LOG.info("Setting Spark Pool name for compaction to " + COMPACT_POOL_NAME); - jssc.setLocalProperty("spark.scheduler.pool", COMPACT_POOL_NAME); + LOG.info("Setting pool name for compaction to " + COMPACT_POOL_NAME); + context.setProperty(EngineProperty.COMPACTION_POOL_NAME, COMPACT_POOL_NAME); while (!isShutdownRequested()) { final HoodieInstant instant = fetchNextCompactionInstant(); diff --git a/hudi-client/src/main/java/org/apache/hudi/async/AbstractAsyncService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java similarity index 95% rename from hudi-client/src/main/java/org/apache/hudi/async/AbstractAsyncService.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java index 714fa6019..32dd04272 100644 --- a/hudi-client/src/main/java/org/apache/hudi/async/AbstractAsyncService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java @@ -34,9 +34,9 @@ import java.util.function.Function; /** * Base Class for running clean/delta-sync/compaction in separate thread and controlling their life-cycle. */ -public abstract class AbstractAsyncService implements Serializable { +public abstract class HoodieAsyncService implements Serializable { - private static final Logger LOG = LogManager.getLogger(AbstractAsyncService.class); + private static final Logger LOG = LogManager.getLogger(HoodieAsyncService.class); // Flag to track if the service is started. private boolean started; @@ -51,11 +51,11 @@ public abstract class AbstractAsyncService implements Serializable { // Run in daemon mode private final boolean runInDaemonMode; - protected AbstractAsyncService() { + protected HoodieAsyncService() { this(false); } - protected AbstractAsyncService(boolean runInDaemonMode) { + protected HoodieAsyncService(boolean runInDaemonMode) { shutdownRequested = false; this.runInDaemonMode = runInDaemonMode; } diff --git a/hudi-client/src/main/java/org/apache/hudi/callback/HoodieWriteCommitCallback.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/HoodieWriteCommitCallback.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/callback/HoodieWriteCommitCallback.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/HoodieWriteCommitCallback.java diff --git a/hudi-client/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java diff --git a/hudi-client/src/main/java/org/apache/hudi/callback/common/HoodieWriteCommitCallbackMessage.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/common/HoodieWriteCommitCallbackMessage.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/callback/common/HoodieWriteCommitCallbackMessage.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/common/HoodieWriteCommitCallbackMessage.java diff --git a/hudi-client/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java similarity index 91% rename from hudi-client/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java index b87ab23c9..bb60879ef 100644 --- a/hudi-client/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java @@ -20,13 +20,12 @@ package org.apache.hudi.callback.impl; import org.apache.hudi.callback.HoodieWriteCommitCallback; import org.apache.hudi.callback.client.http.HoodieWriteCommitHttpCallbackClient; import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage; +import org.apache.hudi.callback.util.HoodieWriteCommitCallbackUtil; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import static org.apache.hudi.callback.util.HoodieWriteCommitCallbackUtil.convertToJsonString; - /** * A http implementation of {@link HoodieWriteCommitCallback}. */ @@ -43,7 +42,7 @@ public class HoodieWriteCommitHttpCallback implements HoodieWriteCommitCallback @Override public void call(HoodieWriteCommitCallbackMessage callbackMessage) { // convert to json - String callbackMsg = convertToJsonString(callbackMessage); + String callbackMsg = HoodieWriteCommitCallbackUtil.convertToJsonString(callbackMessage); LOG.info("Try to send callbackMsg, msg = " + callbackMsg); client.send(callbackMsg); } diff --git a/hudi-client/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java similarity index 93% rename from hudi-client/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java index 9d1e9c354..74eb8b6da 100644 --- a/hudi-client/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java @@ -20,11 +20,10 @@ package org.apache.hudi.callback.util; import org.apache.hudi.callback.HoodieWriteCommitCallback; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.config.HoodieWriteCommitCallbackConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitCallbackException; -import static org.apache.hudi.config.HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_PROP; - /** * Factory help to create {@link HoodieWriteCommitCallback}. */ @@ -40,7 +39,7 @@ public class HoodieCommitCallbackFactory { return (HoodieWriteCommitCallback) instance; } else { throw new HoodieCommitCallbackException(String.format("The value of the config option %s can not be null or " - + "empty", CALLBACK_CLASS_PROP)); + + "empty", HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_PROP)); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/callback/util/HoodieWriteCommitCallbackUtil.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieWriteCommitCallbackUtil.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/callback/util/HoodieWriteCommitCallbackUtil.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieWriteCommitCallbackUtil.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractCompactor.java new file mode 100644 index 000000000..30bc569d7 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractCompactor.java @@ -0,0 +1,41 @@ +/* + * 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.client; + +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; + +import java.io.IOException; +import java.io.Serializable; + +/** + * Run one round of compaction. + */ +public abstract class AbstractCompactor implements Serializable { + + private static final long serialVersionUID = 1L; + + protected transient AbstractHoodieWriteClient compactionClient; + + public AbstractCompactor(AbstractHoodieWriteClient compactionClient) { + this.compactionClient = compactionClient; + } + + public abstract void compact(HoodieInstant instant) throws IOException; +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java similarity index 81% rename from hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java index cdd125e8e..9628e4197 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java @@ -19,17 +19,19 @@ package org.apache.hudi.client; import org.apache.hadoop.conf.Configuration; + +import org.apache.hudi.client.common.EngineProperty; import org.apache.hudi.client.embedded.EmbeddedTimelineService; -import org.apache.hudi.client.utils.ClientUtils; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; import java.io.Serializable; @@ -43,7 +45,7 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl private static final Logger LOG = LogManager.getLogger(AbstractHoodieClient.class); protected final transient FileSystem fs; - protected final transient JavaSparkContext jsc; + protected final transient HoodieEngineContext context; protected final transient Configuration hadoopConf; protected final HoodieWriteConfig config; protected final String basePath; @@ -56,15 +58,15 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl private transient Option timelineServer; private final boolean shouldStopTimelineServer; - protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) { - this(jsc, clientConfig, Option.empty()); + protected AbstractHoodieClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) { + this(context, clientConfig, Option.empty()); } - protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, + protected AbstractHoodieClient(HoodieEngineContext context, HoodieWriteConfig clientConfig, Option timelineServer) { - this.hadoopConf = jsc.hadoopConfiguration(); + this.hadoopConf = context.getHadoopConf().get(); this.fs = FSUtils.getFs(clientConfig.getBasePath(), hadoopConf); - this.jsc = jsc; + this.context = context; this.basePath = clientConfig.getBasePath(); this.config = clientConfig; this.timelineServer = timelineServer; @@ -99,7 +101,8 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl if (!timelineServer.isPresent()) { // Run Embedded Timeline Server LOG.info("Starting Timeline service !!"); - timelineServer = Option.of(new EmbeddedTimelineService(hadoopConf, jsc.getConf(), + Option hostAddr = context.getProperty(EngineProperty.EMBEDDED_SERVER_HOST); + timelineServer = Option.of(new EmbeddedTimelineService(context, hostAddr.orElse(null), config.getClientSpecifiedViewStorageConfig())); try { timelineServer.get().startServer(); @@ -122,6 +125,8 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl } protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoad) { - return ClientUtils.createMetaClient(hadoopConf, config, loadActiveTimelineOnLoad); + return new HoodieTableMetaClient(hadoopConf, config.getBasePath(), loadActiveTimelineOnLoad, + config.getConsistencyGuardConfig(), + Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java similarity index 61% rename from hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index 7970623ee..0f35e270e 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -18,22 +18,27 @@ package org.apache.hudi.client; +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.callback.HoodieWriteCommitCallback; +import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage; +import org.apache.hudi.callback.util.HoodieCommitCallbackFactory; import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieCompactionConfig; @@ -45,109 +50,178 @@ import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.exception.HoodieSavepointException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.metrics.HoodieMetrics; +import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.HoodieTimelineArchiveLog; import org.apache.hudi.table.MarkerFiles; -import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.compact.CompactHelpers; import org.apache.hudi.table.action.savepoint.SavepointHelpers; - -import com.codahale.metrics.Timer; 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 java.io.IOException; +import java.nio.charset.StandardCharsets; import java.text.ParseException; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; /** - * Hoodie Write Client helps you build tables on HDFS [insert()] and then perform efficient mutations on an HDFS - * table [upsert()] - *

- * Note that, at any given time, there can only be one Spark job performing these operations on a Hoodie table. + * Abstract Write Client providing functionality for performing commit, index updates and rollback + * Reused for regular write operations like upsert/insert/bulk-insert.. as well as bootstrap + * + * @param Sub type of HoodieRecordPayload + * @param Type of inputs + * @param Type of keys + * @param Type of outputs */ -public class HoodieWriteClient extends AbstractHoodieWriteClient { +public abstract class AbstractHoodieWriteClient extends AbstractHoodieClient { + protected static final String LOOKUP_STR = "lookup"; private static final long serialVersionUID = 1L; - private static final Logger LOG = LogManager.getLogger(HoodieWriteClient.class); - private static final String LOOKUP_STR = "lookup"; - private final boolean rollbackPending; - private final transient HoodieMetrics metrics; - private transient Timer.Context compactionTimer; - private transient AsyncCleanerService asyncCleanerService; + private static final Logger LOG = LogManager.getLogger(AbstractHoodieWriteClient.class); + + protected final transient HoodieMetrics metrics; + private final transient HoodieIndex index; + + protected transient Timer.Context writeTimer = null; + protected transient Timer.Context compactionTimer; + + private transient WriteOperationType operationType; + private transient HoodieWriteCommitCallback commitCallback; + protected final boolean rollbackPending; + protected transient AsyncCleanerService asyncCleanerService; /** * Create a write client, without cleaning up failed/inflight commits. * - * @param jsc Java Spark Context + * @param context HoodieEngineContext * @param clientConfig instance of HoodieWriteConfig */ - public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) { - this(jsc, clientConfig, false); + public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) { + this(context, clientConfig, false); } /** * Create a write client, with new hudi index. * - * @param jsc Java Spark Context + * @param context HoodieEngineContext * @param writeConfig instance of HoodieWriteConfig * @param rollbackPending whether need to cleanup pending commits */ - public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig writeConfig, boolean rollbackPending) { - this(jsc, writeConfig, rollbackPending, HoodieIndex.createIndex(writeConfig)); - } - - public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig writeConfig, boolean rollbackPending, HoodieIndex index) { - this(jsc, writeConfig, rollbackPending, index, Option.empty()); + public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) { + this(context, writeConfig, rollbackPending, Option.empty()); } /** - * Create a write client, allows to specify all parameters. + * Create a write client, allows to specify all parameters. * - * @param jsc Java Spark Context + * @param context HoodieEngineContext * @param writeConfig instance of HoodieWriteConfig * @param rollbackPending whether need to cleanup pending commits * @param timelineService Timeline Service that runs as part of write client. */ - public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig writeConfig, boolean rollbackPending, - HoodieIndex index, Option timelineService) { - super(jsc, index, writeConfig, timelineService); + public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending, + Option timelineService) { + super(context, writeConfig, timelineService); this.metrics = new HoodieMetrics(config, config.getTableName()); this.rollbackPending = rollbackPending; + this.index = createIndex(writeConfig); + } + + protected abstract HoodieIndex createIndex(HoodieWriteConfig writeConfig); + + public void setOperationType(WriteOperationType operationType) { + this.operationType = operationType; + } + + public WriteOperationType getOperationType() { + return this.operationType; } /** - * Register hudi classes for Kryo serialization. - * - * @param conf instance of SparkConf - * @return SparkConf + * Commit changes performed at the given instantTime marker. */ - public static SparkConf registerClasses(SparkConf conf) { - conf.registerKryoClasses(new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); - return conf; + public boolean commit(String instantTime, O writeStatuses) { + return commit(instantTime, writeStatuses, Option.empty()); + } + + /** + * + * Commit changes performed at the given instantTime marker. + */ + public boolean commit(String instantTime, O writeStatuses, Option> extraMetadata) { + HoodieTableMetaClient metaClient = createMetaClient(false); + String actionType = metaClient.getCommitActionType(); + return commit(instantTime, writeStatuses, extraMetadata, actionType, Collections.emptyMap()); + } + + public abstract boolean commit(String instantTime, O writeStatuses, Option> extraMetadata, + String commitActionType, Map> partitionToReplacedFileIds); + + public boolean commitStats(String instantTime, List stats, Option> extraMetadata, + String commitActionType) { + return commitStats(instantTime, stats, extraMetadata, commitActionType, Collections.emptyMap()); + } + + public boolean commitStats(String instantTime, List stats, Option> extraMetadata, + String commitActionType, Map> partitionToReplaceFileIds) { + LOG.info("Committing " + instantTime + " action " + commitActionType); + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = createTable(config, hadoopConf); + + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getSchema(), commitActionType); + // Finalize write + finalizeWrite(table, instantTime, stats); + + try { + activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime), + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + postCommit(table, metadata, instantTime, extraMetadata); + emitCommitMetrics(instantTime, metadata, commitActionType); + LOG.info("Committed " + instantTime); + } catch (IOException e) { + throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, + e); + } + + // callback if needed. + if (config.writeCommitCallbackOn()) { + if (null == commitCallback) { + commitCallback = HoodieCommitCallbackFactory.create(config); + } + commitCallback.call(new HoodieWriteCommitCallbackMessage(instantTime, config.getTableName(), config.getBasePath())); + } + return true; + } + + protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf); + + void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) { + try { + + if (writeTimer != null) { + long durationInMs = metrics.getDurationInMs(writeTimer.stop()); + metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(instantTime).getTime(), durationInMs, + metadata, actionType); + writeTimer = null; + } + } catch (ParseException e) { + throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime + + "Instant time is not of valid format", e); + } } /** * Filter out HoodieRecords that already exists in the output folder. This is useful in deduplication. * - * @param hoodieRecords Input RDD of Hoodie records. - * @return A subset of hoodieRecords RDD, with existing records filtered out. + * @param hoodieRecords Input Hoodie records. + * @return A subset of hoodieRecords, with existing records filtered out. */ - public JavaRDD> filterExists(JavaRDD> hoodieRecords) { - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.create(config, hadoopConf); - Timer.Context indexTimer = metrics.getIndexCtx(); - JavaRDD> recordsWithLocation = getIndex().tagLocation(hoodieRecords, jsc, table); - metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop())); - return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); - } + public abstract I filterExists(I hoodieRecords); /** * Main API to run bootstrap to hudi. @@ -156,8 +230,8 @@ public class HoodieWriteClient extends AbstractHo if (rollbackPending) { rollBackInflightBootstrap(); } - HoodieTable table = getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS); - table.bootstrap(jsc, extraMetadata); + HoodieTable table = getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS); + table.bootstrap(context, extraMetadata); } /** @@ -165,14 +239,14 @@ public class HoodieWriteClient extends AbstractHo */ protected void rollBackInflightBootstrap() { LOG.info("Rolling back pending bootstrap if present"); - HoodieTable table = HoodieTable.create(config, hadoopConf); + HoodieTable table = createTable(config, hadoopConf); HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); Option instant = Option.fromJavaOptional( inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp).findFirst()); if (instant.isPresent() && HoodieTimeline.compareTimestamps(instant.get(), HoodieTimeline.LESSER_THAN_OR_EQUALS, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) { LOG.info("Found pending bootstrap instants. Rolling them back"); - table.rollbackBootstrap(jsc, HoodieActiveTimeline.createNewInstantTime()); + table.rollbackBootstrap(context, HoodieActiveTimeline.createNewInstantTime()); LOG.info("Finished rolling back pending bootstrap"); } @@ -181,21 +255,11 @@ public class HoodieWriteClient extends AbstractHo /** * Upsert a batch of new records into Hoodie table at the supplied instantTime. * - * @param records JavaRDD of hoodieRecords to upsert + * @param records hoodieRecords to upsert * @param instantTime Instant time of the commit - * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts + * @return WriteStatus to inspect errors and counts */ - public JavaRDD upsert(JavaRDD> records, final String instantTime) { - HoodieTable table = getTableAndInitCtx(WriteOperationType.UPSERT, instantTime); - table.validateUpsertSchema(); - setOperationType(WriteOperationType.UPSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); - HoodieWriteMetadata result = table.upsert(jsc, instantTime, records); - if (result.getIndexLookupDuration().isPresent()) { - metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); - } - return postWrite(result, instantTime, table); - } + public abstract O upsert(I records, final String instantTime); /** * Upserts the given prepared records into the Hoodie table, at the supplied instantTime. @@ -206,14 +270,7 @@ public class HoodieWriteClient extends AbstractHo * @param instantTime Instant time of the commit * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ - public JavaRDD upsertPreppedRecords(JavaRDD> preppedRecords, final String instantTime) { - HoodieTable table = getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime); - table.validateUpsertSchema(); - setOperationType(WriteOperationType.UPSERT_PREPPED); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); - HoodieWriteMetadata result = table.upsertPrepped(jsc,instantTime, preppedRecords); - return postWrite(result, instantTime, table); - } + public abstract O upsertPreppedRecords(I preppedRecords, final String instantTime); /** * Inserts the given HoodieRecords, into the table. This API is intended to be used for normal writes. @@ -225,14 +282,7 @@ public class HoodieWriteClient extends AbstractHo * @param instantTime Instant time of the commit * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ - public JavaRDD insert(JavaRDD> records, final String instantTime) { - HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT, instantTime); - table.validateInsertSchema(); - setOperationType(WriteOperationType.INSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); - HoodieWriteMetadata result = table.insert(jsc,instantTime, records); - return postWrite(result, instantTime, table); - } + public abstract O insert(I records, final String instantTime); /** * Inserts the given prepared records into the Hoodie table, at the supplied instantTime. @@ -245,36 +295,27 @@ public class HoodieWriteClient extends AbstractHo * @param instantTime Instant time of the commit * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ - public JavaRDD insertPreppedRecords(JavaRDD> preppedRecords, final String instantTime) { - HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime); - table.validateInsertSchema(); - setOperationType(WriteOperationType.INSERT_PREPPED); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); - HoodieWriteMetadata result = table.insertPrepped(jsc,instantTime, preppedRecords); - return postWrite(result, instantTime, table); - } + public abstract O insertPreppedRecords(I preppedRecords, final String instantTime); /** * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie * table for the very first time (e.g: converting an existing table to Hoodie). *

* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control - * the numbers of files with less memory compared to the {@link HoodieWriteClient#insert(JavaRDD, String)} + * the numbers of files with less memory compared to the {@link AbstractHoodieWriteClient#insert(I, String)} * * @param records HoodieRecords to insert * @param instantTime Instant time of the commit * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ - public JavaRDD bulkInsert(JavaRDD> records, final String instantTime) { - return bulkInsert(records, instantTime, Option.empty()); - } + public abstract O bulkInsert(I records, final String instantTime); /** * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie * table for the very first time (e.g: converting an existing table to Hoodie). *

* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control - * the numbers of files with less memory compared to the {@link HoodieWriteClient#insert(JavaRDD, String)}. Optionally + * the numbers of files with less memory compared to the {@link AbstractHoodieWriteClient#insert(I, String)}. Optionally * it allows users to specify their own partitioner. If specified then it will be used for repartitioning records. See * {@link BulkInsertPartitioner}. * @@ -284,15 +325,9 @@ public class HoodieWriteClient extends AbstractHo * into hoodie. * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ - public JavaRDD bulkInsert(JavaRDD> records, final String instantTime, - Option userDefinedBulkInsertPartitioner) { - HoodieTable table = getTableAndInitCtx(WriteOperationType.BULK_INSERT, instantTime); - table.validateInsertSchema(); - setOperationType(WriteOperationType.BULK_INSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); - HoodieWriteMetadata result = table.bulkInsert(jsc,instantTime, records, userDefinedBulkInsertPartitioner); - return postWrite(result, instantTime, table); - } + public abstract O bulkInsert(I records, final String instantTime, + Option> userDefinedBulkInsertPartitioner); + /** * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie @@ -300,7 +335,7 @@ public class HoodieWriteClient extends AbstractHo * duplicates if needed. *

* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control - * the numbers of files with less memory compared to the {@link HoodieWriteClient#insert(JavaRDD, String)}. Optionally + * the numbers of files with less memory compared to the {@link AbstractHoodieWriteClient#insert(I, String)}. Optionally * it allows users to specify their own partitioner. If specified then it will be used for repartitioning records. See * {@link BulkInsertPartitioner}. * @@ -310,31 +345,8 @@ public class HoodieWriteClient extends AbstractHo * into hoodie. * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ - public JavaRDD bulkInsertPreppedRecords(JavaRDD> preppedRecords, final String instantTime, - Option bulkInsertPartitioner) { - HoodieTable table = getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED, instantTime); - table.validateInsertSchema(); - setOperationType(WriteOperationType.BULK_INSERT_PREPPED); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); - HoodieWriteMetadata result = table.bulkInsertPrepped(jsc,instantTime, preppedRecords, bulkInsertPartitioner); - return postWrite(result, instantTime, table); - } - - /** - * Removes all existing records from the partitions affected and inserts the given HoodieRecords, into the table. - - * @param records HoodieRecords to insert - * @param instantTime Instant time of the commit - * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts - */ - public HoodieWriteResult insertOverwrite(JavaRDD> records, final String instantTime) { - HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime); - table.validateInsertSchema(); - setOperationType(WriteOperationType.INSERT_OVERWRITE); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); - HoodieWriteMetadata result = table.insertOverwrite(jsc, instantTime, records); - return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); - } + public abstract O bulkInsertPreppedRecords(I preppedRecords, final String instantTime, + Option> bulkInsertPartitioner); /** * Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be @@ -344,12 +356,7 @@ public class HoodieWriteClient extends AbstractHo * @param instantTime Commit time handle * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ - public JavaRDD delete(JavaRDD keys, final String instantTime) { - HoodieTable table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime); - setOperationType(WriteOperationType.DELETE); - HoodieWriteMetadata result = table.delete(jsc,instantTime, keys); - return postWrite(result, instantTime, table); - } + public abstract O delete(K keys, final String instantTime); /** * Common method containing steps to be performed after write (upsert/insert/..) operations including auto-commit. @@ -358,31 +365,21 @@ public class HoodieWriteClient extends AbstractHo * @param hoodieTable Hoodie Table * @return Write Status */ - private JavaRDD postWrite(HoodieWriteMetadata result, String instantTime, HoodieTable hoodieTable) { - if (result.getIndexLookupDuration().isPresent()) { - metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis()); - } - if (result.isCommitted()) { - // Perform post commit operations. - if (result.getFinalizeDuration().isPresent()) { - metrics.updateFinalizeWriteMetrics(result.getFinalizeDuration().get().toMillis(), - result.getWriteStats().get().size()); - } + protected abstract O postWrite(HoodieWriteMetadata result, String instantTime, HoodieTable hoodieTable); - postCommit(hoodieTable, result.getCommitMetadata().get(), instantTime, Option.empty()); - - emitCommitMetrics(instantTime, result.getCommitMetadata().get(), - hoodieTable.getMetaClient().getCommitActionType()); - } - return result.getWriteStatuses(); - } - - @Override - protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata) { + /** + * Post Commit Hook. Derived classes use this method to perform post-commit processing + * + * @param table table to commit on + * @param metadata Commit Metadata corresponding to committed instant + * @param instantTime Instant Time + * @param extraMetadata Additional Metadata passed by user + */ + protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata) { try { // Delete the marker directory for the instant. - new MarkerFiles(table, instantTime).quietDeleteMarkerDir(jsc, config.getMarkersDeleteParallelism()); + new MarkerFiles(table, instantTime).quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); // Do an inline compaction if enabled if (config.isInlineCompaction()) { @@ -393,15 +390,15 @@ public class HoodieWriteClient extends AbstractHo metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false"); } // We cannot have unbounded commit files. Archive commits if we have to archive - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, hadoopConf); - archiveLog.archiveIfRequired(jsc); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table); + archiveLog.archiveIfRequired(context); autoCleanOnCommit(instantTime); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } } - private void runAnyPendingCompactions(HoodieTable table) { + protected void runAnyPendingCompactions(HoodieTable table) { table.getActiveTimeline().getCommitsAndCompactionTimeline().filterPendingCompactionTimeline().getInstants() .forEach(instant -> { LOG.info("Running previously failed inflight compaction at instant " + instant); @@ -411,9 +408,10 @@ public class HoodieWriteClient extends AbstractHo /** * Handle auto clean during commit. + * * @param instantTime */ - private void autoCleanOnCommit(String instantTime) { + protected void autoCleanOnCommit(String instantTime) { if (config.isAutoClean()) { // Call clean to cleanup if there is anything to cleanup after the commit, if (config.isAsyncClean()) { @@ -434,7 +432,7 @@ public class HoodieWriteClient extends AbstractHo * @param comment - Comment for the savepoint */ public void savepoint(String user, String comment) { - HoodieTable table = HoodieTable.create(config, hadoopConf); + HoodieTable table = createTable(config, hadoopConf); if (table.getCompletedCommitsTimeline().empty()) { throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); } @@ -458,8 +456,8 @@ public class HoodieWriteClient extends AbstractHo * @param comment - Comment for the savepoint */ public void savepoint(String instantTime, String user, String comment) { - HoodieTable table = HoodieTable.create(config, hadoopConf); - table.savepoint(jsc, instantTime, user, comment); + HoodieTable table = createTable(config, hadoopConf); + table.savepoint(context, instantTime, user, comment); } /** @@ -470,7 +468,7 @@ public class HoodieWriteClient extends AbstractHo * @return true if the savepoint was deleted successfully */ public void deleteSavepoint(String savepointTime) { - HoodieTable table = HoodieTable.create(config, hadoopConf); + HoodieTable table = createTable(config, hadoopConf); SavepointHelpers.deleteSavepoint(table, savepointTime); } @@ -485,7 +483,7 @@ public class HoodieWriteClient extends AbstractHo * @return true if the savepoint was restored to successfully */ public void restoreToSavepoint(String savepointTime) { - HoodieTable table = HoodieTable.create(config, hadoopConf); + HoodieTable table = createTable(config, hadoopConf); SavepointHelpers.validateSavepointPresence(table, savepointTime); restoreToInstant(savepointTime); SavepointHelpers.validateSavepointRestore(table, savepointTime); @@ -500,16 +498,16 @@ public class HoodieWriteClient extends AbstractHo public boolean rollback(final String commitInstantTime) throws HoodieRollbackException { LOG.info("Begin rollback of instant " + commitInstantTime); final String rollbackInstantTime = HoodieActiveTimeline.createNewInstantTime(); - final Timer.Context context = this.metrics.getRollbackCtx(); + final Timer.Context timerContext = this.metrics.getRollbackCtx(); try { - HoodieTable table = HoodieTable.create(config, hadoopConf); + HoodieTable table = createTable(config, hadoopConf); Option commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants() - .filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime)) - .findFirst()); + .filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime)) + .findFirst()); if (commitInstantOpt.isPresent()) { - HoodieRollbackMetadata rollbackMetadata = table.rollback(jsc, rollbackInstantTime, commitInstantOpt.get(), true); - if (context != null) { - long durationInMs = metrics.getDurationInMs(context.stop()); + HoodieRollbackMetadata rollbackMetadata = table.rollback(context, rollbackInstantTime, commitInstantOpt.get(), true); + if (timerContext != null) { + long durationInMs = metrics.getDurationInMs(timerContext.stop()); metrics.updateRollbackMetrics(durationInMs, rollbackMetadata.getTotalFilesDeleted()); } return true; @@ -531,12 +529,12 @@ public class HoodieWriteClient extends AbstractHo public HoodieRestoreMetadata restoreToInstant(final String instantTime) throws HoodieRestoreException { LOG.info("Begin restore to instant " + instantTime); final String restoreInstantTime = HoodieActiveTimeline.createNewInstantTime(); - Timer.Context context = metrics.getRollbackCtx(); + Timer.Context timerContext = metrics.getRollbackCtx(); try { - HoodieTable table = HoodieTable.create(config, hadoopConf); - HoodieRestoreMetadata restoreMetadata = table.restore(jsc, restoreInstantTime, instantTime); - if (context != null) { - final long durationInMs = metrics.getDurationInMs(context.stop()); + HoodieTable table = createTable(config, hadoopConf); + HoodieRestoreMetadata restoreMetadata = table.restore(context, restoreInstantTime, instantTime); + if (timerContext != null) { + final long durationInMs = metrics.getDurationInMs(timerContext.stop()); final long totalFilesDeleted = restoreMetadata.getHoodieRestoreMetadata().values().stream() .flatMap(Collection::stream) .mapToLong(HoodieRollbackMetadata::getTotalFilesDeleted) @@ -549,16 +547,6 @@ public class HoodieWriteClient extends AbstractHo } } - /** - * Releases any resources used by the client. - */ - @Override - public void close() { - AsyncCleanerService.forceShutdown(asyncCleanerService); - asyncCleanerService = null; - super.close(); - } - /** * Clean up any stale/old files/data lying around (either on file storage or index storage) based on the * configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be @@ -566,10 +554,10 @@ public class HoodieWriteClient extends AbstractHo */ public HoodieCleanMetadata clean(String cleanInstantTime) throws HoodieIOException { LOG.info("Cleaner started"); - final Timer.Context context = metrics.getCleanCtx(); - HoodieCleanMetadata metadata = HoodieTable.create(config, hadoopConf).clean(jsc, cleanInstantTime); - if (context != null && metadata != null) { - long durationMs = metrics.getDurationInMs(context.stop()); + final Timer.Context timerContext = metrics.getCleanCtx(); + HoodieCleanMetadata metadata = createTable(config, hadoopConf).clean(context, cleanInstantTime); + if (timerContext != null && metadata != null) { + long durationMs = metrics.getDurationInMs(timerContext.stop()); metrics.updateCleanMetrics(durationMs, metadata.getTotalFilesDeleted()); LOG.info("Cleaned " + metadata.getTotalFilesDeleted() + " files" + " Earliest Retained Instant :" + metadata.getEarliestCommitToRetain() @@ -634,7 +622,7 @@ public class HoodieWriteClient extends AbstractHo HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), HoodieTimeline.LESSER_THAN, instantTime), "Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :" + latestPending + ", Ingesting at " + instantTime)); - metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(State.REQUESTED, actionType, + metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, actionType, instantTime)); } @@ -656,8 +644,8 @@ public class HoodieWriteClient extends AbstractHo */ public boolean scheduleCompactionAtInstant(String instantTime, Option> extraMetadata) throws HoodieIOException { LOG.info("Scheduling compaction at instant time :" + instantTime); - Option plan = HoodieTable.create(config, hadoopConf) - .scheduleCompaction(jsc, instantTime, extraMetadata); + Option plan = createTable(config, hadoopConf) + .scheduleCompaction(context, instantTime, extraMetadata); return plan.isPresent(); } @@ -667,7 +655,7 @@ public class HoodieWriteClient extends AbstractHo * @param compactionInstantTime Compaction Instant Time * @return RDD of WriteStatus to inspect errors and counts */ - public JavaRDD compact(String compactionInstantTime) { + public O compact(String compactionInstantTime) { return compact(compactionInstantTime, config.shouldAutoCommit()); } @@ -678,38 +666,15 @@ public class HoodieWriteClient extends AbstractHo * @param writeStatuses RDD of WriteStatus to inspect errors and counts * @param extraMetadata Extra Metadata to be stored */ - public void commitCompaction(String compactionInstantTime, JavaRDD writeStatuses, - Option> extraMetadata) throws IOException { - HoodieTable table = HoodieTable.create(config, hadoopConf); - HoodieCommitMetadata metadata = CompactHelpers.createCompactionMetadata( - table, compactionInstantTime, writeStatuses, config.getSchema()); - extraMetadata.ifPresent(m -> m.forEach(metadata::addMetadata)); - completeCompaction(metadata, writeStatuses, table, compactionInstantTime); - } + public abstract void commitCompaction(String compactionInstantTime, O writeStatuses, + Option> extraMetadata) throws IOException; /** * Commit Compaction and track metrics. */ - protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD writeStatuses, HoodieTable table, - String compactionCommitTime) { + protected abstract void completeCompaction(HoodieCommitMetadata metadata, O writeStatuses, + HoodieTable table, String compactionCommitTime); - List writeStats = writeStatuses.map(WriteStatus::getStat).collect(); - finalizeWrite(table, compactionCommitTime, writeStats); - LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata); - CompactHelpers.completeInflightCompaction(table, compactionCommitTime, metadata); - - if (compactionTimer != null) { - long durationInMs = metrics.getDurationInMs(compactionTimer.stop()); - try { - metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(compactionCommitTime).getTime(), - durationInMs, metadata, HoodieActiveTimeline.COMPACTION_ACTION); - } catch (ParseException e) { - throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction " - + config.getBasePath() + " at time " + compactionCommitTime, e); - } - } - LOG.info("Compacted successfully on commit " + compactionCommitTime); - } /** * Rollback failed compactions. Inflight rollbacks for compactions revert the .inflight file to the .requested file @@ -717,8 +682,8 @@ public class HoodieWriteClient extends AbstractHo * @param inflightInstant Inflight Compaction Instant * @param table Hoodie Table */ - public void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTable table) { - table.rollback(jsc, HoodieActiveTimeline.createNewInstantTime(), inflightInstant, false); + public void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTable table) { + table.rollback(context, HoodieActiveTimeline.createNewInstantTime(), inflightInstant, false); table.getActiveTimeline().revertCompactionInflightToRequested(inflightInstant); } @@ -726,7 +691,7 @@ public class HoodieWriteClient extends AbstractHo * Cleanup all pending commits. */ private void rollbackPendingCommits() { - HoodieTable table = HoodieTable.create(config, hadoopConf); + HoodieTable table = createTable(config, hadoopConf); HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); List commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); @@ -747,27 +712,12 @@ public class HoodieWriteClient extends AbstractHo * @param compactionInstantTime Compaction Instant Time * @return RDD of Write Status */ - private JavaRDD compact(String compactionInstantTime, boolean shouldComplete) { - HoodieTable table = HoodieTable.create(config, hadoopConf); - HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); - HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); - if (pendingCompactionTimeline.containsInstant(inflightInstant)) { - rollbackInflightCompaction(inflightInstant, table); - table.getMetaClient().reloadActiveTimeline(); - } - compactionTimer = metrics.getCompactionCtx(); - HoodieWriteMetadata compactionMetadata = table.compact(jsc, compactionInstantTime); - JavaRDD statuses = compactionMetadata.getWriteStatuses(); - if (shouldComplete && compactionMetadata.getCommitMetadata().isPresent()) { - completeCompaction(compactionMetadata.getCommitMetadata().get(), statuses, table, compactionInstantTime); - } - return statuses; - } + protected abstract O compact(String compactionInstantTime, boolean shouldComplete); /** * Performs a compaction operation on a table, serially before or after an insert/upsert action. */ - private Option inlineCompact(Option> extraMetadata) { + protected Option inlineCompact(Option> extraMetadata) { Option compactionInstantTimeOpt = scheduleCompaction(extraMetadata); compactionInstantTimeOpt.ifPresent(compactionInstantTime -> { // inline compaction should auto commit as the user is never given control @@ -775,4 +725,82 @@ public class HoodieWriteClient extends AbstractHo }); return compactionInstantTimeOpt; } + + /** + * Finalize Write operation. + * + * @param table HoodieTable + * @param instantTime Instant Time + * @param stats Hoodie Write Stat + */ + protected void finalizeWrite(HoodieTable table, String instantTime, List stats) { + try { + final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); + table.finalizeWrite(context, instantTime, stats); + if (finalizeCtx != null) { + Option durationInMs = Option.of(metrics.getDurationInMs(finalizeCtx.stop())); + durationInMs.ifPresent(duration -> { + LOG.info("Finalize write elapsed time (milliseconds): " + duration); + metrics.updateFinalizeWriteMetrics(duration, stats.size()); + }); + } + } catch (HoodieIOException ioe) { + throw new HoodieCommitException("Failed to complete commit " + instantTime + " due to finalize errors.", ioe); + } + } + + public HoodieMetrics getMetrics() { + return metrics; + } + + public HoodieIndex getIndex() { + return index; + } + + /** + * Get HoodieTable and init {@link Timer.Context}. + * + * @param operationType write operation type + * @param instantTime current inflight instant time + * @return HoodieTable + */ + protected abstract HoodieTable getTableAndInitCtx(WriteOperationType operationType, String instantTime); + + /** + * Sets write schema from last instant since deletes may not have schema set in the config. + */ + protected void setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) { + try { + HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); + Option lastInstant = + activeTimeline.filterCompletedInstants().filter(s -> s.getAction().equals(metaClient.getCommitActionType())) + .lastInstant(); + if (lastInstant.isPresent()) { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + activeTimeline.getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class); + if (commitMetadata.getExtraMetadata().containsKey(HoodieCommitMetadata.SCHEMA_KEY)) { + config.setSchema(commitMetadata.getExtraMetadata().get(HoodieCommitMetadata.SCHEMA_KEY)); + } else { + throw new HoodieIOException("Latest commit does not have any schema in commit metadata"); + } + } else { + throw new HoodieIOException("Deletes issued without any prior commits"); + } + } catch (IOException e) { + throw new HoodieIOException("IOException thrown while reading last commit metadata", e); + } + } + + @Override + public void close() { + // release AsyncCleanerService + AsyncCleanerService.forceShutdown(asyncCleanerService); + asyncCleanerService = null; + + // Stop timeline-server if running + super.close(); + // Calling this here releases any resources used by your index, so make sure to finish any related operations + // before this point + this.index.close(); + } } diff --git a/hudi-client/src/main/java/org/apache/hudi/client/AsyncCleanerService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java similarity index 90% rename from hudi-client/src/main/java/org/apache/hudi/client/AsyncCleanerService.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java index 6367e7972..e8016c957 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/AsyncCleanerService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java @@ -18,7 +18,7 @@ package org.apache.hudi.client; -import org.apache.hudi.async.AbstractAsyncService; +import org.apache.hudi.async.HoodieAsyncService; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.log4j.LogManager; @@ -31,15 +31,15 @@ import java.util.concurrent.Executors; /** * Clean service running concurrently with write operation. */ -class AsyncCleanerService extends AbstractAsyncService { +class AsyncCleanerService extends HoodieAsyncService { private static final Logger LOG = LogManager.getLogger(AsyncCleanerService.class); - private final HoodieWriteClient writeClient; + private final AbstractHoodieWriteClient writeClient; private final String cleanInstantTime; private final transient ExecutorService executor = Executors.newSingleThreadExecutor(); - protected AsyncCleanerService(HoodieWriteClient writeClient, String cleanInstantTime) { + protected AsyncCleanerService(AbstractHoodieWriteClient writeClient, String cleanInstantTime) { this.writeClient = writeClient; this.cleanInstantTime = cleanInstantTime; } @@ -52,7 +52,7 @@ class AsyncCleanerService extends AbstractAsyncService { }), executor); } - public static AsyncCleanerService startAsyncCleaningIfEnabled(HoodieWriteClient writeClient, + public static AsyncCleanerService startAsyncCleaningIfEnabled(AbstractHoodieWriteClient writeClient, String instantTime) { AsyncCleanerService asyncCleanerService = null; if (writeClient.getConfig().isAutoClean() && writeClient.getConfig().isAsyncClean()) { diff --git a/hudi-client/src/main/java/org/apache/hudi/client/CompactionAdminClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java similarity index 97% rename from hudi-client/src/main/java/org/apache/hudi/client/CompactionAdminClient.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java index 48042e72e..a2ecb6727 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/CompactionAdminClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; @@ -45,7 +46,6 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.action.compact.OperationResult; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.io.FileNotFoundException; import java.io.IOException; @@ -65,8 +65,8 @@ public class CompactionAdminClient extends AbstractHoodieClient { private static final Logger LOG = LogManager.getLogger(CompactionAdminClient.class); - public CompactionAdminClient(JavaSparkContext jsc, String basePath) { - super(jsc, HoodieWriteConfig.newBuilder().withPath(basePath).build()); + public CompactionAdminClient(HoodieEngineContext context, String basePath) { + super(context, HoodieWriteConfig.newBuilder().withPath(basePath).build()); } /** @@ -85,14 +85,14 @@ public class CompactionAdminClient extends AbstractHoodieClient { if (plan.getOperations() != null) { List ops = plan.getOperations().stream() .map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList()); - jsc.setJobGroup(this.getClass().getSimpleName(), "Validate compaction operations"); - return jsc.parallelize(ops, parallelism).map(op -> { + context.setJobStatus(this.getClass().getSimpleName(), "Validate compaction operations"); + return context.map(ops, op -> { try { return validateCompactionOperation(metaClient, compactionInstant, op, Option.of(fsView)); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } - }).collect(); + }, parallelism); } return new ArrayList<>(); } @@ -351,8 +351,8 @@ public class CompactionAdminClient extends AbstractHoodieClient { } else { LOG.info("The following compaction renaming operations needs to be performed to un-schedule"); if (!dryRun) { - jsc.setJobGroup(this.getClass().getSimpleName(), "Execute unschedule operations"); - return jsc.parallelize(renameActions, parallelism).map(lfPair -> { + context.setJobStatus(this.getClass().getSimpleName(), "Execute unschedule operations"); + return context.map(renameActions, lfPair -> { try { LOG.info("RENAME " + lfPair.getLeft().getPath() + " => " + lfPair.getRight().getPath()); renameLogFile(metaClient, lfPair.getLeft(), lfPair.getRight()); @@ -363,7 +363,7 @@ public class CompactionAdminClient extends AbstractHoodieClient { + lfPair.getLeft().getBaseCommitTime() + "\" to recover from failure ***\n\n\n"); return new RenameOpResult(lfPair, false, Option.of(e)); } - }).collect(); + }, parallelism); } else { LOG.info("Dry-Run Mode activated for rename operations"); return renameActions.parallelStream().map(lfPair -> new RenameOpResult(lfPair, false, false, Option.empty())) @@ -394,17 +394,17 @@ public class CompactionAdminClient extends AbstractHoodieClient { "Number of Compaction Operations :" + plan.getOperations().size() + " for instant :" + compactionInstant); List ops = plan.getOperations().stream() .map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList()); - jsc.setJobGroup(this.getClass().getSimpleName(), "Generate compaction unscheduling operations"); - return jsc.parallelize(ops, parallelism).flatMap(op -> { + context.setJobStatus(this.getClass().getSimpleName(), "Generate compaction unscheduling operations"); + return context.flatMap(ops, op -> { try { return getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op, - Option.of(fsView), skipValidation).iterator(); + Option.of(fsView), skipValidation).stream(); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } catch (CompactionValidationException ve) { throw new HoodieException(ve); } - }).collect(); + }, parallelism); } LOG.warn("No operations for compaction instant : " + compactionInstant); return new ArrayList<>(); diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/ReplaceArchivalHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java similarity index 87% rename from hudi-client/src/main/java/org/apache/hudi/table/ReplaceArchivalHelper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java index 317cf0de1..515f43e64 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/ReplaceArchivalHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java @@ -16,21 +16,23 @@ * limitations under the License. */ -package org.apache.hudi.table; +package org.apache.hudi.client; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.hadoop.fs.Path; + import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieRollingStatMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.TableFileSystemView; +import org.apache.hudi.client.common.HoodieEngineContext; + 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 java.io.IOException; import java.io.Serializable; @@ -63,17 +65,17 @@ public class ReplaceArchivalHelper implements Serializable { /** * Delete all files represented by FileSlices in parallel. Return true if all files are deleted successfully. */ - public static boolean deleteReplacedFileGroups(JavaSparkContext jsc, HoodieTableMetaClient metaClient, + public static boolean deleteReplacedFileGroups(HoodieEngineContext context, HoodieTableMetaClient metaClient, TableFileSystemView fileSystemView, HoodieInstant instant, List replacedPartitions) { - JavaRDD partitions = jsc.parallelize(replacedPartitions, replacedPartitions.size()); - return partitions.map(partition -> { + List f = context.map(replacedPartitions, partition -> { Stream fileSlices = fileSystemView.getReplacedFileGroupsBeforeOrOn(instant.getTimestamp(), partition) - .flatMap(g -> g.getAllRawFileSlices()); + .flatMap(HoodieFileGroup::getAllRawFileSlices); + return fileSlices.allMatch(slice -> deleteFileSlice(slice, metaClient, instant)); + }, replacedPartitions.size()); - return fileSlices.map(slice -> deleteFileSlice(slice, metaClient, instant)).allMatch(x -> x); - }).reduce((x, y) -> x & y); + return f.stream().reduce((x, y) -> x & y).orElse(true); } private static boolean deleteFileSlice(FileSlice fileSlice, HoodieTableMetaClient metaClient, HoodieInstant instant) { @@ -95,5 +97,4 @@ public class ReplaceArchivalHelper implements Serializable { return false; } } - } diff --git a/hudi-client/src/main/java/org/apache/hudi/client/WriteStatus.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/WriteStatus.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapMode.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/BootstrapMode.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapMode.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/BootstrapMode.java diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapRecordPayload.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/BootstrapRecordPayload.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapRecordPayload.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/BootstrapRecordPayload.java diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapWriteStatus.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/BootstrapWriteStatus.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapWriteStatus.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/BootstrapWriteStatus.java diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java similarity index 75% rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java index 8b077ad9f..542dad9cc 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java @@ -20,30 +20,28 @@ package org.apache.hudi.client.bootstrap; import java.io.Serializable; import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.collection.Pair; 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 java.util.List; /** - * Creates RDD of Hoodie Records with complete record data, given a list of partitions to be bootstrapped. + * Creates Hoodie Records with complete record data, given a list of partitions to be bootstrapped. */ -public abstract class FullRecordBootstrapDataProvider implements Serializable { +public abstract class FullRecordBootstrapDataProvider implements Serializable { protected static final Logger LOG = LogManager.getLogger(FullRecordBootstrapDataProvider.class); protected final TypedProperties props; - protected final transient JavaSparkContext jsc; + protected final transient HoodieEngineContext context; - public FullRecordBootstrapDataProvider(TypedProperties props, JavaSparkContext jsc) { + public FullRecordBootstrapDataProvider(TypedProperties props, HoodieEngineContext context) { this.props = props; - this.jsc = jsc; + this.context = context; } /** @@ -51,8 +49,8 @@ public abstract class FullRecordBootstrapDataProvider implements Serializable { * @param tableName Hudi Table Name * @param sourceBasePath Source Base Path * @param partitionPaths Partition Paths - * @return JavaRDD of input records + * @return input records */ - public abstract JavaRDD generateInputRecordRDD(String tableName, + public abstract I generateInputRecords(String tableName, String sourceBasePath, List>> partitionPaths); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/HoodieBootstrapSchemaProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/HoodieBootstrapSchemaProvider.java new file mode 100644 index 000000000..f63345d91 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/HoodieBootstrapSchemaProvider.java @@ -0,0 +1,69 @@ +/* + * 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.client.bootstrap; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.avro.Schema; + +import java.util.List; + +/** + * Bootstrap Schema Provider. Schema provided in config is used. If not available, use schema from Parquet + */ +public abstract class HoodieBootstrapSchemaProvider { + + protected final HoodieWriteConfig writeConfig; + + public HoodieBootstrapSchemaProvider(HoodieWriteConfig writeConfig) { + this.writeConfig = writeConfig; + } + + /** + * Main API to select avro schema for bootstrapping. + * @param context HoodieEngineContext + * @param partitions List of partitions with files within them + * @return Avro Schema + */ + public final Schema getBootstrapSchema(HoodieEngineContext context, List>> partitions) { + if (writeConfig.getSchema() != null) { + // Use schema specified by user if set + Schema userSchema = Schema.parse(writeConfig.getSchema()); + if (!HoodieAvroUtils.getNullSchema().equals(userSchema)) { + return userSchema; + } + } + return getBootstrapSourceSchema(context, partitions); + } + + /** + * Select a random file to be used to generate avro schema. + * Override this method to get custom schema selection. + * @param context HoodieEngineContext + * @param partitions List of partitions with files within them + * @return Avro Schema + */ + protected abstract Schema getBootstrapSourceSchema(HoodieEngineContext context, + List>> partitions); + +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapModeSelector.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapModeSelector.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapModeSelector.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapModeSelector.java diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/FullRecordBootstrapModeSelector.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/FullRecordBootstrapModeSelector.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/FullRecordBootstrapModeSelector.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/FullRecordBootstrapModeSelector.java diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/MetadataOnlyBootstrapModeSelector.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/MetadataOnlyBootstrapModeSelector.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/MetadataOnlyBootstrapModeSelector.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/MetadataOnlyBootstrapModeSelector.java diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/UniformBootstrapModeSelector.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/UniformBootstrapModeSelector.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/UniformBootstrapModeSelector.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/UniformBootstrapModeSelector.java diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/BootstrapPartitionPathTranslator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/translator/BootstrapPartitionPathTranslator.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/BootstrapPartitionPathTranslator.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/translator/BootstrapPartitionPathTranslator.java diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/IdentityBootstrapPartitionPathTranslator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/translator/IdentityBootstrapPartitionPathTranslator.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/IdentityBootstrapPartitionPathTranslator.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/translator/IdentityBootstrapPartitionPathTranslator.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/EngineProperty.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/EngineProperty.java new file mode 100644 index 000000000..aeaec3274 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/EngineProperty.java @@ -0,0 +1,33 @@ +/* + * 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.client.common; + +/** + * Properties specific to each engine, that can be set/obtained from. + */ +public enum EngineProperty { + // hostname to bind embedded timeline server to + EMBEDDED_SERVER_HOST, + // Pool/queue to use to run compaction. + COMPACTION_POOL_NAME, + // Amount of total memory available to each engine executor + TOTAL_MEMORY_AVAILABLE, + // Fraction of that memory, that is already in use by the engine + MEMORY_FRACTION_IN_USE, +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.java new file mode 100644 index 000000000..f8d889640 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.java @@ -0,0 +1,71 @@ +/* + * 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.client.common; + +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.client.common.function.SerializableConsumer; +import org.apache.hudi.client.common.function.SerializableFunction; +import org.apache.hudi.client.common.function.SerializablePairFunction; +import org.apache.hudi.common.util.Option; + +import java.util.List; +import java.util.Map; +import java.util.stream.Stream; + +/** + * Base class contains the context information needed by the engine at runtime. It will be extended by different + * engine implementation if needed. + */ +public abstract class HoodieEngineContext { + + /** + * A wrapped hadoop configuration which can be serialized. + */ + private SerializableConfiguration hadoopConf; + + private TaskContextSupplier taskContextSupplier; + + public HoodieEngineContext(SerializableConfiguration hadoopConf, TaskContextSupplier taskContextSupplier) { + this.hadoopConf = hadoopConf; + this.taskContextSupplier = taskContextSupplier; + } + + public SerializableConfiguration getHadoopConf() { + return hadoopConf; + } + + public TaskContextSupplier getTaskContextSupplier() { + return taskContextSupplier; + } + + public abstract List map(List data, SerializableFunction func, int parallelism); + + public abstract List flatMap(List data, SerializableFunction> func, int parallelism); + + public abstract void foreach(List data, SerializableConsumer consumer, int parallelism); + + public abstract Map mapToPair(List data, SerializablePairFunction func, Integer parallelism); + + public abstract void setProperty(EngineProperty key, String value); + + public abstract Option getProperty(EngineProperty key); + + public abstract void setJobStatus(String activeModule, String activityDescription); + +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/TaskContextSupplier.java similarity index 65% rename from hudi-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/TaskContextSupplier.java index 601dd98a2..3a350d61d 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/TaskContextSupplier.java @@ -16,27 +16,23 @@ * limitations under the License. */ -package org.apache.hudi.client; +package org.apache.hudi.client.common; -import org.apache.spark.TaskContext; +import org.apache.hudi.common.util.Option; import java.io.Serializable; import java.util.function.Supplier; /** - * Spark task context supplier. + * Base task context supplier. */ -public class SparkTaskContextSupplier implements Serializable { +public abstract class TaskContextSupplier implements Serializable { - public Supplier getPartitionIdSupplier() { - return () -> TaskContext.getPartitionId(); - } + public abstract Supplier getPartitionIdSupplier(); - public Supplier getStageIdSupplier() { - return () -> TaskContext.get().stageId(); - } + public abstract Supplier getStageIdSupplier(); - public Supplier getAttemptIdSupplier() { - return () -> TaskContext.get().taskAttemptId(); - } + public abstract Supplier getAttemptIdSupplier(); + + public abstract Option getProperty(EngineProperty prop); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableConsumer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableConsumer.java new file mode 100644 index 000000000..d7c420522 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableConsumer.java @@ -0,0 +1,31 @@ +/* + * 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.client.common.function; + +import java.io.Serializable; + +/** + * A wrapped {@link java.util.function.Consumer} which can be serialized. + * + * @param input type + */ +@FunctionalInterface +public interface SerializableConsumer extends Serializable { + void accept(I t) throws Exception; +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableFunction.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableFunction.java new file mode 100644 index 000000000..d3714bc5b --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableFunction.java @@ -0,0 +1,32 @@ +/* + * 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.client.common.function; + +import java.io.Serializable; + +/** + * A wrapped {@link java.util.function.Function} which can be serialized. + * + * @param input data type + * @param output data type + */ +@FunctionalInterface +public interface SerializableFunction extends Serializable { + O apply(I v1) throws Exception; +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializablePairFunction.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializablePairFunction.java new file mode 100644 index 000000000..155837b7f --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializablePairFunction.java @@ -0,0 +1,31 @@ +/* + * 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.client.common.function; + +import scala.Tuple2; + +import java.io.Serializable; + +/** + * A function that returns key-value pairs (Tuple2<K, V>). + */ +@FunctionalInterface +public interface SerializablePairFunction extends Serializable { + Tuple2 call(I t) throws Exception; +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java similarity index 85% rename from hudi-client/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java index 730585b12..ca0c71350 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java @@ -18,6 +18,7 @@ package org.apache.hudi.client.embedded; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; @@ -25,10 +26,8 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageType; import org.apache.hudi.common.util.NetworkUtils; import org.apache.hudi.timeline.service.TimelineService; -import org.apache.hadoop.conf.Configuration; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.SparkConf; import java.io.IOException; @@ -46,13 +45,10 @@ public class EmbeddedTimelineService { private transient FileSystemViewManager viewManager; private transient TimelineService server; - public EmbeddedTimelineService(Configuration hadoopConf, SparkConf sparkConf, FileSystemViewStorageConfig config) { - setHostAddrFromSparkConf(sparkConf); - if (hostAddr == null) { - this.hostAddr = NetworkUtils.getHostname(); - } + public EmbeddedTimelineService(HoodieEngineContext context, String embeddedTimelineServiceHostAddr, FileSystemViewStorageConfig config) { + setHostAddr(embeddedTimelineServiceHostAddr); this.config = config; - this.hadoopConf = new SerializableConfiguration(hadoopConf); + this.hadoopConf = context.getHadoopConf(); this.viewManager = createViewManager(); } @@ -75,13 +71,13 @@ public class EmbeddedTimelineService { LOG.info("Started embedded timeline server at " + hostAddr + ":" + serverPort); } - private void setHostAddrFromSparkConf(SparkConf sparkConf) { - String hostAddr = sparkConf.get("spark.driver.host", null); - if (hostAddr != null) { - LOG.info("Overriding hostIp to (" + hostAddr + ") found in spark-conf. It was " + this.hostAddr); - this.hostAddr = hostAddr; + private void setHostAddr(String embeddedTimelineServiceHostAddr) { + if (embeddedTimelineServiceHostAddr != null) { + LOG.info("Overriding hostIp to (" + embeddedTimelineServiceHostAddr + ") found in spark-conf. It was " + this.hostAddr); + this.hostAddr = embeddedTimelineServiceHostAddr; } else { LOG.warn("Unable to find driver bind address from spark config"); + this.hostAddr = NetworkUtils.getHostname(); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/client/utils/LazyIterableIterator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LazyIterableIterator.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/utils/LazyIterableIterator.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LazyIterableIterator.java diff --git a/hudi-client/src/main/java/org/apache/hudi/client/utils/MergingIterator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MergingIterator.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/utils/MergingIterator.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MergingIterator.java diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java similarity index 96% rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java index 3105fcd13..687033f45 100644 --- a/hudi-client/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java @@ -118,6 +118,8 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig { String.valueOf(DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)); setDefaultOnCondition(props, !props.containsKey(SPILLABLE_MAP_BASE_PATH_PROP), SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH); + setDefaultOnCondition(props, !props.containsKey(MAX_MEMORY_FOR_MERGE_PROP), MAX_MEMORY_FOR_MERGE_PROP, + String.valueOf(DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)); setDefaultOnCondition(props, !props.containsKey(WRITESTATUS_FAILURE_FRACTION_PROP), WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(DEFAULT_WRITESTATUS_FAILURE_FRACTION)); return config; diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieMetricsDatadogConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsDatadogConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieMetricsDatadogConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsDatadogConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieMetricsPrometheusConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsPrometheusConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieMetricsPrometheusConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsPrometheusConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java similarity index 99% rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 089474d15..60ca3c2dd 100644 --- a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -19,7 +19,6 @@ package org.apache.hudi.config; import org.apache.hadoop.hbase.io.compress.Compression; -import org.apache.hudi.client.HoodieWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.bootstrap.BootstrapMode; import org.apache.hudi.common.config.DefaultHoodieConfig; @@ -51,7 +50,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; /** - * Class storing configs for the {@link HoodieWriteClient}. + * Class storing configs for the HoodieWriteClient. */ @Immutable public class HoodieWriteConfig extends DefaultHoodieConfig { @@ -672,7 +671,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { public boolean getPushGatewayRandomJobNameSuffix() { return Boolean.parseBoolean(props.getProperty(HoodieMetricsPrometheusConfig.PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX)); } - + /** * memory configs. */ @@ -755,6 +754,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { return Integer.parseInt(props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM)); } + public Long getMaxMemoryPerPartitionMerge() { + return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP)); + } + public static class Builder { protected final Properties props = new Properties(); diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieAppendException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieAppendException.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieAppendException.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieAppendException.java diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieCommitCallbackException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieCommitCallbackException.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieCommitCallbackException.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieCommitCallbackException.java diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieCommitException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieCommitException.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieCommitException.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieCommitException.java diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieCompactionException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieCompactionException.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieCompactionException.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieCompactionException.java diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieDependentSystemUnavailableException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieDependentSystemUnavailableException.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieDependentSystemUnavailableException.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieDependentSystemUnavailableException.java diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieInsertException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieInsertException.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieInsertException.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieInsertException.java diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieRestoreException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieRestoreException.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieRestoreException.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieRestoreException.java diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieRollbackException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieRollbackException.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieRollbackException.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieRollbackException.java diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieSavepointException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieSavepointException.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieSavepointException.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieSavepointException.java diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieUpgradeDowngradeException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieUpgradeDowngradeException.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieUpgradeDowngradeException.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieUpgradeDowngradeException.java diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieUpsertException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieUpsertException.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieUpsertException.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieUpsertException.java diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java similarity index 84% rename from hudi-client/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java index ae61d8068..8af72f351 100644 --- a/hudi-client/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java @@ -18,13 +18,13 @@ package org.apache.hudi.execution; -import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.execution.LazyInsertIterable.HoodieInsertValueGenResult; +import org.apache.hudi.execution.HoodieLazyInsertIterable.HoodieInsertValueGenResult; import org.apache.hudi.io.HoodieWriteHandle; import org.apache.hudi.io.WriteHandleFactory; import org.apache.hudi.table.HoodieTable; @@ -43,10 +43,10 @@ public class CopyOnWriteInsertHandler private HoodieWriteConfig config; private String instantTime; private boolean areRecordsSorted; - private HoodieTable hoodieTable; + private HoodieTable hoodieTable; private String idPrefix; - private SparkTaskContextSupplier sparkTaskContextSupplier; - private WriteHandleFactory writeHandleFactory; + private TaskContextSupplier taskContextSupplier; + private WriteHandleFactory writeHandleFactory; private final List statuses = new ArrayList<>(); // Stores the open HoodieWriteHandle for each table partition path @@ -55,15 +55,15 @@ public class CopyOnWriteInsertHandler private Map handles = new HashMap<>(); public CopyOnWriteInsertHandler(HoodieWriteConfig config, String instantTime, - boolean areRecordsSorted, HoodieTable hoodieTable, String idPrefix, - SparkTaskContextSupplier sparkTaskContextSupplier, - WriteHandleFactory writeHandleFactory) { + boolean areRecordsSorted, HoodieTable hoodieTable, String idPrefix, + TaskContextSupplier taskContextSupplier, + WriteHandleFactory writeHandleFactory) { this.config = config; this.instantTime = instantTime; this.areRecordsSorted = areRecordsSorted; this.hoodieTable = hoodieTable; this.idPrefix = idPrefix; - this.sparkTaskContextSupplier = sparkTaskContextSupplier; + this.taskContextSupplier = taskContextSupplier; this.writeHandleFactory = writeHandleFactory; } @@ -81,7 +81,7 @@ public class CopyOnWriteInsertHandler } // Lazily initialize the handle, for the first time handle = writeHandleFactory.create(config, instantTime, hoodieTable, - insertPayload.getPartitionPath(), idPrefix, sparkTaskContextSupplier); + insertPayload.getPartitionPath(), idPrefix, taskContextSupplier); handles.put(partitionPath, handle); } @@ -90,7 +90,7 @@ public class CopyOnWriteInsertHandler statuses.add(handle.close()); // Open new handle handle = writeHandleFactory.create(config, instantTime, hoodieTable, - insertPayload.getPartitionPath(), idPrefix, sparkTaskContextSupplier); + insertPayload.getPartitionPath(), idPrefix, taskContextSupplier); handles.put(partitionPath, handle); } handle.write(insertPayload, payload.insertValue, payload.exception); diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/LazyInsertIterable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java similarity index 58% rename from hudi-client/src/main/java/org/apache/hudi/execution/LazyInsertIterable.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java index 572956d27..b435c68de 100644 --- a/hudi-client/src/main/java/org/apache/hudi/execution/LazyInsertIterable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java @@ -18,15 +18,13 @@ package org.apache.hudi.execution; -import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.LazyIterableIterator; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieException; import org.apache.hudi.io.CreateHandleFactory; import org.apache.hudi.io.WriteHandleFactory; import org.apache.hudi.table.HoodieTable; @@ -41,41 +39,39 @@ import java.util.function.Function; /** * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new files. */ -public class LazyInsertIterable +public abstract class HoodieLazyInsertIterable extends LazyIterableIterator, List> { protected final HoodieWriteConfig hoodieConfig; protected final String instantTime; protected boolean areRecordsSorted; - protected final HoodieTable hoodieTable; + protected final HoodieTable hoodieTable; protected final String idPrefix; - protected SparkTaskContextSupplier sparkTaskContextSupplier; - protected WriteHandleFactory writeHandleFactory; + protected TaskContextSupplier taskContextSupplier; + protected WriteHandleFactory writeHandleFactory; - public LazyInsertIterable(Iterator> sortedRecordItr, HoodieWriteConfig config, - String instantTime, HoodieTable hoodieTable, String idPrefix, - SparkTaskContextSupplier sparkTaskContextSupplier) { - this(sortedRecordItr, true, config, instantTime, hoodieTable, idPrefix, sparkTaskContextSupplier); - } - - public LazyInsertIterable(Iterator> recordItr, boolean areRecordsSorted, - HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, - String idPrefix, SparkTaskContextSupplier sparkTaskContextSupplier) { - this(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, sparkTaskContextSupplier, + public HoodieLazyInsertIterable(Iterator> recordItr, + boolean areRecordsSorted, + HoodieWriteConfig config, + String instantTime, + HoodieTable hoodieTable, + String idPrefix, + TaskContextSupplier taskContextSupplier) { + this(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier, new CreateHandleFactory<>()); } - public LazyInsertIterable(Iterator> recordItr, boolean areRecordsSorted, - HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, - String idPrefix, SparkTaskContextSupplier sparkTaskContextSupplier, - WriteHandleFactory writeHandleFactory) { + public HoodieLazyInsertIterable(Iterator> recordItr, boolean areRecordsSorted, + HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + String idPrefix, TaskContextSupplier taskContextSupplier, + WriteHandleFactory writeHandleFactory) { super(recordItr); this.areRecordsSorted = areRecordsSorted; this.hoodieConfig = config; this.instantTime = instantTime; this.hoodieTable = hoodieTable; this.idPrefix = idPrefix; - this.sparkTaskContextSupplier = sparkTaskContextSupplier; + this.taskContextSupplier = taskContextSupplier; this.writeHandleFactory = writeHandleFactory; } @@ -108,32 +104,11 @@ public class LazyInsertIterable @Override protected void start() {} - @Override - protected List computeNext() { - // Executor service used for launching writer thread. - BoundedInMemoryExecutor, HoodieInsertValueGenResult, List> bufferedIteratorExecutor = - null; - try { - final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); - bufferedIteratorExecutor = - new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, getInsertHandler(), getTransformFunction(schema)); - final List result = bufferedIteratorExecutor.execute(); - assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); - return result; - } catch (Exception e) { - throw new HoodieException(e); - } finally { - if (null != bufferedIteratorExecutor) { - bufferedIteratorExecutor.shutdownNow(); - } - } - } - @Override protected void end() {} protected CopyOnWriteInsertHandler getInsertHandler() { return new CopyOnWriteInsertHandler(hoodieConfig, instantTime, areRecordsSorted, hoodieTable, idPrefix, - sparkTaskContextSupplier, writeHandleFactory); + taskContextSupplier, writeHandleFactory); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertSortMode.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertSortMode.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertSortMode.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertSortMode.java diff --git a/hudi-client/src/main/java/org/apache/hudi/index/HoodieIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java similarity index 54% rename from hudi-client/src/main/java/org/apache/hudi/index/HoodieIndex.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java index 4043586dd..c71b34e69 100644 --- a/hudi-client/src/main/java/org/apache/hudi/index/HoodieIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java @@ -21,35 +21,26 @@ package org.apache.hudi.index; import org.apache.hudi.ApiMaturityLevel; import org.apache.hudi.PublicAPIClass; import org.apache.hudi.PublicAPIMethod; -import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ReflectionUtils; -import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; -import org.apache.hudi.index.bloom.HoodieBloomIndex; -import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex; -import org.apache.hudi.index.hbase.HBaseIndex; -import org.apache.hudi.index.simple.HoodieGlobalSimpleIndex; -import org.apache.hudi.index.simple.HoodieSimpleIndex; 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 java.io.Serializable; /** * Base class for different types of indexes to determine the mapping from uuid. + * + * @param Sub type of HoodieRecordPayload + * @param Type of inputs + * @param Type of keys + * @param Type of outputs */ @PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING) -public abstract class HoodieIndex implements Serializable { +public abstract class HoodieIndex implements Serializable { protected final HoodieWriteConfig config; @@ -57,49 +48,13 @@ public abstract class HoodieIndex implements Seri this.config = config; } - public static HoodieIndex createIndex( - HoodieWriteConfig config) throws HoodieIndexException { - // first use index class config to create index. - if (!StringUtils.isNullOrEmpty(config.getIndexClass())) { - Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config); - if (!(instance instanceof HoodieIndex)) { - throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex"); - } - return (HoodieIndex) instance; - } - switch (config.getIndexType()) { - case HBASE: - return new HBaseIndex<>(config); - case INMEMORY: - return new InMemoryHashIndex<>(config); - case BLOOM: - return new HoodieBloomIndex<>(config); - case GLOBAL_BLOOM: - return new HoodieGlobalBloomIndex<>(config); - case SIMPLE: - return new HoodieSimpleIndex<>(config); - case GLOBAL_SIMPLE: - return new HoodieGlobalSimpleIndex<>(config); - default: - throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); - } - } - - /** - * Checks if the given [Keys] exists in the hoodie table and returns [Key, Option[partitionPath, fileID]] If the - * optional is empty, then the key is not found. - */ - @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) - public abstract JavaPairRDD>> fetchRecordLocation( - JavaRDD hoodieKeys, final JavaSparkContext jsc, HoodieTable hoodieTable); - /** * Looks up the index and tags each incoming record with a location of a file that contains the row (if it is actually * present). */ @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) - public abstract JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) throws HoodieIndexException; + public abstract I tagLocation(I records, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException; /** * Extracts the location of written records, and updates the index. @@ -107,11 +62,11 @@ public abstract class HoodieIndex implements Seri * TODO(vc): We may need to propagate the record as well in a WriteStatus class */ @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) - public abstract JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) throws HoodieIndexException; + public abstract O updateLocation(O writeStatusRDD, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException; /** - * Rollback the efffects of the commit made at instantTime. + * Rollback the effects of the commit made at instantTime. */ @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) public abstract boolean rollbackCommit(String instantTime); diff --git a/hudi-client/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java similarity index 75% rename from hudi-client/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java index ebd30b3ba..8cd0cb2f5 100644 --- a/hudi-client/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java @@ -18,6 +18,7 @@ package org.apache.hudi.index; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -26,8 +27,6 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.table.HoodieTable; -import org.apache.spark.api.java.JavaSparkContext; - import java.util.ArrayList; import java.util.List; @@ -42,28 +41,26 @@ public class HoodieIndexUtils { * Fetches Pair of partition path and {@link HoodieBaseFile}s for interested partitions. * * @param partitions list of partitions of interest - * @param jsc instance of {@link JavaSparkContext} to use + * @param context instance of {@link HoodieEngineContext} to use * @param hoodieTable instance of {@link HoodieTable} of interest * @return the list of Pairs of partition path and fileId */ public static List> getLatestBaseFilesForAllPartitions(final List partitions, - final JavaSparkContext jsc, + final HoodieEngineContext context, final HoodieTable hoodieTable) { - jsc.setJobGroup(HoodieIndexUtils.class.getSimpleName(), "Load latest base files from all partitions"); - return jsc.parallelize(partitions, Math.max(partitions.size(), 1)) - .flatMap(partitionPath -> { - Option latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline() - .filterCompletedInstants().lastInstant(); - List> filteredFiles = new ArrayList<>(); - if (latestCommitTime.isPresent()) { - filteredFiles = hoodieTable.getBaseFileOnlyView() - .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp()) - .map(f -> Pair.of(partitionPath, f)) - .collect(toList()); - } - return filteredFiles.iterator(); - }) - .collect(); + context.setJobStatus(HoodieIndexUtils.class.getSimpleName(), "Load latest base files from all partitions"); + return context.flatMap(partitions, partitionPath -> { + Option latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline() + .filterCompletedInstants().lastInstant(); + List> filteredFiles = new ArrayList<>(); + if (latestCommitTime.isPresent()) { + filteredFiles = hoodieTable.getBaseFileOnlyView() + .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp()) + .map(f -> Pair.of(partitionPath, f)) + .collect(toList()); + } + return filteredFiles.stream(); + }, Math.max(partitions.size(), 1)); } /** diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/BloomIndexFileInfo.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BloomIndexFileInfo.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/BloomIndexFileInfo.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BloomIndexFileInfo.java diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/IndexFileFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IndexFileFilter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/IndexFileFilter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IndexFileFilter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedIndexFileFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedIndexFileFilter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedIndexFileFilter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedIndexFileFilter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/KeyRangeLookupTree.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/KeyRangeLookupTree.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/KeyRangeLookupTree.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/KeyRangeLookupTree.java diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/KeyRangeNode.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/KeyRangeNode.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/KeyRangeNode.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/KeyRangeNode.java diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/ListBasedGlobalIndexFileFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedGlobalIndexFileFilter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/ListBasedGlobalIndexFileFilter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedGlobalIndexFileFilter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/ListBasedIndexFileFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedIndexFileFilter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/ListBasedIndexFileFilter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedIndexFileFilter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/index/hbase/DefaultHBaseQPSResourceAllocator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/hbase/DefaultHBaseQPSResourceAllocator.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/index/hbase/DefaultHBaseQPSResourceAllocator.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/hbase/DefaultHBaseQPSResourceAllocator.java diff --git a/hudi-client/src/main/java/org/apache/hudi/index/hbase/HBaseIndexQPSResourceAllocator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/hbase/HBaseIndexQPSResourceAllocator.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/index/hbase/HBaseIndexQPSResourceAllocator.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/hbase/HBaseIndexQPSResourceAllocator.java diff --git a/hudi-client/src/main/java/org/apache/hudi/io/AppendHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java similarity index 73% rename from hudi-client/src/main/java/org/apache/hudi/io/AppendHandleFactory.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java index 4a5554bd9..5c54dce31 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/AppendHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java @@ -18,17 +18,17 @@ package org.apache.hudi.io; -import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -public class AppendHandleFactory extends WriteHandleFactory { +public class AppendHandleFactory extends WriteHandleFactory { @Override - public HoodieAppendHandle create(final HoodieWriteConfig hoodieConfig, final String commitTime, - final HoodieTable hoodieTable, final String partitionPath, - final String fileIdPrefix, final SparkTaskContextSupplier sparkTaskContextSupplier) { + public HoodieAppendHandle create(final HoodieWriteConfig hoodieConfig, final String commitTime, + final HoodieTable hoodieTable, final String partitionPath, + final String fileIdPrefix, final TaskContextSupplier sparkTaskContextSupplier) { return new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable, partitionPath, getNextFileId(fileIdPrefix), sparkTaskContextSupplier); diff --git a/hudi-client/src/main/java/org/apache/hudi/io/CreateHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java similarity index 70% rename from hudi-client/src/main/java/org/apache/hudi/io/CreateHandleFactory.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java index 68d8b4d58..67ebadb2d 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/CreateHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java @@ -18,19 +18,19 @@ package org.apache.hudi.io; -import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -public class CreateHandleFactory extends WriteHandleFactory { +public class CreateHandleFactory extends WriteHandleFactory { @Override - public HoodieWriteHandle create(final HoodieWriteConfig hoodieConfig, final String commitTime, - final HoodieTable hoodieTable, final String partitionPath, - final String fileIdPrefix, SparkTaskContextSupplier sparkTaskContextSupplier) { + public HoodieWriteHandle create(final HoodieWriteConfig hoodieConfig, final String commitTime, + final HoodieTable hoodieTable, final String partitionPath, + final String fileIdPrefix, TaskContextSupplier taskContextSupplier) { return new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, partitionPath, - getNextFileId(fileIdPrefix), sparkTaskContextSupplier); + getNextFileId(fileIdPrefix), taskContextSupplier); } } \ No newline at end of file diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java similarity index 95% rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index 7996a77f2..0c590fe88 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -19,8 +19,8 @@ package org.apache.hudi.io; import org.apache.hudi.avro.HoodieAvroUtils; -import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieDeltaWriteStat; @@ -40,7 +40,9 @@ import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import org.apache.hudi.common.table.view.TableFileSystemView.SliceView; +import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.SizeEstimator; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieAppendException; import org.apache.hudi.exception.HoodieUpsertException; @@ -51,7 +53,6 @@ import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.util.SizeEstimator; import java.io.IOException; import java.util.ArrayList; @@ -64,7 +65,7 @@ import java.util.concurrent.atomic.AtomicLong; /** * IO Operation to append data onto an existing file. */ -public class HoodieAppendHandle extends HoodieWriteHandle { +public class HoodieAppendHandle extends HoodieWriteHandle { private static final Logger LOG = LogManager.getLogger(HoodieAppendHandle.class); // This acts as the sequenceID for records written @@ -101,16 +102,19 @@ public class HoodieAppendHandle extends HoodieWri // Total number of new records inserted into the delta file private long insertRecordsWritten = 0; - public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, - String partitionPath, String fileId, Iterator> recordItr, SparkTaskContextSupplier sparkTaskContextSupplier) { - super(config, instantTime, partitionPath, fileId, hoodieTable, sparkTaskContextSupplier); + private SizeEstimator sizeEstimator; + + public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + String partitionPath, String fileId, Iterator> recordItr, TaskContextSupplier taskContextSupplier) { + super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier); writeStatus.setStat(new HoodieDeltaWriteStat()); this.fileId = fileId; this.recordItr = recordItr; + sizeEstimator = new DefaultSizeEstimator(); } - public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, - String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) { + public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + String partitionPath, String fileId, TaskContextSupplier sparkTaskContextSupplier) { this(config, instantTime, hoodieTable, partitionPath, fileId, null, sparkTaskContextSupplier); } @@ -134,7 +138,7 @@ public class HoodieAppendHandle extends HoodieWri writeStatus.setPartitionPath(partitionPath); writeStatus.getStat().setPartitionPath(partitionPath); writeStatus.getStat().setFileId(fileId); - averageRecordSize = SizeEstimator.estimate(record); + averageRecordSize = sizeEstimator.sizeEstimate(record); try { //save hoodie partition meta in the partition path HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, baseInstantTime, @@ -335,7 +339,7 @@ public class HoodieAppendHandle extends HoodieWri // Recompute averageRecordSize before writing a new block and update existing value with // avg of new and old LOG.info("AvgRecordSize => " + averageRecordSize); - averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2; + averageRecordSize = (averageRecordSize + sizeEstimator.sizeEstimate(record)) / 2; doAppend(header); estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; numberOfRecords = 0; diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java similarity index 87% rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java index 5deeae1d9..b2eaedd53 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java @@ -19,7 +19,7 @@ package org.apache.hudi.io; import org.apache.hudi.avro.HoodieAvroUtils; -import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; @@ -33,13 +33,13 @@ import org.apache.hudi.table.HoodieTable; * writing more than 1 skeleton file for the same bootstrap file. * @param HoodieRecordPayload */ -public class HoodieBootstrapHandle extends HoodieCreateHandle { +public class HoodieBootstrapHandle extends HoodieCreateHandle { - public HoodieBootstrapHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, - String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) { + public HoodieBootstrapHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) { super(config, commitTime, hoodieTable, partitionPath, fileId, Pair.of(HoodieAvroUtils.RECORD_KEY_SCHEMA, - HoodieAvroUtils.addMetadataFields(HoodieAvroUtils.RECORD_KEY_SCHEMA)), sparkTaskContextSupplier); + HoodieAvroUtils.addMetadataFields(HoodieAvroUtils.RECORD_KEY_SCHEMA)), taskContextSupplier); } @Override diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java similarity index 93% rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java index 99532780b..6a8e77350 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java @@ -19,8 +19,8 @@ package org.apache.hudi.io; import org.apache.avro.Schema; -import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; @@ -47,7 +47,7 @@ import java.io.IOException; import java.util.Iterator; import java.util.Map; -public class HoodieCreateHandle extends HoodieWriteHandle { +public class HoodieCreateHandle extends HoodieWriteHandle { private static final Logger LOG = LogManager.getLogger(HoodieCreateHandle.class); @@ -59,17 +59,17 @@ public class HoodieCreateHandle extends HoodieWri private Map> recordMap; private boolean useWriterSchema = false; - public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, - String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) { + public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) { this(config, instantTime, hoodieTable, partitionPath, fileId, getWriterSchemaIncludingAndExcludingMetadataPair(config), - sparkTaskContextSupplier); + taskContextSupplier); } - public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, String partitionPath, String fileId, Pair writerSchemaIncludingAndExcludingMetadataPair, - SparkTaskContextSupplier sparkTaskContextSupplier) { + TaskContextSupplier taskContextSupplier) { super(config, instantTime, partitionPath, fileId, hoodieTable, writerSchemaIncludingAndExcludingMetadataPair, - sparkTaskContextSupplier); + taskContextSupplier); writeStatus.setFileId(fileId); writeStatus.setPartitionPath(partitionPath); @@ -80,7 +80,7 @@ public class HoodieCreateHandle extends HoodieWri new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath)); partitionMetadata.trySave(getPartitionId()); createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension())); - this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, writerSchemaWithMetafields, this.sparkTaskContextSupplier); + this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, writerSchemaWithMetafields, this.taskContextSupplier); } catch (IOException e) { throw new HoodieInsertException("Failed to initialize HoodieStorageWriter for path " + path, e); } @@ -90,10 +90,10 @@ public class HoodieCreateHandle extends HoodieWri /** * Called by the compactor code path. */ - public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, String partitionPath, String fileId, Map> recordMap, - SparkTaskContextSupplier sparkTaskContextSupplier) { - this(config, instantTime, hoodieTable, partitionPath, fileId, sparkTaskContextSupplier); + TaskContextSupplier taskContextSupplier) { + this(config, instantTime, hoodieTable, partitionPath, fileId, taskContextSupplier); this.recordMap = recordMap; this.useWriterSchema = true; } diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieIOHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java similarity index 93% rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieIOHandle.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java index cff91f3b4..c6f9dddef 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieIOHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java @@ -24,14 +24,14 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hadoop.fs.FileSystem; -public abstract class HoodieIOHandle { +public abstract class HoodieIOHandle { protected final String instantTime; protected final HoodieWriteConfig config; protected final FileSystem fs; - protected final HoodieTable hoodieTable; + protected final HoodieTable hoodieTable; - HoodieIOHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable) { + HoodieIOHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable) { this.instantTime = instantTime; this.config = config; this.hoodieTable = hoodieTable; diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java similarity index 96% rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java index 3aa13987e..9194fc042 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java @@ -38,11 +38,11 @@ import scala.Tuple2; * * @param */ -public class HoodieKeyLocationFetchHandle extends HoodieReadHandle { +public class HoodieKeyLocationFetchHandle extends HoodieReadHandle { private final Pair partitionPathBaseFilePair; - public HoodieKeyLocationFetchHandle(HoodieWriteConfig config, HoodieTable hoodieTable, + public HoodieKeyLocationFetchHandle(HoodieWriteConfig config, HoodieTable hoodieTable, Pair partitionPathBaseFilePair) { super(config, null, hoodieTable, Pair.of(partitionPathBaseFilePair.getLeft(), partitionPathBaseFilePair.getRight().getFileId())); this.partitionPathBaseFilePair = partitionPathBaseFilePair; diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java similarity index 98% rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java index 4002b5a9e..ad84e3e97 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java @@ -42,7 +42,7 @@ import java.util.Set; /** * Takes a bunch of keys and returns ones that are present in the file group. */ -public class HoodieKeyLookupHandle extends HoodieReadHandle { +public class HoodieKeyLookupHandle extends HoodieReadHandle { private static final Logger LOG = LogManager.getLogger(HoodieKeyLookupHandle.class); @@ -54,7 +54,7 @@ public class HoodieKeyLookupHandle extends Hoodie private long totalKeysChecked; - public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable hoodieTable, + public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable hoodieTable, Pair partitionPathFilePair) { super(config, null, hoodieTable, partitionPathFilePair); this.tableType = hoodieTable.getMetaClient().getTableType(); diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java similarity index 93% rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java index 650237c79..77fef5c1d 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java @@ -18,9 +18,8 @@ package org.apache.hudi.io; -import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkConfigUtils; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; @@ -54,7 +53,7 @@ import java.util.Map; import java.util.Set; @SuppressWarnings("Duplicates") -public class HoodieMergeHandle extends HoodieWriteHandle { +public class HoodieMergeHandle extends HoodieWriteHandle { private static final Logger LOG = LogManager.getLogger(HoodieMergeHandle.class); @@ -71,9 +70,10 @@ public class HoodieMergeHandle extends HoodieWrit protected boolean useWriterSchema; private HoodieBaseFile baseFileToMerge; - public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, - Iterator> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) { - super(config, instantTime, partitionPath, fileId, hoodieTable, sparkTaskContextSupplier); + public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + Iterator> recordItr, String partitionPath, String fileId, + TaskContextSupplier taskContextSupplier) { + super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier); init(fileId, recordItr); init(fileId, partitionPath, hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId).get()); } @@ -81,10 +81,10 @@ public class HoodieMergeHandle extends HoodieWrit /** * Called by compactor code path. */ - public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, - Map> keyToNewRecords, String partitionPath, String fileId, - HoodieBaseFile dataFileToBeMerged, SparkTaskContextSupplier sparkTaskContextSupplier) { - super(config, instantTime, partitionPath, fileId, hoodieTable, sparkTaskContextSupplier); + public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + Map> keyToNewRecords, String partitionPath, String fileId, + HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) { + super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier); this.keyToNewRecords = keyToNewRecords; this.useWriterSchema = true; init(fileId, this.partitionPath, dataFileToBeMerged); @@ -134,7 +134,7 @@ public class HoodieMergeHandle extends HoodieWrit createMarkerFile(partitionPath, newFileName); // Create the writer for writing the new version file - fileWriter = createNewFileWriter(instantTime, newFilePath, hoodieTable, config, writerSchemaWithMetafields, sparkTaskContextSupplier); + fileWriter = createNewFileWriter(instantTime, newFilePath, hoodieTable, config, writerSchemaWithMetafields, taskContextSupplier); } catch (IOException io) { LOG.error("Error in update task at commit " + instantTime, io); writeStatus.setGlobalError(io); @@ -149,7 +149,7 @@ public class HoodieMergeHandle extends HoodieWrit private void init(String fileId, Iterator> newRecordsItr) { try { // Load the new records in a map - long memoryForMerge = SparkConfigUtils.getMaxMemoryPerPartitionMerge(config.getProps()); + long memoryForMerge = IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config.getProps()); LOG.info("MaxMemoryPerPartitionMerge => " + memoryForMerge); this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(), new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(writerSchema)); diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java similarity index 94% rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java index 2b58583f1..78fa9be69 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java @@ -28,9 +28,9 @@ import java.io.IOException; /** * Extract range information for a given file slice. */ -public class HoodieRangeInfoHandle extends HoodieReadHandle { +public class HoodieRangeInfoHandle extends HoodieReadHandle { - public HoodieRangeInfoHandle(HoodieWriteConfig config, HoodieTable hoodieTable, + public HoodieRangeInfoHandle(HoodieWriteConfig config, HoodieTable hoodieTable, Pair partitionPathFilePair) { super(config, null, hoodieTable, partitionPathFilePair); } diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieReadHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java similarity index 96% rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieReadHandle.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java index 8f2d4de71..a771c33c4 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieReadHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java @@ -34,11 +34,11 @@ import org.apache.hadoop.fs.Path; /** * Base class for read operations done logically on the file group. */ -public abstract class HoodieReadHandle extends HoodieIOHandle { +public abstract class HoodieReadHandle extends HoodieIOHandle { protected final Pair partitionPathFilePair; - public HoodieReadHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + public HoodieReadHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Pair partitionPathFilePair) { super(config, instantTime, hoodieTable); this.partitionPathFilePair = partitionPathFilePair; diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java similarity index 91% rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java index dda7b7257..71610b1aa 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java @@ -18,8 +18,8 @@ package org.apache.hudi.io; -import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -41,24 +41,24 @@ import java.util.Queue; * The implementation performs a merge-sort by comparing the key of the record being written to the list of * keys in newRecordKeys (sorted in-memory). */ -public class HoodieSortedMergeHandle extends HoodieMergeHandle { +public class HoodieSortedMergeHandle extends HoodieMergeHandle { private Queue newRecordKeysSorted = new PriorityQueue<>(); - public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, - Iterator> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) { - super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier); + public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + Iterator> recordItr, String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) { + super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier); newRecordKeysSorted.addAll(keyToNewRecords.keySet()); } /** * Called by compactor code path. */ - public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Map> keyToNewRecordsOrig, String partitionPath, String fileId, - HoodieBaseFile dataFileToBeMerged, SparkTaskContextSupplier sparkTaskContextSupplier) { + HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) { super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged, - sparkTaskContextSupplier); + taskContextSupplier); newRecordKeysSorted.addAll(keyToNewRecords.keySet()); } diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java similarity index 87% rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 5ea8c3802..85898bccc 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -19,8 +19,8 @@ package org.apache.hudi.io; import org.apache.hudi.avro.HoodieAvroUtils; -import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -49,7 +49,7 @@ import java.io.IOException; /** * Base class for all write operations logically performed at the file group level. */ -public abstract class HoodieWriteHandle extends HoodieIOHandle { +public abstract class HoodieWriteHandle extends HoodieIOHandle { private static final Logger LOG = LogManager.getLogger(HoodieWriteHandle.class); @@ -60,17 +60,17 @@ public abstract class HoodieWriteHandle extends H protected final String partitionPath; protected final String fileId; protected final String writeToken; - protected final SparkTaskContextSupplier sparkTaskContextSupplier; + protected final TaskContextSupplier taskContextSupplier; public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath, - String fileId, HoodieTable hoodieTable, SparkTaskContextSupplier sparkTaskContextSupplier) { + String fileId, HoodieTable hoodieTable, TaskContextSupplier taskContextSupplier) { this(config, instantTime, partitionPath, fileId, hoodieTable, - getWriterSchemaIncludingAndExcludingMetadataPair(config), sparkTaskContextSupplier); + getWriterSchemaIncludingAndExcludingMetadataPair(config), taskContextSupplier); } protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath, String fileId, - HoodieTable hoodieTable, Pair writerSchemaIncludingAndExcludingMetadataPair, - SparkTaskContextSupplier sparkTaskContextSupplier) { + HoodieTable hoodieTable, Pair writerSchemaIncludingAndExcludingMetadataPair, + TaskContextSupplier taskContextSupplier) { super(config, instantTime, hoodieTable); this.partitionPath = partitionPath; this.fileId = fileId; @@ -79,7 +79,7 @@ public abstract class HoodieWriteHandle extends H this.timer = new HoodieTimer().startTimer(); this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(), !hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction()); - this.sparkTaskContextSupplier = sparkTaskContextSupplier; + this.taskContextSupplier = taskContextSupplier; this.writeToken = makeWriteToken(); } @@ -179,19 +179,19 @@ public abstract class HoodieWriteHandle extends H } protected int getPartitionId() { - return sparkTaskContextSupplier.getPartitionIdSupplier().get(); + return taskContextSupplier.getPartitionIdSupplier().get(); } protected int getStageId() { - return sparkTaskContextSupplier.getStageIdSupplier().get(); + return taskContextSupplier.getStageIdSupplier().get(); } protected long getAttemptId() { - return sparkTaskContextSupplier.getAttemptIdSupplier().get(); + return taskContextSupplier.getAttemptIdSupplier().get(); } - protected HoodieFileWriter createNewFileWriter(String instantTime, Path path, HoodieTable hoodieTable, - HoodieWriteConfig config, Schema schema, SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException { - return HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, schema, sparkTaskContextSupplier); + protected HoodieFileWriter createNewFileWriter(String instantTime, Path path, HoodieTable hoodieTable, + HoodieWriteConfig config, Schema schema, TaskContextSupplier taskContextSupplier) throws IOException { + return HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, schema, taskContextSupplier); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java new file mode 100644 index 000000000..03719157f --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java @@ -0,0 +1,74 @@ +/* + * 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.io; + +import org.apache.hudi.client.common.EngineProperty; +import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.util.Option; + +import java.util.Properties; + +import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES; +import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION; +import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE; +import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES; +import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP; +import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP; +import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP; +import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP; + +public class IOUtils { + /** + * Dynamic calculation of max memory to use for for spillable map. user.available.memory = executor.memory * + * (1 - memory.fraction) spillable.available.memory = user.available.memory * hoodie.memory.fraction. Anytime + * the engine memory fractions/total memory is changed, the memory used for spillable map changes + * accordingly + */ + public static long getMaxMemoryAllowedForMerge(TaskContextSupplier context, String maxMemoryFraction) { + Option totalMemoryOpt = context.getProperty(EngineProperty.TOTAL_MEMORY_AVAILABLE); + Option memoryFractionOpt = context.getProperty(EngineProperty.MEMORY_FRACTION_IN_USE); + + if (totalMemoryOpt.isPresent() && memoryFractionOpt.isPresent()) { + long executorMemoryInBytes = Long.parseLong(totalMemoryOpt.get()); + double memoryFraction = Double.parseDouble(memoryFractionOpt.get()); + double maxMemoryFractionForMerge = Double.parseDouble(maxMemoryFraction); + double userAvailableMemory = executorMemoryInBytes * (1 - memoryFraction); + long maxMemoryForMerge = (long) Math.floor(userAvailableMemory * maxMemoryFractionForMerge); + return Math.max(DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES, maxMemoryForMerge); + } else { + return DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES; + } + } + + public static long getMaxMemoryPerPartitionMerge(TaskContextSupplier context, Properties properties) { + if (properties.containsKey(MAX_MEMORY_FOR_MERGE_PROP)) { + return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_MERGE_PROP)); + } + String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE); + return getMaxMemoryAllowedForMerge(context, fraction); + } + + public static long getMaxMemoryPerCompaction(TaskContextSupplier context, Properties properties) { + if (properties.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP)) { + return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_COMPACTION_PROP)); + } + String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION); + return getMaxMemoryAllowedForMerge(context, fraction); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/io/WriteHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java similarity index 79% rename from hudi-client/src/main/java/org/apache/hudi/io/WriteHandleFactory.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java index 7039b71e5..c66442a48 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/WriteHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java @@ -18,16 +18,16 @@ package org.apache.hudi.io; -import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -public abstract class WriteHandleFactory { +public abstract class WriteHandleFactory { private int numFilesWritten = 0; - public abstract HoodieWriteHandle create(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, - String partitionPath, String fileIdPrefix, SparkTaskContextSupplier sparkTaskContextSupplier); + public abstract HoodieWriteHandle create(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + String partitionPath, String fileIdPrefix, TaskContextSupplier taskContextSupplier); protected String getNextFileId(String idPfx) { return String.format("%s-%d", idPfx, numFilesWritten++); diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java similarity index 85% rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java index 1d4a9a28e..5f4eec088 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java @@ -19,7 +19,7 @@ package org.apache.hudi.io.storage; import org.apache.hudi.avro.HoodieAvroWriteSupport; -import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.fs.FSUtils; @@ -39,22 +39,22 @@ import static org.apache.hudi.common.model.HoodieFileFormat.HFILE; public class HoodieFileWriterFactory { - public static HoodieFileWriter getFileWriter( - String instantTime, Path path, HoodieTable hoodieTable, HoodieWriteConfig config, Schema schema, - SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException { + public static HoodieFileWriter getFileWriter( + String instantTime, Path path, HoodieTable hoodieTable, HoodieWriteConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { final String extension = FSUtils.getFileExtension(path.getName()); if (PARQUET.getFileExtension().equals(extension)) { - return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, sparkTaskContextSupplier); + return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier); } if (HFILE.getFileExtension().equals(extension)) { - return newHFileFileWriter(instantTime, path, config, schema, hoodieTable, sparkTaskContextSupplier); + return newHFileFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier); } throw new UnsupportedOperationException(extension + " format not supported yet."); } private static HoodieFileWriter newParquetFileWriter( String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable, - SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException { + TaskContextSupplier taskContextSupplier) throws IOException { BloomFilter filter = createBloomFilter(config); HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter); @@ -63,18 +63,18 @@ public class HoodieFileWriterFactory { config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(), hoodieTable.getHadoopConf(), config.getParquetCompressionRatio()); - return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, sparkTaskContextSupplier); + return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, taskContextSupplier); } private static HoodieFileWriter newHFileFileWriter( String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable, - SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException { + TaskContextSupplier taskContextSupplier) throws IOException { BloomFilter filter = createBloomFilter(config); HoodieHFileConfig hfileConfig = new HoodieHFileConfig(hoodieTable.getHadoopConf(), config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(), filter); - return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, sparkTaskContextSupplier); + return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, taskContextSupplier); } private static BloomFilter createBloomFilter(HoodieWriteConfig config) { diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java similarity index 94% rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java index c7f549a09..3684f9d4c 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java @@ -19,7 +19,7 @@ package org.apache.hudi.io.storage; import org.apache.hudi.avro.HoodieAvroUtils; -import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; @@ -60,7 +60,7 @@ public class HoodieHFileWriter getRecordKeyFieldNames(); - String getRecordKey(Row row); - - String getPartitionPath(Row row); - } diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/ConsoleMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/ConsoleMetricsReporter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/ConsoleMetricsReporter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/ConsoleMetricsReporter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/HoodieGauge.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieGauge.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/HoodieGauge.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieGauge.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/InMemoryMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/InMemoryMetricsReporter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/InMemoryMetricsReporter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/InMemoryMetricsReporter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/JmxMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxMetricsReporter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/JmxMetricsReporter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxMetricsReporter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/JmxReporterServer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxReporterServer.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/JmxReporterServer.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxReporterServer.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/Metrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/Metrics.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/MetricsGraphiteReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsGraphiteReporter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/MetricsGraphiteReporter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsGraphiteReporter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/MetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/MetricsReporter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/MetricsReporterType.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterType.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/MetricsReporterType.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterType.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/datadog/DatadogHttpClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogHttpClient.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/datadog/DatadogHttpClient.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogHttpClient.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/datadog/DatadogMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogMetricsReporter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/datadog/DatadogMetricsReporter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogMetricsReporter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayMetricsReporter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayMetricsReporter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayMetricsReporter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayReporter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayReporter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayReporter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/userdefined/AbstractUserDefinedMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/userdefined/AbstractUserDefinedMetricsReporter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/metrics/userdefined/AbstractUserDefinedMetricsReporter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/userdefined/AbstractUserDefinedMetricsReporter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java similarity index 75% rename from hudi-client/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java index 53aee2fb8..b571fd960 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java @@ -18,26 +18,21 @@ package org.apache.hudi.table; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; - -import org.apache.spark.api.java.JavaRDD; - /** * Repartition input records into at least expected number of output spark partitions. It should give below guarantees - * Output spark partition will have records from only one hoodie partition. - Average records per output spark * partitions should be almost equal to (#inputRecords / #outputSparkPartitions) to avoid possible skews. */ -public interface BulkInsertPartitioner { +public interface BulkInsertPartitioner { /** * Repartitions the input records into at least expected number of output spark partitions. * - * @param records Input Hoodie records in RDD - * @param outputSparkPartitions Expected number of output RDD partitions + * @param records Input Hoodie records + * @param outputSparkPartitions Expected number of output partitions * @return */ - JavaRDD> repartitionRecords(JavaRDD> records, int outputSparkPartitions); + I repartitionRecords(I records, int outputSparkPartitions); /** * @return {@code true} if the records within a RDD partition are sorted; {@code false} otherwise. diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java similarity index 72% rename from hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 5c824a6fc..afd1a3664 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -28,7 +28,8 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; -import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.fs.ConsistencyGuard; import org.apache.hudi.common.fs.ConsistencyGuard.FileVisibility; @@ -37,7 +38,6 @@ import org.apache.hudi.common.fs.FailSafeConsistencyGuard; import org.apache.hudi.common.fs.OptimisticConsistencyGuard; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -46,7 +46,6 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.SyncableFileSystemView; @@ -65,8 +64,6 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; 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 java.io.IOException; import java.io.Serializable; @@ -80,29 +77,37 @@ import java.util.stream.Stream; /** * Abstract implementation of a HoodieTable. + * + * @param Sub type of HoodieRecordPayload + * @param Type of inputs + * @param Type of keys + * @param Type of outputs */ -public abstract class HoodieTable implements Serializable { +public abstract class HoodieTable implements Serializable { private static final Logger LOG = LogManager.getLogger(HoodieTable.class); protected final HoodieWriteConfig config; protected final HoodieTableMetaClient metaClient; - protected final HoodieIndex index; + protected final HoodieIndex index; private SerializableConfiguration hadoopConfiguration; private transient FileSystemViewManager viewManager; - protected final SparkTaskContextSupplier sparkTaskContextSupplier = new SparkTaskContextSupplier(); + protected final TaskContextSupplier taskContextSupplier; - protected HoodieTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) { + protected HoodieTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { this.config = config; - this.hadoopConfiguration = new SerializableConfiguration(hadoopConf); - this.viewManager = FileSystemViewManager.createViewManager(new SerializableConfiguration(hadoopConf), + this.hadoopConfiguration = context.getHadoopConf(); + this.viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration, config.getViewStorageConfig()); this.metaClient = metaClient; - this.index = HoodieIndex.createIndex(config); + this.index = getIndex(config); + this.taskContextSupplier = context.getTaskContextSupplier(); } + protected abstract HoodieIndex getIndex(HoodieWriteConfig config); + private synchronized FileSystemViewManager getViewManager() { if (null == viewManager) { viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration, config.getViewStorageConfig()); @@ -110,114 +115,95 @@ public abstract class HoodieTable implements Seri return viewManager; } - public static HoodieTable create(HoodieWriteConfig config, Configuration hadoopConf) { - HoodieTableMetaClient metaClient = new HoodieTableMetaClient( - hadoopConf, - config.getBasePath(), - true, - config.getConsistencyGuardConfig(), - Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())) - ); - return HoodieTable.create(metaClient, config, hadoopConf); - } - - public static HoodieTable create(HoodieTableMetaClient metaClient, - HoodieWriteConfig config, - Configuration hadoopConf) { - switch (metaClient.getTableType()) { - case COPY_ON_WRITE: - return new HoodieCopyOnWriteTable<>(config, hadoopConf, metaClient); - case MERGE_ON_READ: - return new HoodieMergeOnReadTable<>(config, hadoopConf, metaClient); - default: - throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); - } - } - /** * Upsert a batch of new records into Hoodie table at the supplied instantTime. - * @param jsc Java Spark Context jsc + * @param context HoodieEngineContext * @param instantTime Instant Time for the action - * @param records JavaRDD of hoodieRecords to upsert + * @param records hoodieRecords to upsert * @return HoodieWriteMetadata */ - public abstract HoodieWriteMetadata upsert(JavaSparkContext jsc, String instantTime, - JavaRDD> records); + public abstract HoodieWriteMetadata upsert(HoodieEngineContext context, String instantTime, + I records); /** * Insert a batch of new records into Hoodie table at the supplied instantTime. - * @param jsc Java Spark Context jsc + * @param context HoodieEngineContext * @param instantTime Instant Time for the action - * @param records JavaRDD of hoodieRecords to upsert + * @param records hoodieRecords to upsert * @return HoodieWriteMetadata */ - public abstract HoodieWriteMetadata insert(JavaSparkContext jsc, String instantTime, - JavaRDD> records); + public abstract HoodieWriteMetadata insert(HoodieEngineContext context, String instantTime, + I records); /** * Bulk Insert a batch of new records into Hoodie table at the supplied instantTime. - * @param jsc Java Spark Context jsc + * @param context HoodieEngineContext * @param instantTime Instant Time for the action - * @param records JavaRDD of hoodieRecords to upsert + * @param records hoodieRecords to upsert * @param bulkInsertPartitioner User Defined Partitioner * @return HoodieWriteMetadata */ - public abstract HoodieWriteMetadata bulkInsert(JavaSparkContext jsc, String instantTime, - JavaRDD> records, Option bulkInsertPartitioner); + public abstract HoodieWriteMetadata bulkInsert(HoodieEngineContext context, String instantTime, + I records, Option> bulkInsertPartitioner); /** * Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be * de-duped and non existent keys will be removed before deleting. * - * @param jsc Java Spark Context jsc + * @param context HoodieEngineContext * @param instantTime Instant Time for the action * @param keys {@link List} of {@link HoodieKey}s to be deleted * @return HoodieWriteMetadata */ - public abstract HoodieWriteMetadata delete(JavaSparkContext jsc, String instantTime, JavaRDD keys); + public abstract HoodieWriteMetadata delete(HoodieEngineContext context, String instantTime, K keys); /** * Upserts the given prepared records into the Hoodie table, at the supplied instantTime. *

* This implementation requires that the input records are already tagged, and de-duped if needed. - * @param jsc Java Spark Context jsc + * @param context HoodieEngineContext * @param instantTime Instant Time for the action - * @param preppedRecords JavaRDD of hoodieRecords to upsert + * @param preppedRecords hoodieRecords to upsert * @return HoodieWriteMetadata */ - public abstract HoodieWriteMetadata upsertPrepped(JavaSparkContext jsc, String instantTime, - JavaRDD> preppedRecords); + public abstract HoodieWriteMetadata upsertPrepped(HoodieEngineContext context, String instantTime, + I preppedRecords); /** * Inserts the given prepared records into the Hoodie table, at the supplied instantTime. *

* This implementation requires that the input records are already tagged, and de-duped if needed. - * @param jsc Java Spark Context jsc + * @param context HoodieEngineContext * @param instantTime Instant Time for the action - * @param preppedRecords JavaRDD of hoodieRecords to upsert + * @param preppedRecords hoodieRecords to upsert * @return HoodieWriteMetadata */ - public abstract HoodieWriteMetadata insertPrepped(JavaSparkContext jsc, String instantTime, - JavaRDD> preppedRecords); + public abstract HoodieWriteMetadata insertPrepped(HoodieEngineContext context, String instantTime, + I preppedRecords); /** * Bulk Insert the given prepared records into the Hoodie table, at the supplied instantTime. *

* This implementation requires that the input records are already tagged, and de-duped if needed. - * @param jsc Java Spark Context jsc + * @param context HoodieEngineContext * @param instantTime Instant Time for the action - * @param preppedRecords JavaRDD of hoodieRecords to upsert + * @param preppedRecords hoodieRecords to upsert * @param bulkInsertPartitioner User Defined Partitioner * @return HoodieWriteMetadata */ - public abstract HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime, - JavaRDD> preppedRecords, Option bulkInsertPartitioner); + public abstract HoodieWriteMetadata bulkInsertPrepped(HoodieEngineContext context, String instantTime, + I preppedRecords, Option> bulkInsertPartitioner); /** - * Logically delete all existing records and Insert a batch of new records into Hoodie table at the supplied instantTime. + * Replaces all the existing records and inserts the specified new records into Hoodie table at the supplied instantTime, + * for the partition paths contained in input records. + * + * @param context HoodieEngineContext + * @param instantTime Instant time for the replace action + * @param records input records + * @return HoodieWriteMetadata */ - public abstract HoodieWriteMetadata insertOverwrite(JavaSparkContext jsc, String instantTime, - JavaRDD> records); + public abstract HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String instantTime, I records); public HoodieWriteConfig getConfig() { return config; @@ -315,51 +301,51 @@ public abstract class HoodieTable implements Seri /** * Return the index. */ - public HoodieIndex getIndex() { + public HoodieIndex getIndex() { return index; } /** * Schedule compaction for the instant time. * - * @param jsc Spark Context + * @param context HoodieEngineContext * @param instantTime Instant Time for scheduling compaction * @param extraMetadata additional metadata to write into plan * @return */ - public abstract Option scheduleCompaction(JavaSparkContext jsc, + public abstract Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata); /** * Run Compaction on the table. Compaction arranges the data so that it is optimized for data access. * - * @param jsc Spark Context + * @param context HoodieEngineContext * @param compactionInstantTime Instant Time */ - public abstract HoodieWriteMetadata compact(JavaSparkContext jsc, + public abstract HoodieWriteMetadata compact(HoodieEngineContext context, String compactionInstantTime); /** * Perform metadata/full bootstrap of a Hudi table. - * @param jsc JavaSparkContext + * @param context HoodieEngineContext * @param extraMetadata Additional Metadata for storing in commit file. * @return HoodieBootstrapWriteMetadata */ - public abstract HoodieBootstrapWriteMetadata bootstrap(JavaSparkContext jsc, Option> extraMetadata); + public abstract HoodieBootstrapWriteMetadata bootstrap(HoodieEngineContext context, Option> extraMetadata); /** * Perform rollback of bootstrap of a Hudi table. - * @param jsc JavaSparkContext + * @param context HoodieEngineContext */ - public abstract void rollbackBootstrap(JavaSparkContext jsc, String instantTime); + public abstract void rollbackBootstrap(HoodieEngineContext context, String instantTime); /** * Executes a new clean action. * * @return information on cleaned file slices */ - public abstract HoodieCleanMetadata clean(JavaSparkContext jsc, String cleanInstantTime); + public abstract HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime); /** * Rollback the (inflight/committed) record changes with the given commit time. @@ -371,7 +357,7 @@ public abstract class HoodieTable implements Seri * (4) Finally delete .commit or .inflight file, if deleteInstants = true * */ - public abstract HoodieRollbackMetadata rollback(JavaSparkContext jsc, + public abstract HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollbackInstantTime, HoodieInstant commitInstant, boolean deleteInstants); @@ -380,7 +366,7 @@ public abstract class HoodieTable implements Seri * Create a savepoint at the specified instant, so that the table can be restored * to this point-in-timeline later if needed. */ - public abstract HoodieSavepointMetadata savepoint(JavaSparkContext jsc, + public abstract HoodieSavepointMetadata savepoint(HoodieEngineContext context, String instantToSavepoint, String user, String comment); @@ -389,54 +375,53 @@ public abstract class HoodieTable implements Seri * Restore the table to the given instant. Note that this is a admin table recovery operation * that would cause any running queries that are accessing file slices written after the instant to fail. */ - public abstract HoodieRestoreMetadata restore(JavaSparkContext jsc, + public abstract HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore); /** * Finalize the written data onto storage. Perform any final cleanups. * - * @param jsc Spark Context + * @param context HoodieEngineContext * @param stats List of HoodieWriteStats * @throws HoodieIOException if some paths can't be finalized on storage */ - public void finalizeWrite(JavaSparkContext jsc, String instantTs, List stats) throws HoodieIOException { - reconcileAgainstMarkers(jsc, instantTs, stats, config.getConsistencyGuardConfig().isConsistencyCheckEnabled()); + public void finalizeWrite(HoodieEngineContext context, String instantTs, List stats) throws HoodieIOException { + reconcileAgainstMarkers(context, instantTs, stats, config.getConsistencyGuardConfig().isConsistencyCheckEnabled()); } - private void deleteInvalidFilesByPartitions(JavaSparkContext jsc, Map>> invalidFilesByPartition) { + private void deleteInvalidFilesByPartitions(HoodieEngineContext context, Map>> invalidFilesByPartition) { // Now delete partially written files - jsc.parallelize(new ArrayList<>(invalidFilesByPartition.values()), config.getFinalizeWriteParallelism()) - .map(partitionWithFileList -> { - final FileSystem fileSystem = metaClient.getFs(); - LOG.info("Deleting invalid data files=" + partitionWithFileList); - if (partitionWithFileList.isEmpty()) { - return true; - } - // Delete - partitionWithFileList.stream().map(Pair::getValue).forEach(file -> { - try { - fileSystem.delete(new Path(file), false); - } catch (IOException e) { - throw new HoodieIOException(e.getMessage(), e); - } - }); + context.map(new ArrayList<>(invalidFilesByPartition.values()), partitionWithFileList -> { + final FileSystem fileSystem = metaClient.getFs(); + LOG.info("Deleting invalid data files=" + partitionWithFileList); + if (partitionWithFileList.isEmpty()) { + return true; + } + // Delete + partitionWithFileList.stream().map(Pair::getValue).forEach(file -> { + try { + fileSystem.delete(new Path(file), false); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }); - return true; - }).collect(); + return true; + }, config.getFinalizeWriteParallelism()); } /** * Reconciles WriteStats and marker files to detect and safely delete duplicate data files created because of Spark * retries. * - * @param jsc Spark Context + * @param context HoodieEngineContext * @param instantTs Instant Timestamp * @param stats Hoodie Write Stat * @param consistencyCheckEnabled Consistency Check Enabled * @throws HoodieIOException */ - protected void reconcileAgainstMarkers(JavaSparkContext jsc, + protected void reconcileAgainstMarkers(HoodieEngineContext context, String instantTs, List stats, boolean consistencyCheckEnabled) throws HoodieIOException { @@ -452,7 +437,7 @@ public abstract class HoodieTable implements Seri } // we are not including log appends here, since they are already fail-safe. - Set invalidDataPaths = markers.createdAndMergedDataPaths(jsc, config.getFinalizeWriteParallelism()); + Set invalidDataPaths = markers.createdAndMergedDataPaths(context, config.getFinalizeWriteParallelism()); Set validDataPaths = stats.stream() .map(HoodieWriteStat::getPath) .filter(p -> p.endsWith(this.getBaseFileExtension())) @@ -471,17 +456,17 @@ public abstract class HoodieTable implements Seri // Otherwise, we may miss deleting such files. If files are not found even after retries, fail the commit if (consistencyCheckEnabled) { // This will either ensure all files to be deleted are present. - waitForAllFiles(jsc, invalidPathsByPartition, FileVisibility.APPEAR); + waitForAllFiles(context, invalidPathsByPartition, FileVisibility.APPEAR); } // Now delete partially written files - jsc.setJobGroup(this.getClass().getSimpleName(), "Delete all partially written files"); - deleteInvalidFilesByPartitions(jsc, invalidPathsByPartition); + context.setJobStatus(this.getClass().getSimpleName(), "Delete all partially written files"); + deleteInvalidFilesByPartitions(context, invalidPathsByPartition); // Now ensure the deleted files disappear if (consistencyCheckEnabled) { // This will either ensure all files to be deleted are absent. - waitForAllFiles(jsc, invalidPathsByPartition, FileVisibility.DISAPPEAR); + waitForAllFiles(context, invalidPathsByPartition, FileVisibility.DISAPPEAR); } } } catch (IOException ioe) { @@ -492,18 +477,17 @@ public abstract class HoodieTable implements Seri /** * Ensures all files passed either appear or disappear. * - * @param jsc JavaSparkContext + * @param context HoodieEngineContext * @param groupByPartition Files grouped by partition * @param visibility Appear/Disappear */ - private void waitForAllFiles(JavaSparkContext jsc, Map>> groupByPartition, FileVisibility visibility) { + private void waitForAllFiles(HoodieEngineContext context, Map>> groupByPartition, FileVisibility visibility) { // This will either ensure all files to be deleted are present. - jsc.setJobGroup(this.getClass().getSimpleName(), "Wait for all files to appear/disappear"); + context.setJobStatus(this.getClass().getSimpleName(), "Wait for all files to appear/disappear"); boolean checkPassed = - jsc.parallelize(new ArrayList<>(groupByPartition.entrySet()), config.getFinalizeWriteParallelism()) - .map(partitionWithFileList -> waitForCondition(partitionWithFileList.getKey(), - partitionWithFileList.getValue().stream(), visibility)) - .collect().stream().allMatch(x -> x); + context.map(new ArrayList<>(groupByPartition.entrySet()), partitionWithFileList -> waitForCondition(partitionWithFileList.getKey(), + partitionWithFileList.getValue().stream(), visibility), config.getFinalizeWriteParallelism()) + .stream().allMatch(x -> x); if (!checkPassed) { throw new HoodieIOException("Consistency check failed to ensure all files " + visibility); } @@ -535,8 +519,8 @@ public abstract class HoodieTable implements Seri } } - public SparkTaskContextSupplier getSparkTaskContextSupplier() { - return sparkTaskContextSupplier; + public TaskContextSupplier getTaskContextSupplier() { + return taskContextSupplier; } /** @@ -544,7 +528,6 @@ public abstract class HoodieTable implements Seri * * When inserting/updating data, we read records using the last used schema and convert them to the * GenericRecords with writerSchema. Hence, we need to ensure that this conversion can take place without errors. - * */ private void validateSchema() throws HoodieUpsertException, HoodieInsertException { diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java similarity index 93% rename from hudi-client/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java index 5ce0aedcd..af2c2782e 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java @@ -22,14 +22,16 @@ import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; +import org.apache.hudi.client.ReplaceArchivalHelper; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.model.ActionType; import org.apache.hudi.common.model.HoodieArchivedLogFile; +import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieRollingStatMetadata; @@ -55,7 +57,6 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.io.FileNotFoundException; import java.io.IOException; @@ -75,7 +76,7 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN_O /** * Archiver to bound the growth of files under .hoodie meta path. */ -public class HoodieTimelineArchiveLog { +public class HoodieTimelineArchiveLog { private static final Logger LOG = LogManager.getLogger(HoodieTimelineArchiveLog.class); @@ -84,12 +85,12 @@ public class HoodieTimelineArchiveLog { private Writer writer; private final int maxInstantsToKeep; private final int minInstantsToKeep; - private final HoodieTable table; + private final HoodieTable table; private final HoodieTableMetaClient metaClient; - public HoodieTimelineArchiveLog(HoodieWriteConfig config, Configuration configuration) { + public HoodieTimelineArchiveLog(HoodieWriteConfig config, HoodieTable table) { this.config = config; - this.table = HoodieTable.create(config, configuration); + this.table = table; this.metaClient = table.getMetaClient(); this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); this.maxInstantsToKeep = config.getMaxCommitsToKeep(); @@ -123,7 +124,7 @@ public class HoodieTimelineArchiveLog { /** * Check if commits need to be archived. If yes, archive commits. */ - public boolean archiveIfRequired(JavaSparkContext jsc) throws IOException { + public boolean archiveIfRequired(HoodieEngineContext context) throws IOException { try { List instantsToArchive = getInstantsToArchive().collect(Collectors.toList()); @@ -131,7 +132,7 @@ public class HoodieTimelineArchiveLog { if (!instantsToArchive.isEmpty()) { this.writer = openWriter(); LOG.info("Archiving instants " + instantsToArchive); - archive(jsc, instantsToArchive); + archive(context, instantsToArchive); LOG.info("Deleting archived instants " + instantsToArchive); success = deleteArchivedInstants(instantsToArchive); } else { @@ -269,20 +270,20 @@ public class HoodieTimelineArchiveLog { return success; } - public void archive(JavaSparkContext jsc, List instants) throws HoodieCommitException { + public void archive(HoodieEngineContext context, List instants) throws HoodieCommitException { try { HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants(); Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema(); LOG.info("Wrapper schema " + wrapperSchema.toString()); List records = new ArrayList<>(); for (HoodieInstant hoodieInstant : instants) { - boolean deleteSuccess = deleteReplacedFileGroups(jsc, hoodieInstant); + boolean deleteSuccess = deleteReplacedFileGroups(context, hoodieInstant); if (!deleteSuccess) { // throw error and stop archival if deleting replaced file groups failed. throw new HoodieCommitException("Unable to delete file(s) for " + hoodieInstant.getFileName()); } try { - deleteAnyLeftOverMarkerFiles(jsc, hoodieInstant); + deleteAnyLeftOverMarkerFiles(context, hoodieInstant); records.add(convertToAvroRecord(commitTimeline, hoodieInstant)); if (records.size() >= this.config.getCommitArchivalBatchSize()) { writeToFile(wrapperSchema, records); @@ -300,14 +301,14 @@ public class HoodieTimelineArchiveLog { } } - private void deleteAnyLeftOverMarkerFiles(JavaSparkContext jsc, HoodieInstant instant) { + private void deleteAnyLeftOverMarkerFiles(HoodieEngineContext context, HoodieInstant instant) { MarkerFiles markerFiles = new MarkerFiles(table, instant.getTimestamp()); - if (markerFiles.deleteMarkerDir(jsc, config.getMarkersDeleteParallelism())) { + if (markerFiles.deleteMarkerDir(context, config.getMarkersDeleteParallelism())) { LOG.info("Cleaned up left over marker directory for instant :" + instant); } } - private boolean deleteReplacedFileGroups(JavaSparkContext jsc, HoodieInstant instant) { + private boolean deleteReplacedFileGroups(HoodieEngineContext context, HoodieInstant instant) { if (!instant.isCompleted() || !HoodieTimeline.REPLACE_COMMIT_ACTION.equals(instant.getAction())) { // only delete files for completed replace instants return true; @@ -315,7 +316,7 @@ public class HoodieTimelineArchiveLog { TableFileSystemView fileSystemView = this.table.getFileSystemView(); List replacedPartitions = getReplacedPartitions(instant); - return ReplaceArchivalHelper.deleteReplacedFileGroups(jsc, metaClient, fileSystemView, instant, replacedPartitions); + return ReplaceArchivalHelper.deleteReplacedFileGroups(context, metaClient, fileSystemView, instant, replacedPartitions); } private List getReplacedPartitions(HoodieInstant instant) { diff --git a/hudi-client/src/main/java/org/apache/hudi/table/MarkerFiles.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java similarity index 90% rename from hudi-client/src/main/java/org/apache/hudi/table/MarkerFiles.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java index 9577ceab9..8826204cd 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/MarkerFiles.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java @@ -18,6 +18,7 @@ package org.apache.hudi.table; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; @@ -33,7 +34,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; import java.io.Serializable; @@ -63,16 +63,16 @@ public class MarkerFiles implements Serializable { this.basePath = basePath; } - public MarkerFiles(HoodieTable table, String instantTime) { + public MarkerFiles(HoodieTable table, String instantTime) { this(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), table.getMetaClient().getMarkerFolderPath(instantTime), instantTime); } - public void quietDeleteMarkerDir(JavaSparkContext jsc, int parallelism) { + public void quietDeleteMarkerDir(HoodieEngineContext context, int parallelism) { try { - deleteMarkerDir(jsc, parallelism); + deleteMarkerDir(context, parallelism); } catch (HoodieIOException ioe) { LOG.warn("Error deleting marker directory for instant " + instantTime, ioe); } @@ -81,10 +81,10 @@ public class MarkerFiles implements Serializable { /** * Delete Marker directory corresponding to an instant. * - * @param jsc Java Spark Context. - * @param parallelism Spark parallelism for deletion. + * @param context HoodieEngineContext. + * @param parallelism parallelism for deletion. */ - public boolean deleteMarkerDir(JavaSparkContext jsc, int parallelism) { + public boolean deleteMarkerDir(HoodieEngineContext context, int parallelism) { try { if (fs.exists(markerDirPath)) { FileStatus[] fileStatuses = fs.listStatus(markerDirPath); @@ -95,11 +95,11 @@ public class MarkerFiles implements Serializable { if (markerDirSubPaths.size() > 0) { SerializableConfiguration conf = new SerializableConfiguration(fs.getConf()); parallelism = Math.min(markerDirSubPaths.size(), parallelism); - jsc.parallelize(markerDirSubPaths, parallelism).foreach(subPathStr -> { + context.foreach(markerDirSubPaths, subPathStr -> { Path subPath = new Path(subPathStr); FileSystem fileSystem = subPath.getFileSystem(conf.get()); fileSystem.delete(subPath, true); - }); + }, parallelism); } boolean result = fs.delete(markerDirPath, true); @@ -116,7 +116,7 @@ public class MarkerFiles implements Serializable { return fs.exists(markerDirPath); } - public Set createdAndMergedDataPaths(JavaSparkContext jsc, int parallelism) throws IOException { + public Set createdAndMergedDataPaths(HoodieEngineContext context, int parallelism) throws IOException { Set dataFiles = new HashSet<>(); FileStatus[] topLevelStatuses = fs.listStatus(markerDirPath); @@ -135,7 +135,7 @@ public class MarkerFiles implements Serializable { if (subDirectories.size() > 0) { parallelism = Math.min(subDirectories.size(), parallelism); SerializableConfiguration serializedConf = new SerializableConfiguration(fs.getConf()); - dataFiles.addAll(jsc.parallelize(subDirectories, parallelism).flatMap(directory -> { + dataFiles.addAll(context.flatMap(subDirectories, directory -> { Path path = new Path(directory); FileSystem fileSystem = path.getFileSystem(serializedConf.get()); RemoteIterator itr = fileSystem.listFiles(path, true); @@ -147,8 +147,8 @@ public class MarkerFiles implements Serializable { result.add(translateMarkerToDataPath(pathStr)); } } - return result.iterator(); - }).collect()); + return result.stream(); + }, parallelism)); } return dataFiles; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/WorkloadProfile.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/WorkloadProfile.java new file mode 100644 index 000000000..a56710bfb --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/WorkloadProfile.java @@ -0,0 +1,73 @@ +/* + * 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.table; + +import org.apache.hudi.common.util.collection.Pair; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Set; + +/** + * Information about incoming records for upsert/insert obtained either via sampling or introspecting the data fully. + *

+ * TODO(vc): Think about obtaining this directly from index.tagLocation + */ +public class WorkloadProfile implements Serializable { + + /** + * Computed workload profile. + */ + protected final HashMap partitionPathStatMap; + + /** + * Global workloadStat. + */ + protected final WorkloadStat globalStat; + + public WorkloadProfile(Pair, WorkloadStat> profile) { + this.partitionPathStatMap = profile.getLeft(); + this.globalStat = profile.getRight(); + } + + public WorkloadStat getGlobalStat() { + return globalStat; + } + + public Set getPartitionPaths() { + return partitionPathStatMap.keySet(); + } + + public HashMap getPartitionPathStatMap() { + return partitionPathStatMap; + } + + public WorkloadStat getWorkloadStat(String partitionPath) { + return partitionPathStatMap.get(partitionPath); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("WorkloadProfile {"); + sb.append("globalStat=").append(globalStat).append(", "); + sb.append("partitionStat=").append(partitionPathStatMap); + sb.append('}'); + return sb.toString(); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/WorkloadStat.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/WorkloadStat.java similarity index 94% rename from hudi-client/src/main/java/org/apache/hudi/table/WorkloadStat.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/WorkloadStat.java index f69e59901..6fdb217a0 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/WorkloadStat.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/WorkloadStat.java @@ -39,11 +39,11 @@ public class WorkloadStat implements Serializable { updateLocationToCount = new HashMap<>(); } - long addInserts(long numInserts) { + public long addInserts(long numInserts) { return this.numInserts += numInserts; } - long addUpdates(HoodieRecordLocation location, long numUpdates) { + public long addUpdates(HoodieRecordLocation location, long numUpdates) { updateLocationToCount.put(location.getFileId(), Pair.of(location.getInstantTime(), numUpdates)); return this.numUpdates += numUpdates; } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java similarity index 70% rename from hudi-client/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java index 9ee412623..96189e7cf 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java @@ -21,24 +21,25 @@ package org.apache.hudi.table.action; import java.io.Serializable; import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.spark.api.java.JavaSparkContext; -public abstract class BaseActionExecutor implements Serializable { +public abstract class BaseActionExecutor implements Serializable { - protected final transient JavaSparkContext jsc; + protected final transient HoodieEngineContext context; protected final transient Configuration hadoopConf; protected final HoodieWriteConfig config; - protected final HoodieTable table; + protected final HoodieTable table; protected final String instantTime; - public BaseActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, String instantTime) { - this.jsc = jsc; - this.hadoopConf = jsc.hadoopConfiguration(); + public BaseActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime) { + this.context = context; + this.hadoopConf = context.getHadoopConf().get(); this.config = config; this.table = table; this.instantTime = instantTime; diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java similarity index 92% rename from hudi-client/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java index 03ddba36d..5ef204f97 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java @@ -18,11 +18,9 @@ package org.apache.hudi.table.action; -import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.util.Option; -import org.apache.spark.api.java.JavaRDD; import java.time.Duration; import java.util.Collections; @@ -32,9 +30,9 @@ import java.util.Map; /** * Contains metadata, write-statuses and latency times corresponding to a commit/delta-commit action. */ -public class HoodieWriteMetadata { +public class HoodieWriteMetadata { - private JavaRDD writeStatuses; + private O writeStatuses; private Option indexLookupDuration = Option.empty(); // Will be set when auto-commit happens @@ -48,7 +46,7 @@ public class HoodieWriteMetadata { public HoodieWriteMetadata() { } - public JavaRDD getWriteStatuses() { + public O getWriteStatuses() { return writeStatuses; } @@ -56,7 +54,7 @@ public class HoodieWriteMetadata { return commitMetadata; } - public void setWriteStatuses(JavaRDD writeStatuses) { + public void setWriteStatuses(O writeStatuses) { this.writeStatuses = writeStatuses; } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java similarity index 94% rename from hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java index cb4bc2d7b..ade2b1885 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java @@ -26,13 +26,12 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.bootstrap.FileStatusUtils; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.collection.Pair; -import org.apache.spark.api.java.JavaSparkContext; - import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -47,12 +46,12 @@ public class BootstrapUtils { * Returns leaf folders with files under a path. * @param metaClient Hoodie table metadata client * @param fs File System - * @param jsc Java spark context + * @param context JHoodieEngineContext * @return list of partition paths with files under them. * @throws IOException */ public static List>> getAllLeafFoldersWithFiles(HoodieTableMetaClient metaClient, - FileSystem fs, String basePathStr, JavaSparkContext jsc) throws IOException { + FileSystem fs, String basePathStr, HoodieEngineContext context) throws IOException { final Path basePath = new Path(basePathStr); final String baseFileExtension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension(); final Map> levelToPartitions = new HashMap<>(); @@ -77,7 +76,7 @@ public class BootstrapUtils { } if (subDirectories.size() > 0) { - result.addAll(jsc.parallelize(subDirectories, subDirectories.size()).flatMap(directory -> { + result.addAll(context.flatMap(subDirectories, directory -> { PathFilter pathFilter = getFilePathFilter(baseFileExtension); Path path = new Path(directory); FileSystem fileSystem = path.getFileSystem(new Configuration()); @@ -92,8 +91,8 @@ public class BootstrapUtils { res.add(Pair.of(hoodieFileStatus, Pair.of(level, relativePath))); } } - return res.iterator(); - }).collect()); + return res.stream(); + }, subDirectories.size())); } result.forEach(val -> { diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java similarity index 77% rename from hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java index 4e6167ecb..0639876a6 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java @@ -24,22 +24,22 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; /** * Write Result for a bootstrap operation. */ -public class HoodieBootstrapWriteMetadata { +public class HoodieBootstrapWriteMetadata { - private final Option metadataBootstrapResult; - private final Option fullBootstrapResult; + private final Option> metadataBootstrapResult; + private final Option> fullBootstrapResult; - public HoodieBootstrapWriteMetadata(Option metadataBootstrapResult, - Option fullBootstrapResult) { + public HoodieBootstrapWriteMetadata(Option> metadataBootstrapResult, + Option> fullBootstrapResult) { this.metadataBootstrapResult = metadataBootstrapResult; this.fullBootstrapResult = fullBootstrapResult; } - public Option getMetadataBootstrapResult() { + public Option> getMetadataBootstrapResult() { return metadataBootstrapResult; } - public Option getFullBootstrapResult() { + public Option> getFullBootstrapResult() { return fullBootstrapResult; } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java similarity index 59% rename from hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java index 52614476e..588437c51 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java @@ -25,8 +25,9 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCleanFileInfo; import org.apache.hudi.common.HoodieCleanStat; -import org.apache.hudi.common.model.CleanFileInfo; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -42,36 +43,31 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.PairFlatMapFunction; -import scala.Tuple2; import java.io.FileNotFoundException; import java.io.IOException; -import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -public class CleanActionExecutor extends BaseActionExecutor { +public abstract class BaseCleanActionExecutor extends BaseActionExecutor { private static final long serialVersionUID = 1L; - private static final Logger LOG = LogManager.getLogger(CleanActionExecutor.class); + private static final Logger LOG = LogManager.getLogger(BaseCleanActionExecutor.class); - public CleanActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, String instantTime) { - super(jsc, config, table, instantTime); + public BaseCleanActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime) { + super(context, config, table, instantTime); } /** * Generates List of files to be cleaned. * - * @param jsc JavaSparkContext + * @param context HoodieEngineContext * @return Cleaner Plan */ - HoodieCleanerPlan requestClean(JavaSparkContext jsc) { + HoodieCleanerPlan requestClean(HoodieEngineContext context) { try { - CleanPlanner planner = new CleanPlanner<>(table, config); + CleanPlanner planner = new CleanPlanner<>(table, config); Option earliestInstant = planner.getEarliestCommitToRetain(); List partitionsToClean = planner.getPartitionPathsToClean(earliestInstant); @@ -83,11 +79,11 @@ public class CleanActionExecutor extends BaseActionExecutor int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism()); LOG.info("Using cleanerParallelism: " + cleanerParallelism); - jsc.setJobGroup(this.getClass().getSimpleName(), "Generates list of file slices to be cleaned"); - Map> cleanOps = jsc - .parallelize(partitionsToClean, cleanerParallelism) - .map(partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean))) - .collect().stream() + context.setJobStatus(this.getClass().getSimpleName(), "Generates list of file slices to be cleaned"); + + Map> cleanOps = context + .map(partitionsToClean, partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean)), cleanerParallelism) + .stream() .collect(Collectors.toMap(Pair::getKey, y -> CleanerUtils.convertToHoodieCleanFileInfoList(y.getValue()))); return new HoodieCleanerPlan(earliestInstant @@ -99,37 +95,7 @@ public class CleanActionExecutor extends BaseActionExecutor } } - private static PairFlatMapFunction>, String, PartitionCleanStat> - deleteFilesFunc(HoodieTable table) { - return (PairFlatMapFunction>, String, PartitionCleanStat>) iter -> { - Map partitionCleanStatMap = new HashMap<>(); - FileSystem fs = table.getMetaClient().getFs(); - while (iter.hasNext()) { - Tuple2 partitionDelFileTuple = iter.next(); - String partitionPath = partitionDelFileTuple._1(); - Path deletePath = new Path(partitionDelFileTuple._2().getFilePath()); - String deletePathStr = deletePath.toString(); - Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr); - if (!partitionCleanStatMap.containsKey(partitionPath)) { - partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath)); - } - boolean isBootstrapBasePathFile = partitionDelFileTuple._2().isBootstrapBaseFile(); - PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath); - if (isBootstrapBasePathFile) { - // For Bootstrap Base file deletions, store the full file path. - partitionCleanStat.addDeleteFilePatterns(deletePath.toString(), true); - partitionCleanStat.addDeletedFileResult(deletePath.toString(), deletedFileResult, true); - } else { - partitionCleanStat.addDeleteFilePatterns(deletePath.getName(), false); - partitionCleanStat.addDeletedFileResult(deletePath.getName(), deletedFileResult, false); - } - } - return partitionCleanStatMap.entrySet().stream().map(e -> new Tuple2<>(e.getKey(), e.getValue())) - .collect(Collectors.toList()).iterator(); - }; - } - - private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) throws IOException { + protected static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) throws IOException { Path deletePath = new Path(deletePathStr); LOG.debug("Working on delete path :" + deletePath); try { @@ -150,45 +116,7 @@ public class CleanActionExecutor extends BaseActionExecutor * * @throws IllegalArgumentException if unknown cleaning policy is provided */ - List clean(JavaSparkContext jsc, HoodieCleanerPlan cleanerPlan) { - int cleanerParallelism = Math.min( - (int) (cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).count()), - config.getCleanerParallelism()); - LOG.info("Using cleanerParallelism: " + cleanerParallelism); - - jsc.setJobGroup(this.getClass().getSimpleName(), "Perform cleaning of partitions"); - List> partitionCleanStats = jsc - .parallelize(cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream() - .flatMap(x -> x.getValue().stream().map(y -> new Tuple2<>(x.getKey(), - new CleanFileInfo(y.getFilePath(), y.getIsBootstrapBaseFile())))) - .collect(Collectors.toList()), cleanerParallelism) - .mapPartitionsToPair(deleteFilesFunc(table)) - .reduceByKey(PartitionCleanStat::merge).collect(); - - Map partitionCleanStatsMap = partitionCleanStats.stream() - .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2)); - - // Return PartitionCleanStat for each partition passed. - return cleanerPlan.getFilePathsToBeDeletedPerPartition().keySet().stream().map(partitionPath -> { - PartitionCleanStat partitionCleanStat = partitionCleanStatsMap.containsKey(partitionPath) - ? partitionCleanStatsMap.get(partitionPath) - : new PartitionCleanStat(partitionPath); - HoodieActionInstant actionInstant = cleanerPlan.getEarliestInstantToRetain(); - return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath) - .withEarliestCommitRetained(Option.ofNullable( - actionInstant != null - ? new HoodieInstant(HoodieInstant.State.valueOf(actionInstant.getState()), - actionInstant.getAction(), actionInstant.getTimestamp()) - : null)) - .withDeletePathPattern(partitionCleanStat.deletePathPatterns()) - .withSuccessfulDeletes(partitionCleanStat.successDeleteFiles()) - .withFailedDeletes(partitionCleanStat.failedDeleteFiles()) - .withDeleteBootstrapBasePathPatterns(partitionCleanStat.getDeleteBootstrapBasePathPatterns()) - .withSuccessfulDeleteBootstrapBaseFiles(partitionCleanStat.getSuccessfulDeleteBootstrapBaseFiles()) - .withFailedDeleteBootstrapBaseFiles(partitionCleanStat.getFailedDeleteBootstrapBaseFiles()) - .build(); - }).collect(Collectors.toList()); - } + abstract List clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan); /** * Creates a Cleaner plan if there are files to be cleaned and stores them in instant file. @@ -198,7 +126,7 @@ public class CleanActionExecutor extends BaseActionExecutor * @return Cleaner Plan if generated */ Option requestClean(String startCleanTime) { - final HoodieCleanerPlan cleanerPlan = requestClean(jsc); + final HoodieCleanerPlan cleanerPlan = requestClean(context); if ((cleanerPlan.getFilePathsToBeDeletedPerPartition() != null) && !cleanerPlan.getFilePathsToBeDeletedPerPartition().isEmpty() && cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).sum() > 0) { @@ -220,7 +148,7 @@ public class CleanActionExecutor extends BaseActionExecutor /** * Executes the Cleaner plan stored in the instant metadata. */ - void runPendingClean(HoodieTable table, HoodieInstant cleanInstant) { + void runPendingClean(HoodieTable table, HoodieInstant cleanInstant) { try { HoodieCleanerPlan cleanerPlan = CleanerUtils.getCleanerPlan(table.getMetaClient(), cleanInstant); runClean(table, cleanInstant, cleanerPlan); @@ -229,7 +157,7 @@ public class CleanActionExecutor extends BaseActionExecutor } } - private HoodieCleanMetadata runClean(HoodieTable table, HoodieInstant cleanInstant, HoodieCleanerPlan cleanerPlan) { + private HoodieCleanMetadata runClean(HoodieTable table, HoodieInstant cleanInstant, HoodieCleanerPlan cleanerPlan) { ValidationUtils.checkArgument(cleanInstant.getState().equals(HoodieInstant.State.REQUESTED) || cleanInstant.getState().equals(HoodieInstant.State.INFLIGHT)); @@ -244,7 +172,7 @@ public class CleanActionExecutor extends BaseActionExecutor inflightInstant = cleanInstant; } - List cleanStats = clean(jsc, cleanerPlan); + List cleanStats = clean(context, cleanerPlan); if (cleanStats.isEmpty()) { return HoodieCleanMetadata.newBuilder().build(); } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java similarity index 98% rename from hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java index dc891262e..405fc81b7 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java @@ -63,7 +63,7 @@ import java.util.stream.Stream; *

* 2) It bounds the growth of the files in the file system */ -public class CleanPlanner> implements Serializable { +public class CleanPlanner implements Serializable { private static final Logger LOG = LogManager.getLogger(CleanPlanner.class); @@ -74,10 +74,10 @@ public class CleanPlanner> implements Serializa private final SyncableFileSystemView fileSystemView; private final HoodieTimeline commitTimeline; private final Map fgIdToPendingCompactionOperations; - private HoodieTable hoodieTable; + private HoodieTable hoodieTable; private HoodieWriteConfig config; - public CleanPlanner(HoodieTable hoodieTable, HoodieWriteConfig config) { + public CleanPlanner(HoodieTable hoodieTable, HoodieWriteConfig config) { this.hoodieTable = hoodieTable; this.fileSystemView = hoodieTable.getHoodieView(); this.commitTimeline = hoodieTable.getCompletedCommitTimeline(); diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/clean/PartitionCleanStat.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/PartitionCleanStat.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/clean/PartitionCleanStat.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/PartitionCleanStat.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java new file mode 100644 index 000000000..3ead7a07d --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java @@ -0,0 +1,34 @@ +/* + * 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.table.action.commit; + +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +public abstract class AbstractBulkInsertHelper { + + public abstract HoodieWriteMetadata bulkInsert(I inputRecords, String instantTime, + HoodieTable table, HoodieWriteConfig config, + BaseCommitActionExecutor executor, boolean performDedupe, + Option> userDefinedBulkInsertPartitioner); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.java new file mode 100644 index 000000000..c2d2df7eb --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.java @@ -0,0 +1,48 @@ +/* + * 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.table.action.commit; + +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +/** + * Helper class to perform delete keys on hoodie table. + * + * @param + */ +public abstract class AbstractDeleteHelper { + + /** + * Deduplicate Hoodie records, using the given deduplication function. + * + * @param keys HoodieKeys to deduplicate + * @param table target Hoodie table for deduplicating + * @param parallelism parallelism or partitions to be used while reducing/deduplicating + * @return HoodieKey already be deduplicated + */ + public abstract K deduplicateKeys(K keys, HoodieTable table, int parallelism); + + public abstract HoodieWriteMetadata execute(String instantTime, + K keys, HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + BaseCommitActionExecutor deleteExecutor); +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/MergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractMergeHelper.java similarity index 62% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/MergeHelper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractMergeHelper.java index 4daa5c61f..1bbffadb3 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/MergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractMergeHelper.java @@ -30,10 +30,8 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.utils.MergingIterator; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.execution.SparkBoundedInMemoryExecutor; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; @@ -49,67 +47,17 @@ import java.util.Iterator; /** * Helper to read records from previous version of parquet and run Merge. */ -public class MergeHelper { +public abstract class AbstractMergeHelper { /** * Read records from previous version of base file and merge. * @param table Hoodie Table * @param upsertHandle Merge Handle - * @param * @throws IOException in case of error */ - public static > void runMerge(HoodieTable table, HoodieMergeHandle upsertHandle) throws IOException { - final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation(); - Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf()); - HoodieBaseFile baseFile = upsertHandle.baseFileForMerge(); + public abstract void runMerge(HoodieTable table, HoodieMergeHandle upsertHandle) throws IOException; - final GenericDatumWriter gWriter; - final GenericDatumReader gReader; - Schema readSchema; - if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) { - readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), upsertHandle.getOldFilePath()).getSchema(); - gWriter = new GenericDatumWriter<>(readSchema); - gReader = new GenericDatumReader<>(readSchema, upsertHandle.getWriterSchemaWithMetafields()); - } else { - gReader = null; - gWriter = null; - readSchema = upsertHandle.getWriterSchemaWithMetafields(); - } - - BoundedInMemoryExecutor wrapper = null; - HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, upsertHandle.getOldFilePath()); - try { - final Iterator readerIterator; - if (baseFile.getBootstrapBaseFile().isPresent()) { - readerIterator = getMergingIterator(table, upsertHandle, baseFile, reader, readSchema, externalSchemaTransformation); - } else { - readerIterator = reader.getRecordIterator(readSchema); - } - - ThreadLocal encoderCache = new ThreadLocal<>(); - ThreadLocal decoderCache = new ThreadLocal<>(); - wrapper = new SparkBoundedInMemoryExecutor(table.getConfig(), readerIterator, - new UpdateHandler(upsertHandle), record -> { - if (!externalSchemaTransformation) { - return record; - } - return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) record); - }); - wrapper.execute(); - } catch (Exception e) { - throw new HoodieException(e); - } finally { - if (reader != null) { - reader.close(); - } - upsertHandle.close(); - if (null != wrapper) { - wrapper.shutdownNow(); - } - } - } - - private static GenericRecord transformRecordBasedOnNewSchema(GenericDatumReader gReader, GenericDatumWriter gWriter, + protected GenericRecord transformRecordBasedOnNewSchema(GenericDatumReader gReader, GenericDatumWriter gWriter, ThreadLocal encoderCache, ThreadLocal decoderCache, GenericRecord gRec) { ByteArrayOutputStream inStream = null; @@ -141,7 +89,7 @@ public class MergeHelper { * for indexing, writing and other functionality. * */ - private static > Iterator getMergingIterator(HoodieTable table, HoodieMergeHandle mergeHandle, + protected Iterator getMergingIterator(HoodieTable table, HoodieMergeHandle mergeHandle, HoodieBaseFile baseFile, HoodieFileReader reader, Schema readSchema, boolean externalSchemaTransformation) throws IOException { Path externalFilePath = new Path(baseFile.getBootstrapBaseFile().get().getPath()); @@ -161,11 +109,11 @@ public class MergeHelper { /** * Consumer that dequeues records from queue and sends to Merge Handle. */ - private static class UpdateHandler extends BoundedInMemoryQueueConsumer { + protected static class UpdateHandler extends BoundedInMemoryQueueConsumer { private final HoodieMergeHandle upsertHandle; - private UpdateHandler(HoodieMergeHandle upsertHandle) { + protected UpdateHandler(HoodieMergeHandle upsertHandle) { this.upsertHandle = upsertHandle; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java new file mode 100644 index 000000000..f5e5e358c --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java @@ -0,0 +1,91 @@ +/* + * 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.table.action.commit; + +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.time.Duration; +import java.time.Instant; + +public abstract class AbstractWriteHelper { + + public HoodieWriteMetadata write(String instantTime, + I inputRecordsRDD, + HoodieEngineContext context, + HoodieTable table, + boolean shouldCombine, + int shuffleParallelism, + BaseCommitActionExecutor executor, + boolean performTagging) { + try { + // De-dupe/merge if needed + I dedupedRecords = + combineOnCondition(shouldCombine, inputRecordsRDD, shuffleParallelism, table); + + Instant lookupBegin = Instant.now(); + I taggedRecords = dedupedRecords; + if (performTagging) { + // perform index loop up to get existing location of records + taggedRecords = tag(dedupedRecords, context, table); + } + Duration indexLookupDuration = Duration.between(lookupBegin, Instant.now()); + + HoodieWriteMetadata result = executor.execute(taggedRecords); + result.setIndexLookupDuration(indexLookupDuration); + return result; + } catch (Throwable e) { + if (e instanceof HoodieUpsertException) { + throw (HoodieUpsertException) e; + } + throw new HoodieUpsertException("Failed to upsert for commit time " + instantTime, e); + } + } + + private I tag( + I dedupedRecords, HoodieEngineContext context, HoodieTable table) { + // perform index loop up to get existing location of records + return table.getIndex().tagLocation(dedupedRecords, context, table); + } + + public I combineOnCondition( + boolean condition, I records, int parallelism, HoodieTable table) { + return condition ? deduplicateRecords(records, table, parallelism) : records; + } + + /** + * Deduplicate Hoodie records, using the given deduplication function. + * + * @param records hoodieRecords to deduplicate + * @param parallelism parallelism or partitions to be used while reducing/deduplicating + * @return RDD of HoodieRecord already be deduplicated + */ + public I deduplicateRecords( + I records, HoodieTable table, int parallelism) { + return deduplicateRecords(records, table.getIndex(), parallelism); + } + + public abstract I deduplicateRecords( + I records, HoodieIndex index, int parallelism); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java new file mode 100644 index 000000000..71de9b6fc --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -0,0 +1,159 @@ +/* + * 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.table.action.commit; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieInstant.State; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.WorkloadStat; +import org.apache.hudi.table.action.BaseActionExecutor; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public abstract class BaseCommitActionExecutor + extends BaseActionExecutor { + + private static final Logger LOG = LogManager.getLogger(BaseCommitActionExecutor.class); + + protected final Option> extraMetadata; + protected final WriteOperationType operationType; + protected final TaskContextSupplier taskContextSupplier; + + public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable table, String instantTime, WriteOperationType operationType, + Option> extraMetadata) { + super(context, config, table, instantTime); + this.operationType = operationType; + this.extraMetadata = extraMetadata; + this.taskContextSupplier = context.getTaskContextSupplier(); + } + + public abstract HoodieWriteMetadata execute(I inputRecords); + + /** + * Save the workload profile in an intermediate file (here re-using commit files) This is useful when performing + * rollback for MOR tables. Only updates are recorded in the workload profile metadata since updates to log blocks + * are unknown across batches Inserts (which are new parquet files) are rolled back based on commit time. // TODO : + * Create a new WorkloadProfile metadata file instead of using HoodieCommitMetadata + */ + void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, String instantTime) + throws HoodieCommitException { + try { + HoodieCommitMetadata metadata = new HoodieCommitMetadata(); + profile.getPartitionPaths().forEach(path -> { + WorkloadStat partitionStat = profile.getWorkloadStat(path.toString()); + HoodieWriteStat insertStat = new HoodieWriteStat(); + insertStat.setNumInserts(partitionStat.getNumInserts()); + insertStat.setFileId(""); + insertStat.setPrevCommit(HoodieWriteStat.NULL_COMMIT); + metadata.addWriteStat(path, insertStat); + + partitionStat.getUpdateLocationToCount().forEach((key, value) -> { + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId(key); + // TODO : Write baseCommitTime is possible here ? + writeStat.setPrevCommit(value.getKey()); + writeStat.setNumUpdateWrites(value.getValue()); + metadata.addWriteStat(path, writeStat); + }); + }); + metadata.setOperationType(operationType); + + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + String commitActionType = getCommitActionType(); + HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime); + activeTimeline.transitionRequestedToInflight(requested, + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)), + config.shouldAllowMultiWriteOnSameInstant()); + } catch (IOException io) { + throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io); + } + } + + protected String getCommitActionType() { + return table.getMetaClient().getCommitActionType(); + } + + protected void commitOnAutoCommit(HoodieWriteMetadata result) { + if (config.shouldAutoCommit()) { + LOG.info("Auto commit enabled: Committing " + instantTime); + commit(extraMetadata, result); + } else { + LOG.info("Auto commit disabled for " + instantTime); + } + } + + protected abstract void commit(Option> extraMetadata, HoodieWriteMetadata result); + + /** + * Finalize Write operation. + * @param instantTime Instant Time + * @param stats Hoodie Write Stat + */ + protected void finalizeWrite(String instantTime, List stats, HoodieWriteMetadata result) { + try { + Instant start = Instant.now(); + table.finalizeWrite(context, instantTime, stats); + result.setFinalizeDuration(Duration.between(start, Instant.now())); + } catch (HoodieIOException ioe) { + throw new HoodieCommitException("Failed to complete commit " + instantTime + " due to finalize errors.", ioe); + } + } + + /** + * By default, return the writer schema in Write Config for storing in commit. + */ + protected String getSchemaToStoreInCommit() { + return config.getSchema(); + } + + protected boolean isWorkloadProfileNeeded() { + return true; + } + + protected abstract Iterator> handleInsert(String idPfx, + Iterator> recordItr) throws Exception; + + protected abstract Iterator> handleUpdate(String partitionPath, String fileId, + Iterator> recordItr) throws IOException; +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BucketInfo.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BucketInfo.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/BucketInfo.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BucketInfo.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BucketType.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BucketType.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/BucketType.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BucketType.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertBucket.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertBucket.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertBucket.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertBucket.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertBucketCumulativeWeightPair.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertBucketCumulativeWeightPair.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertBucketCumulativeWeightPair.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertBucketCumulativeWeightPair.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/SmallFile.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/SmallFile.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/SmallFile.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/SmallFile.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/AbstractCompactHelpers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/AbstractCompactHelpers.java new file mode 100644 index 000000000..3ff9e625e --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/AbstractCompactHelpers.java @@ -0,0 +1,58 @@ +/* + * 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.table.action.compact; + +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieCompactionException; +import org.apache.hudi.table.HoodieTable; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +/** + * Base class helps to perform compact. + * + * @param Type of payload in {@link org.apache.hudi.common.model.HoodieRecord} + * @param Type of inputs + * @param Type of keys + * @param Type of outputs + */ +public abstract class AbstractCompactHelpers { + public abstract HoodieCommitMetadata createCompactionMetadata(HoodieTable table, + String compactionInstantTime, + O writeStatuses, + String schema) throws IOException; + + public void completeInflightCompaction(HoodieTable table, String compactionCommitTime, HoodieCommitMetadata commitMetadata) { + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + try { + activeTimeline.transitionCompactionInflightToComplete( + new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime), + Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } catch (IOException e) { + throw new HoodieCompactionException( + "Failed to commit " + table.getMetaClient().getBasePath() + " at time " + compactionCommitTime, e); + } + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java similarity index 59% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java index 0d7f6bee8..d21c7d96d 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java @@ -19,71 +19,37 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; -import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieCompactionException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -public class ScheduleCompactionActionExecutor extends BaseActionExecutor> { - - private static final Logger LOG = LogManager.getLogger(ScheduleCompactionActionExecutor.class); +public abstract class BaseScheduleCompactionActionExecutor extends BaseActionExecutor> { private final Option> extraMetadata; - public ScheduleCompactionActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - Option> extraMetadata) { - super(jsc, config, table, instantTime); + public BaseScheduleCompactionActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + Option> extraMetadata) { + super(context, config, table, instantTime); this.extraMetadata = extraMetadata; } - private HoodieCompactionPlan scheduleCompaction() { - LOG.info("Checking if compaction needs to be run on " + config.getBasePath()); - Option lastCompaction = table.getActiveTimeline().getCommitTimeline() - .filterCompletedInstants().lastInstant(); - String lastCompactionTs = "0"; - if (lastCompaction.isPresent()) { - lastCompactionTs = lastCompaction.get().getTimestamp(); - } - - int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline() - .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants(); - if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) { - LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction - + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for " - + config.getInlineCompactDeltaCommitMax()); - return new HoodieCompactionPlan(); - } - - LOG.info("Generating compaction plan for merge on read table " + config.getBasePath()); - HoodieMergeOnReadTableCompactor compactor = new HoodieMergeOnReadTableCompactor(); - try { - return compactor.generateCompactionPlan(jsc, table, config, instantTime, - ((SyncableFileSystemView) table.getSliceView()).getPendingCompactionOperations() - .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) - .collect(Collectors.toSet())); - - } catch (IOException e) { - throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); - } - } + protected abstract HoodieCompactionPlan scheduleCompaction(); @Override public Option execute() { diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java similarity index 72% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java index c81b02885..e5212e8f4 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java @@ -19,14 +19,12 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; - import java.io.IOException; import java.io.Serializable; import java.util.Set; @@ -34,12 +32,12 @@ import java.util.Set; /** * A HoodieCompactor runs compaction on a hoodie table. */ -public interface HoodieCompactor extends Serializable { +public interface HoodieCompactor extends Serializable { /** * Generate a new compaction plan for scheduling. * - * @param jsc Spark Context + * @param context HoodieEngineContext * @param hoodieTable Hoodie Table * @param config Hoodie Write Configuration * @param compactionCommitTime scheduled compaction commit time @@ -47,12 +45,12 @@ public interface HoodieCompactor extends Serializable { * @return Compaction Plan * @throws IOException when encountering errors */ - HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, HoodieTable hoodieTable, HoodieWriteConfig config, - String compactionCommitTime, Set fgIdsInPendingCompactions) throws IOException; + HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, HoodieTable hoodieTable, HoodieWriteConfig config, + String compactionCommitTime, Set fgIdsInPendingCompactions) throws IOException; /** * Execute compaction operations and report back status. */ - JavaRDD compact(JavaSparkContext jsc, HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, + O compact(HoodieEngineContext context, HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config, String compactionInstantTime) throws IOException; } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/OperationResult.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/OperationResult.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/OperationResult.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/OperationResult.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedIOCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedIOCompactionStrategy.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedIOCompactionStrategy.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedIOCompactionStrategy.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java similarity index 88% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java index 597348f47..747e0b2f3 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java @@ -39,7 +39,7 @@ import java.util.stream.Collectors; */ public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionStrategy { - SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_PARTITION_FORMAT); + SimpleDateFormat dateFormat = new SimpleDateFormat(DayBasedCompactionStrategy.DATE_PARTITION_FORMAT); @Override public List orderAndFilter(HoodieWriteConfig writeConfig, @@ -50,8 +50,8 @@ public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionS // Filter out all partitions greater than earliestPartitionPathToCompact return operations.stream().collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet() - .stream().sorted(Map.Entry.comparingByKey(comparator)) - .filter(e -> comparator.compare(earliestPartitionPathToCompact, e.getKey()) >= 0) + .stream().sorted(Map.Entry.comparingByKey(DayBasedCompactionStrategy.comparator)) + .filter(e -> DayBasedCompactionStrategy.comparator.compare(earliestPartitionPathToCompact, e.getKey()) >= 0) .flatMap(e -> e.getValue().stream()).collect(Collectors.toList()); } @@ -63,7 +63,7 @@ public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionS // Get all partitions and sort them return partitionPaths.stream().map(partition -> partition.replace("/", "-")) .sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/")) - .filter(e -> comparator.compare(earliestPartitionPathToCompact, e) >= 0).collect(Collectors.toList()); + .filter(e -> DayBasedCompactionStrategy.comparator.compare(earliestPartitionPathToCompact, e) >= 0).collect(Collectors.toList()); } public static Date getDateAtOffsetFromToday(int offset) { diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java similarity index 98% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java index 0e8e4c674..6c631c462 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.action.compact.HoodieMergeOnReadTableCompactor; import java.io.Serializable; import java.util.HashMap; @@ -38,8 +37,6 @@ import java.util.Map; * implementations of this abstract class can capture the relevant metrics to order and filter the final list of * compaction operation to run in a single compaction. Implementation of CompactionStrategy cannot hold any state. * Difference instantiations can be passed in every time - * - * @see HoodieMergeOnReadTableCompactor */ public abstract class CompactionStrategy implements Serializable { diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/DayBasedCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/DayBasedCompactionStrategy.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/DayBasedCompactionStrategy.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/DayBasedCompactionStrategy.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedCompactionStrategy.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedCompactionStrategy.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedCompactionStrategy.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedPartitionAwareCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedPartitionAwareCompactionStrategy.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedPartitionAwareCompactionStrategy.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedPartitionAwareCompactionStrategy.java diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java similarity index 91% rename from hudi-client/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java index 0b4e4c4d2..0bbc0a05f 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java @@ -20,7 +20,9 @@ package org.apache.hudi.table.action.restore; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -32,7 +34,6 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; import java.util.Collections; @@ -41,18 +42,18 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -public abstract class BaseRestoreActionExecutor extends BaseActionExecutor { +public abstract class BaseRestoreActionExecutor extends BaseActionExecutor { private static final Logger LOG = LogManager.getLogger(BaseRestoreActionExecutor.class); private final String restoreInstantTime; - public BaseRestoreActionExecutor(JavaSparkContext jsc, + public BaseRestoreActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, - HoodieTable table, + HoodieTable table, String instantTime, String restoreInstantTime) { - super(jsc, config, table, instantTime); + super(context, config, table, instantTime); this.restoreInstantTime = restoreInstantTime; } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java similarity index 63% rename from hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java index 2a137b4e0..40526b86f 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java @@ -18,63 +18,57 @@ package org.apache.hudi.table.action.rollback; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.block.HoodieCommandBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.MarkerFiles; - -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; import java.util.Collections; -import java.util.List; import java.util.Map; -import scala.Tuple2; - /** * Performs rollback using marker files generated during the write.. */ -public class MarkerBasedRollbackStrategy implements BaseRollbackActionExecutor.RollbackStrategy { +public abstract class AbstractMarkerBasedRollbackStrategy implements BaseRollbackActionExecutor.RollbackStrategy { - private static final Logger LOG = LogManager.getLogger(MarkerBasedRollbackStrategy.class); + private static final Logger LOG = LogManager.getLogger(AbstractMarkerBasedRollbackStrategy.class); - private final HoodieTable table; + protected final HoodieTable table; - private final transient JavaSparkContext jsc; + protected final transient HoodieEngineContext context; - private final HoodieWriteConfig config; + protected final HoodieWriteConfig config; private final String basePath; private final String instantTime; - public MarkerBasedRollbackStrategy(HoodieTable table, JavaSparkContext jsc, HoodieWriteConfig config, String instantTime) { + public AbstractMarkerBasedRollbackStrategy(HoodieTable table, HoodieEngineContext context, HoodieWriteConfig config, String instantTime) { this.table = table; - this.jsc = jsc; + this.context = context; this.basePath = table.getMetaClient().getBasePath(); this.config = config; this.instantTime = instantTime; } - private HoodieRollbackStat undoMerge(String mergedBaseFilePath) throws IOException { + protected HoodieRollbackStat undoMerge(String mergedBaseFilePath) throws IOException { LOG.info("Rolling back by deleting the merged base file:" + mergedBaseFilePath); return deleteBaseFile(mergedBaseFilePath); } - private HoodieRollbackStat undoCreate(String createdBaseFilePath) throws IOException { + protected HoodieRollbackStat undoCreate(String createdBaseFilePath) throws IOException { LOG.info("Rolling back by deleting the created base file:" + createdBaseFilePath); return deleteBaseFile(createdBaseFilePath); } @@ -89,7 +83,7 @@ public class MarkerBasedRollbackStrategy implements BaseRollbackActionExecutor.R .build(); } - private HoodieRollbackStat undoAppend(String appendBaseFilePath, HoodieInstant instantToRollback) throws IOException, InterruptedException { + protected HoodieRollbackStat undoAppend(String appendBaseFilePath, HoodieInstant instantToRollback) throws IOException, InterruptedException { Path baseFilePathForAppend = new Path(basePath, appendBaseFilePath); String fileId = FSUtils.getFileIdFromFilePath(baseFilePathForAppend); String baseCommitTime = FSUtils.getCommitTime(baseFilePathForAppend.getName()); @@ -131,33 +125,4 @@ public class MarkerBasedRollbackStrategy implements BaseRollbackActionExecutor.R .withRollbackBlockAppendResults(Collections.emptyMap()) .build(); } - - @Override - public List execute(HoodieInstant instantToRollback) { - try { - MarkerFiles markerFiles = new MarkerFiles(table, instantToRollback.getTimestamp()); - List markerFilePaths = markerFiles.allMarkerFilePaths(); - int parallelism = Math.max(Math.min(markerFilePaths.size(), config.getRollbackParallelism()), 1); - return jsc.parallelize(markerFilePaths, parallelism) - .map(markerFilePath -> { - String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1); - IOType type = IOType.valueOf(typeStr); - switch (type) { - case MERGE: - return undoMerge(MarkerFiles.stripMarkerSuffix(markerFilePath)); - case APPEND: - return undoAppend(MarkerFiles.stripMarkerSuffix(markerFilePath), instantToRollback); - case CREATE: - return undoCreate(MarkerFiles.stripMarkerSuffix(markerFilePath)); - default: - throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback); - } - }) - .mapToPair(rollbackStat -> new Tuple2<>(rollbackStat.getPartitionPath(), rollbackStat)) - .reduceByKey(RollbackUtils::mergeRollbackStat) - .map(Tuple2::_2).collect(); - } catch (Exception e) { - throw new HoodieRollbackException("Error rolling back using marker files written for " + instantToRollback, e); - } - } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java similarity index 61% rename from hudi-client/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java index b06dc05b9..c60cb7f5e 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java @@ -18,7 +18,9 @@ package org.apache.hudi.table.action.rollback; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; @@ -27,33 +29,32 @@ import org.apache.hudi.table.HoodieTable; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.util.ArrayList; import java.util.List; -public class CopyOnWriteRollbackActionExecutor extends BaseRollbackActionExecutor { +public abstract class BaseCopyOnWriteRollbackActionExecutor extends BaseRollbackActionExecutor { - private static final Logger LOG = LogManager.getLogger(CopyOnWriteRollbackActionExecutor.class); + private static final Logger LOG = LogManager.getLogger(BaseCopyOnWriteRollbackActionExecutor.class); - public CopyOnWriteRollbackActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - HoodieInstant commitInstant, - boolean deleteInstants) { - super(jsc, config, table, instantTime, commitInstant, deleteInstants); + public BaseCopyOnWriteRollbackActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + HoodieInstant commitInstant, + boolean deleteInstants) { + super(context, config, table, instantTime, commitInstant, deleteInstants); } - public CopyOnWriteRollbackActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - HoodieInstant commitInstant, - boolean deleteInstants, - boolean skipTimelinePublish, - boolean useMarkerBasedStrategy) { - super(jsc, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy); + public BaseCopyOnWriteRollbackActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + HoodieInstant commitInstant, + boolean deleteInstants, + boolean skipTimelinePublish, + boolean useMarkerBasedStrategy) { + super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy); } @Override @@ -87,11 +88,4 @@ public class CopyOnWriteRollbackActionExecutor extends BaseRollbackActionExecuto LOG.info("Time(in ms) taken to finish rollback " + rollbackTimer.endTimer()); return stats; } - - @Override - protected List executeRollbackUsingFileListing(HoodieInstant instantToRollback) { - List rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning()); - return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(jsc, instantToRollback, rollbackRequests); - } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java similarity index 65% rename from hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java index 97e110dc0..0e747a58d 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java @@ -18,43 +18,42 @@ package org.apache.hudi.table.action.rollback; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; - import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; import java.util.ArrayList; import java.util.List; -public class MergeOnReadRollbackActionExecutor extends BaseRollbackActionExecutor { +public abstract class BaseMergeOnReadRollbackActionExecutor extends BaseRollbackActionExecutor { - private static final Logger LOG = LogManager.getLogger(MergeOnReadRollbackActionExecutor.class); + private static final Logger LOG = LogManager.getLogger(BaseMergeOnReadRollbackActionExecutor.class); - public MergeOnReadRollbackActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - HoodieInstant commitInstant, - boolean deleteInstants) { - super(jsc, config, table, instantTime, commitInstant, deleteInstants); + public BaseMergeOnReadRollbackActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + HoodieInstant commitInstant, + boolean deleteInstants) { + super(context, config, table, instantTime, commitInstant, deleteInstants); } - public MergeOnReadRollbackActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - HoodieInstant commitInstant, - boolean deleteInstants, - boolean skipTimelinePublish, - boolean useMarkerBasedStrategy) { - super(jsc, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy); + public BaseMergeOnReadRollbackActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + HoodieInstant commitInstant, + boolean deleteInstants, + boolean skipTimelinePublish, + boolean useMarkerBasedStrategy) { + super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy); } @Override @@ -95,15 +94,4 @@ public class MergeOnReadRollbackActionExecutor extends BaseRollbackActionExecuto LOG.info("Time(in ms) taken to finish rollback " + rollbackTimer.endTimer()); return allRollbackStats; } - - @Override - protected List executeRollbackUsingFileListing(HoodieInstant resolvedInstant) { - List rollbackRequests; - try { - rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(resolvedInstant, table, jsc); - } catch (IOException e) { - throw new HoodieIOException("Error generating rollback requests by file listing.", e); - } - return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(jsc, resolvedInstant, rollbackRequests); - } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java similarity index 92% rename from hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java index 36199c5b5..40206ca45 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java @@ -19,9 +19,11 @@ package org.apache.hudi.table.action.rollback; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.bootstrap.index.BootstrapIndex; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -38,7 +40,6 @@ import org.apache.hudi.table.action.BaseActionExecutor; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; import java.io.Serializable; @@ -47,7 +48,7 @@ import java.util.List; import java.util.Objects; import java.util.stream.Collectors; -public abstract class BaseRollbackActionExecutor extends BaseActionExecutor { +public abstract class BaseRollbackActionExecutor extends BaseActionExecutor { private static final Logger LOG = LogManager.getLogger(BaseRollbackActionExecutor.class); @@ -61,25 +62,25 @@ public abstract class BaseRollbackActionExecutor extends BaseActionExecutor table, + HoodieTable table, String instantTime, HoodieInstant instantToRollback, boolean deleteInstants) { - this(jsc, config, table, instantTime, instantToRollback, deleteInstants, + this(context, config, table, instantTime, instantToRollback, deleteInstants, false, config.shouldRollbackUsingMarkers()); } - public BaseRollbackActionExecutor(JavaSparkContext jsc, + public BaseRollbackActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, - HoodieTable table, + HoodieTable table, String instantTime, HoodieInstant instantToRollback, boolean deleteInstants, boolean skipTimelinePublish, boolean useMarkerBasedStrategy) { - super(jsc, config, table, instantTime); + super(context, config, table, instantTime); this.instantToRollback = instantToRollback; this.deleteInstants = deleteInstants; this.skipTimelinePublish = skipTimelinePublish; @@ -90,13 +91,7 @@ public abstract class BaseRollbackActionExecutor extends BaseActionExecutor executeRollback() throws IOException; @@ -116,7 +111,7 @@ public abstract class BaseRollbackActionExecutor extends BaseActionExecutor generateRollbackRequestsUsingFileListingMOR(HoodieInstant instantToRollback, HoodieTable table, JavaSparkContext jsc) throws IOException { + public static List generateRollbackRequestsUsingFileListingMOR(HoodieInstant instantToRollback, HoodieTable table, HoodieEngineContext context) throws IOException { String commit = instantToRollback.getTimestamp(); HoodieWriteConfig config = table.getConfig(); List partitions = FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()); int sparkPartitions = Math.max(Math.min(partitions.size(), config.getRollbackParallelism()), 1); - jsc.setJobGroup(RollbackUtils.class.getSimpleName(), "Generate all rollback requests"); - return jsc.parallelize(partitions, Math.min(partitions.size(), sparkPartitions)).flatMap(partitionPath -> { + context.setJobStatus(RollbackUtils.class.getSimpleName(), "Generate all rollback requests"); + return context.flatMap(partitions, partitionPath -> { HoodieActiveTimeline activeTimeline = table.getMetaClient().reloadActiveTimeline(); List partitionRollbackRequests = new ArrayList<>(); switch (instantToRollback.getAction()) { @@ -198,8 +198,8 @@ public class RollbackUtils { default: break; } - return partitionRollbackRequests.iterator(); - }).filter(Objects::nonNull).collect(); + return partitionRollbackRequests.stream(); + }, Math.min(partitions.size(), sparkPartitions)).stream().filter(Objects::nonNull).collect(Collectors.toList()); } private static List generateAppendRollbackBlocksAction(String partitionPath, HoodieInstant rollbackInstant, diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java similarity index 86% rename from hudi-client/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java index ac95118d9..16fd9a481 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java @@ -20,8 +20,10 @@ package org.apache.hudi.table.action.savepoint; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -35,7 +37,6 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import scala.Tuple2; import java.io.IOException; @@ -43,20 +44,20 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -public class SavepointActionExecutor extends BaseActionExecutor { +public class SavepointActionExecutor extends BaseActionExecutor { private static final Logger LOG = LogManager.getLogger(SavepointActionExecutor.class); private final String user; private final String comment; - public SavepointActionExecutor(JavaSparkContext jsc, + public SavepointActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, - HoodieTable table, + HoodieTable table, String instantTime, String user, String comment) { - super(jsc, config, table, instantTime); + super(context, config, table, instantTime); this.user = user; this.comment = comment; } @@ -87,19 +88,16 @@ public class SavepointActionExecutor extends BaseActionExecutor> latestFilesMap = jsc.parallelize(FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(), - table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning())) - .mapToPair(partitionPath -> { + context.setJobStatus(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime); + Map> latestFilesMap = context.mapToPair(FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(), + table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()), partitionPath -> { // Scan all partitions files with this commit time LOG.info("Collecting latest files in partition path " + partitionPath); TableFileSystemView.BaseFileOnlyView view = table.getBaseFileOnlyView(); List latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, instantTime) .map(HoodieBaseFile::getFileName).collect(Collectors.toList()); - return new Tuple2<>(partitionPath, latestFiles); - }) - .collectAsMap(); - + return new Tuple2<>(partitionPath, latestFiles); + }, null); HoodieSavepointMetadata metadata = TimelineMetadataUtils.convertSavepointMetadata(user, comment, latestFilesMap); // Nothing to save in the savepoint table.getActiveTimeline().createNewInstant( diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java similarity index 91% rename from hudi-client/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java index 06acd4691..0d51a639a 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java @@ -32,7 +32,7 @@ public class SavepointHelpers { private static final Logger LOG = LogManager.getLogger(SavepointHelpers.class); - public static void deleteSavepoint(HoodieTable table, String savepointTime) { + public static void deleteSavepoint(HoodieTable table, String savepointTime) { if (table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { throw new UnsupportedOperationException("Savepointing is not supported or MergeOnRead table types"); } @@ -48,7 +48,7 @@ public class SavepointHelpers { LOG.info("Savepoint " + savepointTime + " deleted"); } - public static void validateSavepointRestore(HoodieTable table, String savepointTime) { + public static void validateSavepointRestore(HoodieTable table, String savepointTime) { // Make sure the restore was successful table.getMetaClient().reloadActiveTimeline(); Option lastInstant = table.getActiveTimeline() @@ -61,7 +61,7 @@ public class SavepointHelpers { + lastInstant.get().getTimestamp()); } - public static void validateSavepointPresence(HoodieTable table, String savepointTime) { + public static void validateSavepointPresence(HoodieTable table, String savepointTime) { HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); boolean isSavepointPresent = table.getCompletedSavepointTimeline().containsInstant(savePoint); if (!isSavepointPresent) { diff --git a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java similarity index 77% rename from hudi-client/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java index 53af1775c..cafb8167b 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java @@ -18,19 +18,18 @@ package org.apache.hudi.table.upgrade; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieUpgradeDowngradeException; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; import java.util.Date; @@ -39,14 +38,14 @@ import java.util.Properties; /** * Helper class to assist in upgrading/downgrading Hoodie when there is a version change. */ -public class UpgradeDowngrade { +public abstract class AbstractUpgradeDowngrade { - private static final Logger LOG = LogManager.getLogger(UpgradeDowngrade.class); + private static final Logger LOG = LogManager.getLogger(AbstractUpgradeDowngrade.class); public static final String HOODIE_UPDATED_PROPERTY_FILE = "hoodie.properties.updated"; private HoodieTableMetaClient metaClient; - private HoodieWriteConfig config; - private JavaSparkContext jsc; + protected HoodieWriteConfig config; + protected HoodieEngineContext context; private transient FileSystem fs; private Path updatedPropsFilePath; private Path propsFilePath; @@ -73,28 +72,22 @@ public class UpgradeDowngrade { * @param metaClient instance of {@link HoodieTableMetaClient} to use * @param toVersion version to which upgrade or downgrade has to be done. * @param config instance of {@link HoodieWriteConfig} to use. - * @param jsc instance of {@link JavaSparkContext} to use. + * @param context instance of {@link HoodieEngineContext} to use. * @param instantTime current instant time that should not be touched. */ - public static void run(HoodieTableMetaClient metaClient, HoodieTableVersion toVersion, HoodieWriteConfig config, - JavaSparkContext jsc, String instantTime) { - try { - new UpgradeDowngrade(metaClient, config, jsc).run(toVersion, instantTime); - } catch (IOException e) { - throw new HoodieUpgradeDowngradeException("Error during upgrade/downgrade to version:" + toVersion, e); - } - } + public abstract void run(HoodieTableMetaClient metaClient, HoodieTableVersion toVersion, HoodieWriteConfig config, + HoodieEngineContext context, String instantTime); - private UpgradeDowngrade(HoodieTableMetaClient metaClient, HoodieWriteConfig config, JavaSparkContext jsc) { + protected AbstractUpgradeDowngrade(HoodieTableMetaClient metaClient, HoodieWriteConfig config, HoodieEngineContext context) { this.metaClient = metaClient; this.config = config; - this.jsc = jsc; + this.context = context; this.fs = metaClient.getFs(); this.updatedPropsFilePath = new Path(metaClient.getMetaPath(), HOODIE_UPDATED_PROPERTY_FILE); this.propsFilePath = new Path(metaClient.getMetaPath(), HoodieTableConfig.HOODIE_PROPERTIES_FILE); } - private void run(HoodieTableVersion toVersion, String instantTime) throws IOException { + protected void run(HoodieTableVersion toVersion, String instantTime) throws IOException { // Fetch version from property file and current version HoodieTableVersion fromVersion = metaClient.getTableConfig().getTableVersion(); if (toVersion.versionCode() == fromVersion.versionCode()) { @@ -144,19 +137,7 @@ public class UpgradeDowngrade { } } - private void upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) { - if (fromVersion == HoodieTableVersion.ZERO && toVersion == HoodieTableVersion.ONE) { - new ZeroToOneUpgradeHandler().upgrade(config, jsc, instantTime); - } else { - throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), true); - } - } + protected abstract void upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime); - private void downgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) { - if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.ZERO) { - new OneToZeroDowngradeHandler().downgrade(config, jsc, instantTime); - } else { - throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), false); - } - } + protected abstract void downgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime); } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java similarity index 84% rename from hudi-client/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java index 948e44c34..27389d923 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java @@ -18,10 +18,9 @@ package org.apache.hudi.table.upgrade; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.spark.api.java.JavaSparkContext; - /** * Interface to assist in downgrading Hoodie table. */ @@ -31,8 +30,8 @@ public interface DowngradeHandler { * to be invoked to downgrade hoodie table from one version to a lower version. * * @param config instance of {@link HoodieWriteConfig} to be used. - * @param jsc instance of {@link JavaSparkContext} to be used. + * @param context instance of {@link HoodieEngineContext} to be used. * @param instantTime current instant time that should not touched. */ - void downgrade(HoodieWriteConfig config, JavaSparkContext jsc, String instantTime); + void downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime); } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java similarity index 84% rename from hudi-client/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java index 4d56143a7..fdf1261b3 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java @@ -18,10 +18,9 @@ package org.apache.hudi.table.upgrade; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.spark.api.java.JavaSparkContext; - /** * Interface to assist in upgrading Hoodie table. */ @@ -31,8 +30,8 @@ public interface UpgradeHandler { * to be invoked to upgrade hoodie table from one version to a higher version. * * @param config instance of {@link HoodieWriteConfig} to be used. - * @param jsc instance of {@link JavaSparkContext} to be used. + * @param context instance of {@link HoodieEngineContext} to be used. * @param instantTime current instant time that should not be touched. */ - void upgrade(HoodieWriteConfig config, JavaSparkContext jsc, String instantTime); + void upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime); } diff --git a/hudi-client/src/main/resources/log4j.properties b/hudi-client/hudi-client-common/src/main/resources/log4j.properties similarity index 100% rename from hudi-client/src/main/resources/log4j.properties rename to hudi-client/hudi-client-common/src/main/resources/log4j.properties diff --git a/hudi-client/src/test/java/org/apache/hudi/ClientFunctionalTestSuite.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/ClientFunctionalTestSuite.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/ClientFunctionalTestSuite.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/ClientFunctionalTestSuite.java diff --git a/hudi-client/src/test/java/org/apache/hudi/callback/http/TestCallbackHttpClient.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/callback/http/TestCallbackHttpClient.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/callback/http/TestCallbackHttpClient.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/callback/http/TestCallbackHttpClient.java diff --git a/hudi-client/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieConsoleMetrics.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestHoodieConsoleMetrics.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieConsoleMetrics.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestHoodieConsoleMetrics.java diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieJmxMetrics.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestHoodieJmxMetrics.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieJmxMetrics.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestHoodieJmxMetrics.java diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/TestMetricsReporterFactory.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestMetricsReporterFactory.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/metrics/TestMetricsReporterFactory.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestMetricsReporterFactory.java diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogHttpClient.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogHttpClient.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogHttpClient.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogHttpClient.java diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogMetricsReporter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogMetricsReporter.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogMetricsReporter.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogMetricsReporter.java diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogReporter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogReporter.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogReporter.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogReporter.java diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/prometheus/TestPrometheusReporter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/prometheus/TestPrometheusReporter.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/metrics/prometheus/TestPrometheusReporter.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/prometheus/TestPrometheusReporter.java diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/prometheus/TestPushGateWayReporter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/prometheus/TestPushGateWayReporter.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/metrics/prometheus/TestPushGateWayReporter.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/prometheus/TestPushGateWayReporter.java diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/DFSProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/DFSProvider.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/testutils/providers/DFSProvider.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/DFSProvider.java diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieEngineContextProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieEngineContextProvider.java new file mode 100644 index 000000000..e876c08bc --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieEngineContextProvider.java @@ -0,0 +1,26 @@ +/* + * 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.testutils.providers; + +import org.apache.hudi.client.common.HoodieEngineContext; + +public interface HoodieEngineContextProvider { + HoodieEngineContext context(); +} diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieMetaClientProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieMetaClientProvider.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieMetaClientProvider.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieMetaClientProvider.java diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java similarity index 86% rename from hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java index 4840af0be..9bc559deb 100644 --- a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java @@ -19,12 +19,12 @@ package org.apache.hudi.testutils.providers; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.AbstractHoodieWriteClient; import org.apache.hudi.config.HoodieWriteConfig; import java.io.IOException; public interface HoodieWriteClientProvider { - HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws IOException; + AbstractHoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws IOException; } diff --git a/hudi-client/src/test/resources/exampleEvolvedSchema.txt b/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchema.txt similarity index 100% rename from hudi-client/src/test/resources/exampleEvolvedSchema.txt rename to hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchema.txt diff --git a/hudi-client/src/test/resources/exampleSchema.txt b/hudi-client/hudi-client-common/src/test/resources/exampleSchema.txt similarity index 100% rename from hudi-client/src/test/resources/exampleSchema.txt rename to hudi-client/hudi-client-common/src/test/resources/exampleSchema.txt diff --git a/hudi-client/src/test/resources/log4j-surefire-quiet.properties b/hudi-client/hudi-client-common/src/test/resources/log4j-surefire-quiet.properties similarity index 100% rename from hudi-client/src/test/resources/log4j-surefire-quiet.properties rename to hudi-client/hudi-client-common/src/test/resources/log4j-surefire-quiet.properties diff --git a/hudi-client/src/test/resources/log4j-surefire.properties b/hudi-client/hudi-client-common/src/test/resources/log4j-surefire.properties similarity index 100% rename from hudi-client/src/test/resources/log4j-surefire.properties rename to hudi-client/hudi-client-common/src/test/resources/log4j-surefire.properties diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml new file mode 100644 index 000000000..d99346d08 --- /dev/null +++ b/hudi-client/hudi-spark-client/pom.xml @@ -0,0 +1,235 @@ + + + + + hudi-client + org.apache.hudi + 0.6.1-SNAPSHOT + + 4.0.0 + + hudi-spark-client + ${parent.version} + + hudi-spark-client + jar + + + + + org.apache.hudi + hudi-client-common + ${parent.version} + + + + + org.apache.spark + spark-core_${scala.binary.version} + + + org.apache.spark + spark-sql_${scala.binary.version} + + + org.apache.spark + spark-avro_${scala.binary.version} + provided + + + + + org.apache.parquet + parquet-avro + + + + + org.apache.hudi + hudi-common + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-client-common + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-hadoop-mr + ${project.version} + test + + + + + org.apache.hbase + hbase-testing-util + ${hbase.version} + test + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + javax.xml.bind + * + + + + + + + ${hive.groupid} + hive-exec + ${hive.version} + test + ${hive.exec.classifier} + + + ${hive.groupid} + hive-metastore + ${hive.version} + test + + + + + org.junit.jupiter + junit-jupiter-api + test + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.vintage + junit-vintage-engine + test + + + org.junit.jupiter + junit-jupiter-params + test + + + org.mockito + mockito-junit-jupiter + test + + + org.junit.platform + junit-platform-runner + test + + + org.junit.platform + junit-platform-suite-api + test + + + org.junit.platform + junit-platform-commons + test + + + + + + + org.jacoco + jacoco-maven-plugin + + + net.alchim31.maven + scala-maven-plugin + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + compile + + compile + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + test-compile + + + + false + + + + org.apache.rat + apache-rat-plugin + + + + + + src/main/resources + + + src/test/resources + + + + diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/async/SparkAsyncCompactService.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/async/SparkAsyncCompactService.java new file mode 100644 index 000000000..152a901a7 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/async/SparkAsyncCompactService.java @@ -0,0 +1,36 @@ +/* + * 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.async; + +import org.apache.hudi.client.AbstractCompactor; +import org.apache.hudi.client.AbstractHoodieWriteClient; +import org.apache.hudi.client.HoodieSparkCompactor; +import org.apache.hudi.client.common.HoodieEngineContext; + +public class SparkAsyncCompactService extends AsyncCompactService { + + public SparkAsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client) { + super(context, client); + } + + @Override + protected AbstractCompactor createCompactor(AbstractHoodieWriteClient client) { + return new HoodieSparkCompactor(client); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java similarity index 82% rename from hudi-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java index 1a001973a..4fb9f221c 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java @@ -20,6 +20,7 @@ package org.apache.hudi.client; import org.apache.hadoop.conf.Configuration; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -32,12 +33,13 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; 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; @@ -62,42 +64,42 @@ public class HoodieReadClient implements Serializ * TODO: We need to persist the index type into hoodie.properties and be able to access the index just with a simple * basepath pointing to the table. Until, then just always assume a BloomIndex */ - private final transient HoodieIndex index; - private HoodieTable hoodieTable; + private final transient HoodieIndex>, JavaRDD, JavaRDD> index; + private HoodieTable>, JavaRDD, JavaRDD> hoodieTable; private transient Option sqlContextOpt; - private final transient JavaSparkContext jsc; + private final transient HoodieSparkEngineContext context; private final transient Configuration hadoopConf; /** * @param basePath path to Hoodie table */ - public HoodieReadClient(JavaSparkContext jsc, String basePath) { - this(jsc, HoodieWriteConfig.newBuilder().withPath(basePath) + public HoodieReadClient(HoodieSparkEngineContext context, String basePath) { + this(context, HoodieWriteConfig.newBuilder().withPath(basePath) // by default we use HoodieBloomIndex .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build()); } /** - * @param jsc + * @param context * @param basePath * @param sqlContext */ - public HoodieReadClient(JavaSparkContext jsc, String basePath, SQLContext sqlContext) { - this(jsc, basePath); + public HoodieReadClient(HoodieSparkEngineContext context, String basePath, SQLContext sqlContext) { + this(context, basePath); this.sqlContextOpt = Option.of(sqlContext); } /** * @param clientConfig instance of HoodieWriteConfig */ - public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) { - this.jsc = jsc; - this.hadoopConf = jsc.hadoopConfiguration(); + public HoodieReadClient(HoodieSparkEngineContext context, HoodieWriteConfig clientConfig) { + this.context = context; + this.hadoopConf = context.getHadoopConf().get(); final String basePath = clientConfig.getBasePath(); // Create a Hoodie table which encapsulated the commits and files visible HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath, true); - this.hoodieTable = HoodieTable.create(metaClient, clientConfig, hadoopConf); - this.index = HoodieIndex.createIndex(clientConfig); + this.hoodieTable = HoodieSparkTable.create(clientConfig, context, metaClient); + this.index = SparkHoodieIndex.createIndex(clientConfig); this.sqlContextOpt = Option.empty(); } @@ -134,8 +136,7 @@ public class HoodieReadClient implements Serializ */ public Dataset readROView(JavaRDD hoodieKeys, int parallelism) { assertSqlContext(); - JavaPairRDD>> lookupResultRDD = - index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable); + JavaPairRDD>> lookupResultRDD = checkExists(hoodieKeys); JavaPairRDD> keyToFileRDD = lookupResultRDD.mapToPair(r -> new Tuple2<>(r._1, convertToDataFilePath(r._2))); List paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent()) @@ -161,8 +162,12 @@ public class HoodieReadClient implements Serializ * FullFilePath value is not present, then the key is not found. If the FullFilePath value is present, it is the path * component (without scheme) of the URI underlying file */ - public JavaPairRDD> checkExists(JavaRDD hoodieKeys) { - return index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable); + public JavaPairRDD>> checkExists(JavaRDD hoodieKeys) { + return index.tagLocation(hoodieKeys.map(k -> new HoodieRecord<>(k, null)), context, hoodieTable) + .mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown() + ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId())) + : Option.empty()) + ); } /** @@ -184,12 +189,12 @@ public class HoodieReadClient implements Serializ * @return Tagged RDD of Hoodie records */ public JavaRDD> tagLocation(JavaRDD> hoodieRecords) throws HoodieIndexException { - return index.tagLocation(hoodieRecords, jsc, hoodieTable); + return index.tagLocation(hoodieRecords, context, hoodieTable); } /** * Return all pending compactions with instant time for clients to decide what to compact next. - * + * * @return */ public List> getPendingCompactions() { diff --git a/hudi-client/src/main/java/org/apache/hudi/client/Compactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java similarity index 76% rename from hudi-client/src/main/java/org/apache/hudi/client/Compactor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java index b235e25e1..68e6da777 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/Compactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java @@ -18,31 +18,27 @@ package org.apache.hudi.client; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; - import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import java.io.IOException; -import java.io.Serializable; -/** - * Run one round of compaction. - */ -public class Compactor implements Serializable { +public class HoodieSparkCompactor extends AbstractCompactor>, JavaRDD, JavaRDD> { + private static final Logger LOG = LogManager.getLogger(HoodieSparkCompactor.class); - private static final long serialVersionUID = 1L; - private static final Logger LOG = LogManager.getLogger(Compactor.class); - - private transient HoodieWriteClient compactionClient; - - public Compactor(HoodieWriteClient compactionClient) { - this.compactionClient = compactionClient; + public HoodieSparkCompactor(AbstractHoodieWriteClient>, JavaRDD, JavaRDD> compactionClient) { + super(compactionClient); } + @Override public void compact(HoodieInstant instant) throws IOException { LOG.info("Compactor executing compaction " + instant); JavaRDD res = compactionClient.compact(instant.getTimestamp()); diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteResult.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieWriteResult.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteResult.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieWriteResult.java diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java new file mode 100644 index 000000000..56f06898a --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -0,0 +1,319 @@ +/* + * 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.client; + +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.compact.SparkCompactHelpers; +import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade; + +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; + +import java.io.IOException; +import java.text.ParseException; +import java.util.List; +import java.util.Map; + +@SuppressWarnings("checkstyle:LineLength") +public class SparkRDDWriteClient extends + AbstractHoodieWriteClient>, JavaRDD, JavaRDD> { + + private static final Logger LOG = LogManager.getLogger(SparkRDDWriteClient.class); + + public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) { + super(context, clientConfig); + } + + public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) { + super(context, writeConfig, rollbackPending); + } + + public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending, + Option timelineService) { + super(context, writeConfig, rollbackPending, timelineService); + } + + /** + * Register hudi classes for Kryo serialization. + * + * @param conf instance of SparkConf + * @return SparkConf + */ + public static SparkConf registerClasses(SparkConf conf) { + conf.registerKryoClasses(new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); + return conf; + } + + @Override + protected HoodieIndex>, JavaRDD, JavaRDD> createIndex(HoodieWriteConfig writeConfig) { + return SparkHoodieIndex.createIndex(config); + } + + /** + * Complete changes performed at the given instantTime marker with specified action. + */ + @Override + public boolean commit(String instantTime, JavaRDD writeStatuses, Option> extraMetadata, + String commitActionType, Map> partitionToReplacedFileIds) { + List writeStats = writeStatuses.map(WriteStatus::getStat).collect(); + return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds); + } + + @Override + protected HoodieTable>, JavaRDD, JavaRDD> createTable(HoodieWriteConfig config, + Configuration hadoopConf) { + return HoodieSparkTable.create(config, context); + } + + @Override + public JavaRDD> filterExists(JavaRDD> hoodieRecords) { + // Create a Hoodie table which encapsulated the commits and files visible + HoodieSparkTable table = HoodieSparkTable.create(config, context); + Timer.Context indexTimer = metrics.getIndexCtx(); + JavaRDD> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table); + metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop())); + return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); + } + + /** + * Main API to run bootstrap to hudi. + */ + @Override + public void bootstrap(Option> extraMetadata) { + if (rollbackPending) { + rollBackInflightBootstrap(); + } + getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS).bootstrap(context, extraMetadata); + } + + @Override + public JavaRDD upsert(JavaRDD> records, String instantTime) { + HoodieTable>, JavaRDD, JavaRDD> table = + getTableAndInitCtx(WriteOperationType.UPSERT, instantTime); + table.validateUpsertSchema(); + setOperationType(WriteOperationType.UPSERT); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + HoodieWriteMetadata> result = table.upsert(context, instantTime, records); + if (result.getIndexLookupDuration().isPresent()) { + metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); + } + return postWrite(result, instantTime, table); + } + + @Override + public JavaRDD upsertPreppedRecords(JavaRDD> preppedRecords, String instantTime) { + HoodieTable>, JavaRDD, JavaRDD> table = + getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime); + table.validateUpsertSchema(); + setOperationType(WriteOperationType.UPSERT_PREPPED); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, preppedRecords); + return postWrite(result, instantTime, table); + } + + @Override + public JavaRDD insert(JavaRDD> records, String instantTime) { + HoodieTable>, JavaRDD, JavaRDD> table = + getTableAndInitCtx(WriteOperationType.INSERT, instantTime); + table.validateInsertSchema(); + setOperationType(WriteOperationType.INSERT); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + HoodieWriteMetadata> result = table.insert(context,instantTime, records); + return postWrite(result, instantTime, table); + } + + @Override + public JavaRDD insertPreppedRecords(JavaRDD> preppedRecords, String instantTime) { + HoodieTable>, JavaRDD, JavaRDD> table = + getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime); + table.validateInsertSchema(); + setOperationType(WriteOperationType.INSERT_PREPPED); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, preppedRecords); + return postWrite(result, instantTime, table); + } + + /** + * Removes all existing records from the partitions affected and inserts the given HoodieRecords, into the table. + + * @param records HoodieRecords to insert + * @param instantTime Instant time of the commit + * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts + */ + public HoodieWriteResult insertOverwrite(JavaRDD> records, final String instantTime) { + HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime); + table.validateInsertSchema(); + setOperationType(WriteOperationType.INSERT_OVERWRITE); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + HoodieWriteMetadata result = table.insertOverwrite(context, instantTime, records); + return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); + } + + @Override + public JavaRDD bulkInsert(JavaRDD> records, String instantTime) { + return bulkInsert(records, instantTime, Option.empty()); + } + + @Override + public JavaRDD bulkInsert(JavaRDD> records, String instantTime, Option>>> userDefinedBulkInsertPartitioner) { + HoodieTable>, JavaRDD, JavaRDD> table = + getTableAndInitCtx(WriteOperationType.BULK_INSERT, instantTime); + table.validateInsertSchema(); + setOperationType(WriteOperationType.BULK_INSERT); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + HoodieWriteMetadata> result = table.bulkInsert(context,instantTime, records, userDefinedBulkInsertPartitioner); + return postWrite(result, instantTime, table); + } + + @Override + public JavaRDD bulkInsertPreppedRecords(JavaRDD> preppedRecords, String instantTime, Option>>> bulkInsertPartitioner) { + HoodieTable>, JavaRDD, JavaRDD> table = + getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED, instantTime); + table.validateInsertSchema(); + setOperationType(WriteOperationType.BULK_INSERT_PREPPED); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + HoodieWriteMetadata> result = table.bulkInsertPrepped(context,instantTime, preppedRecords, bulkInsertPartitioner); + return postWrite(result, instantTime, table); + } + + @Override + public JavaRDD delete(JavaRDD keys, String instantTime) { + HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime); + setOperationType(WriteOperationType.DELETE); + HoodieWriteMetadata> result = table.delete(context,instantTime, keys); + return postWrite(result, instantTime, table); + } + + @Override + protected JavaRDD postWrite(HoodieWriteMetadata> result, + String instantTime, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { + if (result.getIndexLookupDuration().isPresent()) { + metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis()); + } + if (result.isCommitted()) { + // Perform post commit operations. + if (result.getFinalizeDuration().isPresent()) { + metrics.updateFinalizeWriteMetrics(result.getFinalizeDuration().get().toMillis(), + result.getWriteStats().get().size()); + } + + postCommit(hoodieTable, result.getCommitMetadata().get(), instantTime, Option.empty()); + + emitCommitMetrics(instantTime, result.getCommitMetadata().get(), hoodieTable.getMetaClient().getCommitActionType()); + } + return result.getWriteStatuses(); + } + + @Override + public void commitCompaction(String compactionInstantTime, JavaRDD writeStatuses, Option> extraMetadata) throws IOException { + HoodieSparkTable table = HoodieSparkTable.create(config, context); + HoodieCommitMetadata metadata = SparkCompactHelpers.newInstance().createCompactionMetadata( + table, compactionInstantTime, writeStatuses, config.getSchema()); + extraMetadata.ifPresent(m -> m.forEach(metadata::addMetadata)); + completeCompaction(metadata, writeStatuses, table, compactionInstantTime); + } + + @Override + protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD writeStatuses, + HoodieTable>, JavaRDD, JavaRDD> table, + String compactionCommitTime) { + List writeStats = writeStatuses.map(WriteStatus::getStat).collect(); + finalizeWrite(table, compactionCommitTime, writeStats); + LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata); + SparkCompactHelpers.newInstance().completeInflightCompaction(table, compactionCommitTime, metadata); + + if (compactionTimer != null) { + long durationInMs = metrics.getDurationInMs(compactionTimer.stop()); + try { + metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(compactionCommitTime).getTime(), + durationInMs, metadata, HoodieActiveTimeline.COMPACTION_ACTION); + } catch (ParseException e) { + throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction " + + config.getBasePath() + " at time " + compactionCommitTime, e); + } + } + LOG.info("Compacted successfully on commit " + compactionCommitTime); + } + + @Override + protected JavaRDD compact(String compactionInstantTime, boolean shouldComplete) { + HoodieSparkTable table = HoodieSparkTable.create(config, context); + HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); + HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); + if (pendingCompactionTimeline.containsInstant(inflightInstant)) { + rollbackInflightCompaction(inflightInstant, table); + table.getMetaClient().reloadActiveTimeline(); + } + compactionTimer = metrics.getCompactionCtx(); + HoodieWriteMetadata> compactionMetadata = table.compact(context, compactionInstantTime); + JavaRDD statuses = compactionMetadata.getWriteStatuses(); + if (shouldComplete && compactionMetadata.getCommitMetadata().isPresent()) { + completeCompaction(compactionMetadata.getCommitMetadata().get(), statuses, table, compactionInstantTime); + } + return statuses; + } + + @Override + protected HoodieTable>, JavaRDD, JavaRDD> getTableAndInitCtx(WriteOperationType operationType, String instantTime) { + HoodieTableMetaClient metaClient = createMetaClient(true); + new SparkUpgradeDowngrade(metaClient, config, context).run(metaClient, HoodieTableVersion.current(), config, context, instantTime); + return getTableAndInitCtx(metaClient, operationType); + } + + private HoodieTable>, JavaRDD, JavaRDD> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) { + if (operationType == WriteOperationType.DELETE) { + setWriteSchemaForDeletes(metaClient); + } + // Create a Hoodie table which encapsulated the commits and files visible + HoodieSparkTable table = HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient); + if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) { + writeTimer = metrics.getCommitCtx(); + } else { + writeTimer = metrics.getDeltaCommitCtx(); + } + return table; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java new file mode 100644 index 000000000..563bc1a77 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java @@ -0,0 +1,84 @@ +/* + * 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.client; + +import org.apache.hudi.client.common.EngineProperty; +import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; + +import org.apache.spark.SparkEnv; +import org.apache.spark.TaskContext; +import org.apache.spark.util.Utils; + +import java.io.Serializable; +import java.util.function.Supplier; + +/** + * Spark task context supplier. + */ +public class SparkTaskContextSupplier extends TaskContextSupplier implements Serializable { + + @Override + public Supplier getPartitionIdSupplier() { + return TaskContext::getPartitionId; + } + + @Override + public Supplier getStageIdSupplier() { + return () -> TaskContext.get().stageId(); + } + + @Override + public Supplier getAttemptIdSupplier() { + return () -> TaskContext.get().taskAttemptId(); + } + + @Override + public Option getProperty(EngineProperty prop) { + if (prop == EngineProperty.TOTAL_MEMORY_AVAILABLE) { + // This is hard-coded in spark code {@link + // https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/ + // spark/SparkContext.scala#L471} so have to re-define this here + final String DEFAULT_SPARK_EXECUTOR_MEMORY_MB = "1024"; // in MB + final String SPARK_EXECUTOR_MEMORY_PROP = "spark.executor.memory"; + if (SparkEnv.get() != null) { + // 1 GB is the default conf used by Spark, look at SparkContext.scala + return Option.ofNullable(String.valueOf(Utils.memoryStringToMb(SparkEnv.get().conf() + .get(SPARK_EXECUTOR_MEMORY_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024 * 1024L)); + } + return Option.empty(); + } else if (prop == EngineProperty.MEMORY_FRACTION_IN_USE) { + // This is hard-coded in spark code {@link + // https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/ + // spark/memory/UnifiedMemoryManager.scala#L231} so have to re-define this here + final String DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION = "0.6"; + final String SPARK_EXECUTOR_MEMORY_FRACTION_PROP = "spark.memory.fraction"; + if (SparkEnv.get() != null) { + // 0.6 is the default value used by Spark, + // look at {@link + // https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507} + return Option.ofNullable(SparkEnv.get().conf() + .get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION)); + } + return Option.empty(); + } + throw new HoodieException("Unknown engine property :" + prop); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapSchemaProvider.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java similarity index 59% rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapSchemaProvider.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java index 2018ca9ba..30cde59fe 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapSchemaProvider.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java @@ -18,18 +18,17 @@ package org.apache.hudi.client.bootstrap; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.bootstrap.FileStatusUtils; import org.apache.hudi.common.util.ParquetUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.parquet.schema.MessageType; -import org.apache.spark.api.java.JavaSparkContext; - -import org.apache.avro.Schema; -import org.apache.hadoop.fs.Path; import org.apache.spark.sql.avro.SchemaConverters; import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter; import org.apache.spark.sql.internal.SQLConf; @@ -38,47 +37,17 @@ import org.apache.spark.sql.types.StructType; import java.util.List; import java.util.Objects; -/** - * Bootstrap Schema Provider. Schema provided in config is used. If not available, use schema from Parquet - */ -public class BootstrapSchemaProvider { - - protected final HoodieWriteConfig writeConfig; - - public BootstrapSchemaProvider(HoodieWriteConfig writeConfig) { - this.writeConfig = writeConfig; +public class HoodieSparkBootstrapSchemaProvider extends HoodieBootstrapSchemaProvider { + public HoodieSparkBootstrapSchemaProvider(HoodieWriteConfig writeConfig) { + super(writeConfig); } - /** - * Main API to select avro schema for bootstrapping. - * @param jsc Java Spark Context - * @param partitions List of partitions with files within them - * @return Avro Schema - */ - public final Schema getBootstrapSchema(JavaSparkContext jsc, List>> partitions) { - if (writeConfig.getSchema() != null) { - // Use schema specified by user if set - Schema userSchema = Schema.parse(writeConfig.getSchema()); - if (!HoodieAvroUtils.getNullSchema().equals(userSchema)) { - return userSchema; - } - } - return getBootstrapSourceSchema(jsc, partitions); - } - - /** - * Select a random file to be used to generate avro schema. - * Override this method to get custom schema selection. - * @param jsc Java Spark Context - * @param partitions List of partitions with files within them - * @return Avro Schema - */ - protected Schema getBootstrapSourceSchema(JavaSparkContext jsc, - List>> partitions) { + @Override + protected Schema getBootstrapSourceSchema(HoodieEngineContext context, List>> partitions) { MessageType parquetSchema = partitions.stream().flatMap(p -> p.getValue().stream()).map(fs -> { try { Path filePath = FileStatusUtils.toPath(fs.getPath()); - return ParquetUtils.readSchema(jsc.hadoopConfiguration(), filePath); + return ParquetUtils.readSchema(context.getHadoopConf().get(), filePath); } catch (Exception ex) { return null; } @@ -87,8 +56,8 @@ public class BootstrapSchemaProvider { ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter( - Boolean.parseBoolean(SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()), - Boolean.parseBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString())); + Boolean.parseBoolean(SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()), + Boolean.parseBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString())); StructType sparkSchema = converter.convert(parquetSchema); String tableName = HoodieAvroUtils.sanitizeName(writeConfig.getTableName()); String structName = tableName + "_record"; @@ -96,5 +65,4 @@ public class BootstrapSchemaProvider { return SchemaConverters.toAvroType(sparkSchema, false, structName, recordNamespace); } - } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java new file mode 100644 index 000000000..0f17511b5 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java @@ -0,0 +1,112 @@ +/* + * 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.client.common; + +import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.client.common.function.SerializableConsumer; +import org.apache.hudi.client.common.function.SerializableFunction; +import org.apache.hudi.client.common.function.SerializablePairFunction; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SQLContext; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Stream; + +/** + * A Spark engine implementation of HoodieEngineContext. + */ +public class HoodieSparkEngineContext extends HoodieEngineContext { + + private final JavaSparkContext javaSparkContext; + private SQLContext sqlContext; + + public HoodieSparkEngineContext(JavaSparkContext jsc) { + super(new SerializableConfiguration(jsc.hadoopConfiguration()), new SparkTaskContextSupplier()); + this.javaSparkContext = jsc; + this.sqlContext = SQLContext.getOrCreate(jsc.sc()); + } + + public void setSqlContext(SQLContext sqlContext) { + this.sqlContext = sqlContext; + } + + public JavaSparkContext getJavaSparkContext() { + return javaSparkContext; + } + + public SQLContext getSqlContext() { + return sqlContext; + } + + public static JavaSparkContext getSparkContext(HoodieEngineContext context) { + return ((HoodieSparkEngineContext) context).getJavaSparkContext(); + } + + @Override + public List map(List data, SerializableFunction func, int parallelism) { + return javaSparkContext.parallelize(data, parallelism).map(func::apply).collect(); + } + + @Override + public List flatMap(List data, SerializableFunction> func, int parallelism) { + return javaSparkContext.parallelize(data, parallelism).flatMap(x -> func.apply(x).iterator()).collect(); + } + + @Override + public void foreach(List data, SerializableConsumer consumer, int parallelism) { + javaSparkContext.parallelize(data, parallelism).foreach(consumer::accept); + } + + @Override + public Map mapToPair(List data, SerializablePairFunction func, Integer parallelism) { + if (Objects.nonNull(parallelism)) { + return javaSparkContext.parallelize(data, parallelism).mapToPair(func::call).collectAsMap(); + } else { + return javaSparkContext.parallelize(data).mapToPair(func::call).collectAsMap(); + } + } + + @Override + public void setProperty(EngineProperty key, String value) { + if (key == EngineProperty.COMPACTION_POOL_NAME) { + javaSparkContext.setLocalProperty("spark.scheduler.pool", value); + } else { + throw new HoodieException("Unknown engine property :" + key); + } + } + + @Override + public Option getProperty(EngineProperty key) { + if (key == EngineProperty.EMBEDDED_SERVER_HOST) { + return Option.ofNullable(javaSparkContext.getConf().get("spark.driver.host", null)); + } + throw new HoodieException("Unknown engine property :" + key); + } + + @Override + public void setJobStatus(String activeModule, String activityDescription) { + javaSparkContext.setJobGroup(activeModule, activityDescription); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java new file mode 100644 index 000000000..0fa75dc61 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java @@ -0,0 +1,44 @@ +/* + * 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.client.utils; + +import org.apache.hudi.config.HoodieIndexConfig; + +import org.apache.spark.storage.StorageLevel; + +import java.util.Properties; + +import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL; + +/** + * Spark config utils. + */ +public class SparkMemoryUtils { + public static StorageLevel getWriteStatusStorageLevel(Properties properties) { + return StorageLevel.fromString(properties.getProperty(WRITE_STATUS_STORAGE_LEVEL)); + } + + public static StorageLevel getBloomIndexInputStorageLevel(Properties properties) { + return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL)); + } + + public static StorageLevel getSimpleIndexInputStorageLevel(Properties properties) { + return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL)); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/SparkBoundedInMemoryExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkBoundedInMemoryExecutor.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/execution/SparkBoundedInMemoryExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkBoundedInMemoryExecutor.java diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java new file mode 100644 index 000000000..ec90ef88e --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java @@ -0,0 +1,78 @@ +/* + * 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.execution; + +import org.apache.avro.Schema; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.io.WriteHandleFactory; +import org.apache.hudi.table.HoodieTable; + +import java.util.Iterator; +import java.util.List; + +public class SparkLazyInsertIterable extends HoodieLazyInsertIterable { + + public SparkLazyInsertIterable(Iterator> recordItr, + boolean areRecordsSorted, + HoodieWriteConfig config, + String instantTime, + HoodieTable hoodieTable, + String idPrefix, + TaskContextSupplier taskContextSupplier) { + super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier); + } + + public SparkLazyInsertIterable(Iterator> recordItr, + boolean areRecordsSorted, + HoodieWriteConfig config, + String instantTime, + HoodieTable hoodieTable, + String idPrefix, + TaskContextSupplier taskContextSupplier, + WriteHandleFactory writeHandleFactory) { + super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier, writeHandleFactory); + } + + @Override + protected List computeNext() { + // Executor service used for launching writer thread. + BoundedInMemoryExecutor, HoodieInsertValueGenResult, List> bufferedIteratorExecutor = + null; + try { + final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); + bufferedIteratorExecutor = + new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, getInsertHandler(), getTransformFunction(schema)); + final List result = bufferedIteratorExecutor.execute(); + assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); + return result; + } catch (Exception e) { + throw new HoodieException(e); + } finally { + if (null != bufferedIteratorExecutor) { + bufferedIteratorExecutor.shutdownNow(); + } + } + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java similarity index 87% rename from hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java index 71c10ed6b..db73a9c3e 100644 --- a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java @@ -22,7 +22,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.execution.LazyInsertIterable; +import org.apache.hudi.execution.SparkLazyInsertIterable; import org.apache.hudi.table.HoodieTable; import org.apache.spark.api.java.function.Function2; @@ -39,11 +39,11 @@ public class BulkInsertMapFunction private String instantTime; private boolean areRecordsSorted; private HoodieWriteConfig config; - private HoodieTable hoodieTable; + private HoodieTable hoodieTable; private List fileIDPrefixes; public BulkInsertMapFunction(String instantTime, boolean areRecordsSorted, - HoodieWriteConfig config, HoodieTable hoodieTable, + HoodieWriteConfig config, HoodieTable hoodieTable, List fileIDPrefixes) { this.instantTime = instantTime; this.areRecordsSorted = areRecordsSorted; @@ -54,7 +54,7 @@ public class BulkInsertMapFunction @Override public Iterator> call(Integer partition, Iterator> recordItr) { - return new LazyInsertIterable<>(recordItr, areRecordsSorted, config, instantTime, hoodieTable, - fileIDPrefixes.get(partition), hoodieTable.getSparkTaskContextSupplier()); + return new SparkLazyInsertIterable<>(recordItr, areRecordsSorted, config, instantTime, hoodieTable, + fileIDPrefixes.get(partition), hoodieTable.getTaskContextSupplier()); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java similarity index 97% rename from hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java index 3f4077a83..a184c009a 100644 --- a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java @@ -32,7 +32,7 @@ import org.apache.spark.api.java.JavaRDD; * @param HoodieRecordPayload type */ public class GlobalSortPartitioner - implements BulkInsertPartitioner { + implements BulkInsertPartitioner>> { @Override public JavaRDD> repartitionRecords(JavaRDD> records, diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java similarity index 96% rename from hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java index 571b8aaae..19c90ecb1 100644 --- a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java @@ -31,7 +31,7 @@ import org.apache.spark.api.java.JavaRDD; * @param HoodieRecordPayload type */ public class NonSortPartitioner - implements BulkInsertPartitioner { + implements BulkInsertPartitioner>> { @Override public JavaRDD> repartitionRecords(JavaRDD> records, diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java similarity index 97% rename from hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java index 9fc91a4de..9526ad585 100644 --- a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java @@ -38,7 +38,7 @@ import scala.Tuple2; * @param HoodieRecordPayload type */ public class RDDPartitionSortPartitioner - implements BulkInsertPartitioner { + implements BulkInsertPartitioner>> { @Override public JavaRDD> repartitionRecords(JavaRDD> records, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java new file mode 100644 index 000000000..dd73bf2e7 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java @@ -0,0 +1,85 @@ +/* + * 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.index; + +import org.apache.hudi.ApiMaturityLevel; +import org.apache.hudi.PublicAPIMethod; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.index.bloom.SparkHoodieBloomIndex; +import org.apache.hudi.index.bloom.SparkHoodieGlobalBloomIndex; +import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex; +import org.apache.hudi.index.simple.SparkHoodieGlobalSimpleIndex; +import org.apache.hudi.index.simple.SparkHoodieSimpleIndex; +import org.apache.hudi.table.HoodieTable; + +import org.apache.spark.api.java.JavaRDD; + +@SuppressWarnings("checkstyle:LineLength") +public abstract class SparkHoodieIndex extends HoodieIndex>, JavaRDD, JavaRDD> { + protected SparkHoodieIndex(HoodieWriteConfig config) { + super(config); + } + + public static SparkHoodieIndex createIndex(HoodieWriteConfig config) { + // first use index class config to create index. + if (!StringUtils.isNullOrEmpty(config.getIndexClass())) { + Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config); + if (!(instance instanceof HoodieIndex)) { + throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex"); + } + return (SparkHoodieIndex) instance; + } + switch (config.getIndexType()) { + case HBASE: + return new SparkHoodieHBaseIndex<>(config); + case INMEMORY: + return new SparkInMemoryHashIndex(config); + case BLOOM: + return new SparkHoodieBloomIndex<>(config); + case GLOBAL_BLOOM: + return new SparkHoodieGlobalBloomIndex<>(config); + case SIMPLE: + return new SparkHoodieSimpleIndex(config); + case GLOBAL_SIMPLE: + return new SparkHoodieGlobalSimpleIndex(config); + default: + throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); + } + } + + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + public abstract JavaRDD updateLocation(JavaRDD writeStatusRDD, + HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) throws HoodieIndexException; + + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + public abstract JavaRDD> tagLocation(JavaRDD> records, + HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) throws HoodieIndexException; +} diff --git a/hudi-client/src/main/java/org/apache/hudi/index/InMemoryHashIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java similarity index 69% rename from hudi-client/src/main/java/org/apache/hudi/index/InMemoryHashIndex.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java index 49512ab86..55ce8d2cc 100644 --- a/hudi-client/src/main/java/org/apache/hudi/index/InMemoryHashIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java @@ -19,19 +19,16 @@ package org.apache.hudi.index; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.Pair; 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.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.Function2; import java.util.ArrayList; @@ -45,13 +42,14 @@ import java.util.concurrent.ConcurrentMap; *

* ONLY USE FOR LOCAL TESTING */ -public class InMemoryHashIndex extends HoodieIndex { +@SuppressWarnings("checkstyle:LineLength") +public class SparkInMemoryHashIndex extends SparkHoodieIndex { private static ConcurrentMap recordLocationMap; - public InMemoryHashIndex(HoodieWriteConfig config) { + public SparkInMemoryHashIndex(HoodieWriteConfig config) { super(config); - synchronized (InMemoryHashIndex.class) { + synchronized (SparkInMemoryHashIndex.class) { if (recordLocationMap == null) { recordLocationMap = new ConcurrentHashMap<>(); } @@ -59,37 +57,29 @@ public class InMemoryHashIndex extends HoodieInde } @Override - public JavaPairRDD>> fetchRecordLocation(JavaRDD hoodieKeys, - JavaSparkContext jsc, HoodieTable hoodieTable) { - throw new UnsupportedOperationException("InMemory index does not implement check exist yet"); - } - - @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) { + public JavaRDD> tagLocation(JavaRDD> recordRDD, HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true); } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) { - return writeStatusRDD.map(new Function() { - @Override - public WriteStatus call(WriteStatus writeStatus) { - for (HoodieRecord record : writeStatus.getWrittenRecords()) { - if (!writeStatus.isErrored(record.getKey())) { - HoodieKey key = record.getKey(); - Option newLocation = record.getNewLocation(); - if (newLocation.isPresent()) { - recordLocationMap.put(key, newLocation.get()); - } else { - // Delete existing index for a deleted record - recordLocationMap.remove(key); - } + public JavaRDD updateLocation(JavaRDD writeStatusRDD, + HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { + return writeStatusRDD.map(writeStatus -> { + for (HoodieRecord record : writeStatus.getWrittenRecords()) { + if (!writeStatus.isErrored(record.getKey())) { + HoodieKey key = record.getKey(); + Option newLocation = record.getNewLocation(); + if (newLocation.isPresent()) { + recordLocationMap.put(key, newLocation.get()); + } else { + // Delete existing index for a deleted record + recordLocationMap.remove(key); } } - return writeStatus; } + return writeStatus; }); } diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java similarity index 81% rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java index e03f38ac0..894b41b51 100644 --- a/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java @@ -19,7 +19,8 @@ package org.apache.hudi.index.bloom; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkConfigUtils; +import org.apache.hudi.client.utils.SparkMemoryUtils; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -28,8 +29,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.MetadataNotFoundException; -import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.index.SparkHoodieIndex; import org.apache.hudi.io.HoodieRangeInfoHandle; import org.apache.hudi.table.HoodieTable; @@ -38,7 +39,6 @@ import org.apache.log4j.Logger; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.storage.StorageLevel; import java.util.ArrayList; @@ -57,21 +57,22 @@ import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPar /** * Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in its metadata. */ -public class HoodieBloomIndex extends HoodieIndex { +@SuppressWarnings("checkstyle:LineLength") +public class SparkHoodieBloomIndex extends SparkHoodieIndex { - private static final Logger LOG = LogManager.getLogger(HoodieBloomIndex.class); + private static final Logger LOG = LogManager.getLogger(SparkHoodieBloomIndex.class); - public HoodieBloomIndex(HoodieWriteConfig config) { + public SparkHoodieBloomIndex(HoodieWriteConfig config) { super(config); } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) { + public JavaRDD> tagLocation(JavaRDD> recordRDD, HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { // Step 0: cache the input record RDD if (config.getBloomIndexUseCaching()) { - recordRDD.persist(SparkConfigUtils.getBloomIndexInputStorageLevel(config.getProps())); + recordRDD.persist(SparkMemoryUtils.getBloomIndexInputStorageLevel(config.getProps())); } // Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey) @@ -80,7 +81,7 @@ public class HoodieBloomIndex extends HoodieIndex // Lookup indexes for all the partition/recordkey pair JavaPairRDD keyFilenamePairRDD = - lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable); + lookupIndex(partitionRecordKeyPairRDD, context, hoodieTable); // Cache the result, for subsequent stages. if (config.getBloomIndexUseCaching()) { @@ -102,42 +103,12 @@ public class HoodieBloomIndex extends HoodieIndex return taggedRecordRDD; } - /** - * Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option.Empty if the key is not - * found. - * - * @param hoodieKeys keys to lookup - * @param jsc spark context - * @param hoodieTable hoodie table object - */ - @Override - public JavaPairRDD>> fetchRecordLocation(JavaRDD hoodieKeys, - JavaSparkContext jsc, HoodieTable hoodieTable) { - JavaPairRDD partitionRecordKeyPairRDD = - hoodieKeys.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey())); - - // Lookup indexes for all the partition/recordkey pair - JavaPairRDD recordKeyLocationRDD = - lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable); - JavaPairRDD keyHoodieKeyPairRDD = hoodieKeys.mapToPair(key -> new Tuple2<>(key, null)); - - return keyHoodieKeyPairRDD.leftOuterJoin(recordKeyLocationRDD).mapToPair(keyLoc -> { - Option> partitionPathFileidPair; - if (keyLoc._2._2.isPresent()) { - partitionPathFileidPair = Option.of(Pair.of(keyLoc._1().getPartitionPath(), keyLoc._2._2.get().getFileId())); - } else { - partitionPathFileidPair = Option.empty(); - } - return new Tuple2<>(keyLoc._1, partitionPathFileidPair); - }); - } - /** * Lookup the location for each record key and return the pair for all record keys already * present and drop the record keys if not present. */ private JavaPairRDD lookupIndex( - JavaPairRDD partitionRecordKeyPairRDD, final JavaSparkContext jsc, + JavaPairRDD partitionRecordKeyPairRDD, final HoodieEngineContext context, final HoodieTable hoodieTable) { // Obtain records per partition, in the incoming records Map recordsPerPartition = partitionRecordKeyPairRDD.countByKey(); @@ -145,7 +116,7 @@ public class HoodieBloomIndex extends HoodieIndex // Step 2: Load all involved files as pairs List> fileInfoList = - loadInvolvedFiles(affectedPartitionPathList, jsc, hoodieTable); + loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable); final Map> partitionToFileInfo = fileInfoList.stream().collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList()))); @@ -189,27 +160,27 @@ public class HoodieBloomIndex extends HoodieIndex /** * Load all involved files as pair RDD. */ - List> loadInvolvedFiles(List partitions, final JavaSparkContext jsc, + List> loadInvolvedFiles(List partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) { // Obtain the latest data files from all the partitions. - List> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, jsc, hoodieTable).stream() + List> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable).stream() .map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId())) .collect(toList()); if (config.getBloomIndexPruneByRanges()) { // also obtain file ranges, if range pruning is enabled - jsc.setJobDescription("Obtain key ranges for file slices (range pruning=on)"); - return jsc.parallelize(partitionPathFileIDList, Math.max(partitionPathFileIDList.size(), 1)).mapToPair(pf -> { + context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)"); + return context.map(partitionPathFileIDList, pf -> { try { - HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf); + HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf); String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys(); return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1])); } catch (MetadataNotFoundException me) { LOG.warn("Unable to find range metadata in file :" + pf); return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue())); } - }).collect(); + }, Math.max(partitionPathFileIDList.size(), 1)); } else { return partitionPathFileIDList.stream() .map(pf -> new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue()))).collect(toList()); @@ -320,8 +291,8 @@ public class HoodieBloomIndex extends HoodieIndex } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) { + public JavaRDD updateLocation(JavaRDD writeStatusRDD, HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { return writeStatusRDD; } } diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java similarity index 94% rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java index 4f93b3029..771c01ab8 100644 --- a/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java @@ -18,6 +18,7 @@ package org.apache.hudi.index.bloom; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; @@ -33,7 +34,6 @@ 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.apache.spark.api.java.Optional; import java.io.IOException; @@ -49,9 +49,9 @@ import scala.Tuple2; * This filter will only work with hoodie table since it will only load partitions with .hoodie_partition_metadata * file in it. */ -public class HoodieGlobalBloomIndex extends HoodieBloomIndex { +public class SparkHoodieGlobalBloomIndex extends SparkHoodieBloomIndex { - public HoodieGlobalBloomIndex(HoodieWriteConfig config) { + public SparkHoodieGlobalBloomIndex(HoodieWriteConfig config) { super(config); } @@ -59,13 +59,13 @@ public class HoodieGlobalBloomIndex extends Hoodi * Load all involved files as pair RDD from all partitions in the table. */ @Override - List> loadInvolvedFiles(List partitions, final JavaSparkContext jsc, + List> loadInvolvedFiles(List partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); try { List allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning()); - return super.loadInvolvedFiles(allPartitionPaths, jsc, hoodieTable); + return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable); } catch (IOException e) { throw new HoodieIOException("Failed to load all partitions", e); } diff --git a/hudi-client/src/main/java/org/apache/hudi/index/hbase/HBaseIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java similarity index 94% rename from hudi-client/src/main/java/org/apache/hudi/index/hbase/HBaseIndex.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java index 4540f2f6e..21efd9b27 100644 --- a/hudi-client/src/main/java/org/apache/hudi/index/hbase/HBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java @@ -19,7 +19,9 @@ package org.apache.hudi.index.hbase; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkConfigUtils; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.utils.SparkMemoryUtils; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -28,12 +30,11 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieHBaseIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieDependentSystemUnavailableException; import org.apache.hudi.exception.HoodieIndexException; -import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.SparkHoodieIndex; import org.apache.hudi.table.HoodieTable; import org.apache.hadoop.conf.Configuration; @@ -71,7 +72,7 @@ import scala.Tuple2; /** * Hoodie Index implementation backed by HBase. */ -public class HBaseIndex extends HoodieIndex { +public class SparkHoodieHBaseIndex extends SparkHoodieIndex { public static final String DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME = "spark.executor.instances"; public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME = "spark.dynamicAllocation.enabled"; @@ -84,7 +85,7 @@ public class HBaseIndex extends HoodieIndex { private static final byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path"); private static final int SLEEP_TIME_MILLISECONDS = 100; - private static final Logger LOG = LogManager.getLogger(HBaseIndex.class); + private static final Logger LOG = LogManager.getLogger(SparkHoodieHBaseIndex.class); private static Connection hbaseConnection = null; private HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = null; private float qpsFraction; @@ -98,7 +99,7 @@ public class HBaseIndex extends HoodieIndex { private final String tableName; private HBasePutBatchSizeCalculator putBatchSizeCalculator; - public HBaseIndex(HoodieWriteConfig config) { + public SparkHoodieHBaseIndex(HoodieWriteConfig config) { super(config); this.tableName = config.getHbaseTableName(); addShutDownHook(); @@ -124,12 +125,6 @@ public class HBaseIndex extends HoodieIndex { return new DefaultHBaseQPSResourceAllocator(config); } - @Override - public JavaPairRDD>> fetchRecordLocation(JavaRDD hoodieKeys, - JavaSparkContext jsc, HoodieTable hoodieTable) { - throw new UnsupportedOperationException("HBase index does not implement check exist"); - } - private Connection getHBaseConnection() { Configuration hbaseConfig = HBaseConfiguration.create(); String quorum = config.getHbaseZkQuorum(); @@ -195,7 +190,7 @@ public class HBaseIndex extends HoodieIndex { int multiGetBatchSize = config.getHbaseIndexGetBatchSize(); // Grab the global HBase connection - synchronized (HBaseIndex.class) { + synchronized (SparkHoodieHBaseIndex.class) { if (hbaseConnection == null || hbaseConnection.isClosed()) { hbaseConnection = getHBaseConnection(); } @@ -255,8 +250,9 @@ public class HBaseIndex extends HoodieIndex { } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) { + public JavaRDD> tagLocation(JavaRDD> recordRDD, + HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true); } @@ -266,7 +262,7 @@ public class HBaseIndex extends HoodieIndex { List writeStatusList = new ArrayList<>(); // Grab the global HBase connection - synchronized (HBaseIndex.class) { + synchronized (SparkHoodieHBaseIndex.class) { if (hbaseConnection == null || hbaseConnection.isClosed()) { hbaseConnection = getHBaseConnection(); } @@ -339,20 +335,21 @@ public class HBaseIndex extends HoodieIndex { } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) { + public JavaRDD updateLocation(JavaRDD writeStatusRDD, HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { final HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = createQPSResourceAllocator(this.config); - setPutBatchSize(writeStatusRDD, hBaseIndexQPSResourceAllocator, jsc); + setPutBatchSize(writeStatusRDD, hBaseIndexQPSResourceAllocator, context); LOG.info("multiPutBatchSize: before hbase puts" + multiPutBatchSize); JavaRDD writeStatusJavaRDD = writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true); // caching the index updated status RDD - writeStatusJavaRDD = writeStatusJavaRDD.persist(SparkConfigUtils.getWriteStatusStorageLevel(config.getProps())); + writeStatusJavaRDD = writeStatusJavaRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps())); return writeStatusJavaRDD; } private void setPutBatchSize(JavaRDD writeStatusRDD, - HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator, final JavaSparkContext jsc) { + HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator, final HoodieEngineContext context) { if (config.getHbaseIndexPutBatchSizeAutoCompute()) { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); SparkConf conf = jsc.getConf(); int maxExecutors = conf.getInt(DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME, 1); if (conf.getBoolean(DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME, false)) { @@ -493,6 +490,6 @@ public class HBaseIndex extends HoodieIndex { } public void setHbaseConnection(Connection hbaseConnection) { - HBaseIndex.hbaseConnection = hbaseConnection; + SparkHoodieHBaseIndex.hbaseConnection = hbaseConnection; } } diff --git a/hudi-client/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java similarity index 81% rename from hudi-client/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java index 990f02dce..bdb4991cf 100644 --- a/hudi-client/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java @@ -18,6 +18,8 @@ package org.apache.hudi.index.simple; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieBaseFile; @@ -34,7 +36,6 @@ import org.apache.hudi.index.HoodieIndexUtils; 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 scala.Tuple2; import java.io.IOException; @@ -50,31 +51,33 @@ import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPar * * @param */ -public class HoodieGlobalSimpleIndex extends HoodieSimpleIndex { +@SuppressWarnings("checkstyle:LineLength") +public class SparkHoodieGlobalSimpleIndex extends SparkHoodieSimpleIndex { - public HoodieGlobalSimpleIndex(HoodieWriteConfig config) { + public SparkHoodieGlobalSimpleIndex(HoodieWriteConfig config) { super(config); } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) { - return tagLocationInternal(recordRDD, jsc, hoodieTable); + public JavaRDD> tagLocation(JavaRDD> recordRDD, HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { + return tagLocationInternal(recordRDD, context, hoodieTable); } /** * Tags records location for incoming records. * * @param inputRecordRDD {@link JavaRDD} of incoming records - * @param jsc instance of {@link JavaSparkContext} to use + * @param context instance of {@link HoodieEngineContext} to use * @param hoodieTable instance of {@link HoodieTable} to use * @return {@link JavaRDD} of records with record locations set */ - protected JavaRDD> tagLocationInternal(JavaRDD> inputRecordRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) { + @Override + protected JavaRDD> tagLocationInternal(JavaRDD> inputRecordRDD, HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { JavaPairRDD> keyedInputRecordRDD = inputRecordRDD.mapToPair(entry -> new Tuple2<>(entry.getRecordKey(), entry)); - JavaPairRDD allRecordLocationsInTable = fetchAllRecordLocations(jsc, hoodieTable, + JavaPairRDD allRecordLocationsInTable = fetchAllRecordLocations(context, hoodieTable, config.getGlobalSimpleIndexParallelism()); return getTaggedRecords(keyedInputRecordRDD, allRecordLocationsInTable); } @@ -82,27 +85,28 @@ public class HoodieGlobalSimpleIndex extends Hood /** * Fetch record locations for passed in {@link HoodieKey}s. * - * @param jsc instance of {@link JavaSparkContext} to use + * @param context instance of {@link HoodieEngineContext} to use * @param hoodieTable instance of {@link HoodieTable} of interest * @param parallelism parallelism to use * @return {@link JavaPairRDD} of {@link HoodieKey} and {@link HoodieRecordLocation} */ - protected JavaPairRDD fetchAllRecordLocations(JavaSparkContext jsc, - HoodieTable hoodieTable, + protected JavaPairRDD fetchAllRecordLocations(HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable, int parallelism) { - List> latestBaseFiles = getAllBaseFilesInTable(jsc, hoodieTable); - return fetchRecordLocations(jsc, hoodieTable, parallelism, latestBaseFiles); + List> latestBaseFiles = getAllBaseFilesInTable(context, hoodieTable); + return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles); } /** * Load all files for all partitions as pair RDD. */ - protected List> getAllBaseFilesInTable(final JavaSparkContext jsc, final HoodieTable hoodieTable) { + protected List> getAllBaseFilesInTable(final HoodieEngineContext context, + final HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); try { List allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning()); // Obtain the latest data files from all the partitions. - return getLatestBaseFilesForAllPartitions(allPartitionPaths, jsc, hoodieTable); + return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, hoodieTable); } catch (IOException e) { throw new HoodieIOException("Failed to load all partitions", e); } @@ -149,21 +153,6 @@ public class HoodieGlobalSimpleIndex extends Hood }); } - /** - * Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option.Empty if the key is not. - * found. - * - * @param hoodieKeys keys to lookup - * @param jsc spark context - * @param hoodieTable hoodie table object - */ - @Override - public JavaPairRDD>> fetchRecordLocation(JavaRDD hoodieKeys, - JavaSparkContext jsc, - HoodieTable hoodieTable) { - return fetchRecordLocationInternal(hoodieKeys, jsc, hoodieTable, config.getGlobalSimpleIndexParallelism()); - } - @Override public boolean isGlobal() { return true; diff --git a/hudi-client/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java similarity index 61% rename from hudi-client/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java index af963aa96..3f167e2eb 100644 --- a/hudi-client/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java @@ -19,7 +19,9 @@ package org.apache.hudi.index.simple; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkConfigUtils; +import org.apache.hudi.client.utils.SparkMemoryUtils; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -28,8 +30,8 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.index.SparkHoodieIndex; import org.apache.hudi.io.HoodieKeyLocationFetchHandle; import org.apache.hudi.table.HoodieTable; @@ -49,15 +51,16 @@ import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPar * * @param */ -public class HoodieSimpleIndex extends HoodieIndex { +@SuppressWarnings("checkstyle:LineLength") +public class SparkHoodieSimpleIndex extends SparkHoodieIndex { - public HoodieSimpleIndex(HoodieWriteConfig config) { + public SparkHoodieSimpleIndex(HoodieWriteConfig config) { super(config); } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) { + public JavaRDD updateLocation(JavaRDD writeStatusRDD, HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { return writeStatusRDD; } @@ -82,42 +85,28 @@ public class HoodieSimpleIndex extends HoodieInde } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) { - return tagLocationInternal(recordRDD, jsc, hoodieTable); - } - - /** - * Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option. Empty if the key is not - * found. - * - * @param hoodieKeys keys to lookup - * @param jsc spark context - * @param hoodieTable hoodie table object - */ - @Override - public JavaPairRDD>> fetchRecordLocation(JavaRDD hoodieKeys, - JavaSparkContext jsc, HoodieTable hoodieTable) { - - return fetchRecordLocationInternal(hoodieKeys, jsc, hoodieTable, config.getSimpleIndexParallelism()); + public JavaRDD> tagLocation(JavaRDD> recordRDD, + HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { + return tagLocationInternal(recordRDD, context, hoodieTable); } /** * Tags records location for incoming records. * * @param inputRecordRDD {@link JavaRDD} of incoming records - * @param jsc instance of {@link JavaSparkContext} to use + * @param context instance of {@link HoodieEngineContext} to use * @param hoodieTable instance of {@link HoodieTable} to use * @return {@link JavaRDD} of records with record locations set */ - protected JavaRDD> tagLocationInternal(JavaRDD> inputRecordRDD, JavaSparkContext jsc, - HoodieTable hoodieTable) { + protected JavaRDD> tagLocationInternal(JavaRDD> inputRecordRDD, HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { if (config.getSimpleIndexUseCaching()) { - inputRecordRDD.persist(SparkConfigUtils.getSimpleIndexInputStorageLevel(config.getProps())); + inputRecordRDD.persist(SparkMemoryUtils.getSimpleIndexInputStorageLevel(config.getProps())); } JavaPairRDD> keyedInputRecordRDD = inputRecordRDD.mapToPair(record -> new Tuple2<>(record.getKey(), record)); - JavaPairRDD existingLocationsOnTable = fetchRecordLocationsForAffectedPartitions(keyedInputRecordRDD.keys(), jsc, hoodieTable, + JavaPairRDD existingLocationsOnTable = fetchRecordLocationsForAffectedPartitions(keyedInputRecordRDD.keys(), context, hoodieTable, config.getSimpleIndexParallelism()); JavaRDD> taggedRecordRDD = keyedInputRecordRDD.leftOuterJoin(existingLocationsOnTable) @@ -133,47 +122,29 @@ public class HoodieSimpleIndex extends HoodieInde return taggedRecordRDD; } - /** - * Fetch record locations for passed in {@link JavaRDD} of HoodieKeys. - * - * @param lookupKeys {@link JavaRDD} of {@link HoodieKey}s - * @param jsc instance of {@link JavaSparkContext} to use - * @param hoodieTable instance of {@link HoodieTable} of interest - * @param parallelism parallelism to use - * @return Hoodiekeys mapped to partitionpath and filenames - */ - JavaPairRDD>> fetchRecordLocationInternal(JavaRDD lookupKeys, JavaSparkContext jsc, - HoodieTable hoodieTable, int parallelism) { - JavaPairRDD> keyLocationsRDD = lookupKeys.mapToPair(key -> new Tuple2<>(key, Option.empty())); - JavaPairRDD existingRecords = fetchRecordLocationsForAffectedPartitions(lookupKeys, jsc, hoodieTable, parallelism); - - return keyLocationsRDD.leftOuterJoin(existingRecords) - .mapToPair(entry -> { - final Option locationOpt = Option.ofNullable(entry._2._2.orNull()); - final HoodieKey key = entry._1; - return locationOpt - .map(location -> new Tuple2<>(key, Option.of(Pair.of(key.getPartitionPath(), location.getFileId())))) - .orElse(new Tuple2<>(key, Option.empty())); - }); - } - /** * Fetch record locations for passed in {@link HoodieKey}s. * * @param hoodieKeys {@link JavaRDD} of {@link HoodieKey}s for which locations are fetched - * @param jsc instance of {@link JavaSparkContext} to use + * @param context instance of {@link HoodieEngineContext} to use * @param hoodieTable instance of {@link HoodieTable} of interest * @param parallelism parallelism to use * @return {@link JavaPairRDD} of {@link HoodieKey} and {@link HoodieRecordLocation} */ - protected JavaPairRDD fetchRecordLocationsForAffectedPartitions(JavaRDD hoodieKeys, JavaSparkContext jsc, HoodieTable hoodieTable, + protected JavaPairRDD fetchRecordLocationsForAffectedPartitions(JavaRDD hoodieKeys, + HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable, int parallelism) { List affectedPartitionPathList = hoodieKeys.map(HoodieKey::getPartitionPath).distinct().collect(); - List> latestBaseFiles = getLatestBaseFilesForAllPartitions(affectedPartitionPathList, jsc, hoodieTable); - return fetchRecordLocations(jsc, hoodieTable, parallelism, latestBaseFiles); + List> latestBaseFiles = getLatestBaseFilesForAllPartitions(affectedPartitionPathList, context, hoodieTable); + return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles); } - protected JavaPairRDD fetchRecordLocations(JavaSparkContext jsc, HoodieTable hoodieTable, int parallelism, List> baseFiles) { + protected JavaPairRDD fetchRecordLocations(HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable, + int parallelism, + List> baseFiles) { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); int fetchParallelism = Math.max(1, Math.max(baseFiles.size(), parallelism)); return jsc.parallelize(baseFiles, fetchParallelism) .flatMapToPair(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile).locations()); diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriter.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriter.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriter.java diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriterFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriterFactory.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriterFactory.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriterFactory.java diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java similarity index 92% rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java index 7d0516300..8070c07bf 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java @@ -44,8 +44,8 @@ public class HoodieInternalRowParquetWriter extends ParquetWriter super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()), ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(), parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(), - ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, - ParquetWriter.DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf())); + DEFAULT_IS_DICTIONARY_ENABLED, DEFAULT_IS_VALIDATING_ENABLED, + DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf())); this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()); this.fs = (HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf())); diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetConfig.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetConfig.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetConfig.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetConfig.java diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java new file mode 100644 index 000000000..77abf1514 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java @@ -0,0 +1,31 @@ +/* + * 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.keygen; + +import org.apache.spark.sql.Row; + +/** + * Spark key generator interface. + */ +public interface SparkKeyGeneratorInterface extends KeyGeneratorInterface { + + String getRecordKey(Row row); + + String getPartitionPath(Row row); +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java new file mode 100644 index 000000000..f2b336432 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -0,0 +1,215 @@ +/* + * 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.table; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.avro.model.HoodieSavepointMetadata; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.io.HoodieCreateHandle; +import org.apache.hudi.io.HoodieMergeHandle; +import org.apache.hudi.io.HoodieSortedMergeHandle; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; +import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitActionExecutor; +import org.apache.hudi.table.action.clean.SparkCleanActionExecutor; +import org.apache.hudi.table.action.commit.SparkInsertOverwriteCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkBulkInsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkDeleteCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkInsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkInsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkMergeHelper; +import org.apache.hudi.table.action.commit.SparkUpsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkUpsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.restore.SparkCopyOnWriteRestoreActionExecutor; +import org.apache.hudi.table.action.rollback.SparkCopyOnWriteRollbackActionExecutor; +import org.apache.hudi.table.action.savepoint.SavepointActionExecutor; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * Implementation of a very heavily read-optimized Hoodie Table where, all data is stored in base files, with + * zero read amplification. + *

+ * INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing file, to expand it + *

+ * UPDATES - Produce a new version of the file, just replacing the updated records with new values + */ +public class HoodieSparkCopyOnWriteTable extends HoodieSparkTable { + + private static final Logger LOG = LogManager.getLogger(HoodieSparkCopyOnWriteTable.class); + + public HoodieSparkCopyOnWriteTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { + super(config, context, metaClient); + } + + @Override + public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, JavaRDD> records) { + return new SparkUpsertCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute(); + } + + @Override + public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, JavaRDD> records) { + return new SparkInsertCommitActionExecutor<>((HoodieSparkEngineContext)context, config, this, instantTime, records).execute(); + } + + @Override + public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, JavaRDD> records, + Option>>> userDefinedBulkInsertPartitioner) { + return new SparkBulkInsertCommitActionExecutor((HoodieSparkEngineContext) context, config, + this, instantTime, records, userDefinedBulkInsertPartitioner).execute(); + } + + @Override + public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, JavaRDD keys) { + return new SparkDeleteCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute(); + } + + @Override + public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, + JavaRDD> preppedRecords) { + return new SparkUpsertPreppedCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); + } + + @Override + public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, + JavaRDD> preppedRecords) { + return new SparkInsertPreppedCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); + } + + @Override + public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, + JavaRDD> preppedRecords, Option>>> userDefinedBulkInsertPartitioner) { + return new SparkBulkInsertPreppedCommitActionExecutor((HoodieSparkEngineContext) context, config, + this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); + } + + @Override + public HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String instantTime, JavaRDD> records) { + return new SparkInsertOverwriteCommitActionExecutor(context, config, this, instantTime, records).execute(); + } + + @Override + public Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) { + throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); + } + + @Override + public HoodieWriteMetadata> compact(HoodieEngineContext context, String compactionInstantTime) { + throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); + } + + @Override + public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { + return new SparkBootstrapCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute(); + } + + @Override + public void rollbackBootstrap(HoodieEngineContext context, String instantTime) { + new SparkCopyOnWriteRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); + } + + public Iterator> handleUpdate(String instantTime, String partitionPath, String fileId, + Map> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException { + // these are updates + HoodieMergeHandle upsertHandle = getUpdateHandle(instantTime, partitionPath, fileId, keyToNewRecords, oldDataFile); + return handleUpdateInternal(upsertHandle, instantTime, fileId); + } + + protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String instantTime, + String fileId) throws IOException { + if (upsertHandle.getOldFilePath() == null) { + throw new HoodieUpsertException( + "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); + } else { + SparkMergeHelper.newInstance().runMerge(this, upsertHandle); + } + + // TODO(vc): This needs to be revisited + if (upsertHandle.getWriteStatus().getPartitionPath() == null) { + LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " + + upsertHandle.getWriteStatus()); + } + return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator(); + } + + protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId, + Map> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) { + if (requireSortedRecords()) { + return new HoodieSortedMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId, + dataFileToBeMerged, taskContextSupplier); + } else { + return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId, + dataFileToBeMerged,taskContextSupplier); + } + } + + public Iterator> handleInsert(String instantTime, String partitionPath, String fileId, + Map> recordMap) { + HoodieCreateHandle createHandle = + new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordMap, taskContextSupplier); + createHandle.write(); + return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator(); + } + + @Override + public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime) { + return new SparkCleanActionExecutor((HoodieSparkEngineContext)context, config, this, cleanInstantTime).execute(); + } + + @Override + public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollbackInstantTime, HoodieInstant commitInstant, boolean deleteInstants) { + return new SparkCopyOnWriteRollbackActionExecutor((HoodieSparkEngineContext) context, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute(); + } + + @Override + public HoodieSavepointMetadata savepoint(HoodieEngineContext context, String instantToSavepoint, String user, String comment) { + return new SavepointActionExecutor(context, config, this, instantToSavepoint, user, comment).execute(); + } + + @Override + public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) { + return new SparkCopyOnWriteRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, restoreInstantTime, instantToRestore).execute(); + } + +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java new file mode 100644 index 000000000..0a60dcc50 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java @@ -0,0 +1,163 @@ +/* + * 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.table; + +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; + +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.bootstrap.SparkBootstrapDeltaCommitActionExecutor; +import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; +import org.apache.hudi.table.action.compact.SparkRunCompactionActionExecutor; +import org.apache.hudi.table.action.compact.SparkScheduleCompactionActionExecutor; +import org.apache.hudi.table.action.deltacommit.SparkBulkInsertDeltaCommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.SparkBulkInsertPreppedDeltaCommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.SparkDeleteDeltaCommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.SparkInsertDeltaCommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.SparkInsertPreppedDeltaCommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.SparkUpsertDeltaCommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.SparkUpsertPreppedDeltaCommitActionExecutor; +import org.apache.hudi.table.action.compact.BaseScheduleCompactionActionExecutor; +import org.apache.hudi.table.action.restore.SparkMergeOnReadRestoreActionExecutor; +import org.apache.hudi.table.action.rollback.SparkMergeOnReadRollbackActionExecutor; +import org.apache.spark.api.java.JavaRDD; + +import java.util.List; +import java.util.Map; + +/** + * Implementation of a more real-time Hoodie Table the provides tradeoffs on read and write cost/amplification. + * + *

+ * INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the + * smallest existing file, to expand it + *

+ *

+ * UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the log file into the + * base file. + *

+ *

+ * WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an attempted commit + * action + *

+ */ +public class HoodieSparkMergeOnReadTable extends HoodieSparkCopyOnWriteTable { + + HoodieSparkMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { + super(config, context, metaClient); + } + + @Override + public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, JavaRDD> records) { + return new SparkUpsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute(); + } + + @Override + public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, JavaRDD> records) { + return new SparkInsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute(); + } + + @Override + public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, JavaRDD> records, + Option>>> userDefinedBulkInsertPartitioner) { + return new SparkBulkInsertDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, + this, instantTime, records, userDefinedBulkInsertPartitioner).execute(); + } + + @Override + public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, JavaRDD keys) { + return new SparkDeleteDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute(); + } + + @Override + public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, + JavaRDD> preppedRecords) { + return new SparkUpsertPreppedDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); + } + + @Override + public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, + JavaRDD> preppedRecords) { + return new SparkInsertPreppedDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); + } + + @Override + public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, + JavaRDD> preppedRecords, Option>>> userDefinedBulkInsertPartitioner) { + return new SparkBulkInsertPreppedDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, + this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); + } + + @Override + public Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) { + BaseScheduleCompactionActionExecutor scheduleCompactionExecutor = new SparkScheduleCompactionActionExecutor( + context, config, this, instantTime, extraMetadata); + return scheduleCompactionExecutor.execute(); + } + + @Override + public HoodieWriteMetadata> compact(HoodieEngineContext context, String compactionInstantTime) { + SparkRunCompactionActionExecutor compactionExecutor = new SparkRunCompactionActionExecutor((HoodieSparkEngineContext) context, config, this, compactionInstantTime); + return compactionExecutor.execute(); + } + + @Override + public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { + return new SparkBootstrapDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute(); + } + + @Override + public void rollbackBootstrap(HoodieEngineContext context, String instantTime) { + new SparkMergeOnReadRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); + } + + @Override + public HoodieRollbackMetadata rollback(HoodieEngineContext context, + String rollbackInstantTime, + HoodieInstant commitInstant, + boolean deleteInstants) { + return new SparkMergeOnReadRollbackActionExecutor(context, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute(); + } + + @Override + public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) { + return new SparkMergeOnReadRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, restoreInstantTime, instantToRestore).execute(); + } + + @Override + public void finalizeWrite(HoodieEngineContext context, String instantTs, List stats) + throws HoodieIOException { + // delegate to base class for MOR tables + super.finalizeWrite(context, instantTs, stats); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java new file mode 100644 index 000000000..4292da75f --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -0,0 +1,72 @@ +/* + * 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.table; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.SparkHoodieIndex; + +import org.apache.spark.api.java.JavaRDD; + +public abstract class HoodieSparkTable + extends HoodieTable>, JavaRDD, JavaRDD> { + + protected HoodieSparkTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { + super(config, context, metaClient); + } + + public static HoodieSparkTable create(HoodieWriteConfig config, HoodieEngineContext context) { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient( + context.getHadoopConf().get(), + config.getBasePath(), + true, + config.getConsistencyGuardConfig(), + Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())) + ); + return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient); + } + + public static HoodieSparkTable create(HoodieWriteConfig config, + HoodieSparkEngineContext context, + HoodieTableMetaClient metaClient) { + switch (metaClient.getTableType()) { + case COPY_ON_WRITE: + return new HoodieSparkCopyOnWriteTable<>(config, context, metaClient); + case MERGE_ON_READ: + return new HoodieSparkMergeOnReadTable<>(config, context, metaClient); + default: + throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); + } + } + + @Override + protected HoodieIndex>, JavaRDD, JavaRDD> getIndex(HoodieWriteConfig config) { + return SparkHoodieIndex.createIndex(config); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java similarity index 76% rename from hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index 47791d994..64df78c0d 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -21,24 +21,29 @@ package org.apache.hudi.table.action.bootstrap; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.bootstrap.HoodieBootstrapSchemaProvider; import org.apache.hudi.client.bootstrap.BootstrapMode; import org.apache.hudi.client.bootstrap.BootstrapRecordPayload; -import org.apache.hudi.client.bootstrap.BootstrapSchemaProvider; import org.apache.hudi.client.bootstrap.BootstrapWriteStatus; import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; +import org.apache.hudi.client.bootstrap.HoodieSparkBootstrapSchemaProvider; import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector; import org.apache.hudi.client.bootstrap.translator.BootstrapPartitionPathTranslator; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.utils.SparkMemoryUtils; import org.apache.hudi.common.bootstrap.FileStatusUtils; import org.apache.hudi.common.bootstrap.index.BootstrapIndex; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.BootstrapFileMapping; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -49,16 +54,18 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.execution.SparkBoundedInMemoryExecutor; import org.apache.hudi.io.HoodieBootstrapHandle; import org.apache.hudi.keygen.KeyGeneratorInterface; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; import org.apache.hudi.table.action.commit.BaseCommitActionExecutor; -import org.apache.hudi.table.action.commit.BulkInsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; @@ -66,7 +73,6 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hudi.table.action.commit.CommitActionExecutor; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.avro.AvroParquetReader; @@ -77,27 +83,31 @@ import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; -import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -public class BootstrapCommitActionExecutor> - extends BaseCommitActionExecutor { +public class SparkBootstrapCommitActionExecutor> + extends BaseCommitActionExecutor>, JavaRDD, JavaRDD, HoodieBootstrapWriteMetadata> { - private static final Logger LOG = LogManager.getLogger(BootstrapCommitActionExecutor.class); + private static final Logger LOG = LogManager.getLogger(SparkBootstrapCommitActionExecutor.class); protected String bootstrapSchema = null; private transient FileSystem bootstrapSourceFileSystem; - public BootstrapCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, - Option> extraMetadata) { - super(jsc, new HoodieWriteConfig.Builder().withProps(config.getProps()) + public SparkBootstrapCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, + HoodieTable>, JavaRDD, JavaRDD> table, + Option> extraMetadata) { + super(context, new HoodieWriteConfig.Builder().withProps(config.getProps()) .withAutoCommit(true).withWriteStatusClass(BootstrapWriteStatus.class) .withBulkInsertParallelism(config.getBootstrapParallelism()) .build(), table, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, WriteOperationType.BOOTSTRAP, @@ -160,13 +170,32 @@ public class BootstrapCommitActionExecutor> JavaRDD bootstrapWriteStatuses = runMetadataBootstrap(partitionFilesList); - HoodieWriteMetadata result = new HoodieWriteMetadata(); + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); updateIndexAndCommitIfNeeded(bootstrapWriteStatuses.map(w -> w), result); return Option.of(result); } + private void updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, HoodieWriteMetadata> result) { + // cache writeStatusRDD before updating index, so that all actions before this are not triggered again for future + // RDD actions that are performed after updating the index. + writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps())); + Instant indexStartTime = Instant.now(); + // Update the index back + JavaRDD statuses = table.getIndex().updateLocation(writeStatusRDD, context, + table); + result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); + result.setWriteStatuses(statuses); + commitOnAutoCommit(result); + } + @Override - protected void commit(Option> extraMetadata, HoodieWriteMetadata result) { + public HoodieWriteMetadata> execute(JavaRDD> inputRecords) { + // NO_OP + return null; + } + + @Override + protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { // Perform bootstrap index write and then commit. Make sure both record-key and bootstrap-index // is all done in a single job DAG. Map>> bootstrapSourceAndStats = @@ -188,11 +217,45 @@ public class BootstrapCommitActionExecutor> + config.getBasePath()); } - super.commit(extraMetadata, result, bootstrapSourceAndStats.values().stream() + commit(extraMetadata, result, bootstrapSourceAndStats.values().stream() .flatMap(f -> f.stream().map(Pair::getValue)).collect(Collectors.toList())); LOG.info("Committing metadata bootstrap !!"); } + protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List stats) { + String actionType = table.getMetaClient().getCommitActionType(); + LOG.info("Committing " + instantTime + ", action Type " + actionType); + // Create a Hoodie table which encapsulated the commits and files visible + HoodieSparkTable table = HoodieSparkTable.create(config, context); + + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + HoodieCommitMetadata metadata = new HoodieCommitMetadata(); + + result.setCommitted(true); + stats.forEach(stat -> metadata.addWriteStat(stat.getPartitionPath(), stat)); + result.setWriteStats(stats); + + // Finalize write + finalizeWrite(instantTime, stats, result); + + // add in extra metadata + if (extraMetadata.isPresent()) { + extraMetadata.get().forEach(metadata::addMetadata); + } + metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit()); + metadata.setOperationType(operationType); + + try { + activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime), + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + LOG.info("Committed " + instantTime); + } catch (IOException e) { + throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, + e); + } + result.setCommitMetadata(Option.of(metadata)); + } + /** * Perform Full Bootstrap. * @param partitionFilesList List of partitions and files within that partitions @@ -205,9 +268,9 @@ public class BootstrapCommitActionExecutor> properties.putAll(config.getProps()); FullRecordBootstrapDataProvider inputProvider = (FullRecordBootstrapDataProvider) ReflectionUtils.loadClass(config.getFullBootstrapInputProvider(), - properties, jsc); + properties, context); JavaRDD inputRecordsRDD = - inputProvider.generateInputRecordRDD("bootstrap_source", config.getBootstrapSourceBasePath(), + (JavaRDD) inputProvider.generateInputRecords("bootstrap_source", config.getBootstrapSourceBasePath(), partitionFilesList); // Start Full Bootstrap final HoodieInstant requested = new HoodieInstant(State.REQUESTED, table.getMetaClient().getCommitActionType(), @@ -218,8 +281,8 @@ public class BootstrapCommitActionExecutor> return Option.of(getBulkInsertActionExecutor(inputRecordsRDD).execute()); } - protected CommitActionExecutor getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { - return new BulkInsertCommitActionExecutor(jsc, new HoodieWriteConfig.Builder().withProps(config.getProps()) + protected BaseSparkCommitActionExecutor getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { + return new SparkBulkInsertCommitActionExecutor((HoodieSparkEngineContext) context, new HoodieWriteConfig.Builder().withProps(config.getProps()) .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, inputRecordsRDD, extraMetadata); } @@ -229,7 +292,7 @@ public class BootstrapCommitActionExecutor> Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath()); HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, - table, partitionPath, FSUtils.createNewFileIdPfx(), table.getSparkTaskContextSupplier()); + table, partitionPath, FSUtils.createNewFileIdPfx(), table.getTaskContextSupplier()); Schema avroSchema = null; try { ParquetMetadata readFooter = ParquetFileReader.readFooter(table.getHadoopConf(), sourceFilePath, @@ -281,11 +344,11 @@ public class BootstrapCommitActionExecutor> */ private Map>>> listAndProcessSourcePartitions() throws IOException { List>> folders = BootstrapUtils.getAllLeafFoldersWithFiles( - table.getMetaClient(), bootstrapSourceFileSystem, config.getBootstrapSourceBasePath(), jsc); + table.getMetaClient(), bootstrapSourceFileSystem, config.getBootstrapSourceBasePath(), context); LOG.info("Fetching Bootstrap Schema !!"); - BootstrapSchemaProvider sourceSchemaProvider = new BootstrapSchemaProvider(config); - bootstrapSchema = sourceSchemaProvider.getBootstrapSchema(jsc, folders).toString(); + HoodieBootstrapSchemaProvider sourceSchemaProvider = new HoodieSparkBootstrapSchemaProvider(config); + bootstrapSchema = sourceSchemaProvider.getBootstrapSchema(context, folders).toString(); LOG.info("Bootstrap Schema :" + bootstrapSchema); BootstrapModeSelector selector = @@ -305,6 +368,7 @@ public class BootstrapCommitActionExecutor> } private JavaRDD runMetadataBootstrap(List>> partitions) { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); if (null == partitions || partitions.isEmpty()) { return jsc.emptyRDD(); } @@ -328,17 +392,6 @@ public class BootstrapCommitActionExecutor> partitionFsPair.getRight().getRight(), keyGenerator)); } - //TODO: Once we decouple commit protocol, we should change the class hierarchy to avoid doing this. - @Override - protected Partitioner getUpsertPartitioner(WorkloadProfile profile) { - throw new UnsupportedOperationException("Should not called in bootstrap code path"); - } - - @Override - protected Partitioner getInsertPartitioner(WorkloadProfile profile) { - throw new UnsupportedOperationException("Should not called in bootstrap code path"); - } - @Override protected Iterator> handleInsert(String idPfx, Iterator> recordItr) { throw new UnsupportedOperationException("Should not called in bootstrap code path"); diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java similarity index 58% rename from hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapDeltaCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java index 08760cc3d..59f86662b 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java @@ -19,28 +19,30 @@ package org.apache.hudi.table.action.bootstrap; import java.util.Map; + +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.commit.CommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.BulkInsertDeltaCommitActionExecutor; +import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.SparkBulkInsertDeltaCommitActionExecutor; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class BootstrapDeltaCommitActionExecutor> - extends BootstrapCommitActionExecutor { +public class SparkBootstrapDeltaCommitActionExecutor> + extends SparkBootstrapCommitActionExecutor { - public BootstrapDeltaCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - Option> extraMetadata) { - super(jsc, config, table, extraMetadata); + public SparkBootstrapDeltaCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + Option> extraMetadata) { + super(context, config, table, extraMetadata); } - protected CommitActionExecutor getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { - return new BulkInsertDeltaCommitActionExecutor(jsc, new HoodieWriteConfig.Builder().withProps(config.getProps()) + @Override + protected BaseSparkCommitActionExecutor getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { + return new SparkBulkInsertDeltaCommitActionExecutor((HoodieSparkEngineContext) context, new HoodieWriteConfig.Builder().withProps(config.getProps()) .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, inputRecordsRDD, extraMetadata); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java new file mode 100644 index 000000000..bbd5c1fb0 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java @@ -0,0 +1,134 @@ +/* + * 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.table.action.clean; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.avro.model.HoodieActionInstant; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.HoodieCleanStat; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.CleanFileInfo; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +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.api.java.function.PairFlatMapFunction; +import scala.Tuple2; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +@SuppressWarnings("checkstyle:LineLength") +public class SparkCleanActionExecutor extends + BaseCleanActionExecutor>, JavaRDD, JavaRDD> { + + private static final Logger LOG = LogManager.getLogger(SparkCleanActionExecutor.class); + + public SparkCleanActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, + HoodieTable>, JavaRDD, JavaRDD> table, + String instantTime) { + super(context, config, table, instantTime); + } + + private static PairFlatMapFunction>, String, PartitionCleanStat> + deleteFilesFunc(HoodieTable table) { + return (PairFlatMapFunction>, String, PartitionCleanStat>) iter -> { + Map partitionCleanStatMap = new HashMap<>(); + FileSystem fs = table.getMetaClient().getFs(); + while (iter.hasNext()) { + Tuple2 partitionDelFileTuple = iter.next(); + String partitionPath = partitionDelFileTuple._1(); + Path deletePath = new Path(partitionDelFileTuple._2().getFilePath()); + String deletePathStr = deletePath.toString(); + Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr); + if (!partitionCleanStatMap.containsKey(partitionPath)) { + partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath)); + } + boolean isBootstrapBasePathFile = partitionDelFileTuple._2().isBootstrapBaseFile(); + PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath); + if (isBootstrapBasePathFile) { + // For Bootstrap Base file deletions, store the full file path. + partitionCleanStat.addDeleteFilePatterns(deletePath.toString(), true); + partitionCleanStat.addDeletedFileResult(deletePath.toString(), deletedFileResult, true); + } else { + partitionCleanStat.addDeleteFilePatterns(deletePath.getName(), false); + partitionCleanStat.addDeletedFileResult(deletePath.getName(), deletedFileResult, false); + } + } + return partitionCleanStatMap.entrySet().stream().map(e -> new Tuple2<>(e.getKey(), e.getValue())) + .collect(Collectors.toList()).iterator(); + }; + } + + @Override + List clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan) { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + int cleanerParallelism = Math.min( + (int) (cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).count()), + config.getCleanerParallelism()); + LOG.info("Using cleanerParallelism: " + cleanerParallelism); + + context.setJobStatus(this.getClass().getSimpleName(), "Perform cleaning of partitions"); + List> partitionCleanStats = jsc + .parallelize(cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream() + .flatMap(x -> x.getValue().stream().map(y -> new Tuple2<>(x.getKey(), + new CleanFileInfo(y.getFilePath(), y.getIsBootstrapBaseFile())))) + .collect(Collectors.toList()), cleanerParallelism) + .mapPartitionsToPair(deleteFilesFunc(table)) + .reduceByKey(PartitionCleanStat::merge).collect(); + + Map partitionCleanStatsMap = partitionCleanStats.stream() + .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2)); + + // Return PartitionCleanStat for each partition passed. + return cleanerPlan.getFilePathsToBeDeletedPerPartition().keySet().stream().map(partitionPath -> { + PartitionCleanStat partitionCleanStat = partitionCleanStatsMap.containsKey(partitionPath) + ? partitionCleanStatsMap.get(partitionPath) + : new PartitionCleanStat(partitionPath); + HoodieActionInstant actionInstant = cleanerPlan.getEarliestInstantToRetain(); + return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath) + .withEarliestCommitRetained(Option.ofNullable( + actionInstant != null + ? new HoodieInstant(HoodieInstant.State.valueOf(actionInstant.getState()), + actionInstant.getAction(), actionInstant.getTimestamp()) + : null)) + .withDeletePathPattern(partitionCleanStat.deletePathPatterns()) + .withSuccessfulDeletes(partitionCleanStat.successDeleteFiles()) + .withFailedDeletes(partitionCleanStat.failedDeleteFiles()) + .withDeleteBootstrapBasePathPatterns(partitionCleanStat.getDeleteBootstrapBasePathPatterns()) + .withSuccessfulDeleteBootstrapBaseFiles(partitionCleanStat.getSuccessfulDeleteBootstrapBaseFiles()) + .withFailedDeleteBootstrapBaseFiles(partitionCleanStat.getFailedDeleteBootstrapBaseFiles()) + .build(); + }).collect(Collectors.toList()); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java similarity index 51% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index 0b276391d..36cca8cd1 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -18,35 +18,39 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkConfigUtils; +import org.apache.hudi.client.utils.SparkMemoryUtils; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.execution.SparkLazyInsertIterable; +import org.apache.hudi.io.CreateHandleFactory; +import org.apache.hudi.io.HoodieMergeHandle; +import org.apache.hudi.io.HoodieSortedMergeHandle; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.WorkloadStat; -import org.apache.hudi.table.action.BaseActionExecutor; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.storage.StorageLevel; import scala.Tuple2; @@ -57,29 +61,36 @@ import java.time.Duration; import java.time.Instant; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; -public abstract class BaseCommitActionExecutor, R> - extends BaseActionExecutor { +public abstract class BaseSparkCommitActionExecutor extends + BaseCommitActionExecutor>, JavaRDD, JavaRDD, HoodieWriteMetadata> { - private static final Logger LOG = LogManager.getLogger(BaseCommitActionExecutor.class); + private static final Logger LOG = LogManager.getLogger(BaseSparkCommitActionExecutor.class); - protected final Option> extraMetadata; - private final WriteOperationType operationType; - protected final SparkTaskContextSupplier sparkTaskContextSupplier = new SparkTaskContextSupplier(); - - public BaseCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, - HoodieTable table, String instantTime, WriteOperationType operationType, - Option> extraMetadata) { - super(jsc, config, table, instantTime); - this.operationType = operationType; - this.extraMetadata = extraMetadata; + public BaseSparkCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + WriteOperationType operationType) { + super(context, config, table, instantTime, operationType, Option.empty()); } - public HoodieWriteMetadata execute(JavaRDD> inputRecordsRDD) { - HoodieWriteMetadata result = new HoodieWriteMetadata(); + public BaseSparkCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + WriteOperationType operationType, + Option extraMetadata) { + super(context, config, table, instantTime, operationType, extraMetadata); + } + + @Override + public HoodieWriteMetadata> execute(JavaRDD> inputRecordsRDD) { + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); // Cache the tagged records, so we don't end up computing both // TODO: Consistent contract in HoodieWriteClient regarding preppedRecord storage level handling if (inputRecordsRDD.getStorageLevel() == StorageLevel.NONE()) { @@ -90,7 +101,7 @@ public abstract class BaseCommitActionExecutor, WorkloadProfile profile = null; if (isWorkloadProfileNeeded()) { - profile = new WorkloadProfile(inputRecordsRDD); + profile = new WorkloadProfile(buildProfile(inputRecordsRDD)); LOG.info("Workload profile :" + profile); saveWorkloadProfileMetadataToInflight(profile, instantTime); } @@ -110,44 +121,38 @@ public abstract class BaseCommitActionExecutor, return result; } - /** - * Save the workload profile in an intermediate file (here re-using commit files) This is useful when performing - * rollback for MOR tables. Only updates are recorded in the workload profile metadata since updates to log blocks - * are unknown across batches Inserts (which are new parquet files) are rolled back based on commit time. // TODO : - * Create a new WorkloadProfile metadata file instead of using HoodieCommitMetadata - */ - void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, String instantTime) - throws HoodieCommitException { - try { - HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - profile.getPartitionPaths().forEach(path -> { - WorkloadStat partitionStat = profile.getWorkloadStat(path.toString()); - HoodieWriteStat insertStat = new HoodieWriteStat(); - insertStat.setNumInserts(partitionStat.getNumInserts()); - insertStat.setFileId(""); - insertStat.setPrevCommit(HoodieWriteStat.NULL_COMMIT); - metadata.addWriteStat(path.toString(), insertStat); + private Pair, WorkloadStat> buildProfile(JavaRDD> inputRecordsRDD) { + HashMap partitionPathStatMap = new HashMap<>(); + WorkloadStat globalStat = new WorkloadStat(); - partitionStat.getUpdateLocationToCount().forEach((key, value) -> { - HoodieWriteStat writeStat = new HoodieWriteStat(); - writeStat.setFileId(key); - // TODO : Write baseCommitTime is possible here ? - writeStat.setPrevCommit(value.getKey()); - writeStat.setNumUpdateWrites(value.getValue()); - metadata.addWriteStat(path.toString(), writeStat); - }); - }); - metadata.setOperationType(operationType); + // group the records by partitionPath + currentLocation combination, count the number of + // records in each partition + Map>, Long> partitionLocationCounts = inputRecordsRDD + .mapToPair(record -> new Tuple2<>( + new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record)) + .countByKey(); - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - String commitActionType = getCommitActionType(); - HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime); - activeTimeline.transitionRequestedToInflight(requested, - Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)), - config.shouldAllowMultiWriteOnSameInstant()); - } catch (IOException io) { - throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io); + // count the number of both inserts and updates in each partition, update the counts to workLoadStats + for (Map.Entry>, Long> e : partitionLocationCounts.entrySet()) { + String partitionPath = e.getKey()._1(); + Long count = e.getValue(); + Option locOption = e.getKey()._2(); + + if (!partitionPathStatMap.containsKey(partitionPath)) { + partitionPathStatMap.put(partitionPath, new WorkloadStat()); + } + + if (locOption.isPresent()) { + // update + partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count); + globalStat.addUpdates(locOption.get(), count); + } else { + // insert + partitionPathStatMap.get(partitionPath).addInserts(count); + globalStat.addInserts(count); + } } + return Pair.of(partitionPathStatMap, globalStat); } protected Partitioner getPartitioner(WorkloadProfile profile) { @@ -184,35 +189,26 @@ public abstract class BaseCommitActionExecutor, protected void updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, HoodieWriteMetadata result) { // cache writeStatusRDD before updating index, so that all actions before this are not triggered again for future // RDD actions that are performed after updating the index. - writeStatusRDD = writeStatusRDD.persist(SparkConfigUtils.getWriteStatusStorageLevel(config.getProps())); + writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps())); Instant indexStartTime = Instant.now(); // Update the index back - JavaRDD statuses = ((HoodieTable)table).getIndex().updateLocation(writeStatusRDD, jsc, - (HoodieTable)table); + JavaRDD statuses = table.getIndex().updateLocation(writeStatusRDD, context, table); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(statuses)); commitOnAutoCommit(result); } - protected void commitOnAutoCommit(HoodieWriteMetadata result) { - if (config.shouldAutoCommit()) { - LOG.info("Auto commit enabled: Committing " + instantTime); - commit(extraMetadata, result); - } else { - LOG.info("Auto commit disabled for " + instantTime); - } - } - protected String getCommitActionType() { return table.getMetaClient().getCommitActionType(); } - protected void commit(Option> extraMetadata, HoodieWriteMetadata result) { + @Override + protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collect()); } - protected void commit(Option> extraMetadata, HoodieWriteMetadata result, List writeStats) { + protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List writeStats) { String actionType = getCommitActionType(); LOG.info("Committing " + instantTime + ", action Type " + actionType); result.setCommitted(true); @@ -240,35 +236,9 @@ public abstract class BaseCommitActionExecutor, return Collections.emptyMap(); } - /** - * Finalize Write operation. - * @param instantTime Instant Time - * @param stats Hoodie Write Stat - */ - protected void finalizeWrite(String instantTime, List stats, HoodieWriteMetadata result) { - try { - Instant start = Instant.now(); - table.finalizeWrite(jsc, instantTime, stats); - result.setFinalizeDuration(Duration.between(start, Instant.now())); - } catch (HoodieIOException ioe) { - throw new HoodieCommitException("Failed to complete commit " + instantTime + " due to finalize errors.", ioe); - } - } - - /** - * By default, return the writer schema in Write Config for storing in commit. - */ - protected String getSchemaToStoreInCommit() { - return config.getSchema(); - } - - protected boolean isWorkloadProfileNeeded() { - return true; - } - @SuppressWarnings("unchecked") protected Iterator> handleUpsertPartition(String instantTime, Integer partition, Iterator recordItr, - Partitioner partitioner) { + Partitioner partitioner) { UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner; BucketInfo binfo = upsertPartitioner.getBucketInfo(partition); BucketType btype = binfo.bucketType; @@ -288,23 +258,85 @@ public abstract class BaseCommitActionExecutor, } protected Iterator> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr, - Partitioner partitioner) { + Partitioner partitioner) { return handleUpsertPartition(instantTime, partition, recordItr, partitioner); } - /** - * Provides a partitioner to perform the upsert operation, based on the workload profile. - */ - protected abstract Partitioner getUpsertPartitioner(WorkloadProfile profile); + @Override + public Iterator> handleUpdate(String partitionPath, String fileId, + Iterator> recordItr) + throws IOException { + // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records + if (!recordItr.hasNext()) { + LOG.info("Empty partition with fileId => " + fileId); + return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); + } + // these are updates + HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr); + return handleUpdateInternal(upsertHandle, fileId); + } - /** - * Provides a partitioner to perform the insert operation, based on the workload profile. - */ - protected abstract Partitioner getInsertPartitioner(WorkloadProfile profile); + public Iterator> handleUpdate(String partitionPath, String fileId, + Map> keyToNewRecords, + HoodieBaseFile oldDataFile) throws IOException { + // these are updates + HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, keyToNewRecords, oldDataFile); + return handleUpdateInternal(upsertHandle, fileId); + } - protected abstract Iterator> handleInsert(String idPfx, - Iterator> recordItr) throws Exception; + protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId) + throws IOException { + if (upsertHandle.getOldFilePath() == null) { + throw new HoodieUpsertException( + "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); + } else { + SparkMergeHelper.newInstance().runMerge(table, upsertHandle); + } + + // TODO(vc): This needs to be revisited + if (upsertHandle.getWriteStatus().getPartitionPath() == null) { + LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " + + upsertHandle.getWriteStatus()); + } + return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator(); + } + + protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator> recordItr) { + if (table.requireSortedRecords()) { + return new HoodieSortedMergeHandle<>(config, instantTime, (HoodieSparkTable) table, recordItr, partitionPath, fileId, taskContextSupplier); + } else { + return new HoodieMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier); + } + } + + protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, + Map> keyToNewRecords, + HoodieBaseFile dataFileToBeMerged) { + return new HoodieMergeHandle<>(config, instantTime, table, keyToNewRecords, + partitionPath, fileId, dataFileToBeMerged, taskContextSupplier); + } + + @Override + public Iterator> handleInsert(String idPfx, Iterator> recordItr) + throws Exception { + // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records + if (!recordItr.hasNext()) { + LOG.info("Empty partition"); + return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); + } + return new SparkLazyInsertIterable(recordItr, true, config, instantTime, table, idPfx, + taskContextSupplier, new CreateHandleFactory<>()); + } + + public Partitioner getUpsertPartitioner(WorkloadProfile profile) { + if (profile == null) { + throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner."); + } + return new UpsertPartitioner(profile, context, table, config); + } + + public Partitioner getInsertPartitioner(WorkloadProfile profile) { + return getUpsertPartitioner(profile); + } - protected abstract Iterator> handleUpdate(String partitionPath, String fileId, - Iterator> recordItr) throws IOException; } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java similarity index 67% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java index 162ae2984..fb8b5f9cd 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java @@ -19,6 +19,9 @@ package org.apache.hudi.table.action.commit; import java.util.Map; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -30,32 +33,31 @@ import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class BulkInsertCommitActionExecutor> extends CommitActionExecutor { +public class SparkBulkInsertCommitActionExecutor> extends BaseSparkCommitActionExecutor { private final JavaRDD> inputRecordsRDD; private final Option> bulkInsertPartitioner; - public BulkInsertCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option> bulkInsertPartitioner) { - this(jsc, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty()); + public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD, + Option> bulkInsertPartitioner) { + this(context, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty()); } - public BulkInsertCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, + public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, JavaRDD> inputRecordsRDD, Option> bulkInsertPartitioner, Option> extraMetadata) { - super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); + super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); this.inputRecordsRDD = inputRecordsRDD; this.bulkInsertPartitioner = bulkInsertPartitioner; } @Override - public HoodieWriteMetadata execute() { + public HoodieWriteMetadata> execute() { try { - return BulkInsertHelper.bulkInsert(inputRecordsRDD, instantTime, (HoodieTable) table, config, + return SparkBulkInsertHelper.newInstance().bulkInsert(inputRecordsRDD, instantTime, table, config, this, true, bulkInsertPartitioner); } catch (HoodieInsertException ie) { throw ie; diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java similarity index 56% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertHelper.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java index 4683c8218..9ccd66b2c 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java @@ -20,37 +20,60 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerFactory; import org.apache.hudi.execution.bulkinsert.BulkInsertMapFunction; -import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; + import org.apache.spark.api.java.JavaRDD; import java.util.List; import java.util.stream.Collectors; import java.util.stream.IntStream; -public class BulkInsertHelper> { +/** + * A spark implementation of {@link AbstractBulkInsertHelper}. + * + * @param + */ +@SuppressWarnings("checkstyle:LineLength") +public class SparkBulkInsertHelper extends AbstractBulkInsertHelper>, + JavaRDD, JavaRDD, R> { - public static > HoodieWriteMetadata bulkInsert(JavaRDD> inputRecords, String instantTime, - HoodieTable table, HoodieWriteConfig config, - CommitActionExecutor executor, boolean performDedupe, - Option> userDefinedBulkInsertPartitioner) { + private SparkBulkInsertHelper() { + } + + private static class BulkInsertHelperHolder { + private static final SparkBulkInsertHelper SPARK_BULK_INSERT_HELPER = new SparkBulkInsertHelper(); + } + + public static SparkBulkInsertHelper newInstance() { + return BulkInsertHelperHolder.SPARK_BULK_INSERT_HELPER; + } + + @Override + public HoodieWriteMetadata> bulkInsert(JavaRDD> inputRecords, + String instantTime, + HoodieTable>, JavaRDD, JavaRDD> table, + HoodieWriteConfig config, + BaseCommitActionExecutor>, JavaRDD, JavaRDD, R> executor, + boolean performDedupe, + Option> userDefinedBulkInsertPartitioner) { HoodieWriteMetadata result = new HoodieWriteMetadata(); // De-dupe/merge if needed JavaRDD> dedupedRecords = inputRecords; if (performDedupe) { - dedupedRecords = WriteHelper.combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords, - config.getBulkInsertShuffleParallelism(), ((HoodieTable)table)); + dedupedRecords = (JavaRDD>) SparkWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords, + config.getBulkInsertShuffleParallelism(), table); } final JavaRDD> repartitionedRecords; @@ -58,22 +81,22 @@ public class BulkInsertHelper> { BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.isPresent() ? userDefinedBulkInsertPartitioner.get() : BulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode()); - repartitionedRecords = partitioner.repartitionRecords(dedupedRecords, parallelism); + repartitionedRecords = (JavaRDD>) partitioner.repartitionRecords(dedupedRecords, parallelism); // generate new file ID prefixes for each output partition final List fileIDPrefixes = IntStream.range(0, parallelism).mapToObj(i -> FSUtils.createNewFileIdPfx()).collect(Collectors.toList()); - table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED, - table.getMetaClient().getCommitActionType(), instantTime), Option.empty(), + table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, + table.getMetaClient().getCommitActionType(), instantTime), Option.empty(), config.shouldAllowMultiWriteOnSameInstant()); JavaRDD writeStatusRDD = repartitionedRecords - .mapPartitionsWithIndex(new BulkInsertMapFunction<>(instantTime, + .mapPartitionsWithIndex(new BulkInsertMapFunction(instantTime, partitioner.arePartitionRecordsSorted(), config, table, fileIDPrefixes), true) .flatMap(List::iterator); - executor.updateIndexAndCommitIfNeeded(writeStatusRDD, result); + ((BaseSparkCommitActionExecutor) executor).updateIndexAndCommitIfNeeded(writeStatusRDD, result); return result; } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java similarity index 67% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java index f63d06e95..e6b680949 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -29,27 +31,26 @@ import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class BulkInsertPreppedCommitActionExecutor> - extends CommitActionExecutor { +public class SparkBulkInsertPreppedCommitActionExecutor> + extends BaseSparkCommitActionExecutor { private final JavaRDD> preppedInputRecordRdd; private final Option> userDefinedBulkInsertPartitioner; - public BulkInsertPreppedCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedInputRecordRdd, - Option> userDefinedBulkInsertPartitioner) { - super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT); + public SparkBulkInsertPreppedCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> preppedInputRecordRdd, + Option> userDefinedBulkInsertPartitioner) { + super(context, config, table, instantTime, WriteOperationType.BULK_INSERT); this.preppedInputRecordRdd = preppedInputRecordRdd; this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner; } @Override - public HoodieWriteMetadata execute() { + public HoodieWriteMetadata> execute() { try { - return BulkInsertHelper.bulkInsert(preppedInputRecordRdd, instantTime, (HoodieTable) table, config, + return SparkBulkInsertHelper.newInstance().bulkInsert(preppedInputRecordRdd, instantTime, table, config, this, false, userDefinedBulkInsertPartitioner); } catch (Throwable e) { if (e instanceof HoodieInsertException) { diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/DeleteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java similarity index 63% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/DeleteCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java index 8fa1cb7c5..997c7bf23 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/DeleteCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -26,21 +28,21 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class DeleteCommitActionExecutor> - extends CommitActionExecutor { +public class SparkDeleteCommitActionExecutor> + extends BaseSparkCommitActionExecutor { private final JavaRDD keys; - public DeleteCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD keys) { - super(jsc, config, table, instantTime, WriteOperationType.DELETE); + public SparkDeleteCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD keys) { + super(context, config, table, instantTime, WriteOperationType.DELETE); this.keys = keys; } - public HoodieWriteMetadata execute() { - return DeleteHelper.execute(instantTime, keys, jsc, config, (HoodieTable)table, this); + @Override + public HoodieWriteMetadata> execute() { + return SparkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/DeleteHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java similarity index 63% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/DeleteHelper.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java index b16bf63c2..01f9964b6 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/DeleteHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java @@ -18,38 +18,49 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; - +import org.apache.hudi.table.WorkloadStat; import org.apache.hudi.table.action.HoodieWriteMetadata; + import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import java.time.Duration; import java.time.Instant; +import java.util.HashMap; /** - * Helper class to perform delete keys on hoodie table. + * A spark implementation of {@link AbstractDeleteHelper}. + * * @param */ -public class DeleteHelper> { +@SuppressWarnings("checkstyle:LineLength") +public class SparkDeleteHelper extends + AbstractDeleteHelper>, JavaRDD, JavaRDD, R> { + private SparkDeleteHelper() { + } - /** - * Deduplicate Hoodie records, using the given deduplication function. - * - * @param keys RDD of HoodieKey to deduplicate - * @param table target Hoodie table for deduplicating - * @param parallelism parallelism or partitions to be used while reducing/deduplicating - * @return RDD of HoodieKey already be deduplicated - */ - private static > JavaRDD deduplicateKeys(JavaRDD keys, - HoodieTable table, int parallelism) { + private static class DeleteHelperHolder { + private static final SparkDeleteHelper SPARK_DELETE_HELPER = new SparkDeleteHelper(); + } + + public static SparkDeleteHelper newInstance() { + return DeleteHelperHolder.SPARK_DELETE_HELPER; + } + + @Override + public JavaRDD deduplicateKeys(JavaRDD keys, HoodieTable>, JavaRDD, JavaRDD> table, int parallelism) { boolean isIndexingGlobal = table.getIndex().isGlobal(); if (isIndexingGlobal) { return keys.keyBy(HoodieKey::getRecordKey) @@ -60,10 +71,15 @@ public class DeleteHelper> { } } - public static > HoodieWriteMetadata execute(String instantTime, - JavaRDD keys, JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - CommitActionExecutor deleteExecutor) { + @Override + public HoodieWriteMetadata> execute(String instantTime, + JavaRDD keys, + HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, JavaRDD, JavaRDD> table, + BaseCommitActionExecutor>, JavaRDD, JavaRDD, R> deleteExecutor) { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + try { HoodieWriteMetadata result = null; JavaRDD dedupedKeys = keys; @@ -80,7 +96,7 @@ public class DeleteHelper> { Instant beginTag = Instant.now(); // perform index loop up to get existing location of records JavaRDD> taggedRecords = - ((HoodieTable)table).getIndex().tagLocation(dedupedRecords, jsc, (HoodieTable)table); + table.getIndex().tagLocation(dedupedRecords, context, table); Duration tagLocationDuration = Duration.between(beginTag, Instant.now()); // filter out non existent keys/records @@ -90,7 +106,7 @@ public class DeleteHelper> { result.setIndexLookupDuration(tagLocationDuration); } else { // if entire set of keys are non existent - deleteExecutor.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(jsc.emptyRDD()), instantTime); + deleteExecutor.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); result = new HoodieWriteMetadata(); result.setWriteStatuses(jsc.emptyRDD()); deleteExecutor.commitOnAutoCommit(result); @@ -103,4 +119,5 @@ public class DeleteHelper> { throw new HoodieUpsertException("Failed to delete for commit time " + instantTime, e); } } + } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java similarity index 66% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java index d8944e36f..25891e05a 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -26,23 +28,22 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class InsertCommitActionExecutor> - extends CommitActionExecutor { +public class SparkInsertCommitActionExecutor> + extends BaseSparkCommitActionExecutor { private final JavaRDD> inputRecordsRDD; - public InsertCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { - super(jsc, config, table, instantTime, WriteOperationType.INSERT); + public SparkInsertCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD) { + super(context, config, table, instantTime, WriteOperationType.INSERT); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata execute() { - return WriteHelper.write(instantTime, inputRecordsRDD, jsc, (HoodieTable) table, + public HoodieWriteMetadata> execute() { + return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java similarity index 75% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwriteCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java index 054e8cd10..627e75eda 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwriteCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -31,36 +32,35 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import scala.Tuple2; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -public class InsertOverwriteCommitActionExecutor> - extends CommitActionExecutor { +public class SparkInsertOverwriteCommitActionExecutor> + extends BaseSparkCommitActionExecutor { - private static final Logger LOG = LogManager.getLogger(InsertOverwriteCommitActionExecutor.class); + private static final Logger LOG = LogManager.getLogger(SparkInsertOverwriteCommitActionExecutor.class); private final JavaRDD> inputRecordsRDD; - public InsertOverwriteCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { - super(jsc, config, table, instantTime, WriteOperationType.INSERT_OVERWRITE); + public SparkInsertOverwriteCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD) { + super(context, config, table, instantTime, WriteOperationType.INSERT_OVERWRITE); this.inputRecordsRDD = inputRecordsRDD; } @Override public HoodieWriteMetadata execute() { - return WriteHelper.write(instantTime, inputRecordsRDD, jsc, (HoodieTable) table, + return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); } @Override protected Partitioner getPartitioner(WorkloadProfile profile) { - return new InsertOverwritePartitioner<>(profile, jsc, table, config); + return new SparkInsertOverwritePartitioner(profile, context, table, config); } @Override diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwritePartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java similarity index 74% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwritePartitioner.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java index 80db61277..6f8be79f9 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwritePartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java @@ -18,13 +18,12 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; import java.util.Collections; import java.util.List; @@ -32,13 +31,13 @@ import java.util.List; /** * Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition). */ -public class InsertOverwritePartitioner> extends UpsertPartitioner { +public class SparkInsertOverwritePartitioner extends UpsertPartitioner { - private static final Logger LOG = LogManager.getLogger(InsertOverwritePartitioner.class); + private static final Logger LOG = LogManager.getLogger(SparkInsertOverwritePartitioner.class); - public InsertOverwritePartitioner(WorkloadProfile profile, JavaSparkContext jsc, HoodieTable table, - HoodieWriteConfig config) { - super(profile, jsc, table, config); + public SparkInsertOverwritePartitioner(WorkloadProfile profile, HoodieEngineContext context, HoodieTable table, + HoodieWriteConfig config) { + super(profile, context, table, config); } /** diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java similarity index 66% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPreppedCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java index d8470ea04..400147bb8 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -26,21 +28,21 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class UpsertPreppedCommitActionExecutor> - extends CommitActionExecutor { +public class SparkInsertPreppedCommitActionExecutor> + extends BaseSparkCommitActionExecutor { private final JavaRDD> preppedRecords; - public UpsertPreppedCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { - super(jsc, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); + public SparkInsertPreppedCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> preppedRecords) { + super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED); this.preppedRecords = preppedRecords; } - public HoodieWriteMetadata execute() { + @Override + public HoodieWriteMetadata> execute() { return super.execute(preppedRecords); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java new file mode 100644 index 000000000..697b5ac41 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java @@ -0,0 +1,110 @@ +/* + * 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.table.action.commit; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.execution.SparkBoundedInMemoryExecutor; +import org.apache.hudi.io.HoodieMergeHandle; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.table.HoodieTable; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.BinaryEncoder; +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.api.java.JavaRDD; + +import java.io.IOException; +import java.util.Iterator; + +public class SparkMergeHelper extends AbstractMergeHelper>, + JavaRDD, JavaRDD> { + + private static class MergeHelperHolder { + private static final SparkMergeHelper SPARK_MERGE_HELPER = new SparkMergeHelper(); + } + + public static SparkMergeHelper newInstance() { + return SparkMergeHelper.MergeHelperHolder.SPARK_MERGE_HELPER; + } + + @Override + public void runMerge(HoodieTable>, JavaRDD, JavaRDD> table, + HoodieMergeHandle>, JavaRDD, JavaRDD> upsertHandle) throws IOException { + final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation(); + Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf()); + HoodieMergeHandle>, JavaRDD, JavaRDD> mergeHandle = upsertHandle; + HoodieBaseFile baseFile = mergeHandle.baseFileForMerge(); + + final GenericDatumWriter gWriter; + final GenericDatumReader gReader; + Schema readSchema; + if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) { + readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema(); + gWriter = new GenericDatumWriter<>(readSchema); + gReader = new GenericDatumReader<>(readSchema, mergeHandle.getWriterSchemaWithMetafields()); + } else { + gReader = null; + gWriter = null; + readSchema = mergeHandle.getWriterSchemaWithMetafields(); + } + + BoundedInMemoryExecutor wrapper = null; + HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); + try { + final Iterator readerIterator; + if (baseFile.getBootstrapBaseFile().isPresent()) { + readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation); + } else { + readerIterator = reader.getRecordIterator(readSchema); + } + + ThreadLocal encoderCache = new ThreadLocal<>(); + ThreadLocal decoderCache = new ThreadLocal<>(); + wrapper = new SparkBoundedInMemoryExecutor(table.getConfig(), readerIterator, + new UpdateHandler(mergeHandle), record -> { + if (!externalSchemaTransformation) { + return record; + } + return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) record); + }); + wrapper.execute(); + } catch (Exception e) { + throw new HoodieException(e); + } finally { + if (reader != null) { + reader.close(); + } + mergeHandle.close(); + if (null != wrapper) { + wrapper.shutdownNow(); + } + } + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java similarity index 66% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java index 0c4d08e35..fe90212b0 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -26,23 +28,22 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class UpsertCommitActionExecutor> - extends CommitActionExecutor { +public class SparkUpsertCommitActionExecutor> + extends BaseSparkCommitActionExecutor { private JavaRDD> inputRecordsRDD; - public UpsertCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { - super(jsc, config, table, instantTime, WriteOperationType.UPSERT); + public SparkUpsertCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD) { + super(context, config, table, instantTime, WriteOperationType.UPSERT); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata execute() { - return WriteHelper.write(instantTime, inputRecordsRDD, jsc, (HoodieTable)table, + public HoodieWriteMetadata> execute() { + return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java similarity index 66% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertPreppedCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java index b7d64b124..e36073fd1 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -26,21 +28,21 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class InsertPreppedCommitActionExecutor> - extends CommitActionExecutor { +public class SparkUpsertPreppedCommitActionExecutor> + extends BaseSparkCommitActionExecutor { private final JavaRDD> preppedRecords; - public InsertPreppedCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { - super(jsc, config, table, instantTime, WriteOperationType.INSERT_PREPPED); + public SparkUpsertPreppedCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> preppedRecords) { + super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); this.preppedRecords = preppedRecords; } - public HoodieWriteMetadata execute() { + @Override + public HoodieWriteMetadata> execute() { return super.execute(preppedRecords); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java new file mode 100644 index 000000000..5f1a1ef55 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java @@ -0,0 +1,69 @@ +/* + * 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.table.action.commit; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.index.HoodieIndex; + +import org.apache.spark.api.java.JavaRDD; + +import scala.Tuple2; + +/** + * A spark implementation of {@link AbstractWriteHelper}. + * + * @param + */ +public class SparkWriteHelper extends AbstractWriteHelper>, + JavaRDD, JavaRDD, R> { + private SparkWriteHelper() { + } + + private static class WriteHelperHolder { + private static final SparkWriteHelper SPARK_WRITE_HELPER = new SparkWriteHelper(); + } + + public static SparkWriteHelper newInstance() { + return WriteHelperHolder.SPARK_WRITE_HELPER; + } + + @Override + public JavaRDD> deduplicateRecords(JavaRDD> records, + HoodieIndex>, JavaRDD, JavaRDD> index, + int parallelism) { + boolean isIndexingGlobal = index.isGlobal(); + return records.mapToPair(record -> { + HoodieKey hoodieKey = record.getKey(); + // If index used is global, then records are expected to differ in their partitionPath + Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey; + return new Tuple2<>(key, record); + }).reduceByKey((rec1, rec2) -> { + @SuppressWarnings("unchecked") + T reducedData = (T) rec1.getData().preCombine(rec2.getData()); + // we cannot allow the user to change the key or partitionPath, since that will affect + // everything + // so pick it from one of the records. + return new HoodieRecord(rec1.getKey(), reducedData); + }, parallelism).map(Tuple2::_2); + } + +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java similarity index 95% rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java index 86fa1bfd7..b28c89a53 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -84,11 +86,11 @@ public class UpsertPartitioner> extends Partiti */ private HashMap bucketInfoMap; - protected final HoodieTable table; + protected final HoodieTable table; protected final HoodieWriteConfig config; - public UpsertPartitioner(WorkloadProfile profile, JavaSparkContext jsc, HoodieTable table, + public UpsertPartitioner(WorkloadProfile profile, HoodieEngineContext context, HoodieTable table, HoodieWriteConfig config) { updateLocationToBucket = new HashMap<>(); partitionPathToInsertBucketInfos = new HashMap<>(); @@ -97,7 +99,7 @@ public class UpsertPartitioner> extends Partiti this.table = table; this.config = config; assignUpdates(profile); - assignInserts(profile, jsc); + assignInserts(profile, context); LOG.info("Total Buckets :" + totalBuckets + ", buckets info => " + bucketInfoMap + ", \n" + "Partition to insert buckets => " + partitionPathToInsertBucketInfos + ", \n" @@ -127,7 +129,7 @@ public class UpsertPartitioner> extends Partiti return bucket; } - private void assignInserts(WorkloadProfile profile, JavaSparkContext jsc) { + private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) { // for new inserts, compute buckets depending on how many records we have for each partition Set partitionPaths = profile.getPartitionPaths(); long averageRecordSize = @@ -136,7 +138,7 @@ public class UpsertPartitioner> extends Partiti LOG.info("AvgRecordSize => " + averageRecordSize); Map> partitionSmallFilesMap = - getSmallFilesForPartitions(new ArrayList(partitionPaths), jsc); + getSmallFilesForPartitions(new ArrayList(partitionPaths), context); for (String partitionPath : partitionPaths) { WorkloadStat pStat = profile.getWorkloadStat(partitionPath); @@ -209,11 +211,11 @@ public class UpsertPartitioner> extends Partiti } } - private Map> getSmallFilesForPartitions(List partitionPaths, JavaSparkContext jsc) { - + private Map> getSmallFilesForPartitions(List partitionPaths, HoodieEngineContext context) { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); Map> partitionSmallFilesMap = new HashMap<>(); if (partitionPaths != null && partitionPaths.size() > 0) { - jsc.setJobGroup(this.getClass().getSimpleName(), "Getting small files from partitions"); + context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions"); JavaRDD partitionPathRdds = jsc.parallelize(partitionPaths, partitionPaths.size()); partitionSmallFilesMap = partitionPathRdds.mapToPair((PairFunction>) partitionPath -> new Tuple2<>(partitionPath, getSmallFiles(partitionPath))).collectAsMap(); diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieMergeOnReadTableCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java similarity index 75% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieMergeOnReadTableCompactor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java index c4343f824..505eabb88 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java @@ -21,13 +21,18 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkConfigUtils; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -40,7 +45,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieCopyOnWriteTable; +import org.apache.hudi.io.IOUtils; +import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; @@ -51,7 +57,6 @@ 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.api.java.function.FlatMapFunction; import org.apache.spark.util.AccumulatorV2; import org.apache.spark.util.LongAccumulator; @@ -72,34 +77,36 @@ import static java.util.stream.Collectors.toList; * a normal commit * */ -public class HoodieMergeOnReadTableCompactor implements HoodieCompactor { +@SuppressWarnings("checkstyle:LineLength") +public class HoodieSparkMergeOnReadTableCompactor implements HoodieCompactor>, JavaRDD, JavaRDD> { - private static final Logger LOG = LogManager.getLogger(HoodieMergeOnReadTableCompactor.class); + private static final Logger LOG = LogManager.getLogger(HoodieSparkMergeOnReadTableCompactor.class); // Accumulator to keep track of total log files for a table private AccumulatorV2 totalLogFiles; // Accumulator to keep track of total log file slices for a table private AccumulatorV2 totalFileSlices; @Override - public JavaRDD compact(JavaSparkContext jsc, HoodieCompactionPlan compactionPlan, - HoodieTable hoodieTable, HoodieWriteConfig config, String compactionInstantTime) throws IOException { + public JavaRDD compact(HoodieEngineContext context, HoodieCompactionPlan compactionPlan, + HoodieTable hoodieTable, HoodieWriteConfig config, String compactionInstantTime) throws IOException { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); if (compactionPlan == null || (compactionPlan.getOperations() == null) || (compactionPlan.getOperations().isEmpty())) { return jsc.emptyRDD(); } HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); // Compacting is very similar to applying updates to existing file - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc.hadoopConfiguration(), metaClient); + HoodieSparkCopyOnWriteTable table = new HoodieSparkCopyOnWriteTable(config, context, metaClient); List operations = compactionPlan.getOperations().stream() .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList()); LOG.info("Compactor compacting " + operations + " files"); - jsc.setJobGroup(this.getClass().getSimpleName(), "Compacting file slices"); + context.setJobStatus(this.getClass().getSimpleName(), "Compacting file slices"); return jsc.parallelize(operations, operations.size()) .map(s -> compact(table, metaClient, config, s, compactionInstantTime)).flatMap(List::iterator); } - private List compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient, + private List compact(HoodieSparkCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient, HoodieWriteConfig config, CompactionOperation operation, String instantTime) throws IOException { FileSystem fs = metaClient.getFs(); @@ -116,12 +123,12 @@ public class HoodieMergeOnReadTableCompactor implements HoodieCompactor { .getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) .filterCompletedInstants().lastInstant().get().getTimestamp(); - LOG.info("MaxMemoryPerCompaction => " + SparkConfigUtils.getMaxMemoryPerCompaction(config.getProps())); + long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config.getProps()); + LOG.info("MaxMemoryPerCompaction => " + maxMemoryPerCompaction); List logFiles = operation.getDeltaFileNames().stream().map( p -> new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()), p).toString()) .collect(toList()); - long maxMemoryPerCompaction = SparkConfigUtils.getMaxMemoryPerCompaction(config.getProps()); HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, metaClient.getBasePath(), logFiles, readerSchema, maxInstantTime, maxMemoryPerCompaction, config.getCompactionLazyBlockReadEnabled(), config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(), @@ -162,10 +169,11 @@ public class HoodieMergeOnReadTableCompactor implements HoodieCompactor { } @Override - public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, HoodieTable hoodieTable, - HoodieWriteConfig config, String compactionCommitTime, Set fgIdsInPendingCompactions) + public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable, + HoodieWriteConfig config, String compactionCommitTime, Set fgIdsInPendingCompactions) throws IOException { - + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); totalLogFiles = new LongAccumulator(); totalFileSlices = new LongAccumulator(); jsc.sc().register(totalLogFiles); @@ -192,23 +200,27 @@ public class HoodieMergeOnReadTableCompactor implements HoodieCompactor { SliceView fileSystemView = hoodieTable.getSliceView(); LOG.info("Compaction looking for files to compact in " + partitionPaths + " partitions"); - jsc.setJobGroup(this.getClass().getSimpleName(), "Looking for files to compact"); - List operations = jsc.parallelize(partitionPaths, partitionPaths.size()) - .flatMap((FlatMapFunction) partitionPath -> fileSystemView - .getLatestFileSlices(partitionPath) - .filter(slice -> !fgIdsInPendingCompactions.contains(slice.getFileGroupId())).map(s -> { - List logFiles = - s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); - totalLogFiles.add((long) logFiles.size()); - totalFileSlices.add(1L); - // Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO - // for spark Map operations and collecting them finally in Avro generated classes for storing - // into meta files. - Option dataFile = s.getBaseFile(); - return new CompactionOperation(dataFile, partitionPath, logFiles, - config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles)); - }).filter(c -> !c.getDeltaFileNames().isEmpty()).collect(toList()).iterator()) - .collect().stream().map(CompactionUtils::buildHoodieCompactionOperation).collect(toList()); + context.setJobStatus(this.getClass().getSimpleName(), "Looking for files to compact"); + + List operations = context.flatMap(partitionPaths, partitionPath -> { + return fileSystemView + .getLatestFileSlices(partitionPath) + .filter(slice -> !fgIdsInPendingCompactions.contains(slice.getFileGroupId())) + .map(s -> { + List logFiles = + s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); + totalLogFiles.add((long) logFiles.size()); + totalFileSlices.add(1L); + // Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO + // for spark Map operations and collecting them finally in Avro generated classes for storing + // into meta files. + Option dataFile = s.getBaseFile(); + return new CompactionOperation(dataFile, partitionPath, logFiles, + config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles)); + }) + .filter(c -> !c.getDeltaFileNames().isEmpty()); + }, partitionPaths.size()).stream().map(CompactionUtils::buildHoodieCompactionOperation).collect(toList()); + LOG.info("Total of " + operations.size() + " compactions are retrieved"); LOG.info("Total number of latest files slices " + totalFileSlices.value()); LOG.info("Total number of log files " + totalLogFiles.value()); diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkCompactHelpers.java similarity index 62% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkCompactHelpers.java index 97fdd0fa4..107f533f2 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkCompactHelpers.java @@ -21,26 +21,43 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.exception.HoodieCompactionException; import org.apache.hudi.table.HoodieTable; + import org.apache.spark.api.java.JavaRDD; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.util.List; -public class CompactHelpers { +/** + * A spark implementation of {@link AbstractCompactHelpers}. + * + * @param + */ +public class SparkCompactHelpers extends + AbstractCompactHelpers>, JavaRDD, JavaRDD> { - public static HoodieCommitMetadata createCompactionMetadata(HoodieTable table, - String compactionInstantTime, - JavaRDD writeStatuses, - String schema) throws IOException { + private SparkCompactHelpers() { + } + + private static class CompactHelperHolder { + private static final SparkCompactHelpers SPARK_COMPACT_HELPERS = new SparkCompactHelpers(); + } + + public static SparkCompactHelpers newInstance() { + return CompactHelperHolder.SPARK_COMPACT_HELPERS; + } + + @Override + public HoodieCommitMetadata createCompactionMetadata(HoodieTable>, JavaRDD, JavaRDD> table, + String compactionInstantTime, + JavaRDD writeStatuses, + String schema) throws IOException { byte[] planBytes = table.getActiveTimeline().readCompactionPlanAsBytes( HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get(); HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan(planBytes); @@ -55,16 +72,4 @@ public class CompactHelpers { } return metadata; } - - public static void completeInflightCompaction(HoodieTable table, String compactionCommitTime, HoodieCommitMetadata commitMetadata) { - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - try { - activeTimeline.transitionCompactionInflightToComplete( - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - } catch (IOException e) { - throw new HoodieCompactionException( - "Failed to commit " + table.getMetaClient().getBasePath() + " at time " + compactionCommitTime, e); - } - } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkRunCompactionActionExecutor.java similarity index 70% rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkRunCompactionActionExecutor.java index 2f99fa1ce..ebc3de5b8 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkRunCompactionActionExecutor.java @@ -20,8 +20,12 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkConfigUtils; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.utils.SparkMemoryUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -33,27 +37,25 @@ import org.apache.hudi.exception.HoodieCompactionException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; import org.apache.hudi.table.action.HoodieWriteMetadata; -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 java.io.IOException; import java.util.List; -public class RunCompactionActionExecutor extends BaseActionExecutor { +@SuppressWarnings("checkstyle:LineLength") +public class SparkRunCompactionActionExecutor extends + BaseActionExecutor>, JavaRDD, JavaRDD, HoodieWriteMetadata>> { - private static final Logger LOG = LogManager.getLogger(RunCompactionActionExecutor.class); - - public RunCompactionActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, - HoodieTable table, - String instantTime) { - super(jsc, config, table, instantTime); + public SparkRunCompactionActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, + HoodieTable>, JavaRDD, JavaRDD> table, + String instantTime) { + super(context, config, table, instantTime); } @Override - public HoodieWriteMetadata execute() { + public HoodieWriteMetadata> execute() { HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(instantTime); HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); if (!pendingCompactionTimeline.containsInstant(instant)) { @@ -61,7 +63,7 @@ public class RunCompactionActionExecutor extends BaseActionExecutor> compactionMetadata = new HoodieWriteMetadata<>(); try { HoodieActiveTimeline timeline = table.getActiveTimeline(); HoodieCompactionPlan compactionPlan = @@ -70,10 +72,10 @@ public class RunCompactionActionExecutor extends BaseActionExecutor statuses = compactor.compact(jsc, compactionPlan, table, config, instantTime); + HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor(); + JavaRDD statuses = compactor.compact(context, compactionPlan, table, config, instantTime); - statuses.persist(SparkConfigUtils.getWriteStatusStorageLevel(config.getProps())); + statuses.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps())); List updateStatusMap = statuses.map(WriteStatus::getStat).collect(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(true); for (HoodieWriteStat stat : updateStatusMap) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java new file mode 100644 index 000000000..c5f6c1692 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java @@ -0,0 +1,88 @@ +/* + * 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.table.action.compact; + +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.view.SyncableFileSystemView; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieCompactionException; +import org.apache.hudi.table.HoodieTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +import java.io.IOException; +import java.util.Map; +import java.util.stream.Collectors; + +@SuppressWarnings("checkstyle:LineLength") +public class SparkScheduleCompactionActionExecutor extends + BaseScheduleCompactionActionExecutor>, JavaRDD, JavaRDD> { + + private static final Logger LOG = LogManager.getLogger(SparkScheduleCompactionActionExecutor.class); + + public SparkScheduleCompactionActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, JavaRDD, JavaRDD> table, + String instantTime, + Option> extraMetadata) { + super(context, config, table, instantTime, extraMetadata); + } + + @Override + protected HoodieCompactionPlan scheduleCompaction() { + LOG.info("Checking if compaction needs to be run on " + config.getBasePath()); + Option lastCompaction = table.getActiveTimeline().getCommitTimeline() + .filterCompletedInstants().lastInstant(); + String lastCompactionTs = "0"; + if (lastCompaction.isPresent()) { + lastCompactionTs = lastCompaction.get().getTimestamp(); + } + + int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline() + .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants(); + if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) { + LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction + + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for " + + config.getInlineCompactDeltaCommitMax()); + return new HoodieCompactionPlan(); + } + + LOG.info("Generating compaction plan for merge on read table " + config.getBasePath()); + HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor(); + try { + return compactor.generateCompactionPlan(context, table, config, instantTime, + ((SyncableFileSystemView) table.getSliceView()).getPendingCompactionOperations() + .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) + .collect(Collectors.toSet())); + + } catch (IOException e) { + throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); + } + } + +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java similarity index 64% rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeltaCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java index f616a00b0..64d4c9ce8 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java @@ -20,45 +20,45 @@ package org.apache.hudi.table.action.deltacommit; import java.util.Map; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpsertException; -import org.apache.hudi.execution.LazyInsertIterable; +import org.apache.hudi.execution.SparkLazyInsertIterable; import org.apache.hudi.io.AppendHandleFactory; import org.apache.hudi.io.HoodieAppendHandle; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; -import org.apache.hudi.table.action.commit.CommitActionExecutor; +import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; -import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; import java.util.Collections; import java.util.Iterator; import java.util.List; -public abstract class DeltaCommitActionExecutor> - extends CommitActionExecutor { - private static final Logger LOG = LogManager.getLogger(DeltaCommitActionExecutor.class); +public abstract class AbstractSparkDeltaCommitActionExecutor> + extends BaseSparkCommitActionExecutor { + private static final Logger LOG = LogManager.getLogger(AbstractSparkDeltaCommitActionExecutor.class); // UpsertPartitioner for MergeOnRead table type - private UpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner; + private SparkUpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner; - public DeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, - String instantTime, WriteOperationType operationType) { - this(jsc, config, table, instantTime, operationType, Option.empty()); + public AbstractSparkDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType) { + this(context, config, table, instantTime, operationType, Option.empty()); } - public DeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, - String instantTime, WriteOperationType operationType, - Option> extraMetadata) { - super(jsc, config, table, instantTime, operationType, extraMetadata); + public AbstractSparkDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType, + Option> extraMetadata) { + super(context, config, table, instantTime, operationType, extraMetadata); } @Override @@ -66,7 +66,7 @@ public abstract class DeltaCommitActionExecutor if (profile == null) { throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner."); } - mergeOnReadUpsertPartitioner = new UpsertDeltaCommitPartitioner(profile, jsc, table, config); + mergeOnReadUpsertPartitioner = new SparkUpsertDeltaCommitPartitioner(profile, (HoodieSparkEngineContext) context, table, config); return mergeOnReadUpsertPartitioner; } @@ -79,8 +79,8 @@ public abstract class DeltaCommitActionExecutor LOG.info("Small file corrections for updates for commit " + instantTime + " for file " + fileId); return super.handleUpdate(partitionPath, fileId, recordItr); } else { - HoodieAppendHandle appendHandle = new HoodieAppendHandle<>(config, instantTime, (HoodieTable)table, - partitionPath, fileId, recordItr, sparkTaskContextSupplier); + HoodieAppendHandle appendHandle = new HoodieAppendHandle<>(config, instantTime, table, + partitionPath, fileId, recordItr, taskContextSupplier); appendHandle.doAppend(); appendHandle.close(); return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus())).iterator(); @@ -92,8 +92,8 @@ public abstract class DeltaCommitActionExecutor throws Exception { // If canIndexLogFiles, write inserts to log files else write inserts to base files if (table.getIndex().canIndexLogFiles()) { - return new LazyInsertIterable<>(recordItr, true, config, instantTime, (HoodieTable) table, - idPfx, sparkTaskContextSupplier, new AppendHandleFactory<>()); + return new SparkLazyInsertIterable<>(recordItr, true, config, instantTime, table, + idPfx, taskContextSupplier, new AppendHandleFactory<>()); } else { return super.handleInsert(idPfx, recordItr); } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java similarity index 56% rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertDeltaCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java index 01ff1faa4..281304d95 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java @@ -19,6 +19,9 @@ package org.apache.hudi.table.action.deltacommit; import java.util.Map; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -28,36 +31,35 @@ import org.apache.hudi.exception.HoodieInsertException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.BulkInsertPartitioner; -import org.apache.hudi.table.action.commit.BulkInsertHelper; import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class BulkInsertDeltaCommitActionExecutor> - extends DeltaCommitActionExecutor { +public class SparkBulkInsertDeltaCommitActionExecutor> + extends AbstractSparkDeltaCommitActionExecutor { private final JavaRDD> inputRecordsRDD; private final Option> bulkInsertPartitioner; - public BulkInsertDeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option> bulkInsertPartitioner) { - this(jsc, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty()); + public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD, + Option> bulkInsertPartitioner) { + this(context, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty()); } - public BulkInsertDeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option> bulkInsertPartitioner, - Option> extraMetadata) { - super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); + public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD, + Option> bulkInsertPartitioner, + Option> extraMetadata) { + super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); this.inputRecordsRDD = inputRecordsRDD; this.bulkInsertPartitioner = bulkInsertPartitioner; } @Override - public HoodieWriteMetadata execute() { + public HoodieWriteMetadata> execute() { try { - return BulkInsertHelper.bulkInsert(inputRecordsRDD, instantTime, (HoodieTable) table, config, + return SparkBulkInsertHelper.newInstance().bulkInsert(inputRecordsRDD, instantTime, table, config, this, true, bulkInsertPartitioner); } catch (HoodieInsertException ie) { throw ie; diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java similarity index 65% rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertPreppedDeltaCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java index 7c956008b..21fc013af 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.action.deltacommit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -27,30 +29,29 @@ import org.apache.hudi.exception.HoodieInsertException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.BulkInsertPartitioner; -import org.apache.hudi.table.action.commit.BulkInsertHelper; import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class BulkInsertPreppedDeltaCommitActionExecutor> - extends DeltaCommitActionExecutor { +public class SparkBulkInsertPreppedDeltaCommitActionExecutor> + extends AbstractSparkDeltaCommitActionExecutor { private final JavaRDD> preppedInputRecordRdd; private final Option> bulkInsertPartitioner; - public BulkInsertPreppedDeltaCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedInputRecordRdd, - Option> bulkInsertPartitioner) { - super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT); + public SparkBulkInsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> preppedInputRecordRdd, + Option> bulkInsertPartitioner) { + super(context, config, table, instantTime, WriteOperationType.BULK_INSERT); this.preppedInputRecordRdd = preppedInputRecordRdd; this.bulkInsertPartitioner = bulkInsertPartitioner; } @Override - public HoodieWriteMetadata execute() { + public HoodieWriteMetadata> execute() { try { - return BulkInsertHelper.bulkInsert(preppedInputRecordRdd, instantTime, (HoodieTable) table, config, + return SparkBulkInsertHelper.newInstance().bulkInsert(preppedInputRecordRdd, instantTime, table, config, this, false, bulkInsertPartitioner); } catch (Throwable e) { if (e instanceof HoodieInsertException) { diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeleteDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java similarity index 60% rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeleteDeltaCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java index 53d4d84d7..4fb6a90f9 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeleteDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java @@ -18,30 +18,32 @@ package org.apache.hudi.table.action.deltacommit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.commit.DeleteHelper; import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.commit.SparkDeleteHelper; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class DeleteDeltaCommitActionExecutor> - extends DeltaCommitActionExecutor { +public class SparkDeleteDeltaCommitActionExecutor> + extends AbstractSparkDeltaCommitActionExecutor { private final JavaRDD keys; - public DeleteDeltaCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD keys) { - super(jsc, config, table, instantTime, WriteOperationType.DELETE); + public SparkDeleteDeltaCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD keys) { + super(context, config, table, instantTime, WriteOperationType.DELETE); this.keys = keys; } - public HoodieWriteMetadata execute() { - return DeleteHelper.execute(instantTime, keys, jsc, config, (HoodieTable)table, this); + @Override + public HoodieWriteMetadata> execute() { + return SparkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java similarity index 63% rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertDeltaCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java index 212416555..fcaedee11 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.action.deltacommit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -25,25 +27,24 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.WriteHelper; +import org.apache.hudi.table.action.commit.SparkWriteHelper; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class InsertDeltaCommitActionExecutor> - extends DeltaCommitActionExecutor { +public class SparkInsertDeltaCommitActionExecutor> + extends AbstractSparkDeltaCommitActionExecutor { private final JavaRDD> inputRecordsRDD; - public InsertDeltaCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { - super(jsc, config, table, instantTime, WriteOperationType.INSERT); + public SparkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD) { + super(context, config, table, instantTime, WriteOperationType.INSERT); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata execute() { - return WriteHelper.write(instantTime, inputRecordsRDD, jsc, (HoodieTable) table, + public HoodieWriteMetadata> execute() { + return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(),this, false); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java similarity index 65% rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertPreppedDeltaCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java index d1773f9a5..1f1e0165b 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.action.deltacommit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -25,21 +27,21 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class UpsertPreppedDeltaCommitActionExecutor> - extends DeltaCommitActionExecutor { +public class SparkInsertPreppedDeltaCommitActionExecutor> + extends AbstractSparkDeltaCommitActionExecutor { private final JavaRDD> preppedRecords; - public UpsertPreppedDeltaCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { - super(jsc, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); + public SparkInsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> preppedRecords) { + super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED); this.preppedRecords = preppedRecords; } - public HoodieWriteMetadata execute() { + @Override + public HoodieWriteMetadata> execute() { return super.execute(preppedRecords); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java similarity index 67% rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertDeltaCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java index 1809078cb..82aa08152 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java @@ -18,6 +18,7 @@ package org.apache.hudi.table.action.deltacommit; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -25,25 +26,24 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.WriteHelper; +import org.apache.hudi.table.action.commit.SparkWriteHelper; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class UpsertDeltaCommitActionExecutor> - extends DeltaCommitActionExecutor { +public class SparkUpsertDeltaCommitActionExecutor> + extends AbstractSparkDeltaCommitActionExecutor { private JavaRDD> inputRecordsRDD; - public UpsertDeltaCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { - super(jsc, config, table, instantTime, WriteOperationType.UPSERT); + public SparkUpsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD) { + super(context, config, table, instantTime, WriteOperationType.UPSERT); this.inputRecordsRDD = inputRecordsRDD; } @Override public HoodieWriteMetadata execute() { - return WriteHelper.write(instantTime, inputRecordsRDD, jsc, (HoodieTable) table, + return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(),this, true); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertDeltaCommitPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java similarity index 94% rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertDeltaCommitPartitioner.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java index f5a437058..48a0ff082 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertDeltaCommitPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java @@ -18,6 +18,7 @@ package org.apache.hudi.table.action.deltacommit; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieLogFile; @@ -32,7 +33,6 @@ import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.action.commit.SmallFile; import org.apache.hudi.table.action.commit.UpsertPartitioner; -import org.apache.spark.api.java.JavaSparkContext; import java.util.ArrayList; import java.util.List; @@ -42,11 +42,11 @@ import java.util.stream.Collectors; * UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet files to larger ones * without the need for an index in the logFile. */ -public class UpsertDeltaCommitPartitioner> extends UpsertPartitioner { +public class SparkUpsertDeltaCommitPartitioner> extends UpsertPartitioner { - UpsertDeltaCommitPartitioner(WorkloadProfile profile, JavaSparkContext jsc, HoodieTable table, - HoodieWriteConfig config) { - super(profile, jsc, table, config); + SparkUpsertDeltaCommitPartitioner(WorkloadProfile profile, HoodieSparkEngineContext context, HoodieTable table, + HoodieWriteConfig config) { + super(profile, context, table, config); } @Override diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java similarity index 65% rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertPreppedDeltaCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java index 0fb787e55..3509efa6b 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.action.deltacommit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -25,21 +27,21 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class InsertPreppedDeltaCommitActionExecutor> - extends DeltaCommitActionExecutor { +public class SparkUpsertPreppedDeltaCommitActionExecutor> + extends AbstractSparkDeltaCommitActionExecutor { private final JavaRDD> preppedRecords; - public InsertPreppedDeltaCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { - super(jsc, config, table, instantTime, WriteOperationType.INSERT_PREPPED); + public SparkUpsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> preppedRecords) { + super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); this.preppedRecords = preppedRecords; } - public HoodieWriteMetadata execute() { + @Override + public HoodieWriteMetadata> execute() { return super.execute(preppedRecords); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/restore/SparkCopyOnWriteRestoreActionExecutor.java similarity index 60% rename from hudi-client/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/restore/SparkCopyOnWriteRestoreActionExecutor.java index cbd312759..101b3217d 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/restore/SparkCopyOnWriteRestoreActionExecutor.java @@ -19,30 +19,38 @@ package org.apache.hudi.table.action.restore; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.hudi.table.action.rollback.SparkCopyOnWriteRollbackActionExecutor; -public class CopyOnWriteRestoreActionExecutor extends BaseRestoreActionExecutor { +import org.apache.spark.api.java.JavaRDD; - public CopyOnWriteRestoreActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - String restoreInstantTime) { - super(jsc, config, table, instantTime, restoreInstantTime); +@SuppressWarnings("checkstyle:LineLength") +public class SparkCopyOnWriteRestoreActionExecutor extends + BaseRestoreActionExecutor>, JavaRDD, JavaRDD> { + + public SparkCopyOnWriteRestoreActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + String restoreInstantTime) { + super(context, config, table, instantTime, restoreInstantTime); } @Override protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback) { table.getMetaClient().reloadActiveTimeline(); - CopyOnWriteRollbackActionExecutor rollbackActionExecutor = new CopyOnWriteRollbackActionExecutor( - jsc, + SparkCopyOnWriteRollbackActionExecutor rollbackActionExecutor = new SparkCopyOnWriteRollbackActionExecutor( + (HoodieSparkEngineContext) context, config, table, HoodieActiveTimeline.createNewInstantTime(), diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/restore/SparkMergeOnReadRestoreActionExecutor.java similarity index 64% rename from hudi-client/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/restore/SparkMergeOnReadRestoreActionExecutor.java index edb9acdcc..c32057938 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/restore/SparkMergeOnReadRestoreActionExecutor.java @@ -19,29 +19,37 @@ package org.apache.hudi.table.action.restore; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.hudi.table.action.rollback.SparkMergeOnReadRollbackActionExecutor; -public class MergeOnReadRestoreActionExecutor extends BaseRestoreActionExecutor { +import org.apache.spark.api.java.JavaRDD; - public MergeOnReadRestoreActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - String restoreInstantTime) { - super(jsc, config, table, instantTime, restoreInstantTime); +@SuppressWarnings("checkstyle:LineLength") +public class SparkMergeOnReadRestoreActionExecutor extends + BaseRestoreActionExecutor>, JavaRDD, JavaRDD> { + + public SparkMergeOnReadRestoreActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + String restoreInstantTime) { + super(context, config, table, instantTime, restoreInstantTime); } @Override protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback) { table.getMetaClient().reloadActiveTimeline(); - MergeOnReadRollbackActionExecutor rollbackActionExecutor = new MergeOnReadRollbackActionExecutor( - jsc, + SparkMergeOnReadRollbackActionExecutor rollbackActionExecutor = new SparkMergeOnReadRollbackActionExecutor( + context, config, table, HoodieActiveTimeline.createNewInstantTime(), diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java similarity index 90% rename from hudi-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java index 3c94df489..9cf2434bc 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java @@ -18,7 +18,9 @@ package org.apache.hudi.table.action.rollback; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -68,35 +70,36 @@ public class ListingBasedRollbackHelper implements Serializable { /** * Performs all rollback actions that we have collected in parallel. */ - public List performRollback(JavaSparkContext jsc, HoodieInstant instantToRollback, List rollbackRequests) { + public List performRollback(HoodieEngineContext context, HoodieInstant instantToRollback, List rollbackRequests) { int sparkPartitions = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1); - jsc.setJobGroup(this.getClass().getSimpleName(), "Perform rollback actions"); - JavaPairRDD partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(jsc, instantToRollback, rollbackRequests, sparkPartitions, true); + context.setJobStatus(this.getClass().getSimpleName(), "Perform rollback actions"); + JavaPairRDD partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, sparkPartitions, true); return partitionPathRollbackStatsPairRDD.reduceByKey(RollbackUtils::mergeRollbackStat).map(Tuple2::_2).collect(); } /** * Collect all file info that needs to be rollbacked. */ - public List collectRollbackStats(JavaSparkContext jsc, HoodieInstant instantToRollback, List rollbackRequests) { + public List collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback, List rollbackRequests) { int sparkPartitions = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1); - jsc.setJobGroup(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade"); - JavaPairRDD partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(jsc, instantToRollback, rollbackRequests, sparkPartitions, false); + context.setJobStatus(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade"); + JavaPairRDD partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, sparkPartitions, false); return partitionPathRollbackStatsPairRDD.map(Tuple2::_2).collect(); } /** * May be delete interested files and collect stats or collect stats only. * - * @param jsc instance of {@link JavaSparkContext} to use. + * @param context instance of {@link HoodieEngineContext} to use. * @param instantToRollback {@link HoodieInstant} of interest for which deletion or collect stats is requested. * @param rollbackRequests List of {@link ListingBasedRollbackRequest} to be operated on. * @param sparkPartitions number of spark partitions to use for parallelism. * @param doDelete {@code true} if deletion has to be done. {@code false} if only stats are to be collected w/o performing any deletes. * @return stats collected with or w/o actual deletions. */ - JavaPairRDD maybeDeleteAndCollectStats(JavaSparkContext jsc, HoodieInstant instantToRollback, List rollbackRequests, + JavaPairRDD maybeDeleteAndCollectStats(HoodieEngineContext context, HoodieInstant instantToRollback, List rollbackRequests, int sparkPartitions, boolean doDelete) { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); return jsc.parallelize(rollbackRequests, sparkPartitions).mapToPair(rollbackRequest -> { switch (rollbackRequest.getType()) { case DELETE_DATA_FILES_ONLY: { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java new file mode 100644 index 000000000..965d80559 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java @@ -0,0 +1,73 @@ +/* + * 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.table.action.rollback; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import org.apache.spark.api.java.JavaRDD; + +import java.util.List; + +@SuppressWarnings("checkstyle:LineLength") +public class SparkCopyOnWriteRollbackActionExecutor extends + BaseCopyOnWriteRollbackActionExecutor>, JavaRDD, JavaRDD> { + public SparkCopyOnWriteRollbackActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, + HoodieTable>, JavaRDD, JavaRDD> table, + String instantTime, + HoodieInstant commitInstant, + boolean deleteInstants) { + super(context, config, table, instantTime, commitInstant, deleteInstants); + } + + public SparkCopyOnWriteRollbackActionExecutor(HoodieSparkEngineContext context, + HoodieWriteConfig config, + HoodieTable>, JavaRDD, JavaRDD> table, + String instantTime, + HoodieInstant commitInstant, + boolean deleteInstants, + boolean skipTimelinePublish, + boolean useMarkerBasedStrategy) { + super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy); + } + + @Override + protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() { + if (useMarkerBasedStrategy) { + return new SparkMarkerBasedRollbackStrategy(table, context, config, instantTime); + } else { + return this::executeRollbackUsingFileListing; + } + } + + @Override + protected List executeRollbackUsingFileListing(HoodieInstant instantToRollback) { + List rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), + config.shouldAssumeDatePartitioning()); + return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMarkerBasedRollbackStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMarkerBasedRollbackStrategy.java new file mode 100644 index 000000000..065b22d78 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMarkerBasedRollbackStrategy.java @@ -0,0 +1,77 @@ +/* + * 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.table.action.rollback; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieRollbackException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.MarkerFiles; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import java.util.List; + +import scala.Tuple2; + +@SuppressWarnings("checkstyle:LineLength") +public class SparkMarkerBasedRollbackStrategy extends AbstractMarkerBasedRollbackStrategy>, JavaRDD, JavaRDD> { + public SparkMarkerBasedRollbackStrategy(HoodieTable>, JavaRDD, JavaRDD> table, HoodieEngineContext context, HoodieWriteConfig config, String instantTime) { + super(table, context, config, instantTime); + } + + @Override + public List execute(HoodieInstant instantToRollback) { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + try { + MarkerFiles markerFiles = new MarkerFiles(table, instantToRollback.getTimestamp()); + List markerFilePaths = markerFiles.allMarkerFilePaths(); + int parallelism = Math.max(Math.min(markerFilePaths.size(), config.getRollbackParallelism()), 1); + return jsc.parallelize(markerFilePaths, parallelism) + .map(markerFilePath -> { + String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1); + IOType type = IOType.valueOf(typeStr); + switch (type) { + case MERGE: + return undoMerge(MarkerFiles.stripMarkerSuffix(markerFilePath)); + case APPEND: + return undoAppend(MarkerFiles.stripMarkerSuffix(markerFilePath), instantToRollback); + case CREATE: + return undoCreate(MarkerFiles.stripMarkerSuffix(markerFilePath)); + default: + throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback); + } + }) + .mapToPair(rollbackStat -> new Tuple2<>(rollbackStat.getPartitionPath(), rollbackStat)) + .reduceByKey(RollbackUtils::mergeRollbackStat) + .map(Tuple2::_2).collect(); + } catch (Exception e) { + throw new HoodieRollbackException("Error rolling back using marker files written for " + instantToRollback, e); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMergeOnReadRollbackActionExecutor.java new file mode 100644 index 000000000..459ab128f --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMergeOnReadRollbackActionExecutor.java @@ -0,0 +1,82 @@ +/* + * 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.table.action.rollback; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.HoodieTable; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import java.io.IOException; +import java.util.List; + +@SuppressWarnings("checkstyle:LineLength") +public class SparkMergeOnReadRollbackActionExecutor extends + BaseMergeOnReadRollbackActionExecutor>, JavaRDD, JavaRDD> { + public SparkMergeOnReadRollbackActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, JavaRDD, JavaRDD> table, + String instantTime, + HoodieInstant commitInstant, + boolean deleteInstants) { + super(context, config, table, instantTime, commitInstant, deleteInstants); + } + + public SparkMergeOnReadRollbackActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, JavaRDD, JavaRDD> table, + String instantTime, + HoodieInstant commitInstant, + boolean deleteInstants, + boolean skipTimelinePublish, + boolean useMarkerBasedStrategy) { + super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy); + } + + @Override + protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() { + if (useMarkerBasedStrategy) { + return new SparkMarkerBasedRollbackStrategy(table, context, config, instantTime); + } else { + return this::executeRollbackUsingFileListing; + } + } + + @Override + protected List executeRollbackUsingFileListing(HoodieInstant resolvedInstant) { + List rollbackRequests; + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + try { + rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(resolvedInstant, table, context); + } catch (IOException e) { + throw new HoodieIOException("Error generating rollback requests by file listing.", e); + } + return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, resolvedInstant, rollbackRequests); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java similarity index 79% rename from hudi-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java index 108bdbbf0..52849cb06 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java @@ -18,32 +18,31 @@ package org.apache.hudi.table.upgrade; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.MarkerFiles; -import org.apache.spark.api.java.JavaSparkContext; - import java.util.List; import java.util.stream.Collectors; /** * Downgrade handle to assist in downgrading hoodie table from version 1 to 0. */ -public class OneToZeroDowngradeHandler implements DowngradeHandler { +public class OneToZeroDowngradeHandler implements DowngradeHandler { @Override - public void downgrade(HoodieWriteConfig config, JavaSparkContext jsc, String instantTime) { + public void downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) { // fetch pending commit info - HoodieTable table = HoodieTable.create(config, jsc.hadoopConfiguration()); + HoodieSparkTable table = HoodieSparkTable.create(config, context); HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); List commits = inflightTimeline.getReverseOrderedInstants().collect(Collectors.toList()); for (HoodieInstant commitInstant : commits) { // delete existing marker files MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp()); - markerFiles.quietDeleteMarkerDir(jsc, config.getMarkersDeleteParallelism()); + markerFiles.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); } } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java new file mode 100644 index 000000000..9c13c5a78 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java @@ -0,0 +1,66 @@ +/* + * 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.table.upgrade; + +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieUpgradeDowngradeException; + +import java.io.IOException; + +public class SparkUpgradeDowngrade extends AbstractUpgradeDowngrade { + + public SparkUpgradeDowngrade(HoodieTableMetaClient metaClient, HoodieWriteConfig config, HoodieEngineContext context) { + super(metaClient, config, context); + } + + @Override + public void run(HoodieTableMetaClient metaClient, + HoodieTableVersion toVersion, + HoodieWriteConfig config, + HoodieEngineContext context, + String instantTime) { + try { + new SparkUpgradeDowngrade(metaClient, config, context).run(toVersion, instantTime); + } catch (IOException e) { + throw new HoodieUpgradeDowngradeException("Error during upgrade/downgrade to version:" + toVersion, e); + } + + } + + @Override + protected void upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) { + if (fromVersion == HoodieTableVersion.ZERO && toVersion == HoodieTableVersion.ONE) { + new ZeroToOneUpgradeHandler().upgrade(config, context, instantTime); + } else { + throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), true); + } + } + + @Override + protected void downgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) { + if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.ZERO) { + new OneToZeroDowngradeHandler().downgrade(config, context, instantTime); + } else { + throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), false); + } + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java similarity index 86% rename from hudi-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java index e9c9e28bb..7e3faf32b 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java @@ -18,6 +18,7 @@ package org.apache.hudi.table.upgrade; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieTableType; @@ -28,15 +29,15 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieRollbackException; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.MarkerFiles; import org.apache.hudi.table.action.rollback.ListingBasedRollbackHelper; import org.apache.hudi.table.action.rollback.ListingBasedRollbackRequest; -import org.apache.hudi.table.action.rollback.RollbackUtils; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.hudi.table.action.rollback.RollbackUtils; import java.util.List; import java.util.stream.Collectors; @@ -47,9 +48,9 @@ import java.util.stream.Collectors; public class ZeroToOneUpgradeHandler implements UpgradeHandler { @Override - public void upgrade(HoodieWriteConfig config, JavaSparkContext jsc, String instantTime) { + public void upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) { // fetch pending commit info - HoodieTable table = HoodieTable.create(config, jsc.hadoopConfiguration()); + HoodieSparkTable table = HoodieSparkTable.create(config, context); HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); List commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); @@ -59,7 +60,7 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler { } for (String commit : commits) { // for every pending commit, delete old marker files and re-create marker files in new format - recreateMarkerFiles(commit, table, jsc, config.getMarkersDeleteParallelism()); + recreateMarkerFiles(commit, table, context, config.getMarkersDeleteParallelism()); } } @@ -70,11 +71,14 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler { * Step3: recreate marker files for all interested files. * * @param commitInstantTime instant of interest for which marker files need to be recreated. - * @param table instance of {@link HoodieTable} to use - * @param jsc instance of {@link JavaSparkContext} to use + * @param table instance of {@link HoodieSparkTable} to use + * @param context instance of {@link HoodieEngineContext} to use * @throws HoodieRollbackException on any exception during upgrade. */ - private static void recreateMarkerFiles(final String commitInstantTime, HoodieTable table, JavaSparkContext jsc, int parallelism) throws HoodieRollbackException { + private static void recreateMarkerFiles(final String commitInstantTime, + HoodieSparkTable table, + HoodieEngineContext context, + int parallelism) throws HoodieRollbackException { try { // fetch hoodie instant Option commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants() @@ -83,7 +87,7 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler { if (commitInstantOpt.isPresent()) { // delete existing marker files MarkerFiles markerFiles = new MarkerFiles(table, commitInstantTime); - markerFiles.quietDeleteMarkerDir(jsc, parallelism); + markerFiles.quietDeleteMarkerDir(context, parallelism); // generate rollback stats List rollbackRequests; @@ -91,10 +95,10 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler { rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), table.getConfig().shouldAssumeDatePartitioning()); } else { - rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, jsc); + rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context); } List rollbackStats = new ListingBasedRollbackHelper(table.getMetaClient(), table.getConfig()) - .collectRollbackStats(jsc, commitInstantOpt.get(), rollbackRequests); + .collectRollbackStats(context, commitInstantOpt.get(), rollbackRequests); // recreate marker files adhering to marker based rollback for (HoodieRollbackStat rollbackStat : rollbackStats) { diff --git a/hudi-client/hudi-spark-client/src/main/resources/log4j.properties b/hudi-client/hudi-spark-client/src/main/resources/log4j.properties new file mode 100644 index 000000000..ff268faf6 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/resources/log4j.properties @@ -0,0 +1,23 @@ +### +# 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. +### +log4j.rootLogger=INFO, A1 +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestClientRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java similarity index 95% rename from hudi-client/src/test/java/org/apache/hudi/client/TestClientRollback.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java index da4c002e9..d04a2df1b 100644 --- a/hudi-client/src/test/java/org/apache/hudi/client/TestClientRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java @@ -32,7 +32,7 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.spark.api.java.JavaRDD; @@ -61,7 +61,7 @@ public class TestClientRollback extends HoodieClientTestBase { public void testSavepointAndRollback() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()).build(); - try (HoodieWriteClient client = getHoodieWriteClient(cfg)) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) { HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); /** @@ -102,7 +102,7 @@ public class TestClientRollback extends HoodieClientTestBase { List partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.create(metaClient, getConfig(), hadoopConf); + HoodieSparkTable table = HoodieSparkTable.create(getConfig(), context, metaClient); final BaseFileOnlyView view1 = table.getBaseFileOnlyView(); List dataFiles = partitionPaths.stream().flatMap(s -> { @@ -127,7 +127,7 @@ public class TestClientRollback extends HoodieClientTestBase { assertNoWriteErrors(statuses); metaClient = HoodieTableMetaClient.reload(metaClient); - table = HoodieTable.create(metaClient, getConfig(), hadoopConf); + table = HoodieSparkTable.create(getConfig(), context, metaClient); final BaseFileOnlyView view2 = table.getBaseFileOnlyView(); dataFiles = partitionPaths.stream().flatMap(s -> view2.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("004"))).collect(Collectors.toList()); @@ -143,7 +143,7 @@ public class TestClientRollback extends HoodieClientTestBase { client.restoreToSavepoint(savepoint.getTimestamp()); metaClient = HoodieTableMetaClient.reload(metaClient); - table = HoodieTable.create(metaClient, getConfig(), hadoopConf); + table = HoodieSparkTable.create(getConfig(), context, metaClient); final BaseFileOnlyView view3 = table.getBaseFileOnlyView(); dataFiles = partitionPaths.stream().flatMap(s -> view3.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("002"))).collect(Collectors.toList()); assertEquals(3, dataFiles.size(), "The data files for commit 002 be available"); @@ -201,7 +201,7 @@ public class TestClientRollback extends HoodieClientTestBase { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); - try (HoodieWriteClient client = getHoodieWriteClient(config, false)) { + try (SparkRDDWriteClient client = getHoodieWriteClient(config, false)) { // Rollback commit 1 (this should fail, since commit2 is still around) assertThrows(HoodieRollbackException.class, () -> { @@ -294,7 +294,7 @@ public class TestClientRollback extends HoodieClientTestBase { .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); final String commitTime4 = "20160506030621"; - try (HoodieWriteClient client = getHoodieWriteClient(config, false)) { + try (SparkRDDWriteClient client = getHoodieWriteClient(config, false)) { client.startCommitWithTime(commitTime4); // Check results, nothing changed assertTrue(testTable.commitExists(commitTime1)); @@ -307,7 +307,7 @@ public class TestClientRollback extends HoodieClientTestBase { // Turn auto rollback on final String commitTime5 = "20160506030631"; - try (HoodieWriteClient client = getHoodieWriteClient(config, true)) { + try (SparkRDDWriteClient client = getHoodieWriteClient(config, true)) { client.startCommitWithTime(commitTime5); assertTrue(testTable.commitExists(commitTime1)); assertFalse(testTable.inflightCommitExists(commitTime2)); diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java similarity index 99% rename from hudi-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java index 1200f67cc..03328dddd 100644 --- a/hudi-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java @@ -67,7 +67,7 @@ public class TestCompactionAdminClient extends HoodieClientTestBase { initPath(); initSparkContexts(); metaClient = HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath, MERGE_ON_READ); - client = new CompactionAdminClient(jsc, basePath); + client = new CompactionAdminClient(context, basePath); } @Test diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java similarity index 94% rename from hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java index 47edbbd40..bbb40488b 100644 --- a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java @@ -51,9 +51,10 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.MarkerFiles; -import org.apache.hudi.table.action.commit.WriteHelper; +import org.apache.hudi.table.action.commit.SparkWriteHelper; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.HoodieClientTestUtils; import org.apache.hudi.testutils.HoodieWriteableTestTable; @@ -115,7 +116,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { */ @Test public void testAutoCommitOnInsert() throws Exception { - testAutoCommit(HoodieWriteClient::insert, false); + testAutoCommit(SparkRDDWriteClient::insert, false); } /** @@ -123,7 +124,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { */ @Test public void testAutoCommitOnInsertPrepped() throws Exception { - testAutoCommit(HoodieWriteClient::insertPreppedRecords, true); + testAutoCommit(SparkRDDWriteClient::insertPreppedRecords, true); } /** @@ -131,7 +132,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { */ @Test public void testAutoCommitOnUpsert() throws Exception { - testAutoCommit(HoodieWriteClient::upsert, false); + testAutoCommit(SparkRDDWriteClient::upsert, false); } /** @@ -139,7 +140,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { */ @Test public void testAutoCommitOnUpsertPrepped() throws Exception { - testAutoCommit(HoodieWriteClient::upsertPreppedRecords, true); + testAutoCommit(SparkRDDWriteClient::upsertPreppedRecords, true); } /** @@ -147,7 +148,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { */ @Test public void testAutoCommitOnBulkInsert() throws Exception { - testAutoCommit(HoodieWriteClient::bulkInsert, false); + testAutoCommit(SparkRDDWriteClient::bulkInsert, false); } /** @@ -165,11 +166,11 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { * @param writeFn One of HoodieWriteClient Write API * @throws Exception in case of failure */ - private void testAutoCommit(Function3, HoodieWriteClient, JavaRDD, String> writeFn, + private void testAutoCommit(Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPrepped) throws Exception { // Set autoCommit false HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { String prevCommitTime = "000"; String newCommitTime = "001"; @@ -190,7 +191,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { */ @Test public void testDeduplicationOnInsert() throws Exception { - testDeduplication(HoodieWriteClient::insert); + testDeduplication(SparkRDDWriteClient::insert); } /** @@ -198,7 +199,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { */ @Test public void testDeduplicationOnBulkInsert() throws Exception { - testDeduplication(HoodieWriteClient::bulkInsert); + testDeduplication(SparkRDDWriteClient::bulkInsert); } /** @@ -206,7 +207,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { */ @Test public void testDeduplicationOnUpsert() throws Exception { - testDeduplication(HoodieWriteClient::upsert); + testDeduplication(SparkRDDWriteClient::upsert); } /** @@ -216,7 +217,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { * @throws Exception in case of failure */ private void testDeduplication( - Function3, HoodieWriteClient, JavaRDD, String> writeFn) throws Exception { + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn) throws Exception { String newCommitTime = "001"; String recordKey = UUID.randomUUID().toString(); @@ -238,20 +239,20 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { // Global dedup should be done based on recordKey only HoodieIndex index = mock(HoodieIndex.class); when(index.isGlobal()).thenReturn(true); - List> dedupedRecs = WriteHelper.deduplicateRecords(records, index, 1).collect(); + List> dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect(); assertEquals(1, dedupedRecs.size()); assertNodupesWithinPartition(dedupedRecs); // non-Global dedup should be done based on both recordKey and partitionPath index = mock(HoodieIndex.class); when(index.isGlobal()).thenReturn(false); - dedupedRecs = WriteHelper.deduplicateRecords(records, index, 1).collect(); + dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect(); assertEquals(2, dedupedRecs.size()); assertNodupesWithinPartition(dedupedRecs); // Perform write-action and check JavaRDD recordList = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1); - try (HoodieWriteClient client = getHoodieWriteClient(getConfigBuilder().combineInput(true, true).build(), false);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(getConfigBuilder().combineInput(true, true).build(), false);) { client.startCommitWithTime(newCommitTime); List statuses = writeFn.apply(client, recordList, newCommitTime).collect(); assertNoWriteErrors(statuses); @@ -284,7 +285,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { */ @Test public void testUpserts() throws Exception { - testUpsertsInternal(getConfig(), HoodieWriteClient::upsert, false); + testUpsertsInternal(getConfig(), SparkRDDWriteClient::upsert, false); } /** @@ -292,7 +293,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { */ @Test public void testUpsertsPrepped() throws Exception { - testUpsertsInternal(getConfig(), HoodieWriteClient::upsertPreppedRecords, true); + testUpsertsInternal(getConfig(), SparkRDDWriteClient::upsertPreppedRecords, true); } /** @@ -303,7 +304,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { * @throws Exception in case of error */ private void testUpsertsInternal(HoodieWriteConfig config, - Function3, HoodieWriteClient, JavaRDD, String> writeFn, boolean isPrepped) + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPrepped) throws Exception { // Force using older timeline layout HoodieWriteConfig hoodieWriteConfig = getConfigBuilder().withProps(config.getProps()).withTimelineLayoutVersion( @@ -311,13 +312,13 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), metaClient.getTableType(), metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_0); - HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); + SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); // Write 1 (only inserts) String newCommitTime = "001"; String initCommitTime = "000"; int numRecords = 200; - insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, HoodieWriteClient::insert, + insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, SparkRDDWriteClient::insert, isPrepped, true, numRecords); // Write 2 (updates) @@ -335,7 +336,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { numRecords = 50; deleteBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime, - initCommitTime, numRecords, HoodieWriteClient::delete, isPrepped, true, + initCommitTime, numRecords, SparkRDDWriteClient::delete, isPrepped, true, 0, 150); // Now simulate an upgrade and perform a restore operation @@ -358,7 +359,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { numRecords = 50; deleteBatch(newConfig, client, newCommitTime, prevCommitTime, - initCommitTime, numRecords, HoodieWriteClient::delete, isPrepped, true, + initCommitTime, numRecords, SparkRDDWriteClient::delete, isPrepped, true, 0, 150); HoodieActiveTimeline activeTimeline = new HoodieActiveTimeline(metaClient, false); @@ -382,7 +383,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { */ @Test public void testDeletes() throws Exception { - HoodieWriteClient client = getHoodieWriteClient(getConfig(), false); + SparkRDDWriteClient client = getHoodieWriteClient(getConfig(), false); /** * Write 1 (inserts and deletes) Write actual 200 insert records and ignore 100 delete records @@ -402,7 +403,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { }; writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, // unused as genFn uses hard-coded number of inserts/updates/deletes - -1, recordGenFunction, HoodieWriteClient::upsert, true, 200, 200, 1); + -1, recordGenFunction, SparkRDDWriteClient::upsert, true, 200, 200, 1); /** * Write 2 (deletes+writes). @@ -419,7 +420,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { return recordsInSecondBatch; }; writeBatch(client, newCommitTime, prevCommitTime, Option.empty(), initCommitTime, 100, recordGenFunction, - HoodieWriteClient::upsert, true, 50, 150, 2); + SparkRDDWriteClient::upsert, true, 50, 150, 2); } /** @@ -429,7 +430,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { */ @Test public void testDeletesForInsertsInSameBatch() throws Exception { - HoodieWriteClient client = getHoodieWriteClient(getConfig(), false); + SparkRDDWriteClient client = getHoodieWriteClient(getConfig(), false); /** * Write 200 inserts and issue deletes to a subset(50) of inserts. @@ -449,7 +450,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { }; writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, - -1, recordGenFunction, HoodieWriteClient::upsert, true, 150, 150, 1); + -1, recordGenFunction, SparkRDDWriteClient::upsert, true, 150, 150, 1); } /** @@ -458,7 +459,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = IndexType.class, names = {"GLOBAL_BLOOM", "GLOBAL_SIMPLE"}) public void testUpsertsUpdatePartitionPathGlobalBloom(IndexType indexType) throws Exception { - testUpsertsUpdatePartitionPath(indexType, getConfig(), HoodieWriteClient::upsert); + testUpsertsUpdatePartitionPath(indexType, getConfig(), SparkRDDWriteClient::upsert); } /** @@ -476,7 +477,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { * @param writeFn write function to be used for testing */ private void testUpsertsUpdatePartitionPath(IndexType indexType, HoodieWriteConfig config, - Function3, HoodieWriteClient, JavaRDD, String> writeFn) + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn) throws Exception { // instantiate client @@ -491,7 +492,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), metaClient.getTableType(), metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_0); - HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); + SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); // Write 1 String newCommitTime = "001"; @@ -637,7 +638,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); - HoodieWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config, false); // Inserts => will write file1 String commitTime1 = "001"; @@ -748,7 +749,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { // setup the small file handling params HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); - HoodieWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config, false); // Inserts => will write file1 String commitTime1 = "001"; @@ -829,7 +830,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); - HoodieWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config, false); // Inserts => will write file1 String commitTime1 = "001"; @@ -920,7 +921,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { private void verifyInsertOverwritePartitionHandling(int batch1RecordsCount, int batch2RecordsCount) throws Exception { final String testPartitionPath = "americas"; HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false); - HoodieWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config, false); dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); // Do Inserts @@ -968,7 +969,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { } } - private Pair, List> testUpdates(String instantTime, HoodieWriteClient client, + private Pair, List> testUpdates(String instantTime, SparkRDDWriteClient client, int sizeToInsertAndUpdate, int expectedTotalRecords) throws IOException { client.startCommitWithTime(instantTime); @@ -993,7 +994,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { return Pair.of(keys, inserts); } - private void testDeletes(HoodieWriteClient client, List previousRecords, int sizeToDelete, + private void testDeletes(SparkRDDWriteClient client, List previousRecords, int sizeToDelete, String existingFile, String instantTime, int exepctedRecords, List keys) { client.startCommitWithTime(instantTime); @@ -1039,7 +1040,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit, true); // hold upto 200 records max dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); - HoodieWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config, false); // delete non existent keys String commitTime1 = "001"; @@ -1060,9 +1061,9 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { public void testCommitWritesRelativePaths() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath); - HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieSparkTable table = HoodieSparkTable.create(cfg, context, metaClient); String instantTime = "000"; client.startCommitWithTime(instantTime); @@ -1104,7 +1105,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { @Test public void testMetadataStatsOnCommit() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); - HoodieWriteClient client = getHoodieWriteClient(cfg); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); String instantTime0 = "000"; client.startCommitWithTime(instantTime0); @@ -1170,7 +1171,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { String instantTime = "000"; HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder() .withEnableOptimisticConsistencyGuard(enableOptimisticConsistencyGuard).build()).build(); - HoodieWriteClient client = getHoodieWriteClient(cfg); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); Pair> result = testConsistencyCheck(metaClient, instantTime, enableOptimisticConsistencyGuard); // Delete orphan marker and commit should succeed @@ -1200,7 +1201,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder() .withConsistencyCheckEnabled(true) .withOptimisticConsistencyGuardSleepTimeMs(1).build()).build(); - HoodieWriteClient client = getHoodieWriteClient(cfg); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); testConsistencyCheck(metaClient, instantTime, enableOptimisticConsistencyGuard); if (!enableOptimisticConsistencyGuard) { @@ -1254,7 +1255,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true) .withOptimisticConsistencyGuardSleepTimeMs(1).build()) .build()); - HoodieWriteClient client = getHoodieWriteClient(cfg); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); client.startCommitWithTime(instantTime); JavaRDD writeRecords = jsc.parallelize(dataGen.generateInserts(instantTime, 200), 1); @@ -1291,7 +1292,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false) .withAllowMultiWriteOnSameInstant(true) .build(); - HoodieWriteClient client = getHoodieWriteClient(cfg); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); String firstInstantTime = "0000"; client.startCommitWithTime(firstInstantTime); int numRecords = 200; diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieInternalWriteStatus.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieInternalWriteStatus.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/client/TestHoodieInternalWriteStatus.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieInternalWriteStatus.java diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java similarity index 87% rename from hudi-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java index 80af51397..35ee557ce 100644 --- a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java @@ -52,7 +52,7 @@ public class TestHoodieReadClient extends HoodieClientTestBase { */ @Test public void testReadFilterExistAfterInsert() throws Exception { - testReadFilterExist(getConfig(), HoodieWriteClient::insert); + testReadFilterExist(getConfig(), SparkRDDWriteClient::insert); } /** @@ -60,7 +60,7 @@ public class TestHoodieReadClient extends HoodieClientTestBase { */ @Test public void testReadFilterExistAfterInsertPrepped() throws Exception { - testReadFilterExist(getConfig(), HoodieWriteClient::insertPreppedRecords); + testReadFilterExist(getConfig(), SparkRDDWriteClient::insertPreppedRecords); } /** @@ -68,7 +68,7 @@ public class TestHoodieReadClient extends HoodieClientTestBase { */ @Test public void testReadFilterExistAfterBulkInsert() throws Exception { - testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(), HoodieWriteClient::bulkInsert); + testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(), SparkRDDWriteClient::bulkInsert); } /** @@ -84,7 +84,7 @@ public class TestHoodieReadClient extends HoodieClientTestBase { @Test public void testReadROViewFailsWithoutSqlContext() { - HoodieReadClient readClient = new HoodieReadClient(jsc, getConfig()); + HoodieReadClient readClient = new HoodieReadClient(context, getConfig()); JavaRDD recordsRDD = jsc.parallelize(new ArrayList<>(), 1); assertThrows(IllegalStateException.class, () -> { readClient.readROView(recordsRDD, 1); @@ -100,8 +100,8 @@ public class TestHoodieReadClient extends HoodieClientTestBase { * @throws Exception in case of error */ private void testReadFilterExist(HoodieWriteConfig config, - Function3, HoodieWriteClient, JavaRDD, String> writeFn) throws Exception { - try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) { + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn) throws Exception { + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) { HoodieReadClient readClient = getHoodieReadClient(config.getBasePath()); String newCommitTime = writeClient.startCommit(); List records = dataGen.generateInserts(newCommitTime, 100); @@ -126,7 +126,7 @@ public class TestHoodieReadClient extends HoodieClientTestBase { // check path exists for written keys JavaPairRDD> keyToPathPair = - anotherReadClient.checkExists(recordsRDD.map(r -> r.getKey())); + anotherReadClient.checkExists(recordsRDD.map(HoodieRecord::getKey)); JavaRDD keysWithPaths = keyToPathPair.filter(keyPath -> keyPath._2.isPresent()) .map(keyPath -> keyPath._1); assertEquals(75, keysWithPaths.count()); @@ -153,7 +153,7 @@ public class TestHoodieReadClient extends HoodieClientTestBase { */ @Test public void testTagLocationAfterInsert() throws Exception { - testTagLocation(getConfig(), HoodieWriteClient::insert, HoodieWriteClient::upsert, false); + testTagLocation(getConfig(), SparkRDDWriteClient::insert, SparkRDDWriteClient::upsert, false); } /** @@ -161,7 +161,7 @@ public class TestHoodieReadClient extends HoodieClientTestBase { */ @Test public void testTagLocationAfterInsertPrepped() throws Exception { - testTagLocation(getConfig(), HoodieWriteClient::insertPreppedRecords, HoodieWriteClient::upsertPreppedRecords, + testTagLocation(getConfig(), SparkRDDWriteClient::insertPreppedRecords, SparkRDDWriteClient::upsertPreppedRecords, true); } @@ -170,8 +170,8 @@ public class TestHoodieReadClient extends HoodieClientTestBase { */ @Test public void testTagLocationAfterBulkInsert() throws Exception { - testTagLocation(getConfigBuilder().withBulkInsertParallelism(1).build(), HoodieWriteClient::bulkInsert, - HoodieWriteClient::upsert, false); + testTagLocation(getConfigBuilder().withBulkInsertParallelism(1).build(), SparkRDDWriteClient::bulkInsert, + SparkRDDWriteClient::upsert, false); } /** @@ -182,7 +182,7 @@ public class TestHoodieReadClient extends HoodieClientTestBase { testTagLocation( getConfigBuilder().withBulkInsertParallelism(1).build(), (writeClient, recordRDD, instantTime) -> writeClient .bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty()), - HoodieWriteClient::upsertPreppedRecords, true); + SparkRDDWriteClient::upsertPreppedRecords, true); } /** @@ -195,10 +195,10 @@ public class TestHoodieReadClient extends HoodieClientTestBase { * @throws Exception in case of error */ private void testTagLocation(HoodieWriteConfig hoodieWriteConfig, - Function3, HoodieWriteClient, JavaRDD, String> insertFn, - Function3, HoodieWriteClient, JavaRDD, String> updateFn, boolean isPrepped) + Function3, SparkRDDWriteClient, JavaRDD, String> insertFn, + Function3, SparkRDDWriteClient, JavaRDD, String> updateFn, boolean isPrepped) throws Exception { - try (HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig);) { // Write 1 (only inserts) String newCommitTime = "001"; String initCommitTime = "000"; diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestMultiFS.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java similarity index 97% rename from hudi-client/src/test/java/org/apache/hudi/client/TestMultiFS.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java index b4369d85c..34daed76f 100644 --- a/hudi-client/src/test/java/org/apache/hudi/client/TestMultiFS.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java @@ -82,8 +82,8 @@ public class TestMultiFS extends HoodieClientTestHarness { HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath); HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath); - try (HoodieWriteClient hdfsWriteClient = getHoodieWriteClient(cfg); - HoodieWriteClient localWriteClient = getHoodieWriteClient(localConfig)) { + try (SparkRDDWriteClient hdfsWriteClient = getHoodieWriteClient(cfg); + SparkRDDWriteClient localWriteClient = getHoodieWriteClient(localConfig)) { // Write generated data to hdfs (only inserts) String readCommitTime = hdfsWriteClient.startCommit(); diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java similarity index 92% rename from hudi-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index 0e4874698..8ee0c1634 100644 --- a/hudi-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -132,12 +132,12 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_1); HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA); - HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); + SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); // Initial inserts with TRIP_EXAMPLE_SCHEMA int numRecords = 10; insertFirstBatch(hoodieWriteConfig, client, "001", initCommitTime, - numRecords, HoodieWriteClient::insert, false, false, numRecords); + numRecords, SparkRDDWriteClient::insert, false, false, numRecords); checkLatestDeltaCommit("001"); // Compact once so we can incrementally read later @@ -147,7 +147,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { // Updates with same schema is allowed final int numUpdateRecords = 5; updateBatch(hoodieWriteConfig, client, "003", "002", Option.empty(), - initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, false, 0, 0, 0); + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0); checkLatestDeltaCommit("003"); checkReadRecords("000", numRecords); @@ -155,7 +155,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { final int numDeleteRecords = 2; numRecords -= numDeleteRecords; deleteBatch(hoodieWriteConfig, client, "004", "003", initCommitTime, numDeleteRecords, - HoodieWriteClient::delete, false, false, 0, 0); + SparkRDDWriteClient::delete, false, false, 0, 0); checkLatestDeltaCommit("004"); checkReadRecords("000", numRecords); @@ -167,7 +167,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { // We cannot use insertBatch directly here because we want to insert records // with a devolved schema and insertBatch inserts records using the TRIP_EXMPLE_SCHEMA. writeBatch(client, "005", "004", Option.empty(), "003", numRecords, - (String s, Integer a) -> failedRecords, HoodieWriteClient::insert, false, 0, 0, 0); + (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, false, 0, 0, 0); fail("Insert with devolved scheme should fail"); } catch (HoodieInsertException ex) { // no new commit @@ -179,7 +179,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { // Update with devolved schema is also not allowed try { updateBatch(hoodieDevolvedWriteConfig, client, "005", "004", Option.empty(), - initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, false, 0, 0, 0); + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0); fail("Update with devolved scheme should fail"); } catch (HoodieUpsertException ex) { // no new commit @@ -196,7 +196,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { // with a evolved schemaand insertBatch inserts records using the TRIP_EXMPLE_SCHEMA. final List evolvedRecords = generateInsertsWithSchema("005", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED); writeBatch(client, "005", "004", Option.empty(), initCommitTime, numRecords, - (String s, Integer a) -> evolvedRecords, HoodieWriteClient::insert, false, 0, 0, 0); + (String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, false, 0, 0, 0); // new commit checkLatestDeltaCommit("005"); @@ -205,7 +205,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { // Updates with evolved schema is allowed final List updateRecords = generateUpdatesWithSchema("006", numUpdateRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED); writeBatch(client, "006", "005", Option.empty(), initCommitTime, - numUpdateRecords, (String s, Integer a) -> updateRecords, HoodieWriteClient::upsert, false, 0, 0, 0); + numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, false, 0, 0, 0); // new commit checkLatestDeltaCommit("006"); checkReadRecords("000", 2 * numRecords); @@ -215,7 +215,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { client = getHoodieWriteClient(hoodieWriteConfig, false); try { updateBatch(hoodieWriteConfig, client, "007", "006", Option.empty(), - initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, false, 0, 0, 0); + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0); fail("Update with original scheme should fail"); } catch (HoodieUpsertException ex) { // no new commit @@ -233,7 +233,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { failedRecords.clear(); failedRecords.addAll(dataGen.generateInserts("007", numRecords)); writeBatch(client, "007", "006", Option.empty(), initCommitTime, numRecords, - (String s, Integer a) -> failedRecords, HoodieWriteClient::insert, true, numRecords, numRecords, 1); + (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1); fail("Insert with original scheme should fail"); } catch (HoodieInsertException ex) { // no new commit @@ -257,13 +257,13 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { // Updates with original schema are now allowed client = getHoodieWriteClient(hoodieWriteConfig, false); updateBatch(hoodieWriteConfig, client, "008", "004", Option.empty(), - initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, false, 0, 0, 0); + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0); // new commit checkLatestDeltaCommit("008"); checkReadRecords("000", 2 * numRecords); // Insert with original schema is allowed now - insertBatch(hoodieWriteConfig, client, "009", "008", numRecords, HoodieWriteClient::insert, + insertBatch(hoodieWriteConfig, client, "009", "008", numRecords, SparkRDDWriteClient::insert, false, false, 0, 0, 0); checkLatestDeltaCommit("009"); checkReadRecords("000", 3 * numRecords); @@ -277,18 +277,18 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_1); HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA); - HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); + SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); // Initial inserts with TRIP_EXAMPLE_SCHEMA int numRecords = 10; insertFirstBatch(hoodieWriteConfig, client, "001", initCommitTime, - numRecords, HoodieWriteClient::insert, false, true, numRecords); + numRecords, SparkRDDWriteClient::insert, false, true, numRecords); checkReadRecords("000", numRecords); // Updates with same schema is allowed final int numUpdateRecords = 5; updateBatch(hoodieWriteConfig, client, "002", "001", Option.empty(), - initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, true, + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, numUpdateRecords, numRecords, 2); checkReadRecords("000", numRecords); @@ -296,7 +296,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { final int numDeleteRecords = 2; numRecords -= numDeleteRecords; deleteBatch(hoodieWriteConfig, client, "003", "002", initCommitTime, numDeleteRecords, - HoodieWriteClient::delete, false, true, 0, numRecords); + SparkRDDWriteClient::delete, false, true, 0, numRecords); checkReadRecords("000", numRecords); // Insert with devolved schema is not allowed @@ -307,7 +307,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { // We cannot use insertBatch directly here because we want to insert records // with a devolved schema. writeBatch(client, "004", "003", Option.empty(), "003", numRecords, - (String s, Integer a) -> failedRecords, HoodieWriteClient::insert, true, numRecords, numRecords, 1); + (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1); fail("Insert with devolved scheme should fail"); } catch (HoodieInsertException ex) { // no new commit @@ -319,7 +319,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { // Update with devolved schema is not allowed try { updateBatch(hoodieDevolvedWriteConfig, client, "004", "003", Option.empty(), - initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, true, + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, numUpdateRecords, 2 * numRecords, 5); fail("Update with devolved scheme should fail"); } catch (HoodieUpsertException ex) { @@ -336,7 +336,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { // We cannot use insertBatch directly here because we want to insert records // with a evolved schema. writeBatch(client, "004", "003", Option.empty(), initCommitTime, numRecords, - (String s, Integer a) -> evolvedRecords, HoodieWriteClient::insert, true, numRecords, 2 * numRecords, 4); + (String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, true, numRecords, 2 * numRecords, 4); // new commit HoodieTimeline curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants(); assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("004")); @@ -345,7 +345,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { // Updates with evolved schema is allowed final List updateRecords = generateUpdatesWithSchema("005", numUpdateRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED); writeBatch(client, "005", "004", Option.empty(), initCommitTime, - numUpdateRecords, (String s, Integer a) -> updateRecords, HoodieWriteClient::upsert, true, numUpdateRecords, 2 * numRecords, 5); + numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, true, numUpdateRecords, 2 * numRecords, 5); checkReadRecords("000", 2 * numRecords); // Now even the original schema cannot be used for updates as it is devolved @@ -353,7 +353,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { client = getHoodieWriteClient(hoodieWriteConfig, false); try { updateBatch(hoodieWriteConfig, client, "006", "005", Option.empty(), - initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, true, + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, numUpdateRecords, numRecords, 2); fail("Update with original scheme should fail"); } catch (HoodieUpsertException ex) { @@ -372,7 +372,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { failedRecords.clear(); failedRecords.addAll(dataGen.generateInserts("006", numRecords)); writeBatch(client, "006", "005", Option.empty(), initCommitTime, numRecords, - (String s, Integer a) -> failedRecords, HoodieWriteClient::insert, true, numRecords, numRecords, 1); + (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1); fail("Insert with original scheme should fail"); } catch (HoodieInsertException ex) { // no new commit @@ -397,13 +397,13 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { checkReadRecords("000", numRecords); // Insert with original schema is allowed now - insertBatch(hoodieWriteConfig, client, "007", "003", numRecords, HoodieWriteClient::insert, + insertBatch(hoodieWriteConfig, client, "007", "003", numRecords, SparkRDDWriteClient::insert, false, true, numRecords, 2 * numRecords, 1); checkReadRecords("000", 2 * numRecords); // Update with original schema is allowed now updateBatch(hoodieWriteConfig, client, "008", "007", Option.empty(), - initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, true, + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, numUpdateRecords, 2 * numRecords, 5); checkReadRecords("000", 2 * numRecords); } diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java similarity index 97% rename from hudi-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java index 8b42671d9..805830414 100644 --- a/hudi-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java @@ -29,7 +29,7 @@ import org.apache.hudi.common.util.ParquetUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.HoodieCreateHandle; import org.apache.hudi.io.HoodieMergeHandle; -import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.avro.Schema; @@ -72,7 +72,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness { public void testSchemaEvolutionOnUpdate() throws Exception { // Create a bunch of records with a old version of schema final HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt"); - final HoodieTable table = HoodieTable.create(config, hadoopConf); + final HoodieSparkTable table = HoodieSparkTable.create(config, context); final List statuses = jsc.parallelize(Arrays.asList(1)).map(x -> { String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," @@ -109,7 +109,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness { final WriteStatus insertResult = statuses.get(0); String fileId = insertResult.getFileId(); - final HoodieTable table2 = HoodieTable.create(config2, hadoopConf); + final HoodieSparkTable table2 = HoodieSparkTable.create(config, context); assertEquals(1, jsc.parallelize(Arrays.asList(1)).map(x -> { // New content with values for the newly added field String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java diff --git a/hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestBootstrapRegexModeSelector.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/bootstrap/TestBootstrapRegexModeSelector.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestBootstrapRegexModeSelector.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/bootstrap/TestBootstrapRegexModeSelector.java diff --git a/hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestUniformBootstrapModeSelector.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/bootstrap/TestUniformBootstrapModeSelector.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestUniformBootstrapModeSelector.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/bootstrap/TestUniformBootstrapModeSelector.java diff --git a/hudi-client/src/test/java/org/apache/hudi/client/model/TestHoodieInternalRow.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/model/TestHoodieInternalRow.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/client/model/TestHoodieInternalRow.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/model/TestHoodieInternalRow.java diff --git a/hudi-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java similarity index 93% rename from hudi-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java index 5520ba875..c30635bb1 100644 --- a/hudi-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java @@ -30,7 +30,6 @@ import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.execution.LazyInsertIterable.HoodieInsertValueGenResult; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.avro.generic.IndexedRecord; @@ -53,7 +52,7 @@ import java.util.stream.IntStream; import scala.Tuple2; -import static org.apache.hudi.execution.LazyInsertIterable.getTransformFunction; +import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -83,7 +82,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { public void testRecordReading() throws Exception { final int numRecords = 128; final List hoodieRecords = dataGen.generateInserts(instantTime, numRecords); - final BoundedInMemoryQueue> queue = + final BoundedInMemoryQueue> queue = new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); // Produce Future resFuture = executorService.submit(() -> { @@ -97,7 +96,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { final HoodieRecord originalRecord = originalRecordIterator.next(); final Option originalInsertValue = originalRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA); - final HoodieInsertValueGenResult payload = queue.iterator().next(); + final HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = queue.iterator().next(); // Ensure that record ordering is guaranteed. assertEquals(originalRecord, payload.record); // cached insert value matches the expected insert value. @@ -123,7 +122,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { final int numProducers = 40; final List> recs = new ArrayList<>(); - final BoundedInMemoryQueue> queue = + final BoundedInMemoryQueue> queue = new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); // Record Key to @@ -189,7 +188,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { // Read recs and ensure we have covered all producer recs. while (queue.iterator().hasNext()) { - final HoodieInsertValueGenResult payload = queue.iterator().next(); + final HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = queue.iterator().next(); final HoodieRecord rec = payload.record; Tuple2 producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); Integer lastSeenPos = lastSeenMap.get(producerPos._1()); @@ -217,12 +216,12 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { final List hoodieRecords = dataGen.generateInserts(instantTime, numRecords); // maximum number of records to keep in memory. final int recordLimit = 5; - final SizeEstimator> sizeEstimator = new DefaultSizeEstimator<>(); - HoodieInsertValueGenResult payload = + final SizeEstimator> sizeEstimator = new DefaultSizeEstimator<>(); + HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0)); final long objSize = sizeEstimator.sizeEstimate(payload); final long memoryLimitInBytes = recordLimit * objSize; - final BoundedInMemoryQueue> queue = + final BoundedInMemoryQueue> queue = new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); // Produce @@ -267,7 +266,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { final List hoodieRecords = dataGen.generateInserts(instantTime, numRecords); final SizeEstimator>> sizeEstimator = new DefaultSizeEstimator<>(); // queue memory limit - HoodieInsertValueGenResult payload = + HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0)); final long objSize = sizeEstimator.sizeEstimate(new Tuple2<>(payload.record, payload.insertValue)); final long memoryLimitInBytes = 4 * objSize; diff --git a/hudi-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java similarity index 87% rename from hudi-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java index 487dd2279..fd41a1680 100644 --- a/hudi-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.execution.LazyInsertIterable.HoodieInsertValueGenResult; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.avro.generic.IndexedRecord; @@ -36,7 +35,7 @@ import java.util.List; import scala.Tuple2; -import static org.apache.hudi.execution.LazyInsertIterable.getTransformFunction; +import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.mockito.Mockito.mock; @@ -63,13 +62,13 @@ public class TestSparkBoundedInMemoryExecutor extends HoodieClientTestHarness { HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); - BoundedInMemoryQueueConsumer, Integer> consumer = - new BoundedInMemoryQueueConsumer, Integer>() { + BoundedInMemoryQueueConsumer, Integer> consumer = + new BoundedInMemoryQueueConsumer, Integer>() { private int count = 0; @Override - protected void consumeOneRecord(HoodieInsertValueGenResult record) { + protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { count++; } diff --git a/hudi-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java similarity index 97% rename from hudi-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java index b8ebd5ad9..834229b68 100644 --- a/hudi-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java @@ -80,7 +80,7 @@ public class TestBulkInsertInternalPartitioner extends HoodieClientTestBase { boolean isGloballySorted, boolean isLocallySorted, Map expectedPartitionNumRecords) { int numPartitions = 2; - JavaRDD actualRecords = partitioner.repartitionRecords(records, numPartitions); + JavaRDD actualRecords = (JavaRDD) partitioner.repartitionRecords(records, numPartitions); assertEquals(numPartitions, actualRecords.getNumPartitions()); List collectedActualRecords = actualRecords.collect(); if (isGloballySorted) { diff --git a/hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java similarity index 87% rename from hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java index aefad87c6..9de36c6e0 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java @@ -35,7 +35,9 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex.IndexType; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.testutils.Assertions; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieWriteableTestTable; import org.apache.hudi.testutils.MetadataMergeWriteStatus; @@ -60,7 +62,6 @@ import java.util.UUID; import scala.Tuple2; import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; -import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -99,27 +100,27 @@ public class TestHoodieIndex extends HoodieClientTestHarness { JavaRDD writeRecords = jsc.parallelize(records, 1); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration()); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Test tagLocation without any entries in index - JavaRDD javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); + JavaRDD javaRDD = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); // Insert totalRecords records writeClient.startCommitWithTime(newCommitTime); JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); - assertNoWriteErrors(writeStatues.collect()); + Assertions.assertNoWriteErrors(writeStatues.collect()); // Now tagLocation for these records, index should not tag them since it was a failed // commit - javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); + javaRDD = (JavaRDD) index.tagLocation(writeRecords, context, 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, index should tag them correctly metaClient = HoodieTableMetaClient.reload(metaClient); - hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration()); - javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); + javaRDD = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); Map recordKeyToPartitionPathMap = new HashMap(); List hoodieRecords = writeRecords.collect(); hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath())); @@ -131,7 +132,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { javaRDD.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry.getRecordKey()), entry.getPartitionPath(), "PartitionPath mismatch")); JavaRDD hoodieKeyJavaRDD = writeRecords.map(entry -> entry.getKey()); - JavaPairRDD>> recordLocations = index.fetchRecordLocation(hoodieKeyJavaRDD, jsc, hoodieTable); + JavaPairRDD>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable); List hoodieKeys = hoodieKeyJavaRDD.collect(); assertEquals(totalRecords, recordLocations.collect().size()); assertEquals(totalRecords, recordLocations.map(record -> record._1).distinct().count()); @@ -148,11 +149,11 @@ public class TestHoodieIndex extends HoodieClientTestHarness { List records = dataGen.generateInserts(newCommitTime, totalRecords); JavaRDD writeRecords = jsc.parallelize(records, 1); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration()); + HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); writeClient.startCommitWithTime(newCommitTime); JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); - JavaRDD javaRDD1 = index.tagLocation(writeRecords, jsc, hoodieTable); + JavaRDD javaRDD1 = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); // Duplicate upsert and ensure correctness is maintained // We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not @@ -161,14 +162,14 @@ public class TestHoodieIndex extends HoodieClientTestHarness { metaClient.getFs().delete(new Path(metaClient.getMetaPath(), "001.inflight")); writeClient.upsert(writeRecords, newCommitTime); - assertNoWriteErrors(writeStatues.collect()); + Assertions.assertNoWriteErrors(writeStatues.collect()); // 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 = HoodieTableMetaClient.reload(metaClient); - hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration()); - JavaRDD javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); + JavaRDD javaRDD = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); Map recordKeyToPartitionPathMap = new HashMap(); List hoodieRecords = writeRecords.collect(); @@ -181,7 +182,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { javaRDD.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry.getRecordKey()), entry.getPartitionPath(), "PartitionPath mismatch")); JavaRDD hoodieKeyJavaRDD = writeRecords.map(entry -> entry.getKey()); - JavaPairRDD>> recordLocations = index.fetchRecordLocation(hoodieKeyJavaRDD, jsc, hoodieTable); + JavaPairRDD>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable); List hoodieKeys = hoodieKeyJavaRDD.collect(); assertEquals(totalRecords, recordLocations.collect().size()); assertEquals(totalRecords, recordLocations.map(record -> record._1).distinct().count()); @@ -201,14 +202,14 @@ public class TestHoodieIndex extends HoodieClientTestHarness { // Insert 200 records JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); - assertNoWriteErrors(writeStatues.collect()); + Assertions.assertNoWriteErrors(writeStatues.collect()); // commit this upsert writeClient.commit(newCommitTime, writeStatues); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration()); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Now tagLocation for these records, hbaseIndex should tag them - JavaRDD javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); + JavaRDD javaRDD = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == totalRecords); // check tagged records are tagged with correct fileIds @@ -221,7 +222,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath())); JavaRDD hoodieKeyJavaRDD = writeRecords.map(entry -> entry.getKey()); - JavaPairRDD>> recordLocations = index.fetchRecordLocation(hoodieKeyJavaRDD, jsc, hoodieTable); + JavaPairRDD>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable); List hoodieKeys = hoodieKeyJavaRDD.collect(); assertEquals(totalRecords, recordLocations.collect().size()); assertEquals(totalRecords, recordLocations.map(record -> record._1).distinct().count()); @@ -233,10 +234,10 @@ public class TestHoodieIndex extends HoodieClientTestHarness { // Rollback the last commit writeClient.rollback(newCommitTime); - hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration()); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled // back commit - javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); + javaRDD = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 0); assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0); } @@ -269,9 +270,9 @@ public class TestHoodieIndex extends HoodieClientTestHarness { new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4)); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration()); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - JavaRDD taggedRecordRDD = index.tagLocation(recordRDD, jsc, hoodieTable); + JavaRDD taggedRecordRDD = (JavaRDD) index.tagLocation(recordRDD, context, hoodieTable); // Should not find any files for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -286,9 +287,9 @@ public class TestHoodieIndex extends HoodieClientTestHarness { // We do the tag again metaClient = HoodieTableMetaClient.reload(metaClient); - hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration()); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); - taggedRecordRDD = index.tagLocation(recordRDD, jsc, hoodieTable); + taggedRecordRDD = (JavaRDD) index.tagLocation(recordRDD, context, hoodieTable); // Check results for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -305,8 +306,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { } } - JavaPairRDD>> recordLocations = index.fetchRecordLocation(recordRDD.map(entry -> entry.getKey()), jsc, hoodieTable); - + JavaPairRDD>> recordLocations = getRecordLocations(recordRDD.map(HoodieRecord::getKey), hoodieTable); for (Tuple2>> entry : recordLocations.collect()) { if (entry._1.getRecordKey().equals(rowKey1)) { assertTrue(entry._2.isPresent(), "Row1 should have been present "); @@ -336,7 +336,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { .build()).build(); writeClient = getHoodieWriteClient(config); index = writeClient.getIndex(); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration()); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); final String p1 = "2016/01/31"; final String p2 = "2016/02/28"; @@ -379,7 +379,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { // test against incoming record with a different partition JavaRDD recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord)); - JavaRDD taggedRecordRDD = index.tagLocation(recordRDD, jsc, hoodieTable); + JavaRDD taggedRecordRDD = (JavaRDD) index.tagLocation(recordRDD, context, hoodieTable); assertEquals(2, taggedRecordRDD.count()); for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -400,7 +400,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { // test against incoming record with the same partition JavaRDD recordRDDSamePartition = jsc .parallelize(Collections.singletonList(incomingRecordSamePartition)); - JavaRDD taggedRecordRDDSamePartition = index.tagLocation(recordRDDSamePartition, jsc, hoodieTable); + JavaRDD taggedRecordRDDSamePartition = (JavaRDD) index.tagLocation(recordRDDSamePartition, context, hoodieTable); assertEquals(1, taggedRecordRDDSamePartition.count()); HoodieRecord record = taggedRecordRDDSamePartition.first(); @@ -422,4 +422,12 @@ public class TestHoodieIndex extends HoodieClientTestHarness { .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build()); } + private JavaPairRDD>> getRecordLocations(JavaRDD keyRDD, HoodieTable hoodieTable) { + JavaRDD recordRDD = (JavaRDD) index.tagLocation( + keyRDD.map(k -> new HoodieRecord(k, new EmptyHoodieRecordPayload())), context, hoodieTable); + return recordRDD.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown() + ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId())) + : Option.empty()) + ); + } } diff --git a/hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java similarity index 78% rename from hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java index a68a8720c..9175ebde5 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java @@ -20,26 +20,23 @@ package org.apache.hudi.index; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieHBaseIndexConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.index.HoodieIndex.IndexType; -import org.apache.hudi.index.bloom.HoodieBloomIndex; -import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex; -import org.apache.hudi.index.hbase.HBaseIndex; -import org.apache.hudi.index.simple.HoodieSimpleIndex; +import org.apache.hudi.index.bloom.SparkHoodieBloomIndex; +import org.apache.hudi.index.bloom.SparkHoodieGlobalBloomIndex; +import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex; +import org.apache.hudi.index.simple.SparkHoodieSimpleIndex; 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.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -51,6 +48,7 @@ import java.nio.file.Path; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +@SuppressWarnings("checkstyle:LineLength") public class TestHoodieIndexConfigs { private String basePath; @@ -70,29 +68,29 @@ public class TestHoodieIndexConfigs { case INMEMORY: config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); - assertTrue(HoodieIndex.createIndex(config) instanceof InMemoryHashIndex); + assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkInMemoryHashIndex); break; case BLOOM: config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); - assertTrue(HoodieIndex.createIndex(config) instanceof HoodieBloomIndex); + assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieBloomIndex); break; case GLOBAL_BLOOM: config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(IndexType.GLOBAL_BLOOM).build()).build(); - assertTrue(HoodieIndex.createIndex(config) instanceof HoodieGlobalBloomIndex); + assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieGlobalBloomIndex); break; case SIMPLE: config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(IndexType.SIMPLE).build()).build(); - assertTrue(HoodieIndex.createIndex(config) instanceof HoodieSimpleIndex); + assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieSimpleIndex); break; case HBASE: config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE) .withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder().build()).build()) .build(); - assertTrue(HoodieIndex.createIndex(config) instanceof HBaseIndex); + assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieHBaseIndex); break; default: // no -op. just for checkstyle errors @@ -105,7 +103,7 @@ public class TestHoodieIndexConfigs { HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder(); HoodieWriteConfig config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexClass(DummyHoodieIndex.class.getName()).build()).build(); - assertTrue(HoodieIndex.createIndex(config) instanceof DummyHoodieIndex); + assertTrue(SparkHoodieIndex.createIndex(config) instanceof DummyHoodieIndex); } @Test @@ -115,36 +113,35 @@ public class TestHoodieIndexConfigs { final HoodieWriteConfig config1 = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexClass(IndexWithConstructor.class.getName()).build()).build(); final Throwable thrown1 = assertThrows(HoodieException.class, () -> { - HoodieIndex.createIndex(config1); + SparkHoodieIndex.createIndex(config1); }, "exception is expected"); assertTrue(thrown1.getMessage().contains("is not a subclass of HoodieIndex")); final HoodieWriteConfig config2 = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexClass(IndexWithoutConstructor.class.getName()).build()).build(); final Throwable thrown2 = assertThrows(HoodieException.class, () -> { - HoodieIndex.createIndex(config2); + SparkHoodieIndex.createIndex(config2); }, "exception is expected"); assertTrue(thrown2.getMessage().contains("Unable to instantiate class")); } - public static class DummyHoodieIndex> extends HoodieIndex { + public static class DummyHoodieIndex> extends SparkHoodieIndex { public DummyHoodieIndex(HoodieWriteConfig config) { super(config); } @Override - public JavaPairRDD>> fetchRecordLocation(JavaRDD hoodieKeys, JavaSparkContext jsc, HoodieTable hoodieTable) { + public JavaRDD updateLocation(JavaRDD writeStatusRDD, + HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) throws HoodieIndexException { return null; } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, HoodieTable hoodieTable) throws HoodieIndexException { - return null; - } - - @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, HoodieTable hoodieTable) throws HoodieIndexException { + public JavaRDD> tagLocation(JavaRDD> records, + HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> hoodieTable) throws HoodieIndexException { return null; } diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestBucketizedBloomCheckPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestBucketizedBloomCheckPartitioner.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/index/bloom/TestBucketizedBloomCheckPartitioner.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestBucketizedBloomCheckPartitioner.java diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java similarity index 90% rename from hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java index 458432486..2d091a0ea 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java @@ -30,6 +30,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.HoodieKeyLookupHandle; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieWriteableTestTable; @@ -102,8 +103,8 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { @MethodSource("configParams") public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception { HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); - HoodieBloomIndex index = new HoodieBloomIndex(config); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); // Create some partitions, and put some files @@ -130,7 +131,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); List partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12"); - List> filesList = index.loadInvolvedFiles(partitions, jsc, hoodieTable); + List> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); // Still 0, as no valid commit assertEquals(0, filesList.size()); @@ -139,7 +140,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { .withInserts("2015/03/12", "3", record1) .withInserts("2015/03/12", "4", record2, record3, record4); - filesList = index.loadInvolvedFiles(partitions, jsc, hoodieTable); + filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); assertEquals(4, filesList.size()); if (rangePruning) { @@ -166,7 +167,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { @MethodSource("configParams") public void testRangePruning(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) { HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); - HoodieBloomIndex index = new HoodieBloomIndex(config); + SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config); final Map> partitionToFileIndexInfo = new HashMap<>(); partitionToFileIndexInfo.put("2017/10/22", @@ -236,7 +237,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { Arrays.asList(record1.getRecordKey(), record2.getRecordKey(), record3.getRecordKey(), record4.getRecordKey()); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf); + HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient); HoodieKeyLookupHandle keyHandle = new HoodieKeyLookupHandle<>(config, table, Pair.of(partition, fileId)); List results = keyHandle.checkCandidatesAgainstFile(hadoopConf, uuids, new Path(Paths.get(basePath, partition, filename).toString())); @@ -258,13 +259,13 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { // Also create the metadata and config HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf); + HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient); // Let's tag - HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config); + SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config); assertDoesNotThrow(() -> { - bloomIndex.tagLocation(recordRDD, jsc, table); + bloomIndex.tagLocation(recordRDD, context, table); }, "EmptyRDD should not result in IllegalArgumentException: Positive number of slices required"); } @@ -296,12 +297,12 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { // Also create the metadata and config HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); // Let's tag - HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config); - JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, hoodieTable); + SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config); + JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, hoodieTable); // Should not find any files for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -314,7 +315,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { String fileId3 = testTable.addCommit("003").withInserts("2015/01/31", record4); // We do the tag again - taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, HoodieTable.create(metaClient, config, hadoopConf)); + taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, HoodieSparkTable.create(config, context, metaClient)); // Check results for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -361,16 +362,20 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { // Also create the metadata and config HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); // Let's tag - HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config); - JavaPairRDD>> taggedRecordRDD = - bloomIndex.fetchRecordLocation(keysRDD, jsc, hoodieTable); + SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config); + JavaRDD taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable); + JavaPairRDD>> recordLocationsRDD = taggedRecords + .mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown() + ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId())) + : Option.empty()) + ); // Should not find any files - for (Tuple2>> record : taggedRecordRDD.collect()) { + for (Tuple2>> record : recordLocationsRDD.collect()) { assertTrue(!record._2.isPresent()); } @@ -381,11 +386,16 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { // We do the tag again metaClient = HoodieTableMetaClient.reload(metaClient); - hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); - taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, hoodieTable); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); + taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable); + recordLocationsRDD = taggedRecords + .mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown() + ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId())) + : Option.empty()) + ); // Check results - for (Tuple2>> record : taggedRecordRDD.collect()) { + for (Tuple2>> record : recordLocationsRDD.collect()) { if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) { assertTrue(record._2.isPresent()); assertEquals(fileId1, record._2.get().getRight()); @@ -431,10 +441,10 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2)); HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable table = HoodieSparkTable.create(config, context, metaClient); - HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config); - JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table); + SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config); + JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, table); // Check results for (HoodieRecord record : taggedRecordRDD.collect()) { diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java similarity index 95% rename from hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java index 197a3bb1f..2f68a032e 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieWriteableTestTable; @@ -73,8 +74,8 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { @Test public void testLoadInvolvedFiles() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); // Create some partitions, and put some files, along with the meta file @@ -103,7 +104,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { // intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up List partitions = Arrays.asList("2016/01/21", "2016/04/01"); // partitions will NOT be respected by this loadInvolvedFiles(...) call - List> filesList = index.loadInvolvedFiles(partitions, jsc, hoodieTable); + List> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); // Still 0, as no valid commit assertEquals(0, filesList.size()); @@ -112,7 +113,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { .withInserts("2015/03/12", "3", record1) .withInserts("2015/03/12", "4", record2, record3, record4); - filesList = index.loadInvolvedFiles(partitions, jsc, hoodieTable); + filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); assertEquals(4, filesList.size()); Map filesMap = toFileMap(filesList); @@ -137,7 +138,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { public void testExplodeRecordRDDWithFileComparisons() { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config); + SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); final Map> partitionToFileIndexInfo = new HashMap<>(); partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"), @@ -177,8 +178,8 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { @Test public void testTagLocation() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); // Create some partitions, and put some files, along with the meta file @@ -222,7 +223,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { String fileId4 = testTable.addCommit("4000").withInserts("2015/03/12", record4); // partitions will NOT be respected by this loadInvolvedFiles(...) call - JavaRDD taggedRecordRDD = index.tagLocation(recordRDD, jsc, hoodieTable); + JavaRDD taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable); for (HoodieRecord record : taggedRecordRDD.collect()) { switch (record.getRecordKey()) { @@ -258,8 +259,8 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { .withPath(basePath) .withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(true).build()) .build(); - HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration()); + SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); final String p1 = "2016/01/31"; final String p2 = "2016/02/28"; @@ -302,7 +303,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { // test against incoming record with a different partition JavaRDD recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord)); - JavaRDD taggedRecordRDD = index.tagLocation(recordRDD, jsc, hoodieTable); + JavaRDD taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable); assertEquals(2, taggedRecordRDD.count()); for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -323,7 +324,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { // test against incoming record with the same partition JavaRDD recordRDDSamePartition = jsc .parallelize(Collections.singletonList(incomingRecordSamePartition)); - JavaRDD taggedRecordRDDSamePartition = index.tagLocation(recordRDDSamePartition, jsc, hoodieTable); + JavaRDD taggedRecordRDDSamePartition = index.tagLocation(recordRDDSamePartition, context, hoodieTable); assertEquals(1, taggedRecordRDDSamePartition.count()); HoodieRecord record = taggedRecordRDDSamePartition.first(); diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java diff --git a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java similarity index 88% rename from hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java index db8ca529a..b85851663 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java @@ -18,7 +18,7 @@ package org.apache.hudi.index.hbase; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; @@ -33,6 +33,7 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.FunctionalTestHarness; @@ -136,13 +137,13 @@ public class TestHBaseIndex extends FunctionalTestHarness { // Load to memory HoodieWriteConfig config = getConfig(); - HBaseIndex index = new HBaseIndex(config); - try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) { + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) { metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Test tagLocation without any entries in index - JavaRDD records1 = index.tagLocation(writeRecords, jsc(), hoodieTable); + JavaRDD records1 = index.tagLocation(writeRecords, context(), hoodieTable); assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count()); // Insert 200 records @@ -151,15 +152,15 @@ public class TestHBaseIndex extends FunctionalTestHarness { assertNoWriteErrors(writeStatues.collect()); // Now tagLocation for these records, hbaseIndex should not tag them since commit never occurred - JavaRDD records2 = index.tagLocation(writeRecords, jsc(), hoodieTable); + JavaRDD records2 = index.tagLocation(writeRecords, context(), hoodieTable); assertEquals(0, records2.filter(record -> record.isCurrentLocationKnown()).count()); // 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 = HoodieTableMetaClient.reload(metaClient); - hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); - List records3 = index.tagLocation(writeRecords, jsc(), hoodieTable).collect(); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); + List records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect(); assertEquals(numRecords, records3.stream().filter(record -> record.isCurrentLocationKnown()).count()); assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count()); assertEquals(numRecords, records3.stream().filter(record -> (record.getCurrentLocation() != null @@ -176,14 +177,14 @@ public class TestHBaseIndex extends FunctionalTestHarness { // Load to memory HoodieWriteConfig config = getConfig(); - HBaseIndex index = new HBaseIndex(config); - HoodieWriteClient writeClient = getHoodieWriteClient(config); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); writeClient.startCommitWithTime(newCommitTime); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); - index.tagLocation(writeRecords, jsc(), hoodieTable); + index.tagLocation(writeRecords, context(), hoodieTable); // Duplicate upsert and ensure correctness is maintained // We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not @@ -198,8 +199,8 @@ public class TestHBaseIndex extends FunctionalTestHarness { writeClient.commit(newCommitTime, writeStatues); // Now tagLocation for these records, hbaseIndex should tag them correctly metaClient = HoodieTableMetaClient.reload(metaClient); - hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); - List taggedRecords = index.tagLocation(writeRecords, jsc(), hoodieTable).collect(); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); + List taggedRecords = index.tagLocation(writeRecords, context(), hoodieTable).collect(); assertEquals(numRecords, taggedRecords.stream().filter(HoodieRecord::isCurrentLocationKnown).count()); assertEquals(numRecords, taggedRecords.stream().map(record -> record.getKey().getRecordKey()).distinct().count()); assertEquals(numRecords, taggedRecords.stream().filter(record -> (record.getCurrentLocation() != null @@ -210,8 +211,8 @@ public class TestHBaseIndex extends FunctionalTestHarness { public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { // Load to memory HoodieWriteConfig config = getConfig(); - HBaseIndex index = new HBaseIndex(config); - HoodieWriteClient writeClient = getHoodieWriteClient(config); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); final String newCommitTime = writeClient.startCommit(); final int numRecords = 10; @@ -225,9 +226,9 @@ public class TestHBaseIndex extends FunctionalTestHarness { // commit this upsert writeClient.commit(newCommitTime, writeStatues); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Now tagLocation for these records, hbaseIndex should tag them - List records2 = index.tagLocation(writeRecords, jsc(), hoodieTable).collect(); + List records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect(); assertEquals(numRecords, records2.stream().filter(HoodieRecord::isCurrentLocationKnown).count()); // check tagged records are tagged with correct fileIds @@ -240,10 +241,10 @@ public class TestHBaseIndex extends FunctionalTestHarness { // Rollback the last commit writeClient.rollback(newCommitTime); - hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled // back commit - List records3 = index.tagLocation(writeRecords, jsc(), hoodieTable).collect(); + List records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect(); assertEquals(0, records3.stream().filter(HoodieRecord::isCurrentLocationKnown).count()); assertEquals(0, records3.stream().filter(record -> record.getCurrentLocation() != null).count()); } @@ -251,7 +252,7 @@ public class TestHBaseIndex extends FunctionalTestHarness { @Test public void testTotalGetsBatching() throws Exception { HoodieWriteConfig config = getConfig(); - HBaseIndex index = new HBaseIndex(config); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); // Mock hbaseConnection and related entities Connection hbaseConnection = mock(Connection.class); @@ -262,21 +263,21 @@ public class TestHBaseIndex extends FunctionalTestHarness { // only for test, set the hbaseConnection to mocked object index.setHbaseConnection(hbaseConnection); - HoodieWriteClient writeClient = getHoodieWriteClient(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); // start a commit and generate test data String newCommitTime = writeClient.startCommit(); List records = dataGen.generateInserts(newCommitTime, 250); JavaRDD writeRecords = jsc().parallelize(records, 1); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Insert 250 records JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); assertNoWriteErrors(writeStatues.collect()); // Now tagLocation for these records, hbaseIndex should tag them - index.tagLocation(writeRecords, jsc(), hoodieTable); + index.tagLocation(writeRecords, context(), hoodieTable); // 3 batches should be executed given batchSize = 100 and parallelism = 1 verify(table, times(3)).get((List) any()); @@ -286,15 +287,15 @@ public class TestHBaseIndex extends FunctionalTestHarness { @Test public void testTotalPutsBatching() throws Exception { HoodieWriteConfig config = getConfig(); - HBaseIndex index = new HBaseIndex(config); - HoodieWriteClient writeClient = getHoodieWriteClient(config); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); // start a commit and generate test data String newCommitTime = writeClient.startCommit(); List records = dataGen.generateInserts(newCommitTime, 250); JavaRDD writeRecords = jsc().parallelize(records, 1); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Insert 200 records JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); @@ -314,7 +315,7 @@ public class TestHBaseIndex extends FunctionalTestHarness { // Get all the files generated int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count(); - index.updateLocation(writeStatues, jsc(), hoodieTable); + index.updateLocation(writeStatues, context(), hoodieTable); // 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated, // so each fileId ideally gets updates verify(table, atMost(numberOfDataFileIds)).put((List) any()); @@ -323,7 +324,7 @@ public class TestHBaseIndex extends FunctionalTestHarness { @Test public void testsHBasePutAccessParallelism() { HoodieWriteConfig config = getConfig(); - HBaseIndex index = new HBaseIndex(config); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); final JavaRDD writeStatusRDD = jsc().parallelize( Arrays.asList(getSampleWriteStatus(1, 2), getSampleWriteStatus(0, 3), getSampleWriteStatus(10, 0)), 10); final Tuple2 tuple = index.getHBasePutAccessParallelism(writeStatusRDD); @@ -337,7 +338,7 @@ public class TestHBaseIndex extends FunctionalTestHarness { @Test public void testsHBasePutAccessParallelismWithNoInserts() { HoodieWriteConfig config = getConfig(); - HBaseIndex index = new HBaseIndex(config); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); final JavaRDD writeStatusRDD = jsc().parallelize(Arrays.asList(getSampleWriteStatus(0, 2), getSampleWriteStatus(0, 1)), 10); final Tuple2 tuple = index.getHBasePutAccessParallelism(writeStatusRDD); @@ -357,13 +358,13 @@ public class TestHBaseIndex extends FunctionalTestHarness { // Load to memory HoodieWriteConfig config = getConfig(2); - HBaseIndex index = new HBaseIndex(config); - try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) { + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) { metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Test tagLocation without any entries in index - JavaRDD records1 = index.tagLocation(writeRecords, jsc(), hoodieTable); + JavaRDD records1 = index.tagLocation(writeRecords, context(), hoodieTable); assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count()); // Insert 200 records writeClient.startCommitWithTime(newCommitTime); @@ -372,15 +373,15 @@ public class TestHBaseIndex extends FunctionalTestHarness { // Now tagLocation for these records, hbaseIndex should not tag them since it was a failed // commit - JavaRDD records2 = index.tagLocation(writeRecords, jsc(), hoodieTable); + JavaRDD records2 = index.tagLocation(writeRecords, context(), hoodieTable); assertEquals(0, records2.filter(record -> record.isCurrentLocationKnown()).count()); // 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 = HoodieTableMetaClient.reload(metaClient); - hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); - List records3 = index.tagLocation(writeRecords, jsc(), hoodieTable).collect(); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); + List records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect(); assertEquals(numRecords, records3.stream().filter(record -> record.isCurrentLocationKnown()).count()); assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count()); assertEquals(numRecords, records3.stream().filter(record -> (record.getCurrentLocation() != null @@ -397,13 +398,13 @@ public class TestHBaseIndex extends FunctionalTestHarness { // Load to memory HoodieWriteConfig config = getConfig(); - HBaseIndex index = new HBaseIndex(config); - try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) { + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) { metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Test tagLocation without any entries in index - JavaRDD records1 = index.tagLocation(writeRecords, jsc(), hoodieTable); + JavaRDD records1 = index.tagLocation(writeRecords, context(), hoodieTable); assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count()); // Insert records @@ -414,8 +415,8 @@ public class TestHBaseIndex extends FunctionalTestHarness { // Now tagLocation for these records, hbaseIndex should tag them correctly metaClient = HoodieTableMetaClient.reload(metaClient); - hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); - List records2 = index.tagLocation(writeRecords, jsc(), hoodieTable).collect(); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); + List records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect(); assertEquals(numRecords, records2.stream().filter(record -> record.isCurrentLocationKnown()).count()); assertEquals(numRecords, records2.stream().map(record -> record.getKey().getRecordKey()).distinct().count()); assertEquals(numRecords, records2.stream().filter(record -> (record.getCurrentLocation() != null @@ -430,12 +431,12 @@ public class TestHBaseIndex extends FunctionalTestHarness { newWriteStatus.setStat(new HoodieWriteStat()); return newWriteStatus; }); - JavaRDD deleteStatus = index.updateLocation(deleteWriteStatues, jsc(), hoodieTable); + JavaRDD deleteStatus = index.updateLocation(deleteWriteStatues, context(), hoodieTable); assertEquals(deleteStatus.count(), deleteWriteStatues.count()); assertNoWriteErrors(deleteStatus.collect()); // Ensure no records can be tagged - List records3 = index.tagLocation(writeRecords, jsc(), hoodieTable).collect(); + List records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect(); assertEquals(0, records3.stream().filter(record -> record.isCurrentLocationKnown()).count()); assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count()); assertEquals(0, records3.stream().filter(record -> (record.getCurrentLocation() != null diff --git a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndexUsage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndexUsage.java similarity index 82% rename from hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndexUsage.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndexUsage.java index c1bf1571a..6411b5292 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndexUsage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndexUsage.java @@ -25,7 +25,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.junit.jupiter.MockitoExtension; -import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; @@ -35,11 +34,7 @@ public class TestHBaseIndexUsage { @Test public void testFeatureSupport() { HoodieWriteConfig config = mock(HoodieWriteConfig.class); - HBaseIndex index = new HBaseIndex(config); - + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); assertTrue(index.canIndexLogFiles()); - assertThrows(UnsupportedOperationException.class, () -> { - index.fetchRecordLocation(null, null, null); - }, "HBaseIndex should not support fetchRecordLocation"); } } diff --git a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBasePutBatchSizeCalculator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBasePutBatchSizeCalculator.java similarity index 94% rename from hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBasePutBatchSizeCalculator.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBasePutBatchSizeCalculator.java index 3109942ba..e698eaf7d 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBasePutBatchSizeCalculator.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBasePutBatchSizeCalculator.java @@ -19,8 +19,6 @@ package org.apache.hudi.index.hbase; -import org.apache.hudi.index.hbase.HBaseIndex.HBasePutBatchSizeCalculator; - import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -29,7 +27,7 @@ public class TestHBasePutBatchSizeCalculator { @Test public void testPutBatchSizeCalculation() { - HBasePutBatchSizeCalculator batchSizeCalculator = new HBasePutBatchSizeCalculator(); + SparkHoodieHBaseIndex.HBasePutBatchSizeCalculator batchSizeCalculator = new SparkHoodieHBaseIndex.HBasePutBatchSizeCalculator(); // All asserts cases below are derived out of the first // example below, with change in one parameter at a time. diff --git a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java similarity index 95% rename from hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java index 8972d0040..e9f8b87a7 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java @@ -36,7 +36,7 @@ public class TestHBaseQPSResourceAllocator { @Test public void testsDefaultQPSResourceAllocator() { HoodieWriteConfig config = getConfig(Option.empty()); - HBaseIndex index = new HBaseIndex(config); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config); assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(), DefaultHBaseQPSResourceAllocator.class.getName()); @@ -47,7 +47,7 @@ public class TestHBaseQPSResourceAllocator { @Test public void testsExplicitDefaultQPSResourceAllocator() { HoodieWriteConfig config = getConfig(Option.of(HoodieHBaseIndexConfig.DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS)); - HBaseIndex index = new HBaseIndex(config); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config); assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(), DefaultHBaseQPSResourceAllocator.class.getName()); @@ -58,7 +58,7 @@ public class TestHBaseQPSResourceAllocator { @Test public void testsInvalidQPSResourceAllocator() { HoodieWriteConfig config = getConfig(Option.of("InvalidResourceAllocatorClassName")); - HBaseIndex index = new HBaseIndex(config); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config); assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(), DefaultHBaseQPSResourceAllocator.class.getName()); diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java similarity index 96% rename from hudi-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java index a7cd50325..38cd19c67 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java @@ -18,6 +18,7 @@ package org.apache.hudi.io; +import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; @@ -31,12 +32,12 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieWriteableTestTable; import org.apache.hudi.testutils.MetadataMergeWriteStatus; -import org.apache.spark.api.java.JavaSparkContext; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -85,12 +86,12 @@ public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness { public void testFetchHandle() throws Exception { List records = dataGen.generateInserts(makeNewCommitTime(), 100); Map> partitionRecordsMap = recordsToPartitionRecordsMap(records); - HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration()); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, AVRO_SCHEMA_WITH_METADATA_FIELDS); Map, List>> expectedList = writeToParquetAndGetExpectedRecordLocations(partitionRecordsMap, testTable); - List> partitionPathFileIdPairs = loadAllFilesForPartitions(new ArrayList<>(partitionRecordsMap.keySet()), jsc, hoodieTable); + List> partitionPathFileIdPairs = loadAllFilesForPartitions(new ArrayList<>(partitionRecordsMap.keySet()), context, hoodieTable); for (Tuple2 entry : partitionPathFileIdPairs) { HoodieKeyLocationFetchHandle fetcherHandle = new HoodieKeyLocationFetchHandle(config, hoodieTable, Pair.of(entry._1, entry._2)); @@ -141,10 +142,10 @@ public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness { return expectedList; } - private static List> loadAllFilesForPartitions(List partitions, JavaSparkContext jsc, + private static List> loadAllFilesForPartitions(List partitions, HoodieEngineContext context, HoodieTable hoodieTable) { // Obtain the latest data files from all the partitions. - List> partitionPathFileIDList = HoodieIndexUtils.getLatestBaseFilesForAllPartitions(partitions, jsc, hoodieTable); + List> partitionPathFileIDList = HoodieIndexUtils.getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable); return partitionPathFileIDList.stream() .map(pf -> new Tuple2<>(pf.getKey(), pf.getValue())).collect(toList()); } diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java similarity index 98% rename from hudi-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java index 17ca6c969..95ed61aa4 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java @@ -18,7 +18,7 @@ package org.apache.hudi.io; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecord; @@ -77,7 +77,7 @@ public class TestHoodieMergeHandle extends HoodieClientTestHarness { // Build a write config with bulkinsertparallelism set HoodieWriteConfig cfg = getConfigBuilder().build(); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { FileSystem fs = FSUtils.getFs(basePath, hadoopConf); /** @@ -219,7 +219,7 @@ public class TestHoodieMergeHandle extends HoodieClientTestHarness { public void testHoodieMergeHandleWriteStatMetrics() throws Exception { // insert 100 records HoodieWriteConfig config = getConfigBuilder().build(); - try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) { + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) { String newCommitTime = "100"; writeClient.startCommitWithTime(newCommitTime); diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java similarity index 84% rename from hudi-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java index 93a02531c..c7a313ab9 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java @@ -24,9 +24,11 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieInsertException; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; +import org.apache.hudi.testutils.SparkDatasetTestUtils; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; @@ -40,12 +42,6 @@ import java.util.List; import java.util.Random; import java.util.UUID; -import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER; -import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE; -import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder; -import static org.apache.hudi.testutils.SparkDatasetTestUtils.getInternalRowWithError; -import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows; -import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -56,6 +52,7 @@ import static org.junit.jupiter.api.Assertions.fail; /** * Unit tests {@link HoodieRowCreateHandle}. */ +@SuppressWarnings("checkstyle:LineLength") public class TestHoodieRowCreateHandle extends HoodieClientTestHarness { private static final Random RANDOM = new Random(); @@ -77,8 +74,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness { @Test public void testRowCreateHandle() throws IOException { // init config and table - HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); - HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); List fileNames = new ArrayList<>(); List fileAbsPaths = new ArrayList<>(); @@ -91,10 +88,10 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness { String fileId = UUID.randomUUID().toString(); String instantTime = "000"; - HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE); + HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE); int size = 10 + RANDOM.nextInt(1000); // Generate inputs - Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + Dataset inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size, partitionPath, false); if (totalInputRows == null) { totalInputRows = inputRows; } else { @@ -118,29 +115,29 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness { @Test public void testGlobalFailure() throws IOException { // init config and table - HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); - HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0]; // init some args String fileId = UUID.randomUUID().toString(); String instantTime = "000"; - HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE); + HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE); int size = 10 + RANDOM.nextInt(1000); int totalFailures = 5; // Generate first batch of valid rows - Dataset inputRows = getRandomRows(sqlContext, size / 2, partitionPath, false); - List internalRows = toInternalRows(inputRows, ENCODER); + Dataset inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size / 2, partitionPath, false); + List internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER); // generate some failures rows for (int i = 0; i < totalFailures; i++) { - internalRows.add(getInternalRowWithError(partitionPath)); + internalRows.add(SparkDatasetTestUtils.getInternalRowWithError(partitionPath)); } // generate 2nd batch of valid rows - Dataset inputRows2 = getRandomRows(sqlContext, size / 2, partitionPath, false); - internalRows.addAll(toInternalRows(inputRows2, ENCODER)); + Dataset inputRows2 = SparkDatasetTestUtils.getRandomRows(sqlContext, size / 2, partitionPath, false); + internalRows.addAll(SparkDatasetTestUtils.toInternalRows(inputRows2, SparkDatasetTestUtils.ENCODER)); // issue writes try { @@ -171,11 +168,11 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness { @Test public void testInstantiationFailure() throws IOException { // init config and table - HoodieWriteConfig cfg = getConfigBuilder(basePath).withPath("/dummypath/abc/").build(); - HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).withPath("/dummypath/abc/").build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); try { - new HoodieRowCreateHandle(table, cfg, " def", UUID.randomUUID().toString(), "001", RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE); + new HoodieRowCreateHandle(table, cfg, " def", UUID.randomUUID().toString(), "001", RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE); fail("Should have thrown exception"); } catch (HoodieInsertException ioe) { // expected @@ -183,7 +180,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness { } private HoodieInternalWriteStatus writeAndGetWriteStatus(Dataset inputRows, HoodieRowCreateHandle handle) throws IOException { - List internalRows = toInternalRows(inputRows, ENCODER); + List internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER); // issue writes for (InternalRow internalRow : internalRows) { handle.write(internalRow); diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java similarity index 95% rename from hudi-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java index 46e117a87..88f755a44 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java @@ -37,6 +37,8 @@ import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.HoodieTimelineArchiveLog; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.junit.jupiter.api.AfterEach; @@ -65,10 +67,11 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { initPath(); initSparkContexts(); initMetaClient(); - hadoopConf = metaClient.getHadoopConf(); + hadoopConf = context.getHadoopConf().get(); metaClient.getFs().mkdirs(new Path(basePath)); metaClient = HoodieTestUtils.init(hadoopConf, basePath); wrapperFs = metaClient.getFs(); + hadoopConf.addResource(wrapperFs.getConf()); } @AfterEach @@ -82,8 +85,9 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).forTable("test-trip-table").build(); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf); - boolean result = archiveLog.archiveIfRequired(jsc); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); + boolean result = archiveLog.archiveIfRequired(context); assertTrue(result); } @@ -160,8 +164,9 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { verifyInflightInstants(metaClient, 2); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf); - assertTrue(archiveLog.archiveIfRequired(jsc)); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); + assertTrue(archiveLog.archiveIfRequired(context)); // reload the timeline and remove the remaining commits timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); @@ -228,9 +233,10 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); assertEquals(4, timeline.countInstants(), "Loaded 4 commits and the count should match"); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf); - boolean result = archiveLog.archiveIfRequired(jsc); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); + boolean result = archiveLog.archiveIfRequired(context); assertTrue(result); FileStatus[] allFiles = metaClient.getFs().listStatus(new Path(basePath + "/" + HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)); @@ -252,7 +258,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) .build(); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); // Requested Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf()); @@ -284,7 +291,7 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals(4, timeline.countInstants(), "Loaded 4 commits and the count should match"); - boolean result = archiveLog.archiveIfRequired(jsc); + boolean result = archiveLog.archiveIfRequired(context); assertTrue(result); timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); assertEquals(4, timeline.countInstants(), "Should not archive commits when maxCommitsToKeep is 5"); @@ -317,7 +324,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) .build(); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf()); @@ -327,7 +335,7 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match"); - boolean result = archiveLog.archiveIfRequired(jsc); + boolean result = archiveLog.archiveIfRequired(context); assertTrue(result); timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); assertTrue(timeline.containsOrBeforeTimelineStarts("100"), "Archived commits should always be safe"); @@ -342,6 +350,7 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table") .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) .build(); + HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf()); HoodieTestDataGenerator.createSavepointFile(basePath, "101", wrapperFs.getConf()); @@ -349,11 +358,12 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "104", wrapperFs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf()); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf); + HoodieTable table = HoodieSparkTable.create(cfg, context); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match"); - assertTrue(archiveLog.archiveIfRequired(jsc)); + assertTrue(archiveLog.archiveIfRequired(context)); timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); assertEquals(5, timeline.countInstants(), "Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)"); @@ -383,11 +393,12 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "106", wrapperFs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "107", wrapperFs.getConf()); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, wrapperFs.getConf()); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline(); assertEquals(8, timeline.countInstants(), "Loaded 6 commits and the count should match"); - boolean result = archiveLog.archiveIfRequired(jsc); + boolean result = archiveLog.archiveIfRequired(context); assertTrue(result); timeline = metaClient.getActiveTimeline().reload().getCommitsAndCompactionTimeline(); assertFalse(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "100")), @@ -433,9 +444,9 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { HoodieTestDataGenerator.createCommitFile(basePath, "4", wrapperFs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "5", wrapperFs.getConf()); - - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, wrapperFs.getConf()); - boolean result = archiveLog.archiveIfRequired(jsc); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); + boolean result = archiveLog.archiveIfRequired(context); assertTrue(result); HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(); List archivedInstants = Arrays.asList(instant1, instant2, instant3); @@ -460,7 +471,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) .build(); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); org.apache.hudi.avro.model.HoodieCommitMetadata expectedCommitMetadata = archiveLog.convertCommitMetadata(hoodieCommitMetadata); assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString()); diff --git a/hudi-client/src/test/java/org/apache/hudi/client/utils/TestSparkConfigUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestSparkIOUtils.java similarity index 74% rename from hudi-client/src/test/java/org/apache/hudi/client/utils/TestSparkConfigUtils.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestSparkIOUtils.java index 9463bfb30..ffbf6d103 100644 --- a/hudi-client/src/test/java/org/apache/hudi/client/utils/TestSparkConfigUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestSparkIOUtils.java @@ -16,10 +16,12 @@ * limitations under the License. */ -package org.apache.hudi.client.utils; +package org.apache.hudi.io; +import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.config.HoodieMemoryConfig; import org.apache.hudi.config.HoodieWriteConfig; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -29,10 +31,12 @@ import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_ import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP; import static org.junit.jupiter.api.Assertions.assertEquals; -public class TestSparkConfigUtils { +public class TestSparkIOUtils { @TempDir public java.nio.file.Path basePath; + private final SparkTaskContextSupplier contextSupplier = new SparkTaskContextSupplier(); + @Test public void testMaxMemoryPerPartitionMergeWithMaxSizeDefined() { String path = basePath.toString(); @@ -43,8 +47,8 @@ public class TestSparkConfigUtils { HoodieMemoryConfig memoryConfig = HoodieMemoryConfig.newBuilder().withMaxMemoryMaxSize(mergeMaxSize, compactionMaxSize).build(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(path).withMemoryConfig(memoryConfig).build(); - assertEquals(mergeMaxSize, SparkConfigUtils.getMaxMemoryPerPartitionMerge(config.getProps())); - assertEquals(compactionMaxSize, SparkConfigUtils.getMaxMemoryPerCompaction(config.getProps())); + assertEquals(mergeMaxSize, IOUtils.getMaxMemoryPerPartitionMerge(contextSupplier, config.getProps())); + assertEquals(compactionMaxSize, IOUtils.getMaxMemoryPerCompaction(contextSupplier, config.getProps())); } @Test @@ -54,12 +58,12 @@ public class TestSparkConfigUtils { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(path).build(); String compactionFraction = config.getProps().getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION); - long compactionMaxSize = SparkConfigUtils.getMaxMemoryAllowedForMerge(compactionFraction); + long compactionMaxSize = IOUtils.getMaxMemoryAllowedForMerge(contextSupplier, compactionFraction); String mergeFraction = config.getProps().getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE); - long mergeMaxSize = SparkConfigUtils.getMaxMemoryAllowedForMerge(mergeFraction); + long mergeMaxSize = IOUtils.getMaxMemoryAllowedForMerge(contextSupplier, mergeFraction); - assertEquals(mergeMaxSize, SparkConfigUtils.getMaxMemoryPerPartitionMerge(config.getProps())); - assertEquals(compactionMaxSize, SparkConfigUtils.getMaxMemoryPerCompaction(config.getProps())); + assertEquals(mergeMaxSize, IOUtils.getMaxMemoryPerPartitionMerge(contextSupplier, config.getProps())); + assertEquals(compactionMaxSize, IOUtils.getMaxMemoryPerCompaction(contextSupplier, config.getProps())); } } diff --git a/hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java similarity index 96% rename from hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java index 6ea2b7e1d..26f431a0e 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java @@ -21,6 +21,7 @@ package org.apache.hudi.io.storage; import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestBase; @@ -44,7 +45,7 @@ public class TestHoodieFileWriterFactory extends HoodieClientTestBase { final String instantTime = "100"; final Path parquetPath = new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"); final HoodieWriteConfig cfg = getConfig(); - HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); SparkTaskContextSupplier supplier = new SparkTaskContextSupplier(); HoodieFileWriter parquetWriter = HoodieFileWriterFactory.getFileWriter(instantTime, parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier); diff --git a/hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java similarity index 84% rename from hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java index bcb5aa6c4..37b8cdca8 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java @@ -26,6 +26,7 @@ import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hudi.testutils.SparkDatasetTestUtils; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -39,11 +40,6 @@ import java.util.List; import java.util.Random; import java.util.UUID; -import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER; -import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE; -import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder; -import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows; -import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows; import static org.junit.jupiter.api.Assertions.assertEquals; /** @@ -69,7 +65,7 @@ public class TestHoodieInternalRowParquetWriter extends HoodieClientTestHarness @Test public void endToEndTest() throws IOException { - HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build(); for (int i = 0; i < 5; i++) { // init write support and parquet config HoodieRowParquetWriteSupport writeSupport = getWriteSupport(cfg, hadoopConf); @@ -89,8 +85,8 @@ public class TestHoodieInternalRowParquetWriter extends HoodieClientTestHarness // generate input int size = 10 + RANDOM.nextInt(100); // Generate inputs - Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); - List internalRows = toInternalRows(inputRows, ENCODER); + Dataset inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size, partitionPath, false); + List internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER); // issue writes for (InternalRow internalRow : internalRows) { @@ -112,6 +108,6 @@ public class TestHoodieInternalRowParquetWriter extends HoodieClientTestHarness writeConfig.getBloomFilterFPP(), writeConfig.getDynamicBloomFilterMaxNumEntries(), writeConfig.getBloomFilterType()); - return new HoodieRowParquetWriteSupport(hadoopConf, STRUCT_TYPE, filter); + return new HoodieRowParquetWriteSupport(hadoopConf, SparkDatasetTestUtils.STRUCT_TYPE, filter); } } diff --git a/hudi-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java similarity index 96% rename from hudi-client/src/test/java/org/apache/hudi/table/TestCleaner.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index a4c7b5b7f..730e1ef01 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -24,7 +24,7 @@ import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieFileStatus; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.bootstrap.TestBootstrapIndex; @@ -63,6 +63,7 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.SparkHoodieIndex; import org.apache.hudi.table.action.clean.CleanPlanner; import org.apache.hudi.testutils.HoodieClientTestBase; @@ -124,9 +125,9 @@ public class TestCleaner extends HoodieClientTestBase { * @param insertFn Insertion API for testing * @throws Exception in case of error */ - private void insertFirstBigBatchForClientCleanerTest(HoodieWriteConfig cfg, HoodieWriteClient client, + private void insertFirstBigBatchForClientCleanerTest(HoodieWriteConfig cfg, SparkRDDWriteClient client, Function2, String, Integer> recordGenFunction, - Function3, HoodieWriteClient, JavaRDD, String> insertFn, + Function3, SparkRDDWriteClient, JavaRDD, String> insertFn, HoodieCleaningPolicy cleaningPolicy) throws Exception { /* @@ -147,14 +148,14 @@ public class TestCleaner extends HoodieClientTestBase { HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); assertEquals(1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), "Expecting a single commit."); // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieTable table = HoodieTable.create(metaClient, client.getConfig(), hadoopConf); + HoodieTable table = HoodieSparkTable.create(client.getConfig(), context, metaClient); assertFalse(table.getCompletedCommitsTimeline().empty()); // We no longer write empty cleaner plans when there is nothing to be cleaned. assertTrue(table.getCompletedCleanTimeline().empty()); - HoodieIndex index = HoodieIndex.createIndex(cfg); - List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, table).collect(); + HoodieIndex index = SparkHoodieIndex.createIndex(cfg); + List taggedRecords = ((JavaRDD)index.tagLocation(jsc.parallelize(records, 1), context, table)).collect(); checkTaggedRecords(taggedRecords, newCommitTime); } @@ -163,7 +164,7 @@ public class TestCleaner extends HoodieClientTestBase { */ @Test public void testInsertAndCleanByVersions() throws Exception { - testInsertAndCleanByVersions(HoodieWriteClient::insert, HoodieWriteClient::upsert, false); + testInsertAndCleanByVersions(SparkRDDWriteClient::insert, SparkRDDWriteClient::upsert, false); } /** @@ -171,7 +172,7 @@ public class TestCleaner extends HoodieClientTestBase { */ @Test public void testInsertPreppedAndCleanByVersions() throws Exception { - testInsertAndCleanByVersions(HoodieWriteClient::insertPreppedRecords, HoodieWriteClient::upsertPreppedRecords, + testInsertAndCleanByVersions(SparkRDDWriteClient::insertPreppedRecords, SparkRDDWriteClient::upsertPreppedRecords, true); } @@ -180,7 +181,7 @@ public class TestCleaner extends HoodieClientTestBase { */ @Test public void testBulkInsertAndCleanByVersions() throws Exception { - testInsertAndCleanByVersions(HoodieWriteClient::bulkInsert, HoodieWriteClient::upsert, false); + testInsertAndCleanByVersions(SparkRDDWriteClient::bulkInsert, SparkRDDWriteClient::upsert, false); } /** @@ -190,7 +191,7 @@ public class TestCleaner extends HoodieClientTestBase { public void testBulkInsertPreppedAndCleanByVersions() throws Exception { testInsertAndCleanByVersions( (client, recordRDD, instantTime) -> client.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty()), - HoodieWriteClient::upsertPreppedRecords, true); + SparkRDDWriteClient::upsertPreppedRecords, true); } /** @@ -203,8 +204,8 @@ public class TestCleaner extends HoodieClientTestBase { * @throws Exception in case of errors */ private void testInsertAndCleanByVersions( - Function3, HoodieWriteClient, JavaRDD, String> insertFn, - Function3, HoodieWriteClient, JavaRDD, String> upsertFn, boolean isPreppedAPI) + Function3, SparkRDDWriteClient, JavaRDD, String> insertFn, + Function3, SparkRDDWriteClient, JavaRDD, String> upsertFn, boolean isPreppedAPI) throws Exception { int maxVersions = 2; // keep upto 2 versions for each file HoodieWriteConfig cfg = getConfigBuilder() @@ -213,7 +214,7 @@ public class TestCleaner extends HoodieClientTestBase { .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) .build(); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { final Function2, String, Integer> recordInsertGenWrappedFunction = generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts); @@ -226,7 +227,7 @@ public class TestCleaner extends HoodieClientTestBase { Map compactionFileIdToLatestFileSlice = new HashMap<>(); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.create(metaClient, getConfig(), hadoopConf); + HoodieTable table = HoodieSparkTable.create(getConfig(), context, metaClient); for (String partitionPath : dataGen.getPartitionPaths()) { TableFileSystemView fsView = table.getFileSystemView(); Option added = Option.fromJavaOptional(fsView.getAllFileGroups(partitionPath).findFirst().map(fg -> { @@ -263,7 +264,7 @@ public class TestCleaner extends HoodieClientTestBase { assertNoWriteErrors(statuses); metaClient = HoodieTableMetaClient.reload(metaClient); - table = HoodieTable.create(metaClient, getConfig(), hadoopConf); + table = HoodieSparkTable.create(getConfig(), context, metaClient); HoodieTimeline timeline = table.getMetaClient().getCommitsTimeline(); TableFileSystemView fsView = table.getFileSystemView(); @@ -325,7 +326,7 @@ public class TestCleaner extends HoodieClientTestBase { */ @Test public void testInsertAndCleanByCommits() throws Exception { - testInsertAndCleanByCommits(HoodieWriteClient::insert, HoodieWriteClient::upsert, false); + testInsertAndCleanByCommits(SparkRDDWriteClient::insert, SparkRDDWriteClient::upsert, false); } /** @@ -333,7 +334,7 @@ public class TestCleaner extends HoodieClientTestBase { */ @Test public void testInsertPreppedAndCleanByCommits() throws Exception { - testInsertAndCleanByCommits(HoodieWriteClient::insertPreppedRecords, HoodieWriteClient::upsertPreppedRecords, true); + testInsertAndCleanByCommits(SparkRDDWriteClient::insertPreppedRecords, SparkRDDWriteClient::upsertPreppedRecords, true); } /** @@ -343,7 +344,7 @@ public class TestCleaner extends HoodieClientTestBase { public void testBulkInsertPreppedAndCleanByCommits() throws Exception { testInsertAndCleanByCommits( (client, recordRDD, instantTime) -> client.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty()), - HoodieWriteClient::upsertPreppedRecords, true); + SparkRDDWriteClient::upsertPreppedRecords, true); } /** @@ -351,7 +352,7 @@ public class TestCleaner extends HoodieClientTestBase { */ @Test public void testBulkInsertAndCleanByCommits() throws Exception { - testInsertAndCleanByCommits(HoodieWriteClient::bulkInsert, HoodieWriteClient::upsert, false); + testInsertAndCleanByCommits(SparkRDDWriteClient::bulkInsert, SparkRDDWriteClient::upsert, false); } /** @@ -364,8 +365,8 @@ public class TestCleaner extends HoodieClientTestBase { * @throws Exception in case of errors */ private void testInsertAndCleanByCommits( - Function3, HoodieWriteClient, JavaRDD, String> insertFn, - Function3, HoodieWriteClient, JavaRDD, String> upsertFn, boolean isPreppedAPI) + Function3, SparkRDDWriteClient, JavaRDD, String> insertFn, + Function3, SparkRDDWriteClient, JavaRDD, String> upsertFn, boolean isPreppedAPI) throws Exception { int maxCommits = 3; // keep upto 3 commits from the past HoodieWriteConfig cfg = getConfigBuilder() @@ -374,7 +375,7 @@ public class TestCleaner extends HoodieClientTestBase { .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) .build(); - HoodieWriteClient client = getHoodieWriteClient(cfg); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); final Function2, String, Integer> recordInsertGenWrappedFunction = generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts); @@ -396,7 +397,7 @@ public class TestCleaner extends HoodieClientTestBase { assertNoWriteErrors(statuses); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table1 = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable table1 = HoodieSparkTable.create(cfg, context, metaClient); HoodieTimeline activeTimeline = table1.getCompletedCommitsTimeline(); // NOTE: See CleanPlanner#getFilesToCleanKeepingLatestCommits. We explicitly keep one commit before earliest // commit @@ -452,7 +453,7 @@ public class TestCleaner extends HoodieClientTestBase { * @param config HoodieWriteConfig */ private List runCleaner(HoodieWriteConfig config, boolean simulateRetryFailure, int firstCommitSequence) throws IOException { - HoodieWriteClient writeClient = getHoodieWriteClient(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); String cleanInstantTs = makeNewCommitTime(firstCommitSequence); HoodieCleanMetadata cleanMetadata1 = writeClient.clean(cleanInstantTs); @@ -986,11 +987,11 @@ public class TestCleaner extends HoodieClientTestBase { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable table = HoodieSparkTable.create(config, context, metaClient); table.getActiveTimeline().transitionRequestedToInflight( new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "000"), Option.empty()); metaClient.reloadActiveTimeline(); - table.rollback(jsc, "001", new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "000"), true); + table.rollback(context, "001", new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "000"), true); final int numTempFilesAfter = testTable.listAllFilesInTempFolder().size(); assertEquals(0, numTempFilesAfter, "All temp files are deleted."); } @@ -1129,7 +1130,7 @@ public class TestCleaner extends HoodieClientTestBase { if (j == i && j <= maxNumFileIdsForCompaction) { expFileIdToPendingCompaction.put(fileId, compactionInstants[j]); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable table = HoodieSparkTable.create(config, context, metaClient); FileSlice slice = table.getSliceView().getLatestFileSlices(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) .filter(fs -> fs.getFileId().equals(fileId)).findFirst().get(); @@ -1171,7 +1172,7 @@ public class TestCleaner extends HoodieClientTestBase { // Test for safety final HoodieTableMetaClient newMetaClient = HoodieTableMetaClient.reload(metaClient); - final HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf); + final HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); expFileIdToPendingCompaction.forEach((fileId, value) -> { String baseInstantForCompaction = fileIdToLatestInstantBeforeCompaction.get(fileId); diff --git a/hudi-client/src/test/java/org/apache/hudi/table/TestConsistencyGuard.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestConsistencyGuard.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/table/TestConsistencyGuard.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestConsistencyGuard.java diff --git a/hudi-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java similarity index 94% rename from hudi-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index f60305175..e7db7ada4 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -19,8 +19,9 @@ package org.apache.hudi.table; import org.apache.hudi.client.HoodieReadClient; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -57,8 +58,8 @@ import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; -import org.apache.hudi.table.action.deltacommit.DeleteDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.DeltaCommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.AbstractSparkDeltaCommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.SparkDeleteDeltaCommitActionExecutor; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieClientTestUtils; import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; @@ -110,6 +111,8 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { initDFS(); initSparkContexts("TestHoodieMergeOnReadTable"); hadoopConf.addResource(dfs.getConf()); + jsc.hadoopConfiguration().addResource(dfs.getConf()); + context = new HoodieSparkEngineContext(jsc); initPath(); dfs.mkdirs(new Path(basePath)); metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ, baseFileFormat); @@ -133,7 +136,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { @Test public void testSimpleInsertAndUpdate() throws Exception { HoodieWriteConfig cfg = getConfig(true); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { /** * Write 1 (only inserts) @@ -155,7 +158,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { String compactionCommitTime = client.scheduleCompaction(Option.empty()).get().toString(); client.compact(compactionCommitTime); - HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath()); tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); @@ -181,7 +184,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { init(HoodieFileFormat.HFILE); HoodieWriteConfig cfg = getConfig(true); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { /** * Write 1 (only inserts) @@ -203,7 +206,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { String compactionCommitTime = client.scheduleCompaction(Option.empty()).get().toString(); client.compact(compactionCommitTime); - HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath()); tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); @@ -230,7 +233,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { String partitionPath = "2020/02/20"; // use only one partition for this test dataGen = new HoodieTestDataGenerator(new String[] { partitionPath }); HoodieWriteConfig cfg = getConfig(true); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { /** * Write 1 (only inserts) @@ -327,7 +330,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { @Test public void testMetadataAggregateFromWriteStatus() throws Exception { HoodieWriteConfig cfg = getConfigBuilder(false).withWriteStatusClass(MetadataMergeWriteStatus.class).build(); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { String newCommitTime = "001"; List records = dataGen.generateInserts(newCommitTime, 200); @@ -350,7 +353,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { @Test public void testSimpleInsertUpdateAndDelete() throws Exception { HoodieWriteConfig cfg = getConfig(true); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { /** * Write 1 (only inserts, written as parquet file) @@ -365,7 +368,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { assertNoWriteErrors(statuses); metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -432,7 +435,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE); HoodieWriteConfig cfg = getConfig(false, rollbackUsingMarkers); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { /** * Write 1 (only inserts) @@ -472,7 +475,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { client.rollback(newCommitTime); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath()); tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); @@ -494,7 +497,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { private void testRollbackWithDeltaAndCompactionCommit(Boolean rollbackUsingMarkers) throws Exception { HoodieWriteConfig cfg = getConfig(false, rollbackUsingMarkers); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { // Test delta commit rollback /** @@ -512,7 +515,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { assertNoWriteErrors(statuses); metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -537,7 +540,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { */ final String commitTime1 = "002"; // WriteClient with custom config (disable small file handling) - try (HoodieWriteClient secondClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff());) { + try (SparkRDDWriteClient secondClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff());) { secondClient.startCommitWithTime(commitTime1); List copyOfRecords = new ArrayList<>(records); @@ -570,7 +573,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { * Write 3 (inserts + updates - testing successful delta commit) */ final String commitTime2 = "002"; - try (HoodieWriteClient thirdClient = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient thirdClient = getHoodieWriteClient(cfg);) { thirdClient.startCommitWithTime(commitTime2); List copyOfRecords = new ArrayList<>(records); @@ -596,7 +599,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { .filter(file -> file.getPath().getName().contains(commitTime2)).count()); metaClient = HoodieTableMetaClient.reload(metaClient); - hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf); + hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); dataFiles = tableView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList()); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles, basePath); @@ -652,7 +655,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { @Test public void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception { HoodieWriteConfig cfg = getConfig(false); - try (final HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (final SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { /** * Write 1 (only inserts) */ @@ -668,7 +671,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { assertNoWriteErrors(statuses); metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -692,7 +695,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { */ newCommitTime = "002"; // WriteClient with custom config (disable small file handling) - HoodieWriteClient nClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff()); + SparkRDDWriteClient nClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff()); nClient.startCommitWithTime(newCommitTime); List copyOfRecords = new ArrayList<>(records); @@ -753,7 +756,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { compactionInstantTime = "006"; client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty()); - JavaRDD ws = client.compact(compactionInstantTime); + JavaRDD ws = (JavaRDD) client.compact(compactionInstantTime); client.commitCompaction(compactionInstantTime, ws, Option.empty()); allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath()); @@ -815,7 +818,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { @Test public void testUpsertPartitioner() throws Exception { HoodieWriteConfig cfg = getConfig(true); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { /** * Write 1 (only inserts, written as parquet file) @@ -830,7 +833,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { assertNoWriteErrors(statuses); metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -895,7 +898,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { public void testLogFileCountsAfterCompaction() throws Exception { // insert 100 records HoodieWriteConfig config = getConfig(true); - try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) { + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) { String newCommitTime = "100"; writeClient.startCommitWithTime(newCommitTime); @@ -912,7 +915,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { List updatedRecords = dataGen.generateUpdates(newCommitTime, records); JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); - HoodieReadClient readClient = new HoodieReadClient(jsc, config); + HoodieReadClient readClient = new HoodieReadClient(context, config); updatedRecords = readClient.tagLocation(updatedRecordsRDD).collect(); // Write them to corresponding avro logfiles @@ -921,7 +924,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { // Verify that all data file has one log file metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable table = HoodieSparkTable.create(config, context, metaClient); // In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state ((SyncableFileSystemView) (table.getSliceView())).reset(); @@ -941,11 +944,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { // Do a compaction String compactionInstantTime = writeClient.scheduleCompaction(Option.empty()).get().toString(); - JavaRDD result = writeClient.compact(compactionInstantTime); + JavaRDD result = (JavaRDD) writeClient.compact(compactionInstantTime); // Verify that recently written compacted data file has no log file metaClient = HoodieTableMetaClient.reload(metaClient); - table = HoodieTable.create(metaClient, config, hadoopConf); + table = HoodieSparkTable.create(config, context, metaClient); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); assertTrue(HoodieTimeline @@ -969,7 +972,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { // insert 100 records // Setting IndexType to be InMemory to simulate Global Index nature HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build(); - try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) { + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) { String newCommitTime = "100"; writeClient.startCommitWithTime(newCommitTime); @@ -979,7 +982,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { writeClient.commit(newCommitTime, statuses); HoodieTable table = - HoodieTable.create(getHoodieMetaClient(hadoopConf, basePath), config, hadoopConf); + HoodieSparkTable.create(config, context, getHoodieMetaClient(hadoopConf, basePath)); SliceView tableRTFileSystemView = table.getSliceView(); long numLogFiles = 0; @@ -994,7 +997,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { assertTrue(numLogFiles > 0); // Do a compaction String instantTime = writeClient.scheduleCompaction(Option.empty()).get().toString(); - statuses = writeClient.compact(instantTime); + statuses = (JavaRDD) writeClient.compact(instantTime); assertEquals(statuses.map(status -> status.getStat().getPath().contains("parquet")).count(), numLogFiles); assertEquals(statuses.count(), numLogFiles); writeClient.commitCompaction(instantTime, statuses, Option.empty()); @@ -1005,7 +1008,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { // insert 100 records // Setting IndexType to be InMemory to simulate Global Index nature HoodieWriteConfig config = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY).build(); - try (HoodieWriteClient writeClient = getHoodieWriteClient(config)) { + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config)) { String newCommitTime = "100"; writeClient.startCommitWithTime(newCommitTime); @@ -1059,7 +1062,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { writeClient.rollback(newCommitTime); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.create(config, hadoopConf); + HoodieTable table = HoodieSparkTable.create(config, context); SliceView tableRTFileSystemView = table.getSliceView(); long numLogFiles = 0; @@ -1102,7 +1105,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { // insert 100 records // Setting IndexType to be InMemory to simulate Global Index nature HoodieWriteConfig config = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY).build(); - try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) { + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) { String newCommitTime = "100"; writeClient.startCommitWithTime(newCommitTime); @@ -1113,7 +1116,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { // trigger an action statuses.collect(); - HoodieTable table = HoodieTable.create(getHoodieMetaClient(hadoopConf, basePath), config, hadoopConf); + HoodieTable table = HoodieSparkTable.create(config, context, getHoodieMetaClient(hadoopConf, basePath)); SliceView tableRTFileSystemView = table.getSliceView(); long numLogFiles = 0; @@ -1127,7 +1130,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { assertTrue(numLogFiles > 0); // Do a compaction newCommitTime = writeClient.scheduleCompaction(Option.empty()).get().toString(); - statuses = writeClient.compact(newCommitTime); + statuses = (JavaRDD) writeClient.compact(newCommitTime); // Ensure all log files have been compacted into parquet files assertEquals(statuses.map(status -> status.getStat().getPath().contains("parquet")).count(), numLogFiles); assertEquals(statuses.count(), numLogFiles); @@ -1136,7 +1139,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { table.getActiveTimeline().reload(); writeClient.rollbackInflightCompaction(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, newCommitTime), table); - table = HoodieTable.create(getHoodieMetaClient(hadoopConf, basePath), config, hadoopConf); + table = HoodieSparkTable.create(config, context, getHoodieMetaClient(hadoopConf, basePath)); tableRTFileSystemView = table.getSliceView(); ((SyncableFileSystemView) tableRTFileSystemView).reset(); @@ -1165,9 +1168,9 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { public void testMetadataStatsOnCommit(Boolean rollbackUsingMarkers) throws Exception { HoodieWriteConfig cfg = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY) .withAutoCommit(false).build(); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { metaClient = getHoodieMetaClient(hadoopConf, basePath); - HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); // Create a commit without metadata stats in metadata to test backwards compatibility HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); @@ -1188,7 +1191,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { assertTrue(client.commit(instantTime, statuses), "Commit should succeed"); // Read from commit file - table = HoodieTable.create(cfg, hadoopConf); + table = HoodieSparkTable.create(cfg, context); HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes( table.getActiveTimeline().getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class); @@ -1221,7 +1224,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { client.rollback(instantTime); // Read from commit file - table = HoodieTable.create(cfg, hadoopConf); + table = HoodieSparkTable.create(cfg, context); metadata = HoodieCommitMetadata.fromBytes( table.getActiveTimeline() .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), @@ -1258,7 +1261,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { @Test public void testRollingStatsWithSmallFileHandling() throws Exception { HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build(); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { Map fileIdToInsertsMap = new HashMap<>(); Map fileIdToUpsertsMap = new HashMap<>(); @@ -1272,7 +1275,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { assertTrue(client.commit(instantTime, statuses), "Commit should succeed"); // Read from commit file - HoodieTable table = HoodieTable.create(cfg, hadoopConf); + HoodieTable table = HoodieSparkTable.create(cfg, context); HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes( table.getActiveTimeline() .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), @@ -1297,7 +1300,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { assertTrue(client.commit(instantTime, statuses), "Commit should succeed"); // Read from commit file - table = HoodieTable.create(cfg, hadoopConf); + table = HoodieSparkTable.create(cfg, context); metadata = HoodieCommitMetadata.fromBytes( table.getActiveTimeline() .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), @@ -1319,11 +1322,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { // Test small file handling after compaction instantTime = "002"; client.scheduleCompactionAtInstant(instantTime, Option.of(metadata.getExtraMetadata())); - statuses = client.compact(instantTime); + statuses = (JavaRDD) client.compact(instantTime); client.commitCompaction(instantTime, statuses, Option.empty()); // Read from commit file - table = HoodieTable.create(cfg, hadoopConf); + table = HoodieSparkTable.create(cfg, context); HoodieCommitMetadata metadata1 = HoodieCommitMetadata.fromBytes( table.getActiveTimeline() .getInstantDetails(table.getActiveTimeline().getCommitsTimeline().lastInstant().get()).get(), @@ -1347,7 +1350,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { assertTrue(client.commit(instantTime, statuses), "Commit should succeed"); // Read from commit file - table = HoodieTable.create(cfg, hadoopConf); + table = HoodieSparkTable.create(cfg, context); metadata = HoodieCommitMetadata.fromBytes( table.getActiveTimeline() .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), @@ -1373,7 +1376,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { @Test public void testHandleUpdateWithMultiplePartitions() throws Exception { HoodieWriteConfig cfg = getConfig(true); - try (HoodieWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { /** * Write 1 (only inserts, written as parquet file) @@ -1388,7 +1391,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { assertNoWriteErrors(statuses); metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath()); - HoodieMergeOnReadTable hoodieTable = (HoodieMergeOnReadTable) HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieSparkMergeOnReadTable hoodieTable = (HoodieSparkMergeOnReadTable) HoodieSparkTable.create(cfg, context, metaClient); Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -1432,9 +1435,9 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { JavaRDD deleteRDD = jsc.parallelize(fewRecordsForDelete, 1); // initialize partitioner - DeltaCommitActionExecutor actionExecutor = new DeleteDeltaCommitActionExecutor(jsc, cfg, hoodieTable, + AbstractSparkDeltaCommitActionExecutor actionExecutor = new SparkDeleteDeltaCommitActionExecutor(context, cfg, hoodieTable, newDeleteTime, deleteRDD); - actionExecutor.getUpsertPartitioner(new WorkloadProfile(deleteRDD)); + actionExecutor.getUpsertPartitioner(new WorkloadProfile(buildProfile(deleteRDD))); final List> deleteStatus = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleUpdate(partitionPath, fileId, fewRecordsForDelete.iterator()); }).map(Transformations::flatten).collect(); @@ -1479,7 +1482,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { .withRollbackUsingMarkers(rollbackUsingMarkers); } - private FileStatus[] insertAndGetFilePaths(List records, HoodieWriteClient client, + private FileStatus[] insertAndGetFilePaths(List records, SparkRDDWriteClient client, HoodieWriteConfig cfg, String commitTime) throws IOException { JavaRDD writeRecords = jsc.parallelize(records, 1); @@ -1487,7 +1490,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { assertNoWriteErrors(statuses); metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); assertTrue(deltaCommit.isPresent()); @@ -1509,7 +1512,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { return allFiles; } - private FileStatus[] updateAndGetFilePaths(List records, HoodieWriteClient client, + private FileStatus[] updateAndGetFilePaths(List records, SparkRDDWriteClient client, HoodieWriteConfig cfg, String commitTime) throws IOException { Map recordsMap = new HashMap<>(); for (HoodieRecord rec : records) { @@ -1529,7 +1532,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness { Option commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); return listAllDataFilesInPath(hoodieTable, cfg.getBasePath()); } diff --git a/hudi-client/src/test/java/org/apache/hudi/table/TestMarkerFiles.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestMarkerFiles.java similarity index 93% rename from hudi-client/src/test/java/org/apache/hudi/table/TestMarkerFiles.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestMarkerFiles.java index 55b7b50af..b25427baf 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/TestMarkerFiles.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestMarkerFiles.java @@ -18,6 +18,7 @@ package org.apache.hudi.table; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.testutils.FileSystemTestUtils; @@ -49,12 +50,14 @@ public class TestMarkerFiles extends HoodieCommonTestHarness { private FileSystem fs; private Path markerFolderPath; private JavaSparkContext jsc; + private HoodieSparkEngineContext context; @BeforeEach public void setup() throws IOException { initPath(); initMetaClient(); this.jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest(TestMarkerFiles.class.getName())); + this.context = new HoodieSparkEngineContext(jsc); this.fs = FSUtils.getFs(metaClient.getBasePath(), metaClient.getHadoopConf()); this.markerFolderPath = new Path(metaClient.getMarkerFolderPath("000")); this.markerFiles = new MarkerFiles(fs, metaClient.getBasePath(), markerFolderPath.toString(), "000"); @@ -63,6 +66,7 @@ public class TestMarkerFiles extends HoodieCommonTestHarness { @AfterEach public void cleanup() { jsc.stop(); + context = null; } private void createSomeMarkerFiles() { @@ -107,7 +111,7 @@ public class TestMarkerFiles extends HoodieCommonTestHarness { // then assertTrue(markerFiles.doesMarkerDirExist()); - assertTrue(markerFiles.deleteMarkerDir(jsc, 2)); + assertTrue(markerFiles.deleteMarkerDir(context, 2)); assertFalse(markerFiles.doesMarkerDirExist()); } @@ -115,7 +119,7 @@ public class TestMarkerFiles extends HoodieCommonTestHarness { public void testDeletionWhenMarkerDirNotExists() throws IOException { // then assertFalse(markerFiles.doesMarkerDirExist()); - assertFalse(markerFiles.deleteMarkerDir(jsc, 2)); + assertFalse(markerFiles.deleteMarkerDir(context, 2)); } @Test @@ -130,7 +134,7 @@ public class TestMarkerFiles extends HoodieCommonTestHarness { // then assertIterableEquals(CollectionUtils.createImmutableList( "2020/06/01/file1", "2020/06/03/file3"), - markerFiles.createdAndMergedDataPaths(jsc, 2).stream().sorted().collect(Collectors.toList()) + markerFiles.createdAndMergedDataPaths(context, 2).stream().sorted().collect(Collectors.toList()) ); } diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java similarity index 95% rename from hudi-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java index 64db6cf6d..b57e1c527 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java @@ -64,14 +64,14 @@ public class TestBootstrapUtils extends HoodieClientTestBase { }); List>> collected = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, - metaClient.getFs(), basePath, jsc); + metaClient.getFs(), basePath, context); assertEquals(3, collected.size()); collected.stream().forEach(k -> { assertEquals(2, k.getRight().size()); }); // Simulate reading from un-partitioned dataset - collected = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath + "/" + folders.get(0), jsc); + collected = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath + "/" + folders.get(0), context); assertEquals(1, collected.size()); collected.stream().forEach(k -> { assertEquals(2, k.getRight().size()); diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java similarity index 91% rename from hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java index 89fd592f8..852f8029c 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.fs.FSUtils; @@ -37,7 +37,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.io.HoodieCreateHandle; -import org.apache.hudi.table.HoodieCopyOnWriteTable; +import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.MetadataMergeWriteStatus; @@ -90,7 +91,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { String instantTime = makeNewCommitTime(); HoodieWriteConfig config = makeHoodieClientConfig(); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf); + HoodieTable table = HoodieSparkTable.create(config, context, metaClient); Pair newPathWithWriteToken = jsc.parallelize(Arrays.asList(1)).map(x -> { HoodieRecord record = mock(HoodieRecord.class); @@ -120,12 +121,12 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { // Prepare the AvroParquetIO HoodieWriteConfig config = makeHoodieClientConfig(); String firstCommitTime = makeNewCommitTime(); - HoodieWriteClient writeClient = getHoodieWriteClient(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); writeClient.startCommitWithTime(firstCommitTime); metaClient = HoodieTableMetaClient.reload(metaClient); String partitionPath = "2016/01/31"; - HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf); + HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); // Get some records belong to the same partition (2016/01/31) String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," @@ -146,7 +147,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); // Insert new records - final HoodieCopyOnWriteTable cowTable = table; + final HoodieSparkCopyOnWriteTable cowTable = table; writeClient.insert(jsc.parallelize(records, 1), firstCommitTime); FileStatus[] allFiles = getIncrementalFiles(partitionPath, "0", -1); @@ -267,7 +268,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { String firstCommitTime = makeNewCommitTime(); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf); + HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); // Get some records belong to the same partition (2016/01/31) String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," @@ -286,7 +287,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); // Insert new records - CommitActionExecutor actionExecutor = new InsertCommitActionExecutor(jsc, config, table, + BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table, firstCommitTime, jsc.parallelize(records)); List writeStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator()); @@ -320,7 +321,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { HoodieWriteConfig config = makeHoodieClientConfig(); String instantTime = makeNewCommitTime(); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf); + HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); // Case 1: // 10 records for partition 1, 1 record for partition 2. @@ -329,7 +330,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { // Insert new records final List recs2 = records; - CommitActionExecutor actionExecutor = new InsertPreppedCommitActionExecutor(jsc, config, table, + BaseSparkCommitActionExecutor actionExecutor = new SparkInsertPreppedCommitActionExecutor(context, config, table, instantTime, jsc.parallelize(recs2)); List returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs2.iterator()); @@ -350,7 +351,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { // Insert new records final List recs3 = records; - CommitActionExecutor newActionExecutor = new UpsertPreppedCommitActionExecutor(jsc, config, table, + BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertPreppedCommitActionExecutor(context, config, table, instantTime, jsc.parallelize(recs3)); returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> { return newActionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs3.iterator()); @@ -371,7 +372,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { .parquetBlockSize(64 * 1024).parquetPageSize(64 * 1024).build()).build(); String instantTime = makeNewCommitTime(); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf); + HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); List records = new ArrayList<>(); // Approx 1150 records are written for block size of 64KB @@ -383,7 +384,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { } // Insert new records - CommitActionExecutor actionExecutor = new UpsertCommitActionExecutor(jsc, config, table, + BaseSparkCommitActionExecutor actionExecutor = new SparkUpsertCommitActionExecutor(context, config, table, instantTime, jsc.parallelize(records)); jsc.parallelize(Arrays.asList(1)) .map(i -> actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator())) @@ -407,11 +408,11 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { .withStorageConfig(HoodieStorageConfig.newBuilder() .parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build(); metaClient = HoodieTableMetaClient.reload(metaClient); - final HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf); + final HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); String instantTime = "000"; // Perform inserts of 100 records to test CreateHandle and BufferedExecutor final List inserts = dataGen.generateInsertsWithHoodieAvroPayload(instantTime, 100); - CommitActionExecutor actionExecutor = new InsertCommitActionExecutor(jsc, config, table, + BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table, instantTime, jsc.parallelize(inserts)); final List> ws = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleInsert(UUID.randomUUID().toString(), inserts.iterator()); @@ -424,7 +425,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { String partitionPath = writeStatus.getPartitionPath(); long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count(); - CommitActionExecutor newActionExecutor = new UpsertCommitActionExecutor(jsc, config, table, + BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertCommitActionExecutor(context, config, table, instantTime, jsc.parallelize(updates)); final List> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> { return newActionExecutor.handleUpdate(partitionPath, fileId, updates.iterator()); @@ -437,16 +438,16 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { .withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA) .withBulkInsertParallelism(2).withBulkInsertSortMode(bulkInsertMode).build(); String instantTime = makeNewCommitTime(); - HoodieWriteClient writeClient = getHoodieWriteClient(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); writeClient.startCommitWithTime(instantTime); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf); + HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); // Insert new records final JavaRDD inputRecords = generateTestRecordsForBulkInsert(jsc); - BulkInsertCommitActionExecutor bulkInsertExecutor = new BulkInsertCommitActionExecutor( - jsc, config, table, instantTime, inputRecords, Option.empty()); - List returnedStatuses = bulkInsertExecutor.execute().getWriteStatuses().collect(); + SparkBulkInsertCommitActionExecutor bulkInsertExecutor = new SparkBulkInsertCommitActionExecutor( + context, config, table, instantTime, inputRecords, Option.empty()); + List returnedStatuses = ((JavaRDD)bulkInsertExecutor.execute().getWriteStatuses()).collect(); verifyStatusResult(returnedStatuses, generateExpectedPartitionNumRecords(inputRecords)); } diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java similarity index 86% rename from hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java index 8fda8ae09..d9dc6ac97 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java @@ -17,23 +17,25 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.index.bloom.HoodieBloomIndex; +import org.apache.hudi.index.bloom.SparkHoodieBloomIndex; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; + import org.apache.spark.Partition; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; - import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; -import scala.Tuple2; import java.util.Collections; import java.util.List; @@ -62,12 +64,13 @@ public class TestDeleteHelper { private static final boolean WITHOUT_COMBINE = false; private static final int DELETE_PARALLELISM = 200; - @Mock private HoodieBloomIndex index; - @Mock private HoodieTable table; - @Mock private CommitActionExecutor executor; + @Mock private SparkHoodieBloomIndex index; + @Mock private HoodieTable, JavaRDD, JavaRDD> table; + @Mock private BaseSparkCommitActionExecutor executor; @Mock private HoodieWriteMetadata metadata; @Mock private JavaPairRDD keyPairs; @Mock private JavaSparkContext jsc; + @Mock private HoodieSparkEngineContext context; private JavaRDD rddToDelete; private HoodieWriteConfig config; @@ -75,6 +78,7 @@ public class TestDeleteHelper { @BeforeEach public void setUp() { when(table.getIndex()).thenReturn(index); + when(context.getJavaSparkContext()).thenReturn(jsc); } @Test @@ -82,7 +86,7 @@ public class TestDeleteHelper { rddToDelete = mockEmptyHoodieKeyRdd(); config = newWriteConfig(WITHOUT_COMBINE); - DeleteHelper.execute("test-time", rddToDelete, jsc, config, table, executor); + SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); verify(rddToDelete, never()).repartition(DELETE_PARALLELISM); verifyNoDeleteExecution(); @@ -93,7 +97,7 @@ public class TestDeleteHelper { rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.None); config = newWriteConfig(WITHOUT_COMBINE); - DeleteHelper.execute("test-time", rddToDelete, jsc, config, table, executor); + SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); verify(rddToDelete, times(1)).repartition(DELETE_PARALLELISM); verifyDeleteExecution(); @@ -104,7 +108,7 @@ public class TestDeleteHelper { rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.NoneGlobalIndex); config = newWriteConfig(WITH_COMBINE); - DeleteHelper.execute("test-time", rddToDelete, jsc, config, table, executor); + SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); verify(rddToDelete, times(1)).distinct(DELETE_PARALLELISM); verifyDeleteExecution(); @@ -116,7 +120,7 @@ public class TestDeleteHelper { config = newWriteConfig(WITH_COMBINE); when(index.isGlobal()).thenReturn(true); - DeleteHelper.execute("test-time", rddToDelete, jsc, config, table, executor); + SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); verify(keyPairs, times(1)).reduceByKey(any(), eq(DELETE_PARALLELISM)); verifyDeleteExecution(); @@ -171,10 +175,6 @@ public class TestDeleteHelper { doReturn(Collections.emptyList()).when(emptyRdd).partitions(); doReturn(emptyRdd).when(emptyRdd).map(any()); - JavaPairRDD emptyPairRdd = mock(JavaPairRDD.class); - doReturn(Collections.emptyMap()).when(emptyPairRdd).countByKey(); - doReturn(emptyPairRdd).when(emptyRdd).mapToPair(any()); - doReturn(emptyRdd).when(index).tagLocation(any(), any(), any()); doReturn(emptyRdd).when(emptyRdd).filter(any()); diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java similarity index 96% rename from hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java index 2fc9fe8b2..6b3426b0e 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java @@ -31,8 +31,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieCopyOnWriteTable; -import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.testutils.HoodieClientTestBase; @@ -77,7 +77,7 @@ public class TestUpsertPartitioner extends HoodieClientTestBase { FileCreateUtils.createCommit(basePath, "001"); FileCreateUtils.createBaseFile(basePath, testPartitionPath, "001", "file1", fileSize); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf); + HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath}); List insertRecords = dataGenerator.generateInserts("001", numInserts); @@ -90,8 +90,8 @@ public class TestUpsertPartitioner extends HoodieClientTestBase { List records = new ArrayList<>(); records.addAll(insertRecords); records.addAll(updateRecords); - WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(records)); - UpsertPartitioner partitioner = new UpsertPartitioner(profile, jsc, table, config); + WorkloadProfile profile = new WorkloadProfile(buildProfile(jsc.parallelize(records))); + UpsertPartitioner partitioner = new UpsertPartitioner(profile, context, table, config); assertEquals(0, partitioner.getPartition( new Tuple2<>(updateRecords.get(0).getKey(), Option.ofNullable(updateRecords.get(0).getCurrentLocation()))), "Update record should have gone to the 1 update partition"); @@ -196,12 +196,12 @@ public class TestUpsertPartitioner extends HoodieClientTestBase { FileCreateUtils.createCommit(basePath, "001"); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf); + HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath}); List insertRecords = dataGenerator.generateInserts("001", totalInsertNum); - WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(insertRecords)); - UpsertPartitioner partitioner = new UpsertPartitioner(profile, jsc, table, config); + WorkloadProfile profile = new WorkloadProfile(buildProfile(jsc.parallelize(insertRecords))); + UpsertPartitioner partitioner = new UpsertPartitioner(profile, context, table, config); List insertBuckets = partitioner.getInsertBuckets(testPartitionPath); float bucket0Weight = 0.2f; diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java similarity index 96% rename from hudi-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java index bd8c8c8ff..d1d31f824 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java @@ -21,7 +21,7 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.HoodieReadClient; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; @@ -102,7 +102,7 @@ public class CompactionTestBase extends HoodieClientTestBase { }); } - protected List runNextDeltaCommits(HoodieWriteClient client, final HoodieReadClient readClient, List deltaInstants, + protected List runNextDeltaCommits(SparkRDDWriteClient client, final HoodieReadClient readClient, List deltaInstants, List records, HoodieWriteConfig cfg, boolean insertFirst, List expPendingCompactionInstants) throws Exception { @@ -155,20 +155,20 @@ public class CompactionTestBase extends HoodieClientTestBase { assertTrue(instant.isInflight(), "Instant must be marked inflight"); } - protected void scheduleCompaction(String compactionInstantTime, HoodieWriteClient client, HoodieWriteConfig cfg) { + protected void scheduleCompaction(String compactionInstantTime, SparkRDDWriteClient client, HoodieWriteConfig cfg) { client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath()); HoodieInstant instant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().get(); assertEquals(compactionInstantTime, instant.getTimestamp(), "Last compaction instant must be the one set"); } - protected void scheduleAndExecuteCompaction(String compactionInstantTime, HoodieWriteClient client, HoodieTable table, + protected void scheduleAndExecuteCompaction(String compactionInstantTime, SparkRDDWriteClient client, HoodieTable table, HoodieWriteConfig cfg, int expectedNumRecs, boolean hasDeltaCommitAfterPendingCompaction) throws IOException { scheduleCompaction(compactionInstantTime, client, cfg); executeCompaction(compactionInstantTime, client, table, cfg, expectedNumRecs, hasDeltaCommitAfterPendingCompaction); } - protected void executeCompaction(String compactionInstantTime, HoodieWriteClient client, HoodieTable table, + protected void executeCompaction(String compactionInstantTime, SparkRDDWriteClient client, HoodieTable table, HoodieWriteConfig cfg, int expectedNumRecs, boolean hasDeltaCommitAfterPendingCompaction) throws IOException { client.compact(compactionInstantTime); @@ -200,7 +200,7 @@ public class CompactionTestBase extends HoodieClientTestBase { } - protected List createNextDeltaCommit(String instantTime, List records, HoodieWriteClient client, + protected List createNextDeltaCommit(String instantTime, List records, SparkRDDWriteClient client, HoodieTableMetaClient metaClient, HoodieWriteConfig cfg, boolean skipCommit) { JavaRDD writeRecords = jsc.parallelize(records, 1); diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java similarity index 95% rename from hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java index 81840b978..8da1f3ddc 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java @@ -21,13 +21,14 @@ package org.apache.hudi.table.action.compact; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hudi.client.HoodieReadClient; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.junit.jupiter.api.Test; @@ -52,7 +53,7 @@ public class TestAsyncCompaction extends CompactionTestBase { public void testRollbackForInflightCompaction() throws Exception { // Rollback inflight compaction HoodieWriteConfig cfg = getConfig(false); - try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) { HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; String secondInstantTime = "004"; @@ -79,7 +80,7 @@ public class TestAsyncCompaction extends CompactionTestBase { // Reload and rollback inflight compaction metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf); + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); // hoodieTable.rollback(jsc, // new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), false); @@ -114,7 +115,7 @@ public class TestAsyncCompaction extends CompactionTestBase { int numRecs = 2000; - try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) { HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); List records = dataGen.generateInserts(firstInstantTime, numRecs); records = runNextDeltaCommits(client, readClient, Arrays.asList(firstInstantTime, secondInstantTime), records, cfg, true, @@ -156,7 +157,7 @@ public class TestAsyncCompaction extends CompactionTestBase { public void testInflightCompaction() throws Exception { // There is inflight compaction. Subsequent compaction run must work correctly HoodieWriteConfig cfg = getConfig(true); - try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) { HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; String secondInstantTime = "004"; @@ -189,7 +190,7 @@ public class TestAsyncCompaction extends CompactionTestBase { public void testScheduleIngestionBeforePendingCompaction() throws Exception { // Case: Failure case. Latest pending compaction instant time must be earlier than this instant time HoodieWriteConfig cfg = getConfig(false); - HoodieWriteClient client = getHoodieWriteClient(cfg, true); + SparkRDDWriteClient client = getHoodieWriteClient(cfg, true); HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; @@ -220,7 +221,7 @@ public class TestAsyncCompaction extends CompactionTestBase { // Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time HoodieWriteConfig cfg = getConfig(false); - HoodieWriteClient client = getHoodieWriteClient(cfg, true); + SparkRDDWriteClient client = getHoodieWriteClient(cfg, true); HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; @@ -252,7 +253,7 @@ public class TestAsyncCompaction extends CompactionTestBase { // Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time HoodieWriteConfig cfg = getConfig(false); - HoodieWriteClient client = getHoodieWriteClient(cfg, true); + SparkRDDWriteClient client = getHoodieWriteClient(cfg, true); HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); final String firstInstantTime = "001"; @@ -287,7 +288,7 @@ public class TestAsyncCompaction extends CompactionTestBase { public void testCompactionAfterTwoDeltaCommits() throws Exception { // No Delta Commits after compaction request HoodieWriteConfig cfg = getConfig(true); - try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) { HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; String secondInstantTime = "004"; @@ -308,7 +309,7 @@ public class TestAsyncCompaction extends CompactionTestBase { public void testInterleavedCompaction() throws Exception { // Case: Two delta commits before and after compaction schedule HoodieWriteConfig cfg = getConfig(true); - try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) { HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; String secondInstantTime = "004"; diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java similarity index 86% rename from hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java index 09a9ccad4..7655a75ce 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; @@ -40,7 +40,8 @@ import org.apache.hudi.config.HoodieStorageConfig; 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.index.bloom.SparkHoodieBloomIndex; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; @@ -104,11 +105,11 @@ public class TestHoodieCompactor extends HoodieClientTestHarness { @Test public void testCompactionOnCopyOnWriteFail() throws Exception { metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE); - HoodieTable table = HoodieTable.create(metaClient, getConfig(), hadoopConf); + HoodieTable table = HoodieSparkTable.create(getConfig(), context, metaClient); String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime(); assertThrows(HoodieNotSupportedException.class, () -> { - table.scheduleCompaction(jsc, compactionInstantTime, Option.empty()); - table.compact(jsc, compactionInstantTime); + table.scheduleCompaction(context, compactionInstantTime, Option.empty()); + table.compact(context, compactionInstantTime); }); } @@ -116,8 +117,8 @@ public class TestHoodieCompactor extends HoodieClientTestHarness { public void testCompactionEmpty() throws Exception { HoodieWriteConfig config = getConfig(); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf); - try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) { + HoodieTable table = HoodieSparkTable.create(getConfig(), context, metaClient); + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) { String newCommitTime = writeClient.startCommit(); List records = dataGen.generateInserts(newCommitTime, 100); @@ -125,7 +126,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness { writeClient.insert(recordsRDD, newCommitTime).collect(); String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime(); - Option plan = table.scheduleCompaction(jsc, compactionInstantTime, Option.empty()); + Option plan = table.scheduleCompaction(context, compactionInstantTime, Option.empty()); assertFalse(plan.isPresent(), "If there is nothing to compact, result will be empty"); } } @@ -134,7 +135,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness { public void testWriteStatusContentsAfterCompaction() throws Exception { // insert 100 records HoodieWriteConfig config = getConfig(); - try (HoodieWriteClient writeClient = getHoodieWriteClient(config)) { + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config)) { String newCommitTime = "100"; writeClient.startCommitWithTime(newCommitTime); @@ -143,14 +144,14 @@ public class TestHoodieCompactor extends HoodieClientTestHarness { writeClient.insert(recordsRDD, newCommitTime).collect(); // Update all the 100 records - HoodieTable table = HoodieTable.create(config, hadoopConf); + HoodieTable table = HoodieSparkTable.create(config, context); newCommitTime = "101"; writeClient.startCommitWithTime(newCommitTime); List updatedRecords = dataGen.generateUpdates(newCommitTime, records); JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); - HoodieIndex index = new HoodieBloomIndex<>(config); - updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect(); + HoodieIndex index = new SparkHoodieBloomIndex<>(config); + updatedRecords = ((JavaRDD)index.tagLocation(updatedRecordsRDD, context, table)).collect(); // Write them to corresponding avro logfiles. Also, set the state transition properly. HoodieTestUtils.writeRecordsToLogFiles(fs, metaClient.getBasePath(), @@ -161,7 +162,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness { metaClient.reloadActiveTimeline(); // Verify that all data file has one log file - table = HoodieTable.create(config, hadoopConf); + table = HoodieSparkTable.create(config, context); for (String partitionPath : dataGen.getPartitionPaths()) { List groupedLogFiles = table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList()); @@ -174,11 +175,11 @@ public class TestHoodieCompactor extends HoodieClientTestHarness { createInflightDeltaCommit(basePath, newCommitTime); // Do a compaction - table = HoodieTable.create(config, hadoopConf); + table = HoodieSparkTable.create(config, context); String compactionInstantTime = "102"; - table.scheduleCompaction(jsc, compactionInstantTime, Option.empty()); + table.scheduleCompaction(context, compactionInstantTime, Option.empty()); table.getMetaClient().reloadActiveTimeline(); - JavaRDD result = table.compact(jsc, compactionInstantTime).getWriteStatuses(); + JavaRDD result = (JavaRDD) table.compact(context, compactionInstantTime).getWriteStatuses(); // Verify that all partition paths are present in the WriteStatus result for (String partitionPath : dataGen.getPartitionPaths()) { diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java similarity index 93% rename from hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java index 4cbb4610d..066a9656f 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.client.HoodieReadClient; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -50,7 +50,7 @@ public class TestInlineCompaction extends CompactionTestBase { public void testCompactionIsNotScheduledEarly() throws Exception { // Given: make two commits HoodieWriteConfig cfg = getConfigForInlineCompaction(3); - try (HoodieWriteClient writeClient = getHoodieWriteClient(cfg)) { + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(cfg)) { List records = dataGen.generateInserts("000", 100); HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); runNextDeltaCommits(writeClient, readClient, Arrays.asList("000", "001"), records, cfg, true, new ArrayList<>()); @@ -67,7 +67,7 @@ public class TestInlineCompaction extends CompactionTestBase { HoodieWriteConfig cfg = getConfigForInlineCompaction(3); List instants = IntStream.range(0, 2).mapToObj(i -> HoodieActiveTimeline.createNewInstantTime()).collect(Collectors.toList()); - try (HoodieWriteClient writeClient = getHoodieWriteClient(cfg)) { + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(cfg)) { List records = dataGen.generateInserts(instants.get(0), 100); HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>()); @@ -92,7 +92,7 @@ public class TestInlineCompaction extends CompactionTestBase { .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) .build(); List instants = CollectionUtils.createImmutableList("000", "001"); - try (HoodieWriteClient writeClient = getHoodieWriteClient(cfg)) { + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(cfg)) { List records = dataGen.generateInserts(instants.get(0), 100); HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>()); @@ -103,7 +103,7 @@ public class TestInlineCompaction extends CompactionTestBase { // When: a third commit happens HoodieWriteConfig inlineCfg = getConfigForInlineCompaction(2); - try (HoodieWriteClient writeClient = getHoodieWriteClient(inlineCfg)) { + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(inlineCfg)) { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath()); createNextDeltaCommit("003", dataGen.generateUpdates("003", 100), writeClient, metaClient, inlineCfg, false); } diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/HoodieClientRollbackTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/HoodieClientRollbackTestBase.java similarity index 97% rename from hudi-client/src/test/java/org/apache/hudi/table/action/rollback/HoodieClientRollbackTestBase.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/HoodieClientRollbackTestBase.java index ed97f43cc..eb0e8711a 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/HoodieClientRollbackTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/HoodieClientRollbackTestBase.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.action.rollback; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieFileGroup; @@ -50,7 +50,7 @@ public class HoodieClientRollbackTestBase extends HoodieClientTestBase { dataGen = new HoodieTestDataGenerator(new String[]{DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH}); //1. prepare data HoodieTestDataGenerator.writePartitionMetadata(fs, new String[]{DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH}, basePath); - HoodieWriteClient client = getHoodieWriteClient(cfg); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); /** * Write 1 (only inserts) */ diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java similarity index 94% rename from hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java index a5bf03288..e14dbf9c6 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java @@ -83,8 +83,8 @@ public class TestCopyOnWriteRollbackActionExecutor extends HoodieClientRollbackT HoodieInstant needRollBackInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "002"); // execute CopyOnWriteRollbackActionExecutor with filelisting mode - CopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new CopyOnWriteRollbackActionExecutor(jsc, table.getConfig(), table, "003", needRollBackInstant, true); - assertFalse(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof MarkerBasedRollbackStrategy); + SparkCopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new SparkCopyOnWriteRollbackActionExecutor(context, table.getConfig(), table, "003", needRollBackInstant, true); + assertFalse(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy); List hoodieRollbackStats = copyOnWriteRollbackActionExecutor.executeRollback(); // assert hoodieRollbackStats @@ -133,7 +133,7 @@ public class TestCopyOnWriteRollbackActionExecutor extends HoodieClientRollbackT List secondPartitionCommit2FileSlices = new ArrayList<>(); HoodieWriteConfig cfg = getConfigBuilder().withRollbackUsingMarkers(isUsingMarkers).withAutoCommit(false).build(); this.twoUpsertCommitDataWithTwoPartitions(firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices, cfg, !isUsingMarkers); - HoodieTable table = this.getHoodieTable(metaClient, cfg); + HoodieTable table = this.getHoodieTable(metaClient, cfg); //2. rollback HoodieInstant commitInstant; @@ -143,11 +143,11 @@ public class TestCopyOnWriteRollbackActionExecutor extends HoodieClientRollbackT commitInstant = table.getCompletedCommitTimeline().lastInstant().get(); } - CopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new CopyOnWriteRollbackActionExecutor(jsc, cfg, table, "003", commitInstant, false); + SparkCopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new SparkCopyOnWriteRollbackActionExecutor(context, cfg, table, "003", commitInstant, false); if (!isUsingMarkers) { - assertFalse(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof MarkerBasedRollbackStrategy); + assertFalse(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy); } else { - assertTrue(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof MarkerBasedRollbackStrategy); + assertTrue(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy); } Map rollbackMetadata = copyOnWriteRollbackActionExecutor.execute().getPartitionMetadata(); diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java similarity index 93% rename from hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java index 8c4da54b2..4ab189adf 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java @@ -24,7 +24,7 @@ import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestTable; -import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hadoop.fs.FileStatus; @@ -69,7 +69,7 @@ public class TestMarkerBasedRollbackStrategy extends HoodieClientTestBase { .withMarkerFile("partA", f2, IOType.CREATE); // when - List stats = new MarkerBasedRollbackStrategy(HoodieTable.create(metaClient, getConfig(), hadoopConf), jsc, getConfig(), "002") + List stats = new SparkMarkerBasedRollbackStrategy(HoodieSparkTable.create(getConfig(), context, metaClient), context, getConfig(), "002") .execute(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001")); // then: ensure files are deleted correctly, non-existent files reported as failed deletes @@ -102,7 +102,7 @@ public class TestMarkerBasedRollbackStrategy extends HoodieClientTestBase { .withMarkerFile("partB", f4, IOType.APPEND); // when - List stats = new MarkerBasedRollbackStrategy(HoodieTable.create(metaClient, getConfig(), hadoopConf), jsc, getConfig(), "002") + List stats = new SparkMarkerBasedRollbackStrategy(HoodieSparkTable.create(getConfig(), context, metaClient), context, getConfig(), "002") .execute(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001")); // then: ensure files are deleted, rollback block is appended (even if append does not exist) diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java similarity index 95% rename from hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java index 1edebcd96..f64ce8be7 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java @@ -88,8 +88,8 @@ public class TestMergeOnReadRollbackActionExecutor extends HoodieClientRollbackT //2. rollback HoodieInstant rollBackInstant = new HoodieInstant(isUsingMarkers, HoodieTimeline.DELTA_COMMIT_ACTION, "002"); - MergeOnReadRollbackActionExecutor mergeOnReadRollbackActionExecutor = new MergeOnReadRollbackActionExecutor( - jsc, + SparkMergeOnReadRollbackActionExecutor mergeOnReadRollbackActionExecutor = new SparkMergeOnReadRollbackActionExecutor( + context, cfg, table, "003", @@ -97,9 +97,9 @@ public class TestMergeOnReadRollbackActionExecutor extends HoodieClientRollbackT true); // assert is filelist mode if (!isUsingMarkers) { - assertFalse(mergeOnReadRollbackActionExecutor.getRollbackStrategy() instanceof MarkerBasedRollbackStrategy); + assertFalse(mergeOnReadRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy); } else { - assertTrue(mergeOnReadRollbackActionExecutor.getRollbackStrategy() instanceof MarkerBasedRollbackStrategy); + assertTrue(mergeOnReadRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy); } //3. assert the rollback stat @@ -144,8 +144,8 @@ public class TestMergeOnReadRollbackActionExecutor extends HoodieClientRollbackT public void testFailForCompletedInstants() { Assertions.assertThrows(IllegalArgumentException.class, () -> { HoodieInstant rollBackInstant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "002"); - new MergeOnReadRollbackActionExecutor( - jsc, + new SparkMergeOnReadRollbackActionExecutor( + context, getConfigBuilder().build(), getHoodieTable(metaClient, getConfigBuilder().build()), "003", diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java diff --git a/hudi-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java similarity index 95% rename from hudi-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java index e1dc4cefa..b8e02b905 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.upgrade; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieFileGroup; @@ -69,7 +69,7 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; /** - * Unit tests {@link UpgradeDowngrade}. + * Unit tests {@link SparkUpgradeDowngrade}. */ public class TestUpgradeDowngrade extends HoodieClientTestBase { @@ -110,14 +110,14 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase { metaClient = HoodieTestUtils.init(dfs.getConf(), dfsBasePath, HoodieTableType.MERGE_ON_READ); } HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(false).withProps(params).build(); - HoodieWriteClient client = getHoodieWriteClient(cfg); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); // prepare data. Make 2 commits, in which 2nd is not committed. List firstPartitionCommit2FileSlices = new ArrayList<>(); List secondPartitionCommit2FileSlices = new ArrayList<>(); Pair, List> inputRecords = twoUpsertCommitDataWithTwoPartitions(firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices, cfg, client, false); - HoodieTable table = this.getHoodieTable(metaClient, cfg); + HoodieTable table = this.getHoodieTable(metaClient, cfg); HoodieInstant commitInstant = table.getPendingCommitTimeline().lastInstant().get(); // delete one of the marker files in 2nd commit if need be. @@ -137,7 +137,7 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase { } // should re-create marker files for 2nd commit since its pending. - UpgradeDowngrade.run(metaClient, HoodieTableVersion.ONE, cfg, jsc, null); + new SparkUpgradeDowngrade(metaClient, cfg, context).run(metaClient, HoodieTableVersion.ONE, cfg, context, null); // assert marker files assertMarkerFilesForUpgrade(table, commitInstant, firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices); @@ -152,7 +152,7 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase { // Check the entire dataset has all records only from 1st commit and 3rd commit since 2nd is expected to be rolledback. assertRows(inputRecords.getKey(), thirdBatch); if (induceResiduesFromPrevUpgrade) { - assertFalse(dfs.exists(new Path(metaClient.getMetaPath(), UpgradeDowngrade.HOODIE_UPDATED_PROPERTY_FILE))); + assertFalse(dfs.exists(new Path(metaClient.getMetaPath(), SparkUpgradeDowngrade.HOODIE_UPDATED_PROPERTY_FILE))); } } @@ -166,14 +166,14 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase { metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ); } HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(false).withProps(params).build(); - HoodieWriteClient client = getHoodieWriteClient(cfg); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); // prepare data. Make 2 commits, in which 2nd is not committed. List firstPartitionCommit2FileSlices = new ArrayList<>(); List secondPartitionCommit2FileSlices = new ArrayList<>(); Pair, List> inputRecords = twoUpsertCommitDataWithTwoPartitions(firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices, cfg, client, false); - HoodieTable table = this.getHoodieTable(metaClient, cfg); + HoodieTable table = this.getHoodieTable(metaClient, cfg); HoodieInstant commitInstant = table.getPendingCommitTimeline().lastInstant().get(); // delete one of the marker files in 2nd commit if need be. @@ -189,7 +189,7 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase { prepForDowngrade(); // downgrade should be performed. all marker files should be deleted - UpgradeDowngrade.run(metaClient, HoodieTableVersion.ZERO, cfg, jsc, null); + new SparkUpgradeDowngrade(metaClient, cfg, context).run(metaClient, HoodieTableVersion.ZERO, cfg, context, null); // assert marker files assertMarkerFilesForDowngrade(table, commitInstant); @@ -285,7 +285,7 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase { params.put(HOODIE_TABLE_TYPE_PROP_NAME, HoodieTableType.MERGE_ON_READ.name()); } HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(enableMarkedBasedRollback).withProps(params).build(); - HoodieWriteClient client = getHoodieWriteClient(cfg, true); + SparkRDDWriteClient client = getHoodieWriteClient(cfg, true); client.startCommitWithTime(newCommitTime); @@ -330,13 +330,13 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase { * @param firstPartitionCommit2FileSlices list to hold file slices in first partition. * @param secondPartitionCommit2FileSlices list of hold file slices from second partition. * @param cfg instance of {@link HoodieWriteConfig} - * @param client instance of {@link HoodieWriteClient} to use. + * @param client instance of {@link SparkRDDWriteClient} to use. * @param commitSecondUpsert true if 2nd commit needs to be committed. false otherwise. * @return a pair of list of records from 1st and 2nd batch. */ private Pair, List> twoUpsertCommitDataWithTwoPartitions(List firstPartitionCommit2FileSlices, List secondPartitionCommit2FileSlices, - HoodieWriteConfig cfg, HoodieWriteClient client, + HoodieWriteConfig cfg, SparkRDDWriteClient client, boolean commitSecondUpsert) throws IOException { //just generate two partitions dataGen = new HoodieTestDataGenerator(new String[] {DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH}); @@ -398,7 +398,7 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase { private void createResidualFile() throws IOException { Path propertyFile = new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE); - Path updatedPropertyFile = new Path(metaClient.getMetaPath() + "/" + UpgradeDowngrade.HOODIE_UPDATED_PROPERTY_FILE); + Path updatedPropertyFile = new Path(metaClient.getMetaPath() + "/" + SparkUpgradeDowngrade.HOODIE_UPDATED_PROPERTY_FILE); // Step1: Copy hoodie.properties to hoodie.properties.orig FileUtil.copy(metaClient.getFs(), propertyFile, metaClient.getFs(), updatedPropertyFile, diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/Assertions.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/Assertions.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/testutils/Assertions.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/Assertions.java diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java similarity index 90% rename from hudi-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java index 27db072c7..1020e932b 100644 --- a/hudi-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java @@ -20,13 +20,14 @@ package org.apache.hudi.testutils; import org.apache.hudi.client.HoodieReadClient; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.minicluster.HdfsTestService; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.testutils.providers.DFSProvider; import org.apache.hudi.testutils.providers.HoodieMetaClientProvider; import org.apache.hudi.testutils.providers.HoodieWriteClientProvider; @@ -58,6 +59,7 @@ public class FunctionalTestHarness implements SparkProvider, DFSProvider, Hoodie private static transient SparkSession spark; private static transient SQLContext sqlContext; private static transient JavaSparkContext jsc; + protected static transient HoodieSparkEngineContext context; private static transient HdfsTestService hdfsTestService; private static transient MiniDFSCluster dfsCluster; @@ -104,6 +106,11 @@ public class FunctionalTestHarness implements SparkProvider, DFSProvider, Hoodie return dfs.getWorkingDirectory(); } + @Override + public HoodieEngineContext context() { + return context; + } + public HoodieTableMetaClient getHoodieMetaClient(Configuration hadoopConf, String basePath) throws IOException { return getHoodieMetaClient(hadoopConf, basePath, new Properties()); } @@ -118,8 +125,8 @@ public class FunctionalTestHarness implements SparkProvider, DFSProvider, Hoodie } @Override - public HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws IOException { - return new HoodieWriteClient(jsc, cfg, false, HoodieIndex.createIndex(cfg)); + public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws IOException { + return new SparkRDDWriteClient(context(), cfg, false); } @BeforeEach @@ -127,11 +134,12 @@ public class FunctionalTestHarness implements SparkProvider, DFSProvider, Hoodie initialized = spark != null && hdfsTestService != null; if (!initialized) { SparkConf sparkConf = conf(); - HoodieWriteClient.registerClasses(sparkConf); + SparkRDDWriteClient.registerClasses(sparkConf); HoodieReadClient.addHoodieSupport(sparkConf); spark = SparkSession.builder().config(sparkConf).getOrCreate(); sqlContext = spark.sqlContext(); jsc = new JavaSparkContext(spark.sparkContext()); + context = new HoodieSparkEngineContext(jsc); hdfsTestService = new HdfsTestService(); dfsCluster = hdfsTestService.start(true); diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java similarity index 91% rename from hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index c176dcddb..1caf9c04f 100644 --- a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -18,7 +18,7 @@ package org.apache.hudi.testutils; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.fs.ConsistencyGuardConfig; @@ -40,9 +40,9 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; -import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -138,8 +138,8 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build()); } - public HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) { - HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf); + public HoodieSparkTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) { + HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient); ((SyncableFileSystemView) (table.getSliceView())).reset(); return table; } @@ -218,11 +218,11 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { private Function2, String, Integer> wrapRecordsGenFunctionForPreppedCalls( final HoodieWriteConfig writeConfig, final Function2, String, Integer> recordGenFunction) { return (commit, numRecords) -> { - final HoodieIndex index = HoodieIndex.createIndex(writeConfig); + final SparkHoodieIndex index = SparkHoodieIndex.createIndex(writeConfig); List records = recordGenFunction.apply(commit, numRecords); final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath, true); - HoodieTable table = HoodieTable.create(metaClient, writeConfig, hadoopConf); - JavaRDD taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, table); + HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient); + JavaRDD taggedRecords = index.tagLocation(jsc.parallelize(records, 1), context, table); return taggedRecords.collect(); }; } @@ -239,13 +239,13 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { private Function> wrapDeleteKeysGenFunctionForPreppedCalls( final HoodieWriteConfig writeConfig, final Function> keyGenFunction) { return (numRecords) -> { - final HoodieIndex index = HoodieIndex.createIndex(writeConfig); + final SparkHoodieIndex index = SparkHoodieIndex.createIndex(writeConfig); List records = keyGenFunction.apply(numRecords); final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath, true); - HoodieTable table = HoodieTable.create(metaClient, writeConfig, hadoopConf); + HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient); JavaRDD recordsToDelete = jsc.parallelize(records, 1) .map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload())); - JavaRDD taggedRecords = index.tagLocation(recordsToDelete, jsc, table); + JavaRDD taggedRecords = index.tagLocation(recordsToDelete, context, table); return taggedRecords.map(record -> record.getKey()).collect(); }; } @@ -300,9 +300,9 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { * @return RDD of write-status * @throws Exception in case of error */ - public JavaRDD insertFirstBatch(HoodieWriteConfig writeConfig, HoodieWriteClient client, String newCommitTime, + public JavaRDD insertFirstBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, - Function3, HoodieWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, boolean assertForCommit, int expRecordsInThisCommit) throws Exception { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); @@ -328,9 +328,9 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { * @return RDD of write-status * @throws Exception in case of error */ - public JavaRDD insertBatch(HoodieWriteConfig writeConfig, HoodieWriteClient client, String newCommitTime, + public JavaRDD insertBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, - Function3, HoodieWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); @@ -358,11 +358,11 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { * @return RDD of write-status * @throws Exception in case of error */ - public JavaRDD updateBatch(HoodieWriteConfig writeConfig, HoodieWriteClient client, String newCommitTime, - String prevCommitTime, Option> commitTimesBetweenPrevAndNew, String initCommitTime, - int numRecordsInThisCommit, - Function3, HoodieWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception { + public JavaRDD updateBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, + String prevCommitTime, Option> commitTimesBetweenPrevAndNew, String initCommitTime, + int numRecordsInThisCommit, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateUniqueUpdates); @@ -388,10 +388,10 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { * @return RDD of write-status * @throws Exception in case of error */ - public JavaRDD deleteBatch(HoodieWriteConfig writeConfig, HoodieWriteClient client, String newCommitTime, + public JavaRDD deleteBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, String initCommitTime, int numRecordsInThisCommit, - Function3, HoodieWriteClient, JavaRDD, String> deleteFn, boolean isPreppedAPI, + Function3, SparkRDDWriteClient, JavaRDD, String> deleteFn, boolean isPreppedAPI, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords) throws Exception { final Function> keyGenFunction = generateWrapDeleteKeysFn(isPreppedAPI, writeConfig, dataGen::generateUniqueDeletes); @@ -418,10 +418,10 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { * @param expTotalCommits Expected number of commits (including this commit) * @throws Exception in case of error */ - public JavaRDD writeBatch(HoodieWriteClient client, String newCommitTime, String prevCommitTime, + public JavaRDD writeBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function2, String, Integer> recordGenFunction, - Function3, HoodieWriteClient, JavaRDD, String> writeFn, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception { // Write 1 (only inserts) @@ -486,10 +486,10 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { * @param expTotalRecords Expected number of records when scanned * @throws Exception in case of error */ - public JavaRDD deleteBatch(HoodieWriteClient client, String newCommitTime, String prevCommitTime, + public JavaRDD deleteBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, String initCommitTime, int numRecordsInThisCommit, Function> keyGenFunction, - Function3, HoodieWriteClient, JavaRDD, String> deleteFn, + Function3, SparkRDDWriteClient, JavaRDD, String> deleteFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords) throws Exception { // Delete 1 (only deletes) diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java similarity index 79% rename from hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java index f1e3f175e..b10781e3b 100644 --- a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java @@ -18,9 +18,12 @@ package org.apache.hudi.testutils; import org.apache.hudi.client.HoodieReadClient; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; @@ -28,8 +31,9 @@ import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.minicluster.HdfsTestService; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.index.HoodieIndex; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -38,15 +42,20 @@ 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.table.WorkloadStat; 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.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestInfo; +import scala.Tuple2; import java.io.IOException; import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -59,13 +68,14 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im private String testMethodName; protected transient JavaSparkContext jsc = null; + protected transient HoodieSparkEngineContext context = null; protected transient Configuration hadoopConf = null; protected transient SQLContext sqlContext; protected transient FileSystem fs; protected transient HoodieTestDataGenerator dataGen = null; protected transient ExecutorService executorService; protected transient HoodieTableMetaClient metaClient; - protected transient HoodieWriteClient writeClient; + protected transient SparkRDDWriteClient writeClient; protected transient HoodieReadClient readClient; protected transient HoodieTableFileSystemView tableView; @@ -123,6 +133,8 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im // SQLContext stuff sqlContext = new SQLContext(jsc); + context = new HoodieSparkEngineContext(jsc); + hadoopConf = context.getHadoopConf().get(); } /** @@ -149,6 +161,11 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im jsc.stop(); jsc = null; } + + if (context != null) { + LOG.info("Closing spark engine context used in previous test-case"); + context = null; + } } /** @@ -197,7 +214,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im throw new IllegalStateException("The Spark context has not been initialized."); } - metaClient = HoodieTestUtils.init(hadoopConf, basePath, getTableType()); + metaClient = HoodieTestUtils.init(context.getHadoopConf().get(), basePath, getTableType()); } /** @@ -325,26 +342,21 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im } } - public HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) { + public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) { return getHoodieWriteClient(cfg, false); } - public HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit) { - return getHoodieWriteClient(cfg, rollbackInflightCommit, HoodieIndex.createIndex(cfg)); - } - public HoodieReadClient getHoodieReadClient(String basePath) { - readClient = new HoodieReadClient(jsc, basePath, SQLContext.getOrCreate(jsc.sc())); + readClient = new HoodieReadClient(context, basePath, SQLContext.getOrCreate(jsc.sc())); return readClient; } - public HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit, - HoodieIndex index) { + public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit) { if (null != writeClient) { writeClient.close(); writeClient = null; } - writeClient = new HoodieWriteClient(jsc, cfg, rollbackInflightCommit, index); + writeClient = new SparkRDDWriteClient(context, cfg, rollbackInflightCommit); return writeClient; } @@ -362,4 +374,38 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im } return tableView; } + + protected Pair, WorkloadStat> buildProfile(JavaRDD inputRecordsRDD) { + HashMap partitionPathStatMap = new HashMap<>(); + WorkloadStat globalStat = new WorkloadStat(); + + // group the records by partitionPath + currentLocation combination, count the number of + // records in each partition + Map>, Long> partitionLocationCounts = inputRecordsRDD + .mapToPair(record -> new Tuple2<>( + new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record)) + .countByKey(); + + // count the number of both inserts and updates in each partition, update the counts to workLoadStats + for (Map.Entry>, Long> e : partitionLocationCounts.entrySet()) { + String partitionPath = e.getKey()._1(); + Long count = e.getValue(); + Option locOption = e.getKey()._2(); + + if (!partitionPathStatMap.containsKey(partitionPath)) { + partitionPathStatMap.put(partitionPath, new WorkloadStat()); + } + + if (locOption.isPresent()) { + // update + partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count); + globalStat.addUpdates(locOption.get(), count); + } else { + // insert + partitionPathStatMap.get(partitionPath).addInserts(count); + globalStat.addInserts(count); + } + } + return Pair.of(partitionPathStatMap, globalStat); + } } diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java similarity index 100% rename from hudi-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/SparkProvider.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/providers/SparkProvider.java similarity index 94% rename from hudi-client/src/test/java/org/apache/hudi/testutils/providers/SparkProvider.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/providers/SparkProvider.java index cdf5ac4ad..be15dc85d 100644 --- a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/SparkProvider.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/providers/SparkProvider.java @@ -27,7 +27,7 @@ import org.apache.spark.sql.SparkSession; import java.util.Collections; import java.util.Map; -public interface SparkProvider { +public interface SparkProvider extends org.apache.hudi.testutils.providers.HoodieEngineContextProvider { SparkSession spark(); @@ -52,4 +52,4 @@ public interface SparkProvider { default SparkConf conf() { return conf(Collections.emptyMap()); } -} +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire-quiet.properties b/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire-quiet.properties new file mode 100644 index 000000000..2b94ea290 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire-quiet.properties @@ -0,0 +1,30 @@ +### +# 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. +### +log4j.rootLogger=WARN, CONSOLE +log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR + +# CONSOLE is set to be a ConsoleAppender. +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +# CONSOLE uses PatternLayout. +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n +log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter +log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true +log4j.appender.CONSOLE.filter.a.LevelMin=WARN +log4j.appender.CONSOLE.filter.a.LevelMax=FATAL \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire.properties b/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire.properties new file mode 100644 index 000000000..32af46209 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire.properties @@ -0,0 +1,31 @@ +### +# 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. +### +log4j.rootLogger=WARN, CONSOLE +log4j.logger.org.apache=INFO +log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR + +# A1 is set to be a ConsoleAppender. +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n +log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter +log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true +log4j.appender.CONSOLE.filter.a.LevelMin=WARN +log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-client/src/test/resources/testDataGeneratorSchema.txt b/hudi-client/hudi-spark-client/src/test/resources/testDataGeneratorSchema.txt similarity index 100% rename from hudi-client/src/test/resources/testDataGeneratorSchema.txt rename to hudi-client/hudi-spark-client/src/test/resources/testDataGeneratorSchema.txt diff --git a/hudi-client/pom.xml b/hudi-client/pom.xml index 1a051759b..1ab04793a 100644 --- a/hudi-client/pom.xml +++ b/hudi-client/pom.xml @@ -24,294 +24,14 @@ 4.0.0 hudi-client - jar + pom ${project.parent.basedir} - - - - org.jacoco - jacoco-maven-plugin - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - test-compile - - - - false - - - - org.apache.rat - apache-rat-plugin - - - - - - src/main/resources - - - src/test/resources - - - - - - - - org.apache.hudi - hudi-common - ${project.version} - - - org.apache.hudi - hudi-timeline-service - ${project.version} - - - - - log4j - log4j - - - - - org.apache.parquet - parquet-avro - - - - - org.apache.spark - spark-core_${scala.binary.version} - - - org.apache.spark - spark-sql_${scala.binary.version} - - - org.apache.spark - spark-avro_${scala.binary.version} - provided - - - - - io.dropwizard.metrics - metrics-graphite - - - com.rabbitmq - * - - - - - io.dropwizard.metrics - metrics-core - - - io.dropwizard.metrics - metrics-jmx - - - io.prometheus - simpleclient - - - io.prometheus - simpleclient_httpserver - - - io.prometheus - simpleclient_dropwizard - - - io.prometheus - simpleclient_pushgateway - - - - com.beust - jcommander - test - - - - - org.apache.hadoop - hadoop-client - - - javax.servlet - * - - - - - org.apache.hadoop - hadoop-hdfs - tests - - - - org.mortbay.jetty - * - - - javax.servlet.jsp - * - - - javax.servlet - * - - - - - org.apache.hadoop - hadoop-common - tests - - - org.mortbay.jetty - * - - - javax.servlet.jsp - * - - - javax.servlet - * - - - - - - - org.apache.hbase - hbase-client - ${hbase.version} - - - org.apache.hbase - hbase-server - ${hbase.version} - provided - - - - - org.apache.hudi - hudi-common - ${project.version} - tests - test-jar - test - - - org.apache.hudi - hudi-hadoop-mr - ${project.version} - test - - - - - org.apache.hbase - hbase-testing-util - ${hbase.version} - test - - - org.codehaus.jackson - jackson-mapper-asl - - - org.codehaus.jackson - jackson-core-asl - - - javax.xml.bind - * - - - - - - - ${hive.groupid} - hive-exec - ${hive.version} - test - ${hive.exec.classifier} - - - ${hive.groupid} - hive-metastore - ${hive.version} - test - - - - org.junit.jupiter - junit-jupiter-api - test - - - - org.junit.jupiter - junit-jupiter-engine - test - - - - org.junit.vintage - junit-vintage-engine - test - - - - org.junit.jupiter - junit-jupiter-params - test - - - - org.mockito - mockito-junit-jupiter - test - - - - org.junit.platform - junit-platform-runner - test - - - - org.junit.platform - junit-platform-suite-api - test - - - - org.junit.platform - junit-platform-commons - test - - + + hudi-client-common + hudi-spark-client + diff --git a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java deleted file mode 100644 index 06e980db3..000000000 --- a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ /dev/null @@ -1,266 +0,0 @@ -/* - * 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.client; - -import com.codahale.metrics.Timer; -import org.apache.hudi.callback.HoodieWriteCommitCallback; -import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage; -import org.apache.hudi.callback.util.HoodieCommitCallbackFactory; -import org.apache.hudi.client.embedded.EmbeddedTimelineService; -import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.HoodieWriteStat; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.HoodieTableVersion; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.CommitUtils; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieCommitException; -import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.metrics.HoodieMetrics; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.upgrade.UpgradeDowngrade; -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 java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.text.ParseException; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -/** - * Abstract Write Client providing functionality for performing commit, index updates and rollback - * Reused for regular write operations like upsert/insert/bulk-insert.. as well as bootstrap - * @param Sub type of HoodieRecordPayload - */ -public abstract class AbstractHoodieWriteClient extends AbstractHoodieClient { - - private static final Logger LOG = LogManager.getLogger(AbstractHoodieWriteClient.class); - - private final transient HoodieMetrics metrics; - private final transient HoodieIndex index; - - private transient Timer.Context writeContext = null; - private transient WriteOperationType operationType; - private transient HoodieWriteCommitCallback commitCallback; - - public void setOperationType(WriteOperationType operationType) { - this.operationType = operationType; - } - - public WriteOperationType getOperationType() { - return this.operationType; - } - - protected AbstractHoodieWriteClient(JavaSparkContext jsc, HoodieIndex index, HoodieWriteConfig clientConfig, - Option timelineServer) { - super(jsc, clientConfig, timelineServer); - this.metrics = new HoodieMetrics(config, config.getTableName()); - this.index = index; - } - - /** - * Commit changes performed at the given instantTime marker. - */ - public boolean commit(String instantTime, JavaRDD writeStatuses) { - return commit(instantTime, writeStatuses, Option.empty()); - } - - /** - * - * Commit changes performed at the given instantTime marker. - */ - public boolean commit(String instantTime, JavaRDD writeStatuses, - Option> extraMetadata) { - HoodieTableMetaClient metaClient = createMetaClient(false); - String actionType = metaClient.getCommitActionType(); - return commit(instantTime, writeStatuses, extraMetadata, actionType, Collections.emptyMap()); - } - - /** - * Complete changes performed at the given instantTime marker with specified action. - */ - public boolean commit(String instantTime, JavaRDD writeStatuses, - Option> extraMetadata, String commitActionType, Map> partitionToReplacedFileIds) { - List writeStats = writeStatuses.map(WriteStatus::getStat).collect(); - return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds); - } - - public boolean commitStats(String instantTime, List stats, Option> extraMetadata, - String commitActionType) { - return commitStats(instantTime, stats, extraMetadata, commitActionType, Collections.emptyMap()); - } - - public boolean commitStats(String instantTime, List stats, Option> extraMetadata, - String commitActionType, Map> partitionToReplaceFileIds) { - LOG.info("Committing " + instantTime + " action " + commitActionType); - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.create(config, hadoopConf); - - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getSchema(), commitActionType); - // Finalize write - finalizeWrite(table, instantTime, stats); - - try { - activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime), - Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - postCommit(table, metadata, instantTime, extraMetadata); - emitCommitMetrics(instantTime, metadata, commitActionType); - LOG.info("Committed " + instantTime); - } catch (IOException e) { - throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, - e); - } - - // callback if needed. - if (config.writeCommitCallbackOn()) { - if (null == commitCallback) { - commitCallback = HoodieCommitCallbackFactory.create(config); - } - commitCallback.call(new HoodieWriteCommitCallbackMessage(instantTime, config.getTableName(), config.getBasePath())); - } - return true; - } - - void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) { - try { - - if (writeContext != null) { - long durationInMs = metrics.getDurationInMs(writeContext.stop()); - metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(instantTime).getTime(), durationInMs, - metadata, actionType); - writeContext = null; - } - } catch (ParseException e) { - throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime - + "Instant time is not of valid format", e); - } - } - - /** - * Post Commit Hook. Derived classes use this method to perform post-commit processing - * - * @param table table to commit on - * @param metadata Commit Metadata corresponding to committed instant - * @param instantTime Instant Time - * @param extraMetadata Additional Metadata passed by user - */ - protected abstract void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata); - - /** - * Finalize Write operation. - * @param table HoodieTable - * @param instantTime Instant Time - * @param stats Hoodie Write Stat - */ - protected void finalizeWrite(HoodieTable table, String instantTime, List stats) { - try { - final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); - table.finalizeWrite(jsc, instantTime, stats); - if (finalizeCtx != null) { - Option durationInMs = Option.of(metrics.getDurationInMs(finalizeCtx.stop())); - durationInMs.ifPresent(duration -> { - LOG.info("Finalize write elapsed time (milliseconds): " + duration); - metrics.updateFinalizeWriteMetrics(duration, stats.size()); - }); - } - } catch (HoodieIOException ioe) { - throw new HoodieCommitException("Failed to complete commit " + instantTime + " due to finalize errors.", ioe); - } - } - - public HoodieMetrics getMetrics() { - return metrics; - } - - public HoodieIndex getIndex() { - return index; - } - - /** - * Get HoodieTable and init {@link Timer.Context}. - * - * @param operationType write operation type - * @param instantTime current inflight instant time - * @return HoodieTable - */ - protected HoodieTable getTableAndInitCtx(WriteOperationType operationType, String instantTime) { - HoodieTableMetaClient metaClient = createMetaClient(true); - UpgradeDowngrade.run(metaClient, HoodieTableVersion.current(), config, jsc, instantTime); - return getTableAndInitCtx(metaClient, operationType); - } - - private HoodieTable getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) { - if (operationType == WriteOperationType.DELETE) { - setWriteSchemaForDeletes(metaClient); - } - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf); - if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) { - writeContext = metrics.getCommitCtx(); - } else { - writeContext = metrics.getDeltaCommitCtx(); - } - return table; - } - - /** - * Sets write schema from last instant since deletes may not have schema set in the config. - */ - private void setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) { - try { - HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); - Option lastInstant = - activeTimeline.filterCompletedInstants().filter(s -> s.getAction().equals(metaClient.getCommitActionType())) - .lastInstant(); - if (lastInstant.isPresent()) { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( - activeTimeline.getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class); - if (commitMetadata.getExtraMetadata().containsKey(HoodieCommitMetadata.SCHEMA_KEY)) { - config.setSchema(commitMetadata.getExtraMetadata().get(HoodieCommitMetadata.SCHEMA_KEY)); - } else { - throw new HoodieIOException("Latest commit does not have any schema in commit metadata"); - } - } else { - throw new HoodieIOException("Deletes issued without any prior commits"); - } - } catch (IOException e) { - throw new HoodieIOException("IOException thrown while reading last commit metadata", e); - } - } - - @Override - public void close() { - // Stop timeline-server if running - super.close(); - // Calling this here releases any resources used by your index, so make sure to finish any related operations - // before this point - this.index.close(); - } -} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/utils/ClientUtils.java b/hudi-client/src/main/java/org/apache/hudi/client/utils/ClientUtils.java deleted file mode 100644 index 0cfe0c1bb..000000000 --- a/hudi-client/src/main/java/org/apache/hudi/client/utils/ClientUtils.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.client.utils; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; - -public class ClientUtils { - - /** - * Create Consistency Aware MetaClient. - * - * @param hadoopConf Configuration - * @param config HoodieWriteConfig - * @param loadActiveTimelineOnLoad early loading of timeline - */ - public static HoodieTableMetaClient createMetaClient(Configuration hadoopConf, HoodieWriteConfig config, - boolean loadActiveTimelineOnLoad) { - return new HoodieTableMetaClient(hadoopConf, config.getBasePath(), loadActiveTimelineOnLoad, - config.getConsistencyGuardConfig(), - Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))); - } -} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/utils/SparkConfigUtils.java b/hudi-client/src/main/java/org/apache/hudi/client/utils/SparkConfigUtils.java deleted file mode 100644 index 0a6b6080b..000000000 --- a/hudi-client/src/main/java/org/apache/hudi/client/utils/SparkConfigUtils.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * 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.client.utils; - -import org.apache.hudi.config.HoodieIndexConfig; - -import org.apache.spark.SparkEnv; -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.util.Utils; - -import java.util.Properties; - -import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES; -import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION; -import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE; -import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES; -import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP; -import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP; -import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP; -import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP; -import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL; - -/** - * Spark config utils. - */ -public class SparkConfigUtils { - - /** - * Dynamic calculation of max memory to use for for spillable map. user.available.memory = spark.executor.memory * - * (1 - spark.memory.fraction) spillable.available.memory = user.available.memory * hoodie.memory.fraction. Anytime - * the spark.executor.memory or the spark.memory.fraction is changed, the memory used for spillable map changes - * accordingly - */ - public static long getMaxMemoryAllowedForMerge(String maxMemoryFraction) { - final String SPARK_EXECUTOR_MEMORY_PROP = "spark.executor.memory"; - final String SPARK_EXECUTOR_MEMORY_FRACTION_PROP = "spark.memory.fraction"; - // This is hard-coded in spark code {@link - // https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/ - // spark/memory/UnifiedMemoryManager.scala#L231} so have to re-define this here - final String DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION = "0.6"; - // This is hard-coded in spark code {@link - // https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/ - // spark/SparkContext.scala#L471} so have to re-define this here - final String DEFAULT_SPARK_EXECUTOR_MEMORY_MB = "1024"; // in MB - if (SparkEnv.get() != null) { - // 1 GB is the default conf used by Spark, look at SparkContext.scala - long executorMemoryInBytes = Utils.memoryStringToMb( - SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024 * 1024L; - // 0.6 is the default value used by Spark, - // look at {@link - // https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507} - double memoryFraction = Double.parseDouble( - SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION)); - double maxMemoryFractionForMerge = Double.parseDouble(maxMemoryFraction); - double userAvailableMemory = executorMemoryInBytes * (1 - memoryFraction); - long maxMemoryForMerge = (long) Math.floor(userAvailableMemory * maxMemoryFractionForMerge); - return Math.max(DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES, maxMemoryForMerge); - } else { - return DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES; - } - } - - public static StorageLevel getWriteStatusStorageLevel(Properties properties) { - return StorageLevel.fromString(properties.getProperty(WRITE_STATUS_STORAGE_LEVEL)); - } - - public static StorageLevel getBloomIndexInputStorageLevel(Properties properties) { - return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL)); - } - - public static long getMaxMemoryPerPartitionMerge(Properties properties) { - if (properties.containsKey(MAX_MEMORY_FOR_MERGE_PROP)) { - return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_MERGE_PROP)); - } - String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE); - return getMaxMemoryAllowedForMerge(fraction); - } - - public static long getMaxMemoryPerCompaction(Properties properties) { - if (properties.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP)) { - return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_COMPACTION_PROP)); - } - String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION); - return getMaxMemoryAllowedForMerge(fraction); - } - - public static StorageLevel getSimpleIndexInputStorageLevel(Properties properties) { - return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL)); - } -} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java deleted file mode 100644 index a4bcbbf83..000000000 --- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java +++ /dev/null @@ -1,216 +0,0 @@ -/* - * 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.table; - -import org.apache.hudi.avro.model.HoodieCleanMetadata; -import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.avro.model.HoodieRestoreMetadata; -import org.apache.hudi.avro.model.HoodieRollbackMetadata; -import org.apache.hudi.avro.model.HoodieSavepointMetadata; -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieNotSupportedException; -import org.apache.hudi.exception.HoodieUpsertException; -import org.apache.hudi.io.HoodieCreateHandle; -import org.apache.hudi.io.HoodieMergeHandle; -import org.apache.hudi.io.HoodieSortedMergeHandle; -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.bootstrap.BootstrapCommitActionExecutor; -import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; -import org.apache.hudi.table.action.clean.CleanActionExecutor; -import org.apache.hudi.table.action.commit.BulkInsertCommitActionExecutor; -import org.apache.hudi.table.action.commit.BulkInsertPreppedCommitActionExecutor; -import org.apache.hudi.table.action.commit.DeleteCommitActionExecutor; -import org.apache.hudi.table.action.commit.InsertCommitActionExecutor; -import org.apache.hudi.table.action.commit.InsertOverwriteCommitActionExecutor; -import org.apache.hudi.table.action.commit.InsertPreppedCommitActionExecutor; -import org.apache.hudi.table.action.commit.MergeHelper; -import org.apache.hudi.table.action.commit.UpsertCommitActionExecutor; -import org.apache.hudi.table.action.commit.UpsertPreppedCommitActionExecutor; -import org.apache.hudi.table.action.restore.CopyOnWriteRestoreActionExecutor; -import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; -import org.apache.hudi.table.action.savepoint.SavepointActionExecutor; - -import org.apache.hadoop.conf.Configuration; -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 java.io.IOException; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -/** - * Implementation of a very heavily read-optimized Hoodie Table where, all data is stored in base files, with - * zero read amplification. - *

- * INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing file, to expand it - *

- * UPDATES - Produce a new version of the file, just replacing the updated records with new values - */ -public class HoodieCopyOnWriteTable extends HoodieTable { - - private static final Logger LOG = LogManager.getLogger(HoodieCopyOnWriteTable.class); - - public HoodieCopyOnWriteTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) { - super(config, hadoopConf, metaClient); - } - - @Override - public HoodieWriteMetadata upsert(JavaSparkContext jsc, String instantTime, JavaRDD> records) { - return new UpsertCommitActionExecutor<>(jsc, config, this, instantTime, records).execute(); - } - - @Override - public HoodieWriteMetadata insert(JavaSparkContext jsc, String instantTime, JavaRDD> records) { - return new InsertCommitActionExecutor<>(jsc, config, this, instantTime, records).execute(); - } - - @Override - public HoodieWriteMetadata bulkInsert(JavaSparkContext jsc, String instantTime, JavaRDD> records, - Option userDefinedBulkInsertPartitioner) { - return new BulkInsertCommitActionExecutor(jsc, config, - this, instantTime, records, userDefinedBulkInsertPartitioner).execute(); - } - - @Override - public HoodieWriteMetadata delete(JavaSparkContext jsc, String instantTime, JavaRDD keys) { - return new DeleteCommitActionExecutor<>(jsc, config, this, instantTime, keys).execute(); - } - - @Override - public HoodieWriteMetadata upsertPrepped(JavaSparkContext jsc, String instantTime, - JavaRDD> preppedRecords) { - return new UpsertPreppedCommitActionExecutor<>(jsc, config, this, instantTime, preppedRecords).execute(); - } - - @Override - public HoodieWriteMetadata insertPrepped(JavaSparkContext jsc, String instantTime, - JavaRDD> preppedRecords) { - return new InsertPreppedCommitActionExecutor<>(jsc, config, this, instantTime, preppedRecords).execute(); - } - - @Override - public HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime, - JavaRDD> preppedRecords, Option userDefinedBulkInsertPartitioner) { - return new BulkInsertPreppedCommitActionExecutor(jsc, config, - this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); - } - - @Override - public HoodieWriteMetadata insertOverwrite(JavaSparkContext jsc, String instantTime, - JavaRDD> records) { - return new InsertOverwriteCommitActionExecutor<>(jsc, config, this, instantTime, records).execute(); - } - - @Override - public Option scheduleCompaction(JavaSparkContext jsc, String instantTime, Option> extraMetadata) { - throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); - } - - @Override - public HoodieWriteMetadata compact(JavaSparkContext jsc, String compactionInstantTime) { - throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); - } - - @Override - public HoodieBootstrapWriteMetadata bootstrap(JavaSparkContext jsc, Option> extraMetadata) { - return new BootstrapCommitActionExecutor(jsc, config, this, extraMetadata).execute(); - } - - @Override - public void rollbackBootstrap(JavaSparkContext jsc, String instantTime) { - new CopyOnWriteRestoreActionExecutor(jsc, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); - } - - public Iterator> handleUpdate(String instantTime, String partitionPath, String fileId, - Map> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException { - // these are updates - HoodieMergeHandle upsertHandle = getUpdateHandle(instantTime, partitionPath, fileId, keyToNewRecords, oldDataFile); - return handleUpdateInternal(upsertHandle, instantTime, fileId); - } - - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String instantTime, - String fileId) throws IOException { - if (upsertHandle.getOldFilePath() == null) { - throw new HoodieUpsertException( - "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); - } else { - MergeHelper.runMerge(this, upsertHandle); - } - - // TODO(vc): This needs to be revisited - if (upsertHandle.getWriteStatus().getPartitionPath() == null) { - LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " - + upsertHandle.getWriteStatus()); - } - return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator(); - } - - protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId, - Map> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) { - if (requireSortedRecords()) { - return new HoodieSortedMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId, - dataFileToBeMerged, sparkTaskContextSupplier); - } else { - return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId, - dataFileToBeMerged, sparkTaskContextSupplier); - } - } - - public Iterator> handleInsert(String instantTime, String partitionPath, String fileId, - Map> recordMap) { - HoodieCreateHandle createHandle = - new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordMap, sparkTaskContextSupplier); - createHandle.write(); - return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator(); - } - - @Override - public HoodieCleanMetadata clean(JavaSparkContext jsc, String cleanInstantTime) { - return new CleanActionExecutor(jsc, config, this, cleanInstantTime).execute(); - } - - @Override - public HoodieRollbackMetadata rollback(JavaSparkContext jsc, String rollbackInstantTime, HoodieInstant commitInstant, boolean deleteInstants) { - return new CopyOnWriteRollbackActionExecutor(jsc, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute(); - } - - @Override - public HoodieSavepointMetadata savepoint(JavaSparkContext jsc, String instantToSavepoint, String user, String comment) { - return new SavepointActionExecutor(jsc, config, this, instantToSavepoint, user, comment).execute(); - } - - @Override - public HoodieRestoreMetadata restore(JavaSparkContext jsc, String restoreInstantTime, String instantToRestore) { - return new CopyOnWriteRestoreActionExecutor(jsc, config, this, restoreInstantTime, instantToRestore).execute(); - } - -} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieMergeOnReadTable.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieMergeOnReadTable.java deleted file mode 100644 index a236cdb94..000000000 --- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieMergeOnReadTable.java +++ /dev/null @@ -1,161 +0,0 @@ -/* - * 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.table; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.avro.model.HoodieRestoreMetadata; -import org.apache.hudi.avro.model.HoodieRollbackMetadata; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.HoodieWriteStat; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; - -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.bootstrap.BootstrapDeltaCommitActionExecutor; -import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; -import org.apache.hudi.table.action.compact.RunCompactionActionExecutor; -import org.apache.hudi.table.action.deltacommit.BulkInsertDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.BulkInsertPreppedDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.DeleteDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.InsertDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.InsertPreppedDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.UpsertDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.UpsertPreppedDeltaCommitActionExecutor; -import org.apache.hudi.table.action.compact.ScheduleCompactionActionExecutor; -import org.apache.hudi.table.action.restore.MergeOnReadRestoreActionExecutor; -import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; - -import java.util.List; -import java.util.Map; - -/** - * Implementation of a more real-time Hoodie Table the provides tradeoffs on read and write cost/amplification. - * - *

- * INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the - * smallest existing file, to expand it - *

- *

- * UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the log file into the - * base file. - *

- *

- * WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an attempted commit - * action - *

- */ -public class HoodieMergeOnReadTable extends HoodieCopyOnWriteTable { - - HoodieMergeOnReadTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) { - super(config, hadoopConf, metaClient); - } - - @Override - public HoodieWriteMetadata upsert(JavaSparkContext jsc, String instantTime, JavaRDD> records) { - return new UpsertDeltaCommitActionExecutor<>(jsc, config, this, instantTime, records).execute(); - } - - @Override - public HoodieWriteMetadata insert(JavaSparkContext jsc, String instantTime, JavaRDD> records) { - return new InsertDeltaCommitActionExecutor<>(jsc, config, this, instantTime, records).execute(); - } - - @Override - public HoodieWriteMetadata bulkInsert(JavaSparkContext jsc, String instantTime, JavaRDD> records, - Option userDefinedBulkInsertPartitioner) { - return new BulkInsertDeltaCommitActionExecutor(jsc, config, - this, instantTime, records, userDefinedBulkInsertPartitioner).execute(); - } - - @Override - public HoodieWriteMetadata delete(JavaSparkContext jsc, String instantTime, JavaRDD keys) { - return new DeleteDeltaCommitActionExecutor<>(jsc, config, this, instantTime, keys).execute(); - } - - @Override - public HoodieWriteMetadata upsertPrepped(JavaSparkContext jsc, String instantTime, - JavaRDD> preppedRecords) { - return new UpsertPreppedDeltaCommitActionExecutor<>(jsc, config, this, instantTime, preppedRecords).execute(); - } - - @Override - public HoodieWriteMetadata insertPrepped(JavaSparkContext jsc, String instantTime, - JavaRDD> preppedRecords) { - return new InsertPreppedDeltaCommitActionExecutor<>(jsc, config, this, instantTime, preppedRecords).execute(); - } - - @Override - public HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime, - JavaRDD> preppedRecords, Option userDefinedBulkInsertPartitioner) { - return new BulkInsertPreppedDeltaCommitActionExecutor(jsc, config, - this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); - } - - @Override - public Option scheduleCompaction(JavaSparkContext jsc, String instantTime, Option> extraMetadata) { - ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor( - jsc, config, this, instantTime, extraMetadata); - return scheduleCompactionExecutor.execute(); - } - - @Override - public HoodieWriteMetadata compact(JavaSparkContext jsc, String compactionInstantTime) { - RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor(jsc, config, this, compactionInstantTime); - return compactionExecutor.execute(); - } - - @Override - public HoodieBootstrapWriteMetadata bootstrap(JavaSparkContext jsc, Option> extraMetadata) { - return new BootstrapDeltaCommitActionExecutor(jsc, config, this, extraMetadata).execute(); - } - - @Override - public void rollbackBootstrap(JavaSparkContext jsc, String instantTime) { - new MergeOnReadRestoreActionExecutor(jsc, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); - } - - @Override - public HoodieRollbackMetadata rollback(JavaSparkContext jsc, - String rollbackInstantTime, - HoodieInstant commitInstant, - boolean deleteInstants) { - return new MergeOnReadRollbackActionExecutor(jsc, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute(); - } - - @Override - public HoodieRestoreMetadata restore(JavaSparkContext jsc, String restoreInstantTime, String instantToRestore) { - return new MergeOnReadRestoreActionExecutor(jsc, config, this, restoreInstantTime, instantToRestore).execute(); - } - - @Override - public void finalizeWrite(JavaSparkContext jsc, String instantTs, List stats) - throws HoodieIOException { - // delegate to base class for MOR tables - super.finalizeWrite(jsc, instantTs, stats); - } -} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/WorkloadProfile.java b/hudi-client/src/main/java/org/apache/hudi/table/WorkloadProfile.java deleted file mode 100644 index 8179c90a3..000000000 --- a/hudi-client/src/main/java/org/apache/hudi/table/WorkloadProfile.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * 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.table; - -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.Option; - -import org.apache.spark.api.java.JavaRDD; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - -import scala.Tuple2; - -/** - * Information about incoming records for upsert/insert obtained either via sampling or introspecting the data fully. - *

- * TODO(vc): Think about obtaining this directly from index.tagLocation - */ -public class WorkloadProfile implements Serializable { - - /** - * Input workload. - */ - private final JavaRDD> taggedRecords; - - /** - * Computed workload profile. - */ - private final HashMap partitionPathStatMap; - - /** - * Global workloadStat. - */ - private final WorkloadStat globalStat; - - public WorkloadProfile(JavaRDD> taggedRecords) { - this.taggedRecords = taggedRecords; - this.partitionPathStatMap = new HashMap<>(); - this.globalStat = new WorkloadStat(); - buildProfile(); - } - - /** - * Method help to build WorkloadProfile. - */ - private void buildProfile() { - // group the records by partitionPath + currentLocation combination, count the number of - // records in each partition - Map>, Long> partitionLocationCounts = taggedRecords - .mapToPair(record -> new Tuple2<>( - new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record)) - .countByKey(); - - // count the number of both inserts and updates in each partition, update the counts to workLoadStats - for (Map.Entry>, Long> e : partitionLocationCounts.entrySet()) { - String partitionPath = e.getKey()._1(); - Long count = e.getValue(); - Option locOption = e.getKey()._2(); - - if (!partitionPathStatMap.containsKey(partitionPath)) { - partitionPathStatMap.put(partitionPath, new WorkloadStat()); - } - - if (locOption.isPresent()) { - // update - partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count); - globalStat.addUpdates(locOption.get(), count); - } else { - // insert - partitionPathStatMap.get(partitionPath).addInserts(count); - globalStat.addInserts(count); - } - } - } - - public WorkloadStat getGlobalStat() { - return globalStat; - } - - public Set getPartitionPaths() { - return partitionPathStatMap.keySet(); - } - - public HashMap getPartitionPathStatMap() { - return partitionPathStatMap; - } - - public WorkloadStat getWorkloadStat(String partitionPath) { - return partitionPathStatMap.get(partitionPath); - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder("WorkloadProfile {"); - sb.append("globalStat=").append(globalStat).append(", "); - sb.append("partitionStat=").append(partitionPathStatMap); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/CommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/CommitActionExecutor.java deleted file mode 100644 index f35acaf6b..000000000 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/CommitActionExecutor.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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.table.action.commit; - -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieUpsertException; -import org.apache.hudi.execution.LazyInsertIterable; -import org.apache.hudi.io.HoodieMergeHandle; -import org.apache.hudi.io.HoodieSortedMergeHandle; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.WorkloadProfile; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.Partitioner; -import org.apache.spark.api.java.JavaSparkContext; - -import java.io.IOException; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -public abstract class CommitActionExecutor> - extends BaseCommitActionExecutor { - - private static final Logger LOG = LogManager.getLogger(CommitActionExecutor.class); - - public CommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, - String instantTime, WriteOperationType operationType) { - this(jsc, config, table, instantTime, operationType, Option.empty()); - } - - public CommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, - String instantTime, WriteOperationType operationType, - Option> extraMetadata) { - super(jsc, config, table, instantTime, operationType, extraMetadata); - } - - @Override - public Iterator> handleUpdate(String partitionPath, String fileId, - Iterator> recordItr) - throws IOException { - // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records - if (!recordItr.hasNext()) { - LOG.info("Empty partition with fileId => " + fileId); - return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); - } - // these are updates - HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr); - return handleUpdateInternal(upsertHandle, fileId); - } - - public Iterator> handleUpdate(String partitionPath, String fileId, - Map> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException { - // these are updates - HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, keyToNewRecords, oldDataFile); - return handleUpdateInternal(upsertHandle, fileId); - } - - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId) - throws IOException { - if (upsertHandle.getOldFilePath() == null) { - throw new HoodieUpsertException( - "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); - } else { - MergeHelper.runMerge(table, upsertHandle); - } - - // TODO(vc): This needs to be revisited - if (upsertHandle.getWriteStatus().getPartitionPath() == null) { - LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " - + upsertHandle.getWriteStatus()); - } - return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator(); - } - - protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator> recordItr) { - if (table.requireSortedRecords()) { - return new HoodieSortedMergeHandle<>(config, instantTime, (HoodieTable)table, recordItr, partitionPath, fileId, sparkTaskContextSupplier); - } else { - return new HoodieMergeHandle<>(config, instantTime, (HoodieTable)table, recordItr, partitionPath, fileId, sparkTaskContextSupplier); - } - } - - protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, - Map> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) { - return new HoodieMergeHandle<>(config, instantTime, (HoodieTable)table, keyToNewRecords, - partitionPath, fileId, dataFileToBeMerged, sparkTaskContextSupplier); - } - - @Override - public Iterator> handleInsert(String idPfx, Iterator> recordItr) - throws Exception { - // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records - if (!recordItr.hasNext()) { - LOG.info("Empty partition"); - return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); - } - return new LazyInsertIterable<>(recordItr, config, instantTime, (HoodieTable)table, idPfx, - sparkTaskContextSupplier); - } - - @Override - public Partitioner getUpsertPartitioner(WorkloadProfile profile) { - if (profile == null) { - throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner."); - } - return new UpsertPartitioner(profile, jsc, table, config); - } - - @Override - public Partitioner getInsertPartitioner(WorkloadProfile profile) { - return getUpsertPartitioner(profile); - } -} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/WriteHelper.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/WriteHelper.java deleted file mode 100644 index 92dcbb628..000000000 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/WriteHelper.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * 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.table.action.commit; - -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.exception.HoodieUpsertException; -import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.table.HoodieTable; - -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; - -import java.time.Duration; -import java.time.Instant; -import scala.Tuple2; - -public class WriteHelper> { - - public static > HoodieWriteMetadata write(String instantTime, - JavaRDD> inputRecordsRDD, JavaSparkContext jsc, - HoodieTable table, boolean shouldCombine, - int shuffleParallelism, CommitActionExecutor executor, boolean performTagging) { - try { - // De-dupe/merge if needed - JavaRDD> dedupedRecords = - combineOnCondition(shouldCombine, inputRecordsRDD, shuffleParallelism, table); - - Instant lookupBegin = Instant.now(); - JavaRDD> taggedRecords = dedupedRecords; - if (performTagging) { - // perform index loop up to get existing location of records - taggedRecords = tag(dedupedRecords, jsc, table); - } - Duration indexLookupDuration = Duration.between(lookupBegin, Instant.now()); - - HoodieWriteMetadata result = executor.execute(taggedRecords); - result.setIndexLookupDuration(indexLookupDuration); - return result; - } catch (Throwable e) { - if (e instanceof HoodieUpsertException) { - throw (HoodieUpsertException) e; - } - throw new HoodieUpsertException("Failed to upsert for commit time " + instantTime, e); - } - } - - private static > JavaRDD> tag( - JavaRDD> dedupedRecords, JavaSparkContext jsc, HoodieTable table) { - // perform index loop up to get existing location of records - return table.getIndex().tagLocation(dedupedRecords, jsc, table); - } - - public static > JavaRDD> combineOnCondition( - boolean condition, JavaRDD> records, int parallelism, HoodieTable table) { - return condition ? deduplicateRecords(records, table, parallelism) : records; - } - - /** - * Deduplicate Hoodie records, using the given deduplication function. - * - * @param records hoodieRecords to deduplicate - * @param parallelism parallelism or partitions to be used while reducing/deduplicating - * @return RDD of HoodieRecord already be deduplicated - */ - public static > JavaRDD> deduplicateRecords( - JavaRDD> records, HoodieTable table, int parallelism) { - return deduplicateRecords(records, table.getIndex(), parallelism); - } - - public static > JavaRDD> deduplicateRecords( - JavaRDD> records, HoodieIndex index, int parallelism) { - boolean isIndexingGlobal = index.isGlobal(); - return records.mapToPair(record -> { - HoodieKey hoodieKey = record.getKey(); - // If index used is global, then records are expected to differ in their partitionPath - Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey; - return new Tuple2<>(key, record); - }).reduceByKey((rec1, rec2) -> { - @SuppressWarnings("unchecked") - T reducedData = (T) rec1.getData().preCombine(rec2.getData()); - // we cannot allow the user to change the key or partitionPath, since that will affect - // everything - // so pick it from one of the records. - return new HoodieRecord(rec1.getKey(), reducedData); - }, parallelism).map(Tuple2::_2); - } -} diff --git a/hudi-examples/pom.xml b/hudi-examples/pom.xml index 5fe3ddc32..ba132904e 100644 --- a/hudi-examples/pom.xml +++ b/hudi-examples/pom.xml @@ -129,7 +129,13 @@ org.apache.hudi - hudi-client + hudi-client-common + ${project.version} + + + + org.apache.hudi + hudi-spark-client ${project.version} diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java index 655e549d9..b606c527b 100644 --- a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java @@ -18,8 +18,9 @@ package org.apache.hudi.examples.spark; -import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieKey; @@ -48,7 +49,7 @@ import java.util.stream.Collectors; /** - * Simple examples of #{@link HoodieWriteClient}. + * Simple examples of #{@link SparkRDDWriteClient}. * * To run this example, you should *

@@ -94,7 +95,7 @@ public class HoodieWriteClientExample {
               .withDeleteParallelism(2).forTable(tableName)
               .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
               .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(20, 30).build()).build();
-      HoodieWriteClient client = new HoodieWriteClient<>(jsc, cfg);
+      SparkRDDWriteClient client = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jsc), cfg);
 
       // inserts
       String newCommitTime = client.startCommit();
diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml
index fff52a44a..e9fcc61e0 100644
--- a/hudi-integ-test/pom.xml
+++ b/hudi-integ-test/pom.xml
@@ -191,7 +191,13 @@
     
     
       org.apache.hudi
-      hudi-client
+      hudi-client-common
+      ${project.version}
+      test-jar
+    
+    
+      org.apache.hudi
+      hudi-spark-client
       ${project.version}
       test-jar
     
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java
index b22faca22..ff4f0a673 100644
--- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java
@@ -27,8 +27,9 @@ import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
 import org.apache.hudi.client.HoodieReadClient;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.util.Option;
@@ -50,12 +51,12 @@ import org.apache.spark.api.java.JavaSparkContext;
 /**
  * A writer abstraction for the Hudi test suite. This class wraps different implementations of writers used to perform
  * write operations into the target hudi dataset. Current supported writers are {@link HoodieDeltaStreamerWrapper}
- * and {@link HoodieWriteClient}.
+ * and {@link SparkRDDWriteClient}.
  */
 public class HoodieTestSuiteWriter {
 
   private HoodieDeltaStreamerWrapper deltaStreamerWrapper;
-  private HoodieWriteClient writeClient;
+  private SparkRDDWriteClient writeClient;
   protected HoodieTestSuiteConfig cfg;
   private Option lastCheckpoint;
   private HoodieReadClient hoodieReadClient;
@@ -76,10 +77,11 @@ public class HoodieTestSuiteWriter {
     // We ensure that only 1 instance of HoodieWriteClient is instantiated for a HoodieTestSuiteWriter
     // This does not instantiate a HoodieWriteClient until a
     // {@link HoodieDeltaStreamer#commit(HoodieWriteClient, JavaRDD, Option)} is invoked.
+    HoodieSparkEngineContext context = new HoodieSparkEngineContext(jsc);
     this.deltaStreamerWrapper = new HoodieDeltaStreamerWrapper(cfg, jsc);
-    this.hoodieReadClient = new HoodieReadClient(jsc, cfg.targetBasePath);
+    this.hoodieReadClient = new HoodieReadClient(context, cfg.targetBasePath);
     if (!cfg.useDeltaStreamer) {
-      this.writeClient = new HoodieWriteClient(jsc, getHoodieClientConfig(cfg, props, schema), rollbackInflight);
+      this.writeClient = new SparkRDDWriteClient(context, getHoodieClientConfig(cfg, props, schema), rollbackInflight);
     }
     this.cfg = cfg;
     this.configuration = jsc.hadoopConfiguration();
@@ -162,7 +164,7 @@ public class HoodieTestSuiteWriter {
         }
       }
       if (instantTime.isPresent()) {
-        return writeClient.compact(instantTime.get());
+        return (JavaRDD) writeClient.compact(instantTime.get());
       } else {
         return null;
       }
@@ -183,19 +185,19 @@ public class HoodieTestSuiteWriter {
     if (!cfg.useDeltaStreamer) {
       Map extraMetadata = new HashMap<>();
       /** Store the checkpoint in the commit metadata just like
-       * {@link HoodieDeltaStreamer#commit(HoodieWriteClient, JavaRDD, Option)} **/
+       * {@link HoodieDeltaStreamer#commit(SparkRDDWriteClient, JavaRDD, Option)} **/
       extraMetadata.put(HoodieDeltaStreamerWrapper.CHECKPOINT_KEY, lastCheckpoint.get());
       writeClient.commit(instantTime.get(), records, Option.of(extraMetadata));
     }
   }
 
-  public HoodieWriteClient getWriteClient(DagNode dagNode) throws IllegalAccessException {
+  public SparkRDDWriteClient getWriteClient(DagNode dagNode) throws IllegalAccessException {
     if (cfg.useDeltaStreamer & !allowWriteClientAccess(dagNode)) {
       throw new IllegalAccessException("cannot access write client when testing in deltastreamer mode");
     }
     synchronized (this) {
       if (writeClient == null) {
-        this.writeClient = new HoodieWriteClient(this.sparkContext, getHoodieClientConfig(cfg, props, schema), false);
+        this.writeClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(this.sparkContext), getHoodieClientConfig(cfg, props, schema), false);
       }
     }
     return writeClient;
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSHoodieDatasetInputReader.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSHoodieDatasetInputReader.java
index 15e6f7072..1caf8f80f 100644
--- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSHoodieDatasetInputReader.java
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSHoodieDatasetInputReader.java
@@ -26,8 +26,9 @@ import java.util.List;
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.common.testutils.HoodieTestUtils;
@@ -70,7 +71,7 @@ public class TestDFSHoodieDatasetInputReader extends UtilitiesTestBase {
   public void testSimpleHoodieDatasetReader() throws Exception {
 
     HoodieWriteConfig config = makeHoodieClientConfig();
-    HoodieWriteClient client = new HoodieWriteClient(jsc, config);
+    SparkRDDWriteClient client = new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config);
     String commitTime = client.startCommit();
     HoodieTestDataGenerator generator = new HoodieTestDataGenerator();
     // Insert 100 records across 3 partitions
diff --git a/hudi-spark/pom.xml b/hudi-spark/pom.xml
index a3c846497..094232754 100644
--- a/hudi-spark/pom.xml
+++ b/hudi-spark/pom.xml
@@ -168,7 +168,12 @@
     
     
       org.apache.hudi
-      hudi-client
+      hudi-client-common
+      ${project.version}
+    
+    
+      org.apache.hudi
+      hudi-spark-client
       ${project.version}
     
     
@@ -340,7 +345,15 @@
     
     
       org.apache.hudi
-      hudi-client
+      hudi-client-common
+      ${project.version}
+      tests
+      test-jar
+      test
+    
+    
+      org.apache.hudi
+      hudi-spark-client
       ${project.version}
       tests
       test-jar
diff --git a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java
index b61d3a6b3..a4d47157a 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java
@@ -22,8 +22,9 @@ import org.apache.avro.generic.GenericRecord;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.client.HoodieReadClient;
-import org.apache.hudi.client.HoodieWriteClient;
 import org.apache.hudi.client.HoodieWriteResult;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
@@ -171,9 +172,9 @@ public class DataSourceUtils {
         .withProps(parameters).build();
   }
 
-  public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
-      String tblName, Map parameters) {
-    return new HoodieWriteClient<>(jssc, createHoodieConfig(schemaStr, basePath, tblName, parameters), true);
+  public static SparkRDDWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
+                                                       String tblName, Map parameters) {
+    return new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), createHoodieConfig(schemaStr, basePath, tblName, parameters), true);
   }
 
   public static String getCommitActionType(WriteOperationType operation, HoodieTableType tableType) {
@@ -184,7 +185,7 @@ public class DataSourceUtils {
     }
   }
 
-  public static HoodieWriteResult doWriteOperation(HoodieWriteClient client, JavaRDD hoodieRecords,
+  public static HoodieWriteResult doWriteOperation(SparkRDDWriteClient client, JavaRDD hoodieRecords,
                                                    String instantTime, WriteOperationType operation) throws HoodieException {
     switch (operation) {
       case BULK_INSERT:
@@ -202,7 +203,7 @@ public class DataSourceUtils {
     }
   }
 
-  public static HoodieWriteResult doDeleteOperation(HoodieWriteClient client, JavaRDD hoodieKeys,
+  public static HoodieWriteResult doDeleteOperation(SparkRDDWriteClient client, JavaRDD hoodieKeys,
       String instantTime) {
     return new HoodieWriteResult(client.delete(hoodieKeys, instantTime));
   }
@@ -224,7 +225,7 @@ public class DataSourceUtils {
   public static JavaRDD dropDuplicates(JavaSparkContext jssc, JavaRDD incomingHoodieRecords,
       HoodieWriteConfig writeConfig) {
     try {
-      HoodieReadClient client = new HoodieReadClient<>(jssc, writeConfig);
+      HoodieReadClient client = new HoodieReadClient<>(new HoodieSparkEngineContext(jssc), writeConfig);
       return client.tagLocation(incomingHoodieRecords)
           .filter(r -> !((HoodieRecord) r).isCurrentLocationKnown());
     } catch (TableNotFoundException e) {
diff --git a/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java b/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java
index ae0ad7357..d1a415b17 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java
@@ -18,8 +18,10 @@
 
 package org.apache.hudi.async;
 
-import org.apache.hudi.client.HoodieWriteClient;
-import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.hudi.client.AbstractCompactor;
+import org.apache.hudi.client.AbstractHoodieWriteClient;
+import org.apache.hudi.client.HoodieSparkCompactor;
+import org.apache.hudi.client.common.HoodieEngineContext;
 
 /**
  * Async Compaction Service used by Structured Streaming. Here, async compaction is run in daemon mode to prevent
@@ -29,7 +31,12 @@ public class SparkStreamingAsyncCompactService extends AsyncCompactService {
 
   private static final long serialVersionUID = 1L;
 
-  public SparkStreamingAsyncCompactService(JavaSparkContext jssc, HoodieWriteClient client) {
-    super(jssc, client, true);
+  public SparkStreamingAsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client) {
+    super(context, client, true);
+  }
+
+  @Override
+  protected AbstractCompactor createCompactor(AbstractHoodieWriteClient client) {
+    return new HoodieSparkCompactor(client);
   }
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java b/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java
index 3a5a79647..022abe308 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java
@@ -23,6 +23,7 @@ import org.apache.hudi.DataSourceUtils;
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.avro.model.HoodieFileStatus;
 import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.bootstrap.FileStatusUtils;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.model.HoodieRecord;
@@ -32,7 +33,6 @@ import org.apache.hudi.keygen.KeyGenerator;
 
 import org.apache.avro.generic.GenericRecord;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.rdd.RDD;
 import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.SparkSession;
@@ -43,18 +43,18 @@ import java.util.List;
 /**
  * Spark Data frame based bootstrap input provider.
  */
-public class SparkParquetBootstrapDataProvider extends FullRecordBootstrapDataProvider {
+public class SparkParquetBootstrapDataProvider extends FullRecordBootstrapDataProvider> {
 
   private final transient SparkSession sparkSession;
 
   public SparkParquetBootstrapDataProvider(TypedProperties props,
-                                           JavaSparkContext jsc) {
-    super(props, jsc);
-    this.sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate();
+                                           HoodieSparkEngineContext context) {
+    super(props, context);
+    this.sparkSession = SparkSession.builder().config(context.getJavaSparkContext().getConf()).getOrCreate();
   }
 
   @Override
-  public JavaRDD generateInputRecordRDD(String tableName, String sourceBasePath,
+  public JavaRDD generateInputRecords(String tableName, String sourceBasePath,
       List>> partitionPathsWithFiles) {
     String[] filePaths = partitionPathsWithFiles.stream().map(Pair::getValue)
         .flatMap(f -> f.stream().map(fs -> FileStatusUtils.toPath(fs.getPath()).toString()))
diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java
index a4bfd5910..e8cbff80a 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java
@@ -20,7 +20,8 @@ package org.apache.hudi.internal;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.DataSourceUtils;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.model.WriteOperationType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -30,6 +31,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -59,7 +61,7 @@ public class HoodieDataSourceInternalWriter implements DataSourceWriter {
   private final HoodieTableMetaClient metaClient;
   private final HoodieWriteConfig writeConfig;
   private final StructType structType;
-  private final HoodieWriteClient writeClient;
+  private final SparkRDDWriteClient writeClient;
   private final HoodieTable hoodieTable;
   private final WriteOperationType operationType;
 
@@ -69,11 +71,11 @@ public class HoodieDataSourceInternalWriter implements DataSourceWriter {
     this.writeConfig = writeConfig;
     this.structType = structType;
     this.operationType = WriteOperationType.BULK_INSERT;
-    this.writeClient  = new HoodieWriteClient<>(new JavaSparkContext(sparkSession.sparkContext()), writeConfig, true);
+    this.writeClient  = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), writeConfig, true);
     writeClient.setOperationType(operationType);
     writeClient.startCommitWithTime(instantTime);
     this.metaClient = new HoodieTableMetaClient(configuration, writeConfig.getBasePath());
-    this.hoodieTable = HoodieTable.create(metaClient, writeConfig, metaClient.getHadoopConf());
+    this.hoodieTable = HoodieSparkTable.create(writeConfig, new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), metaClient);
   }
 
   @Override
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
index 26efa5eed..7c67f9aec 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
@@ -36,7 +36,7 @@ import java.util.List;
  * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
  */
 @PublicAPIClass(maturity = ApiMaturityLevel.STABLE)
-public abstract class KeyGenerator implements Serializable, KeyGeneratorInterface {
+public abstract class KeyGenerator implements Serializable, SparkKeyGeneratorInterface {
 
   private static final String STRUCT_NAME = "hoodieRowTopLevelField";
   private static final String NAMESPACE = "hoodieRow";
diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
index df7960e5b..7173a2834 100644
--- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
+++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
@@ -27,7 +27,8 @@ import org.apache.hadoop.fs.{FileSystem, Path}
 import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hudi.DataSourceWriteOptions._
 import org.apache.hudi.avro.HoodieAvroUtils
-import org.apache.hudi.client.{HoodieWriteClient, HoodieWriteResult}
+import org.apache.hudi.client.{SparkRDDWriteClient, HoodieWriteResult}
+import org.apache.hudi.client.{SparkRDDWriteClient, WriteStatus}
 import org.apache.hudi.common.config.TypedProperties
 import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType}
 import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
@@ -59,11 +60,11 @@ private[hudi] object HoodieSparkSqlWriter {
             parameters: Map[String, String],
             df: DataFrame,
             hoodieTableConfigOpt: Option[HoodieTableConfig] = Option.empty,
-            hoodieWriteClient: Option[HoodieWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty,
-            asyncCompactionTriggerFn: Option[Function1[HoodieWriteClient[HoodieRecordPayload[Nothing]], Unit]] = Option.empty
+            hoodieWriteClient: Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty,
+            asyncCompactionTriggerFn: Option[Function1[SparkRDDWriteClient[HoodieRecordPayload[Nothing]], Unit]] = Option.empty
            )
   : (Boolean, common.util.Option[String], common.util.Option[String],
-     HoodieWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = {
+    SparkRDDWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = {
 
     val sparkContext = sqlContext.sparkContext
     val path = parameters.get("path")
@@ -126,7 +127,7 @@ private[hudi] object HoodieSparkSqlWriter {
       }
       // scalastyle:on
 
-      val (writeResult, writeClient: HoodieWriteClient[HoodieRecordPayload[Nothing]]) =
+      val (writeResult, writeClient: SparkRDDWriteClient[HoodieRecordPayload[Nothing]]) =
         if (operation != WriteOperationType.DELETE) {
           // register classes & schemas
           val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tblName)
@@ -151,7 +152,7 @@ private[hudi] object HoodieSparkSqlWriter {
           // Create a HoodieWriteClient & issue the write.
           val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, schema.toString, path.get,
             tblName, mapAsJavaMap(parameters)
-          )).asInstanceOf[HoodieWriteClient[HoodieRecordPayload[Nothing]]]
+          )).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
 
           if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
             asyncCompactionTriggerFn.get.apply(client)
@@ -190,7 +191,7 @@ private[hudi] object HoodieSparkSqlWriter {
           // Create a HoodieWriteClient & issue the delete.
           val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
             Schema.create(Schema.Type.NULL).toString, path.get, tblName,
-            mapAsJavaMap(parameters))).asInstanceOf[HoodieWriteClient[HoodieRecordPayload[Nothing]]]
+            mapAsJavaMap(parameters))).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
 
           if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
             asyncCompactionTriggerFn.get.apply(client)
@@ -389,7 +390,7 @@ private[hudi] object HoodieSparkSqlWriter {
 
   private def commitAndPerformPostOperations(writeResult: HoodieWriteResult,
                                              parameters: Map[String, String],
-                                             client: HoodieWriteClient[HoodieRecordPayload[Nothing]],
+                                             client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]],
                                              tableConfig: HoodieTableConfig,
                                              jsc: JavaSparkContext,
                                              tableInstantInfo: TableInstantInfo
@@ -446,7 +447,7 @@ private[hudi] object HoodieSparkSqlWriter {
     }
   }
 
-  private def isAsyncCompactionEnabled(client: HoodieWriteClient[HoodieRecordPayload[Nothing]],
+  private def isAsyncCompactionEnabled(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]],
                                        tableConfig: HoodieTableConfig,
                                        parameters: Map[String, String], configuration: Configuration) : Boolean = {
     log.info(s"Config.isInlineCompaction ? ${client.getConfig.isInlineCompaction}")
diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
index 1600ab02b..9f57fb53e 100644
--- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
+++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
@@ -20,7 +20,8 @@ import java.lang
 import java.util.function.{Function, Supplier}
 
 import org.apache.hudi.async.{AsyncCompactService, SparkStreamingAsyncCompactService}
-import org.apache.hudi.client.HoodieWriteClient
+import org.apache.hudi.client.SparkRDDWriteClient
+import org.apache.hudi.client.common.HoodieSparkEngineContext
 import org.apache.hudi.common.model.HoodieRecordPayload
 import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
 import org.apache.hudi.common.table.timeline.HoodieInstant.State
@@ -60,7 +61,7 @@ class HoodieStreamingSink(sqlContext: SQLContext,
     }
 
   private var asyncCompactorService : AsyncCompactService = _
-  private var writeClient : Option[HoodieWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty
+  private var writeClient : Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty
   private var hoodieTableConfig : Option[HoodieTableConfig] = Option.empty
 
   override def addBatch(batchId: Long, data: DataFrame): Unit = this.synchronized {
@@ -153,10 +154,10 @@ class HoodieStreamingSink(sqlContext: SQLContext,
     }
   }
 
-  protected def triggerAsyncCompactor(client: HoodieWriteClient[HoodieRecordPayload[Nothing]]): Unit = {
+  protected def triggerAsyncCompactor(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]]): Unit = {
     if (null == asyncCompactorService) {
       log.info("Triggering Async compaction !!")
-      asyncCompactorService = new SparkStreamingAsyncCompactService(new JavaSparkContext(sqlContext.sparkContext),
+      asyncCompactorService = new SparkStreamingAsyncCompactService(new HoodieSparkEngineContext(new JavaSparkContext(sqlContext.sparkContext)),
         client)
       asyncCompactorService.start(new Function[java.lang.Boolean, java.lang.Boolean] {
         override def apply(errored: lang.Boolean): lang.Boolean = {
diff --git a/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala b/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala
index e894b0653..e113d4af5 100644
--- a/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala
+++ b/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala
@@ -17,7 +17,6 @@
 
 package org.apache.hudi
 
-
 import com.google.common.collect.Lists
 import org.apache.avro.Schema
 import org.apache.hadoop.fs.GlobPattern
@@ -30,10 +29,11 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline
 import org.apache.hudi.common.util.ParquetUtils
 import org.apache.hudi.config.HoodieWriteConfig
 import org.apache.hudi.exception.HoodieException
-import org.apache.hudi.table.HoodieTable
-
 import org.apache.hadoop.fs.GlobPattern
+import org.apache.hudi.client.common.HoodieSparkEngineContext
+import org.apache.hudi.table.HoodieSparkTable
 import org.apache.log4j.LogManager
+import org.apache.spark.api.java.JavaSparkContext
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.sources.{BaseRelation, TableScan}
 import org.apache.spark.sql.types.{StringType, StructField, StructType}
@@ -64,8 +64,9 @@ class IncrementalRelation(val sqlContext: SQLContext,
     throw new HoodieException("Incremental view not implemented yet, for merge-on-read tables")
   }
   // TODO : Figure out a valid HoodieWriteConfig
-  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(),
-    sqlContext.sparkContext.hadoopConfiguration)
+  private val hoodieTable = HoodieSparkTable.create(HoodieWriteConfig.newBuilder().withPath(basePath).build(),
+    new HoodieSparkEngineContext(new JavaSparkContext(sqlContext.sparkContext)),
+    metaClient)
   private val commitTimeline = hoodieTable.getMetaClient.getCommitTimeline.filterCompletedInstants()
   if (commitTimeline.empty()) {
     throw new HoodieException("No instants to incrementally pull")
diff --git a/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java b/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java
index 9ff114e46..97948b9ee 100644
--- a/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java
+++ b/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java
@@ -19,7 +19,7 @@
 package org.apache.hudi;
 
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -63,7 +63,7 @@ import static org.mockito.Mockito.when;
 public class TestDataSourceUtils {
 
   @Mock
-  private HoodieWriteClient hoodieWriteClient;
+  private SparkRDDWriteClient hoodieWriteClient;
 
   @Mock
   private JavaRDD hoodieRecords;
@@ -172,7 +172,7 @@ public class TestDataSourceUtils {
   }
 
   public static class NoOpBulkInsertPartitioner
-      implements BulkInsertPartitioner {
+      implements BulkInsertPartitioner>> {
 
     @Override
     public JavaRDD> repartitionRecords(JavaRDD> records, int outputSparkPartitions) {
diff --git a/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java b/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java
index 14f36d469..7e13a5e4b 100644
--- a/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java
+++ b/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java
@@ -25,6 +25,7 @@ import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider;
 import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector;
 import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector;
 import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.bootstrap.FileStatusUtils;
 import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
 import org.apache.hudi.common.config.TypedProperties;
@@ -166,7 +167,7 @@ public class TestBootstrap extends HoodieClientTestBase {
       df.write().format("parquet").mode(SaveMode.Overwrite).save(srcPath);
     }
     String filePath = FileStatusUtils.toPath(BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(),
-            srcPath, jsc).stream().findAny().map(p -> p.getValue().stream().findAny())
+            srcPath, context).stream().findAny().map(p -> p.getValue().stream().findAny())
             .orElse(null).get().getPath()).toString();
     ParquetFileReader reader = ParquetFileReader.open(metaClient.getHadoopConf(), new Path(filePath));
     MessageType schema = reader.getFooter().getFileMetaData().getSchema();
@@ -249,7 +250,7 @@ public class TestBootstrap extends HoodieClientTestBase {
             .withBootstrapParallelism(3)
             .withBootstrapModeSelector(bootstrapModeSelectorClass).build())
         .build();
-    HoodieWriteClient client = new HoodieWriteClient(jsc, config);
+    SparkRDDWriteClient client = new SparkRDDWriteClient(context, config);
     client.bootstrap(Option.empty());
     checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap,
         numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants);
@@ -260,14 +261,14 @@ public class TestBootstrap extends HoodieClientTestBase {
     client.rollBackInflightBootstrap();
     metaClient.reloadActiveTimeline();
     assertEquals(0, metaClient.getCommitsTimeline().countInstants());
-    assertEquals(0L, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath, jsc)
+    assertEquals(0L, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath, context)
             .stream().flatMap(f -> f.getValue().stream()).count());
 
     BootstrapIndex index = BootstrapIndex.getBootstrapIndex(metaClient);
     assertFalse(index.useIndex());
 
     // Run bootstrap again
-    client = new HoodieWriteClient(jsc, config);
+    client = new SparkRDDWriteClient(context, config);
     client.bootstrap(Option.empty());
 
     metaClient.reloadActiveTimeline();
@@ -286,7 +287,7 @@ public class TestBootstrap extends HoodieClientTestBase {
     String updateSPath = tmpFolder.toAbsolutePath().toString() + "/data2";
     generateNewDataSetAndReturnSchema(updateTimestamp, totalRecords, partitions, updateSPath);
     JavaRDD updateBatch =
-        generateInputBatch(jsc, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), updateSPath, jsc),
+        generateInputBatch(jsc, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), updateSPath, context),
                 schema);
     String newInstantTs = client.startCommit();
     client.upsert(updateBatch, newInstantTs);
@@ -348,7 +349,7 @@ public class TestBootstrap extends HoodieClientTestBase {
     original.registerTempTable("original");
     if (checkNumRawFiles) {
       List files = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(),
-          bootstrapBasePath, jsc).stream().flatMap(x -> x.getValue().stream()).collect(Collectors.toList());
+          bootstrapBasePath, context).stream().flatMap(x -> x.getValue().stream()).collect(Collectors.toList());
       assertEquals(files.size() * numVersions,
           sqlContext.sql("select distinct _hoodie_file_name from bootstrapped").count());
     }
@@ -466,18 +467,19 @@ public class TestBootstrap extends HoodieClientTestBase {
     assertEquals(totalRecords, seenKeys.size());
   }
 
-  public static class TestFullBootstrapDataProvider extends FullRecordBootstrapDataProvider {
+  public static class TestFullBootstrapDataProvider extends FullRecordBootstrapDataProvider> {
 
-    public TestFullBootstrapDataProvider(TypedProperties props, JavaSparkContext jsc) {
-      super(props, jsc);
+    public TestFullBootstrapDataProvider(TypedProperties props, HoodieSparkEngineContext context) {
+      super(props, context);
     }
 
     @Override
-    public JavaRDD generateInputRecordRDD(String tableName, String sourceBasePath,
+    public JavaRDD generateInputRecords(String tableName, String sourceBasePath,
         List>> partitionPaths) {
       String filePath = FileStatusUtils.toPath(partitionPaths.stream().flatMap(p -> p.getValue().stream())
           .findAny().get().getPath()).toString();
       ParquetFileReader reader = null;
+      JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
       try {
         reader = ParquetFileReader.open(jsc.hadoopConfiguration(), new Path(filePath));
       } catch (IOException e) {
diff --git a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java
index 884e11ca8..5a5d8b270 100644
--- a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java
+++ b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java
@@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 
@@ -76,7 +77,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends HoodieClientTestHarn
   public void testDataInternalWriter() throws IOException {
     // init config and table
     HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
-    HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
     // execute N rounds
     for (int i = 0; i < 5; i++) {
       String instantTime = "00" + i;
@@ -121,7 +122,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends HoodieClientTestHarn
   public void testGlobalFailure() throws IOException {
     // init config and table
     HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
-    HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
     String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
 
     String instantTime = "001";
diff --git a/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala b/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala
index bcd83dbef..15872dd22 100644
--- a/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala
+++ b/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala
@@ -22,7 +22,7 @@ import java.util.{Date, UUID}
 
 import org.apache.commons.io.FileUtils
 import org.apache.hudi.DataSourceWriteOptions._
-import org.apache.hudi.client.HoodieWriteClient
+import org.apache.hudi.client.SparkRDDWriteClient
 import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordPayload}
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator
 import org.apache.hudi.config.HoodieWriteConfig
@@ -250,7 +250,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
             schema.toString,
             path.toAbsolutePath.toString,
             hoodieFooTableName,
-            mapAsJavaMap(fooTableParams)).asInstanceOf[HoodieWriteClient[HoodieRecordPayload[Nothing]]])
+            mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
 
           // write to Hudi
           HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df, Option.empty,
diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml
index 02100a27c..ab51475de 100644
--- a/hudi-utilities/pom.xml
+++ b/hudi-utilities/pom.xml
@@ -94,7 +94,12 @@
     
     
       org.apache.hudi
-      hudi-client
+      hudi-client-common
+      ${project.version}
+    
+    
+      org.apache.hudi
+      hudi-spark-client
       ${project.version}
     
     
@@ -347,7 +352,15 @@
     
     
       org.apache.hudi
-      hudi-client
+      hudi-client-common
+      ${project.version}
+      tests
+      test-jar
+      test
+    
+    
+      org.apache.hudi
+      hudi-spark-client
       ${project.version}
       tests
       test-jar
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java
index c15585b79..af66c1761 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java
@@ -18,8 +18,10 @@
 
 package org.apache.hudi.utilities;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.HoodieJsonPayload;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.fs.FSUtils;
@@ -142,7 +144,7 @@ public class HDFSParquetImporter implements Serializable {
       // Get schema.
       String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile);
 
-      HoodieWriteClient client =
+      SparkRDDWriteClient client =
           UtilHelpers.createHoodieClient(jsc, cfg.targetPath, schemaStr, cfg.parallelism, Option.empty(), props);
 
       JavaRDD> hoodieRecords = buildHoodieRecordsForImport(jsc, schemaStr);
@@ -166,7 +168,8 @@ public class HDFSParquetImporter implements Serializable {
     AvroReadSupport.setAvroReadSchema(jsc.hadoopConfiguration(), (new Schema.Parser().parse(schemaStr)));
     ParquetInputFormat.setReadSupportClass(job, (AvroReadSupport.class));
 
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Build records for import");
+    HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
+    context.setJobStatus(this.getClass().getSimpleName(), "Build records for import");
     return jsc.newAPIHadoopFile(cfg.srcPath, ParquetInputFormat.class, Void.class, GenericRecord.class,
             job.getConfiguration())
         // To reduce large number of tasks.
@@ -203,8 +206,8 @@ public class HDFSParquetImporter implements Serializable {
    * @param hoodieRecords Hoodie Records
    * @param  Type
    */
-  protected  JavaRDD load(HoodieWriteClient client, String instantTime,
-      JavaRDD> hoodieRecords) {
+  protected  JavaRDD load(SparkRDDWriteClient client, String instantTime,
+                                                                      JavaRDD> hoodieRecords) {
     switch (cfg.command.toLowerCase()) {
       case "upsert": {
         return client.upsert(hoodieRecords, instantTime);
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java
index bf1fedb82..94798eaa4 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java
@@ -18,7 +18,8 @@
 
 package org.apache.hudi.utilities;
 
-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.common.fs.FSUtils;
 import org.apache.hudi.config.HoodieWriteConfig;
@@ -68,7 +69,7 @@ public class HoodieCleaner {
 
   public void run() {
     HoodieWriteConfig hoodieCfg = getHoodieClientConfig();
-    HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg, false);
+    SparkRDDWriteClient client = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg, false);
     client.clean();
   }
 
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactionAdminTool.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactionAdminTool.java
index d30355fa1..d3e4dbab7 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactionAdminTool.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactionAdminTool.java
@@ -21,6 +21,7 @@ package org.apache.hudi.utilities;
 import org.apache.hudi.client.CompactionAdminClient;
 import org.apache.hudi.client.CompactionAdminClient.RenameOpResult;
 import org.apache.hudi.client.CompactionAdminClient.ValidationOpResult;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieFileGroupId;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -60,7 +61,7 @@ public class HoodieCompactionAdminTool {
    */
   public void run(JavaSparkContext jsc) throws Exception {
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath);
-    try (CompactionAdminClient admin = new CompactionAdminClient(jsc, cfg.basePath)) {
+    try (CompactionAdminClient admin = new CompactionAdminClient(new HoodieSparkEngineContext(jsc), cfg.basePath)) {
       final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
       if (cfg.outputPath != null && fs.exists(new Path(cfg.outputPath))) {
         throw new IllegalStateException("Output File Path already exists");
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java
index a09112417..30d5445b3 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.utilities;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.fs.FSUtils;
@@ -130,15 +130,15 @@ public class HoodieCompactor {
   private int doCompact(JavaSparkContext jsc) throws Exception {
     // Get schema.
     String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile);
-    HoodieWriteClient client =
+    SparkRDDWriteClient client =
         UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props);
-    JavaRDD writeResponse = client.compact(cfg.compactionInstantTime);
+    JavaRDD writeResponse = (JavaRDD) client.compact(cfg.compactionInstantTime);
     return UtilHelpers.handleErrors(jsc, cfg.compactionInstantTime, writeResponse);
   }
 
   private int doSchedule(JavaSparkContext jsc) throws Exception {
     // Get schema.
-    HoodieWriteClient client =
+    SparkRDDWriteClient client =
         UtilHelpers.createHoodieClient(jsc, cfg.basePath, "", cfg.parallelism, Option.of(cfg.strategyClassName), props);
     client.scheduleCompactionAtInstant(cfg.compactionInstantTime, Option.empty());
     return 0;
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java
index 916d01905..05b462730 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.utilities;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.SerializableConfiguration;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieBaseFile;
@@ -97,8 +99,10 @@ public class HoodieSnapshotCopier implements Serializable {
         fs.delete(new Path(outputDir), true);
       }
 
-      jsc.setJobGroup(this.getClass().getSimpleName(), "Creating a snapshot");
-      jsc.parallelize(partitions, partitions.size()).flatMap(partition -> {
+      HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
+      context.setJobStatus(this.getClass().getSimpleName(), "Creating a snapshot");
+
+      List> filesToCopy = context.flatMap(partitions, partition -> {
         // Only take latest version files <= latestCommit.
         FileSystem fs1 = FSUtils.getFs(baseDir, serConf.newCopy());
         List> filePaths = new ArrayList<>();
@@ -112,8 +116,10 @@ public class HoodieSnapshotCopier implements Serializable {
           filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString()));
         }
 
-        return filePaths.iterator();
-      }).foreach(tuple -> {
+        return filePaths.stream();
+      }, partitions.size());
+
+      context.foreach(filesToCopy, tuple -> {
         String partition = tuple._1();
         Path sourceFilePath = new Path(tuple._2());
         Path toPartitionPath = new Path(outputDir, partition);
@@ -124,8 +130,8 @@ public class HoodieSnapshotCopier implements Serializable {
         }
         FileUtil.copy(ifs, sourceFilePath, ifs, new Path(toPartitionPath, sourceFilePath.getName()), false,
             ifs.getConf());
-      });
-
+      }, filesToCopy.size());
+      
       // Also copy the .commit files
       LOG.info(String.format("Copying .commit files which are no-late-than %s.", latestCommitTimestamp));
       FileStatus[] commitFilesToCopy =
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java
index 0743839ff..cf69dd220 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.utilities;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.SerializableConfiguration;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieBaseFile;
@@ -175,7 +177,8 @@ public class HoodieSnapshotExporter {
         ? defaultPartitioner
         : ReflectionUtils.loadClass(cfg.outputPartitioner);
 
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Exporting as non-HUDI dataset");
+    HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
+    context.setJobStatus(this.getClass().getSimpleName(), "Exporting as non-HUDI dataset");
     final BaseFileOnlyView fsView = getBaseFileOnlyView(jsc, cfg);
     Iterator exportingFilePaths = jsc
         .parallelize(partitions, partitions.size())
@@ -193,14 +196,16 @@ public class HoodieSnapshotExporter {
 
   private void exportAsHudi(JavaSparkContext jsc, Config cfg, List partitions, String latestCommitTimestamp) throws IOException {
     final BaseFileOnlyView fsView = getBaseFileOnlyView(jsc, cfg);
-    final SerializableConfiguration serConf = new SerializableConfiguration(jsc.hadoopConfiguration());
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Exporting as HUDI dataset");
-    jsc.parallelize(partitions, partitions.size()).flatMap(partition -> {
+
+    final HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
+    final SerializableConfiguration serConf = context.getHadoopConf();
+    context.setJobStatus(this.getClass().getSimpleName(), "Exporting as HUDI dataset");
+
+    List> files = context.flatMap(partitions, partition -> {
       // Only take latest version files <= latestCommit.
       List> filePaths = new ArrayList<>();
       Stream dataFiles = fsView.getLatestBaseFilesBeforeOrOn(partition, latestCommitTimestamp);
       dataFiles.forEach(hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath())));
-
       // also need to copy over partition metadata
       Path partitionMetaFile =
           new Path(new Path(cfg.sourceBasePath, partition), HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE);
@@ -208,9 +213,10 @@ public class HoodieSnapshotExporter {
       if (fs.exists(partitionMetaFile)) {
         filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString()));
       }
+      return filePaths.stream();
+    }, partitions.size());
 
-      return filePaths.iterator();
-    }).foreach(tuple -> {
+    context.foreach(files, tuple -> {
       String partition = tuple._1();
       Path sourceFilePath = new Path(tuple._2());
       Path toPartitionPath = new Path(cfg.targetOutputPath, partition);
@@ -221,7 +227,7 @@ public class HoodieSnapshotExporter {
       }
       FileUtil.copy(fs, sourceFilePath, fs, new Path(toPartitionPath, sourceFilePath.getName()), false,
           fs.getConf());
-    });
+    }, files.size());
 
     // Also copy the .commit files
     LOG.info(String.format("Copying .commit files which are no-late-than %s.", latestCommitTimestamp));
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieWithTimelineServer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieWithTimelineServer.java
index 06c1084c9..8974061c4 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieWithTimelineServer.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieWithTimelineServer.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.utilities;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.util.ValidationUtils;
 
 import com.beust.jcommander.JCommander;
@@ -86,8 +88,10 @@ public class HoodieWithTimelineServer implements Serializable {
     System.out.println("Driver Hostname is :" + driverHost);
     List messages = new ArrayList<>();
     IntStream.range(0, cfg.numPartitions).forEach(i -> messages.add("Hello World"));
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Sending requests to driver host");
-    List gotMessages = jsc.parallelize(messages).map(msg -> sendRequest(driverHost, cfg.serverPort)).collect();
+
+    HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
+    context.setJobStatus(this.getClass().getSimpleName(), "Sending requests to driver host");
+    List gotMessages = context.map(messages, msg -> sendRequest(driverHost, cfg.serverPort), messages.size());
     System.out.println("Got Messages :" + gotMessages);
     ValidationUtils.checkArgument(gotMessages.equals(messages), "Got expected reply from Server");
   }
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
index 0f4a64cd2..29fc19572 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
@@ -20,8 +20,9 @@ package org.apache.hudi.utilities;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.AvroConversionUtils;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.DFSPropertiesConfiguration;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.util.Option;
@@ -230,7 +231,7 @@ public class UtilHelpers {
     sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
 
     additionalConfigs.forEach(sparkConf::set);
-    return HoodieWriteClient.registerClasses(sparkConf);
+    return SparkRDDWriteClient.registerClasses(sparkConf);
   }
 
   public static JavaSparkContext buildSparkContext(String appName, String defaultMaster, Map configs) {
@@ -260,8 +261,8 @@ public class UtilHelpers {
    * @param schemaStr   Schema
    * @param parallelism Parallelism
    */
-  public static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, String schemaStr,
-                                                     int parallelism, Option compactionStrategyClass, TypedProperties properties) {
+  public static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, String schemaStr,
+                                                             int parallelism, Option compactionStrategyClass, TypedProperties properties) {
     HoodieCompactionConfig compactionConfig = compactionStrategyClass
         .map(strategy -> HoodieCompactionConfig.newBuilder().withInlineCompaction(false)
             .withCompactionStrategy(ReflectionUtils.loadClass(strategy)).build())
@@ -274,7 +275,7 @@ public class UtilHelpers {
             .withSchema(schemaStr).combineInput(true, true).withCompactionConfig(compactionConfig)
             .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
             .withProps(properties).build();
-    return new HoodieWriteClient(jsc, config);
+    return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config);
   }
 
   public static int handleErrors(JavaSparkContext jsc, String instantTime, JavaRDD writeResponse) {
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java
index a137cac5d..c7974b3ee 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java
@@ -20,7 +20,8 @@ package org.apache.hudi.utilities.deltastreamer;
 
 import org.apache.hudi.DataSourceUtils;
 import org.apache.hudi.DataSourceWriteOptions;
-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.common.model.HoodieTableType;
 import org.apache.hudi.common.table.HoodieTableConfig;
@@ -137,7 +138,7 @@ public class BootstrapExecutor  implements Serializable {
    */
   public void execute() throws IOException {
     initializeTable();
-    HoodieWriteClient bootstrapClient = new HoodieWriteClient(jssc, bootstrapConfig, true);
+    SparkRDDWriteClient bootstrapClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(jssc), bootstrapConfig, true);
 
     try {
       HashMap checkpointCommitMetadata = new HashMap<>();
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
index afe686213..a8d2ac108 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
@@ -21,8 +21,9 @@ package org.apache.hudi.utilities.deltastreamer;
 import org.apache.hudi.AvroConversionUtils;
 import org.apache.hudi.DataSourceUtils;
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -158,7 +159,7 @@ public class DeltaSync implements Serializable {
   /**
    * Callback when write client is instantiated.
    */
-  private transient Function onInitializingHoodieWriteClient;
+  private transient Function onInitializingHoodieWriteClient;
 
   /**
    * Timeline with completed commits.
@@ -168,13 +169,13 @@ public class DeltaSync implements Serializable {
   /**
    * Write Client.
    */
-  private transient HoodieWriteClient writeClient;
+  private transient SparkRDDWriteClient writeClient;
 
   private transient HoodieDeltaStreamerMetrics metrics;
 
   public DeltaSync(HoodieDeltaStreamer.Config cfg, SparkSession sparkSession, SchemaProvider schemaProvider,
                    TypedProperties props, JavaSparkContext jssc, FileSystem fs, Configuration conf,
-                   Function onInitializingHoodieWriteClient) throws IOException {
+                   Function onInitializingHoodieWriteClient) throws IOException {
 
     this.cfg = cfg;
     this.jssc = jssc;
@@ -543,7 +544,7 @@ public class DeltaSync implements Serializable {
     if ((null != schemaProvider) && (null == writeClient)) {
       registerAvroSchemas(schemaProvider);
       HoodieWriteConfig hoodieCfg = getHoodieClientConfig(schemaProvider);
-      writeClient = new HoodieWriteClient<>(jssc, hoodieCfg, true);
+      writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg, true);
       onInitializingHoodieWriteClient.apply(writeClient);
     }
   }
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
index 692422dd6..5e6e655a2 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
@@ -18,10 +18,12 @@
 
 package org.apache.hudi.utilities.deltastreamer;
 
-import org.apache.hudi.async.AbstractAsyncService;
+import org.apache.hudi.async.HoodieAsyncService;
 import org.apache.hudi.async.AsyncCompactService;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.async.SparkAsyncCompactService;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.fs.FSUtils;
@@ -470,7 +472,7 @@ public class HoodieDeltaStreamer implements Serializable {
   /**
    * Syncs data either in single-run or in continuous mode.
    */
-  public static class DeltaSyncService extends AbstractAsyncService {
+  public static class DeltaSyncService extends HoodieAsyncService {
 
     private static final long serialVersionUID = 1L;
     /**
@@ -620,9 +622,9 @@ public class HoodieDeltaStreamer implements Serializable {
      * @param writeClient HoodieWriteClient
      * @return
      */
-    protected Boolean onInitializingWriteClient(HoodieWriteClient writeClient) {
+    protected Boolean onInitializingWriteClient(SparkRDDWriteClient writeClient) {
       if (cfg.isAsyncCompactionEnabled()) {
-        asyncCompactService = new AsyncCompactService(jssc, writeClient);
+        asyncCompactService = new SparkAsyncCompactService(new HoodieSparkEngineContext(jssc), writeClient);
         // Enqueue existing pending compactions first
         HoodieTableMetaClient meta =
             new HoodieTableMetaClient(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, true);
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java
index 6aaa6bd26..f338e52cb 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.utilities.perf;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -132,8 +134,9 @@ public class TimelineServerPerf implements Serializable {
 
   public List runLookups(JavaSparkContext jsc, List partitionPaths, SyncableFileSystemView fsView,
       int numIterations, int concurrency) {
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Lookup all performance stats");
-    return jsc.parallelize(partitionPaths, cfg.numExecutors).flatMap(p -> {
+    HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
+    context.setJobStatus(this.getClass().getSimpleName(), "Lookup all performance stats");
+    return context.flatMap(partitionPaths, p -> {
       ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(100);
       final List result = new ArrayList<>();
       final List> futures = new ArrayList<>();
@@ -141,7 +144,7 @@ public class TimelineServerPerf implements Serializable {
       String fileId = slices.isEmpty() ? "dummyId"
           : slices.get(new Random(Double.doubleToLongBits(Math.random())).nextInt(slices.size())).getFileId();
       IntStream.range(0, concurrency).forEach(i -> futures.add(executor.schedule(() -> runOneRound(fsView, p, fileId,
-              i, numIterations), 0, TimeUnit.NANOSECONDS)));
+          i, numIterations), 0, TimeUnit.NANOSECONDS)));
       futures.forEach(x -> {
         try {
           result.add(x.get());
@@ -151,8 +154,8 @@ public class TimelineServerPerf implements Serializable {
       });
       System.out.println("SLICES are=");
       slices.forEach(s -> System.out.println("\t\tFileSlice=" + s));
-      return result.iterator();
-    }).collect();
+      return result.stream();
+    }, cfg.numExecutors);
   }
 
   private static PerfStats runOneRound(SyncableFileSystemView fsView, String partition, String fileId, int id,
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java
index 39341b2ee..5f511740c 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.utilities.functional;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.common.model.HoodieAvroPayload;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieTableType;
@@ -85,7 +85,7 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
 
     // Prepare data as source Hudi dataset
     HoodieWriteConfig cfg = getHoodieWriteConfig(sourcePath);
-    HoodieWriteClient hdfsWriteClient = new HoodieWriteClient(jsc(), cfg);
+    SparkRDDWriteClient hdfsWriteClient = new SparkRDDWriteClient(context(), cfg);
     hdfsWriteClient.startCommitWithTime(COMMIT_TIME);
     HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(new String[] {PARTITION_PATH});
     List records = dataGen.generateInserts(COMMIT_TIME, NUM_RECORDS);
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java
index a25d45d1a..0bbdb2346 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java
@@ -19,6 +19,8 @@
 package org.apache.hudi.utilities.testutils;
 
 import java.io.FileInputStream;
+
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieTableType;
@@ -84,6 +86,7 @@ public class UtilitiesTestBase {
   protected static MiniDFSCluster dfsCluster;
   protected static DistributedFileSystem dfs;
   protected transient JavaSparkContext jsc = null;
+  protected transient HoodieSparkEngineContext context = null;
   protected transient SparkSession sparkSession = null;
   protected transient SQLContext sqlContext;
   protected static HiveServer2 hiveServer;
@@ -129,6 +132,7 @@ public class UtilitiesTestBase {
   public void setup() throws Exception {
     TestDataSource.initDataGen();
     jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[2]");
+    context = new HoodieSparkEngineContext(jsc);
     sqlContext = new SQLContext(jsc);
     sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate();
   }
@@ -139,6 +143,9 @@ public class UtilitiesTestBase {
     if (jsc != null) {
       jsc.stop();
     }
+    if (context != null) {
+      context = null;
+    }
   }
 
   /**
diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml
index 48dd16924..84285dcdc 100644
--- a/packaging/hudi-integ-test-bundle/pom.xml
+++ b/packaging/hudi-integ-test-bundle/pom.xml
@@ -69,7 +69,8 @@
                   commons-pool:commons-pool
 
                   org.apache.hudi:hudi-common
-                  org.apache.hudi:hudi-client
+                  org.apache.hudi:hudi-client-common
+                  org.apache.hudi:hudi-spark-client
                   org.apache.hudi:hudi-utilities_${scala.binary.version}
                   org.apache.hudi:hudi-spark_${scala.binary.version}
                   org.apache.hudi:hudi-hive-sync
@@ -431,7 +432,12 @@
 
     
       org.apache.hudi
-      hudi-client
+      hudi-client-common
+      ${project.version}
+    
+    
+      org.apache.hudi
+      hudi-spark-client
       ${project.version}
     
 
@@ -450,7 +456,15 @@
 
     
       org.apache.hudi
-      hudi-client
+      hudi-client-common
+      ${project.version}
+      tests
+      test-jar
+      test
+    
+    
+      org.apache.hudi
+      hudi-spark-client
       ${project.version}
       tests
       test-jar
diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml
index 92bf79766..2c22ac3e8 100644
--- a/packaging/hudi-spark-bundle/pom.xml
+++ b/packaging/hudi-spark-bundle/pom.xml
@@ -65,7 +65,8 @@
                 
                   com.yammer.metrics:metrics-core
                   org.apache.hudi:hudi-common
-                  org.apache.hudi:hudi-client
+                  org.apache.hudi:hudi-client-common
+                  org.apache.hudi:hudi-spark-client
                   org.apache.hudi:hudi-spark_${scala.binary.version}
                   org.apache.hudi:hudi-hive-sync
                   org.apache.hudi:hudi-sync-common
@@ -209,7 +210,12 @@
     
     
       org.apache.hudi
-      hudi-client
+      hudi-client-common
+      ${project.version}
+    
+    
+      org.apache.hudi
+      hudi-spark-client
       ${project.version}
     
     
diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml
index eb4fee7a3..6bd5fec3f 100644
--- a/packaging/hudi-utilities-bundle/pom.xml
+++ b/packaging/hudi-utilities-bundle/pom.xml
@@ -66,7 +66,8 @@
               
                 
                   org.apache.hudi:hudi-common
-                  org.apache.hudi:hudi-client
+                  org.apache.hudi:hudi-client-common
+                  org.apache.hudi:hudi-spark-client
                   org.apache.hudi:hudi-utilities_${scala.binary.version}
                   org.apache.hudi:hudi-spark_${scala.binary.version}
                   org.apache.hudi:hudi-hive-sync
@@ -211,7 +212,12 @@
     
     
       org.apache.hudi
-      hudi-client
+      hudi-client-common
+      ${project.version}
+    
+    
+      org.apache.hudi
+      hudi-spark-client
       ${project.version}
     
     
diff --git a/style/checkstyle.xml b/style/checkstyle.xml
index 86575c91e..c816abe99 100644
--- a/style/checkstyle.xml
+++ b/style/checkstyle.xml
@@ -46,10 +46,12 @@
         
         
     
+    
     
         
             
         
+